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:41 UTC

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

Repository: hive
Updated Branches:
  refs/heads/master d717d3853 -> 12041d39f


http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStoreSchemaMethods.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStoreSchemaMethods.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStoreSchemaMethods.java
new file mode 100644
index 0000000..2b8fbd1
--- /dev/null
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStoreSchemaMethods.java
@@ -0,0 +1,573 @@
+/*
+ * 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;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.annotation.MetastoreCheckinTest;
+import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.ISchema;
+import org.apache.hadoop.hive.metastore.api.ISchemaName;
+import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+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.SchemaVersionDescriptor;
+import org.apache.hadoop.hive.metastore.api.SchemaVersionState;
+import org.apache.hadoop.hive.metastore.api.SerDeInfo;
+import org.apache.hadoop.hive.metastore.client.builder.DatabaseBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.ISchemaBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.SchemaVersionBuilder;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.thrift.TException;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_DATABASE_NAME;
+
+@Category(MetastoreCheckinTest.class)
+public class TestObjectStoreSchemaMethods {
+  private RawStore objectStore;
+
+  @Before
+  public void setUp() throws Exception {
+    Configuration conf = MetastoreConf.newMetastoreConf();
+    MetastoreConf.setVar(conf, MetastoreConf.ConfVars.EXPRESSION_PROXY_CLASS,
+        DefaultPartitionExpressionProxy.class.getName());
+
+    objectStore = new ObjectStore();
+    objectStore.setConf(conf);
+  }
+
+  @Test
+  public void iSchema() throws TException {
+    String dbName = createUniqueDatabaseForTest();
+    ISchema schema = objectStore.getISchema(new ISchemaName(dbName, "no.such.schema"));
+    Assert.assertNull(schema);
+
+    String schemaName = "schema1";
+    String schemaGroup = "group1";
+    String description = "This is a description";
+    schema = new ISchemaBuilder()
+        .setSchemaType(SchemaType.AVRO)
+        .setName(schemaName)
+        .setDbName(dbName)
+        .setCompatibility(SchemaCompatibility.FORWARD)
+        .setValidationLevel(SchemaValidation.LATEST)
+        .setCanEvolve(false)
+        .setSchemaGroup(schemaGroup)
+        .setDescription(description)
+        .build();
+    objectStore.createISchema(schema);
+
+    schema = objectStore.getISchema(new ISchemaName(dbName, schemaName));
+    Assert.assertNotNull(schema);
+
+    Assert.assertEquals(SchemaType.AVRO, schema.getSchemaType());
+    Assert.assertEquals(schemaName, schema.getName());
+    Assert.assertEquals(SchemaCompatibility.FORWARD, schema.getCompatibility());
+    Assert.assertEquals(SchemaValidation.LATEST, schema.getValidationLevel());
+    Assert.assertFalse(schema.isCanEvolve());
+    Assert.assertEquals(schemaGroup, schema.getSchemaGroup());
+    Assert.assertEquals(description, schema.getDescription());
+
+    schemaGroup = "new group";
+    description = "new description";
+    schema.setCompatibility(SchemaCompatibility.BOTH);
+    schema.setValidationLevel(SchemaValidation.ALL);
+    schema.setCanEvolve(true);
+    schema.setSchemaGroup(schemaGroup);
+    schema.setDescription(description);
+    objectStore.alterISchema(new ISchemaName(dbName, schemaName), schema);
+
+    schema = objectStore.getISchema(new ISchemaName(dbName, schemaName));
+    Assert.assertNotNull(schema);
+
+    Assert.assertEquals(SchemaType.AVRO, schema.getSchemaType());
+    Assert.assertEquals(schemaName, schema.getName());
+    Assert.assertEquals(SchemaCompatibility.BOTH, schema.getCompatibility());
+    Assert.assertEquals(SchemaValidation.ALL, schema.getValidationLevel());
+    Assert.assertTrue(schema.isCanEvolve());
+    Assert.assertEquals(schemaGroup, schema.getSchemaGroup());
+    Assert.assertEquals(description, schema.getDescription());
+
+    objectStore.dropISchema(new ISchemaName(dbName, schemaName));
+    schema = objectStore.getISchema(new ISchemaName(dbName, schemaName));
+    Assert.assertNull(schema);
+  }
+
+  @Test(expected = NoSuchObjectException.class)
+  public void schemaWithInvalidDatabase() throws MetaException, AlreadyExistsException,
+      NoSuchObjectException {
+    ISchema schema = new ISchemaBuilder()
+        .setName("thisSchemaDoesntHaveADb")
+        .setDbName("no.such.database")
+        .setSchemaType(SchemaType.AVRO)
+        .build();
+    objectStore.createISchema(schema);
+  }
+
+  @Test(expected = AlreadyExistsException.class)
+  public void schemaAlreadyExists() throws TException {
+    String dbName = createUniqueDatabaseForTest();
+    String schemaName = "schema2";
+    ISchema schema = new ISchemaBuilder()
+        .setSchemaType(SchemaType.HIVE)
+        .setName(schemaName)
+        .setDbName(dbName)
+        .build();
+    objectStore.createISchema(schema);
+
+    schema = objectStore.getISchema(new ISchemaName(dbName, schemaName));
+    Assert.assertNotNull(schema);
+
+    Assert.assertEquals(SchemaType.HIVE, schema.getSchemaType());
+    Assert.assertEquals(schemaName, schema.getName());
+    Assert.assertEquals(SchemaCompatibility.BACKWARD, schema.getCompatibility());
+    Assert.assertEquals(SchemaValidation.ALL, schema.getValidationLevel());
+    Assert.assertTrue(schema.isCanEvolve());
+
+    // This second attempt to create it should throw
+    objectStore.createISchema(schema);
+  }
+
+  @Test(expected = NoSuchObjectException.class)
+  public void alterNonExistentSchema() throws MetaException, NoSuchObjectException {
+    String schemaName = "noSuchSchema";
+    ISchema schema = new ISchemaBuilder()
+        .setSchemaType(SchemaType.HIVE)
+        .setName(schemaName)
+        .setDescription("a new description")
+        .build();
+    objectStore.alterISchema(new ISchemaName(DEFAULT_DATABASE_NAME, schemaName), schema);
+  }
+
+  @Test(expected = NoSuchObjectException.class)
+  public void dropNonExistentSchema() throws MetaException, NoSuchObjectException {
+    objectStore.dropISchema(new ISchemaName(DEFAULT_DATABASE_NAME, "no_such_schema"));
+  }
+
+  @Test(expected = NoSuchObjectException.class)
+  public void createVersionOfNonExistentSchema() throws MetaException, AlreadyExistsException,
+      NoSuchObjectException, InvalidObjectException {
+    SchemaVersion schemaVersion = new SchemaVersionBuilder()
+        .setSchemaName("noSchemaOfThisNameExists")
+        .setDbName(DEFAULT_DATABASE_NAME)
+        .setVersion(1)
+        .addCol("a", ColumnType.STRING_TYPE_NAME)
+        .build();
+    objectStore.addSchemaVersion(schemaVersion);
+  }
+
+  @Test
+  public void addSchemaVersion() throws TException {
+    String dbName = createUniqueDatabaseForTest();
+    String schemaName = "schema37";
+    int version = 1;
+    SchemaVersion schemaVersion = objectStore.getSchemaVersion(new SchemaVersionDescriptor(new ISchemaName(dbName, schemaName), version));
+    Assert.assertNull(schemaVersion);
+
+    ISchema schema = new ISchemaBuilder()
+        .setSchemaType(SchemaType.AVRO)
+        .setName(schemaName)
+        .setDbName(dbName)
+        .build();
+    objectStore.createISchema(schema);
+
+    String description = "very descriptive";
+    String schemaText = "this should look like json, but oh well";
+    String fingerprint = "this should be an md5 string";
+    String versionName = "why would I name a version?";
+    long creationTime = 10;
+    String serdeName = "serde_for_schema37";
+    String serializer = "org.apache.hadoop.hive.metastore.test.Serializer";
+    String deserializer = "org.apache.hadoop.hive.metastore.test.Deserializer";
+    String serdeDescription = "how do you describe a serde?";
+    schemaVersion = new SchemaVersionBuilder()
+        .versionOf(schema)
+        .setVersion(version)
+        .addCol("a", ColumnType.INT_TYPE_NAME)
+        .addCol("b", ColumnType.FLOAT_TYPE_NAME)
+        .setCreatedAt(creationTime)
+        .setState(SchemaVersionState.INITIATED)
+        .setDescription(description)
+        .setSchemaText(schemaText)
+        .setFingerprint(fingerprint)
+        .setName(versionName)
+        .setSerdeName(serdeName)
+        .setSerdeSerializerClass(serializer)
+        .setSerdeDeserializerClass(deserializer)
+        .setSerdeDescription(serdeDescription)
+        .build();
+    objectStore.addSchemaVersion(schemaVersion);
+
+    schemaVersion = objectStore.getSchemaVersion(new SchemaVersionDescriptor(new ISchemaName(dbName, schemaName), version));
+    Assert.assertNotNull(schemaVersion);
+    Assert.assertEquals(schemaName, schemaVersion.getSchema().getSchemaName());
+    Assert.assertEquals(dbName, schemaVersion.getSchema().getDbName());
+    Assert.assertEquals(version, schemaVersion.getVersion());
+    Assert.assertEquals(creationTime, schemaVersion.getCreatedAt());
+    Assert.assertEquals(SchemaVersionState.INITIATED, schemaVersion.getState());
+    Assert.assertEquals(description, schemaVersion.getDescription());
+    Assert.assertEquals(schemaText, schemaVersion.getSchemaText());
+    Assert.assertEquals(fingerprint, schemaVersion.getFingerprint());
+    Assert.assertEquals(versionName, schemaVersion.getName());
+    Assert.assertEquals(serdeName, schemaVersion.getSerDe().getName());
+    Assert.assertEquals(serializer, schemaVersion.getSerDe().getSerializerClass());
+    Assert.assertEquals(deserializer, schemaVersion.getSerDe().getDeserializerClass());
+    Assert.assertEquals(serdeDescription, schemaVersion.getSerDe().getDescription());
+    Assert.assertEquals(2, schemaVersion.getColsSize());
+    List<FieldSchema> cols = schemaVersion.getCols();
+    Collections.sort(cols);
+    Assert.assertEquals("a", cols.get(0).getName());
+    Assert.assertEquals(ColumnType.INT_TYPE_NAME, cols.get(0).getType());
+    Assert.assertEquals("b", cols.get(1).getName());
+    Assert.assertEquals(ColumnType.FLOAT_TYPE_NAME, cols.get(1).getType());
+
+    objectStore.dropSchemaVersion(new SchemaVersionDescriptor(new ISchemaName(dbName, schemaName), version));
+    schemaVersion = objectStore.getSchemaVersion(new SchemaVersionDescriptor(new ISchemaName(dbName, schemaName), version));
+    Assert.assertNull(schemaVersion);
+  }
+
+  // Test that adding multiple versions of the same schema
+  @Test
+  public void multipleSchemaVersions() throws TException {
+    String dbName = createUniqueDatabaseForTest();
+    String schemaName = "schema195";
+
+    ISchema schema = new ISchemaBuilder()
+        .setSchemaType(SchemaType.AVRO)
+        .setName(schemaName)
+        .setDbName(dbName)
+        .build();
+    objectStore.createISchema(schema);
+    SchemaVersion schemaVersion = new SchemaVersionBuilder()
+        .versionOf(schema)
+        .setVersion(1)
+        .addCol("a", ColumnType.BIGINT_TYPE_NAME)
+        .build();
+    objectStore.addSchemaVersion(schemaVersion);
+
+    schemaVersion = new SchemaVersionBuilder()
+        .versionOf(schema)
+        .setVersion(2)
+        .addCol("a", ColumnType.BIGINT_TYPE_NAME)
+        .addCol("b", ColumnType.DATE_TYPE_NAME)
+        .build();
+    objectStore.addSchemaVersion(schemaVersion);
+
+    schemaVersion = new SchemaVersionBuilder()
+        .versionOf(schema)
+        .setVersion(3)
+        .addCol("a", ColumnType.BIGINT_TYPE_NAME)
+        .addCol("b", ColumnType.DATE_TYPE_NAME)
+        .addCol("c", ColumnType.TIMESTAMP_TYPE_NAME)
+        .build();
+    objectStore.addSchemaVersion(schemaVersion);
+
+    schemaVersion = objectStore.getLatestSchemaVersion(new ISchemaName(dbName, schemaName));
+    Assert.assertEquals(3, schemaVersion.getVersion());
+    Assert.assertEquals(3, schemaVersion.getColsSize());
+    List<FieldSchema> cols = schemaVersion.getCols();
+    Collections.sort(cols);
+    Assert.assertEquals("a", cols.get(0).getName());
+    Assert.assertEquals("b", cols.get(1).getName());
+    Assert.assertEquals("c", cols.get(2).getName());
+    Assert.assertEquals(ColumnType.BIGINT_TYPE_NAME, cols.get(0).getType());
+    Assert.assertEquals(ColumnType.DATE_TYPE_NAME, cols.get(1).getType());
+    Assert.assertEquals(ColumnType.TIMESTAMP_TYPE_NAME, cols.get(2).getType());
+
+    schemaVersion = objectStore.getLatestSchemaVersion(new ISchemaName(dbName, "no.such.schema.with.this.name"));
+    Assert.assertNull(schemaVersion);
+
+    List<SchemaVersion> versions =
+        objectStore.getAllSchemaVersion(new ISchemaName(dbName, "there.really.isnt.a.schema.named.this"));
+    Assert.assertNull(versions);
+
+    versions = objectStore.getAllSchemaVersion(new ISchemaName(dbName, schemaName));
+    Assert.assertEquals(3, versions.size());
+    versions.sort(Comparator.comparingInt(SchemaVersion::getVersion));
+    Assert.assertEquals(1, versions.get(0).getVersion());
+    Assert.assertEquals(1, versions.get(0).getColsSize());
+    Assert.assertEquals(ColumnType.BIGINT_TYPE_NAME, versions.get(0).getCols().get(0).getType());
+
+    Assert.assertEquals(2, versions.get(1).getVersion());
+    Assert.assertEquals(2, versions.get(1).getColsSize());
+    cols = versions.get(1).getCols();
+    Collections.sort(cols);
+    Assert.assertEquals("a", cols.get(0).getName());
+    Assert.assertEquals("b", cols.get(1).getName());
+    Assert.assertEquals(ColumnType.BIGINT_TYPE_NAME, cols.get(0).getType());
+    Assert.assertEquals(ColumnType.DATE_TYPE_NAME, cols.get(1).getType());
+
+    Assert.assertEquals(3, versions.get(2).getVersion());
+    Assert.assertEquals(3, versions.get(2).getColsSize());
+    cols = versions.get(2).getCols();
+    Collections.sort(cols);
+    Assert.assertEquals("a", cols.get(0).getName());
+    Assert.assertEquals("b", cols.get(1).getName());
+    Assert.assertEquals("c", cols.get(2).getName());
+    Assert.assertEquals(ColumnType.BIGINT_TYPE_NAME, cols.get(0).getType());
+    Assert.assertEquals(ColumnType.DATE_TYPE_NAME, cols.get(1).getType());
+    Assert.assertEquals(ColumnType.TIMESTAMP_TYPE_NAME, cols.get(2).getType());
+  }
+
+  @Test(expected = AlreadyExistsException.class)
+  public void addDuplicateSchemaVersion() throws TException {
+    String dbName = createUniqueDatabaseForTest();
+    String schemaName = "schema1234";
+    int version = 1;
+    SchemaVersion schemaVersion = objectStore.getSchemaVersion(new SchemaVersionDescriptor(new ISchemaName(dbName, schemaName), version));
+    Assert.assertNull(schemaVersion);
+
+    ISchema schema = new ISchemaBuilder()
+        .setSchemaType(SchemaType.AVRO)
+        .setName(schemaName)
+        .setDbName(dbName)
+        .build();
+    objectStore.createISchema(schema);
+
+    schemaVersion = new SchemaVersionBuilder()
+        .versionOf(schema)
+        .setVersion(version)
+        .addCol("a", ColumnType.INT_TYPE_NAME)
+        .addCol("b", ColumnType.FLOAT_TYPE_NAME)
+        .build();
+    objectStore.addSchemaVersion(schemaVersion);
+
+    objectStore.addSchemaVersion(schemaVersion);
+  }
+
+  @Test
+  public void alterSchemaVersion() throws TException {
+    String dbName = createUniqueDatabaseForTest();
+    String schemaName = "schema371234";
+    int version = 1;
+    SchemaVersion schemaVersion = objectStore.getSchemaVersion(new SchemaVersionDescriptor(new ISchemaName(dbName, schemaName), version));
+    Assert.assertNull(schemaVersion);
+
+    ISchema schema = new ISchemaBuilder()
+        .setSchemaType(SchemaType.AVRO)
+        .setName(schemaName)
+        .setDbName(dbName)
+        .build();
+    objectStore.createISchema(schema);
+
+    schemaVersion = new SchemaVersionBuilder()
+        .versionOf(schema)
+        .setVersion(version)
+        .addCol("a", ColumnType.INT_TYPE_NAME)
+        .addCol("b", ColumnType.FLOAT_TYPE_NAME)
+        .setState(SchemaVersionState.INITIATED)
+        .build();
+    objectStore.addSchemaVersion(schemaVersion);
+
+    schemaVersion = objectStore.getSchemaVersion(new SchemaVersionDescriptor(new ISchemaName(dbName, schemaName), version));
+    Assert.assertNotNull(schemaVersion);
+    Assert.assertEquals(schemaName, schemaVersion.getSchema().getSchemaName());
+    Assert.assertEquals(dbName, schemaVersion.getSchema().getDbName());
+    Assert.assertEquals(version, schemaVersion.getVersion());
+    Assert.assertEquals(SchemaVersionState.INITIATED, schemaVersion.getState());
+
+    schemaVersion.setState(SchemaVersionState.REVIEWED);
+    String serdeName = "serde for " + schemaName;
+    SerDeInfo serde = new SerDeInfo(serdeName, "", Collections.emptyMap());
+    String serializer = "org.apache.hadoop.hive.metastore.test.Serializer";
+    String deserializer = "org.apache.hadoop.hive.metastore.test.Deserializer";
+    serde.setSerializerClass(serializer);
+    serde.setDeserializerClass(deserializer);
+    schemaVersion.setSerDe(serde);
+    objectStore.alterSchemaVersion(new SchemaVersionDescriptor(new ISchemaName(dbName, schemaName), version), schemaVersion);
+
+    schemaVersion = objectStore.getSchemaVersion(new SchemaVersionDescriptor(new ISchemaName(dbName, schemaName), version));
+    Assert.assertNotNull(schemaVersion);
+    Assert.assertEquals(schemaName, schemaVersion.getSchema().getSchemaName());
+    Assert.assertEquals(dbName, schemaVersion.getSchema().getDbName());
+    Assert.assertEquals(version, schemaVersion.getVersion());
+    Assert.assertEquals(SchemaVersionState.REVIEWED, schemaVersion.getState());
+    Assert.assertEquals(serdeName, schemaVersion.getSerDe().getName());
+    Assert.assertEquals(serializer, schemaVersion.getSerDe().getSerializerClass());
+    Assert.assertEquals(deserializer, schemaVersion.getSerDe().getDeserializerClass());
+  }
+
+  @Test(expected = NoSuchObjectException.class)
+  public void alterNonExistentSchemaVersion() throws MetaException, AlreadyExistsException,
+      NoSuchObjectException {
+    String schemaName = "schema3723asdflj";
+    int version = 37;
+    SchemaVersion schemaVersion = new SchemaVersionBuilder()
+        .setSchemaName(schemaName)
+        .setDbName(DEFAULT_DATABASE_NAME)
+        .setVersion(version)
+        .addCol("a", ColumnType.INT_TYPE_NAME)
+        .addCol("b", ColumnType.FLOAT_TYPE_NAME)
+        .setState(SchemaVersionState.INITIATED)
+        .build();
+    objectStore.alterSchemaVersion(new SchemaVersionDescriptor(new ISchemaName(DEFAULT_DATABASE_NAME, schemaName), version), schemaVersion);
+  }
+
+  @Test(expected = NoSuchObjectException.class)
+  public void dropNonExistentSchemaVersion() throws NoSuchObjectException, MetaException {
+    objectStore.dropSchemaVersion(new SchemaVersionDescriptor(new ISchemaName(DEFAULT_DATABASE_NAME, "ther is no schema named this"), 23));
+  }
+
+  @Test
+  public void schemaQuery() throws TException {
+    String dbName = createUniqueDatabaseForTest();
+    String schemaName1 = "a_schema1";
+    ISchema schema1 = new ISchemaBuilder()
+        .setSchemaType(SchemaType.AVRO)
+        .setName(schemaName1)
+        .setDbName(dbName)
+        .build();
+    objectStore.createISchema(schema1);
+
+    String schemaName2 = "a_schema2";
+    ISchema schema2 = new ISchemaBuilder()
+        .setSchemaType(SchemaType.AVRO)
+        .setName(schemaName2)
+        .setDbName(dbName)
+        .build();
+    objectStore.createISchema(schema2);
+
+    SchemaVersion schemaVersion1_1 = new SchemaVersionBuilder()
+        .versionOf(schema1)
+        .setVersion(1)
+        .addCol("alpha", ColumnType.BIGINT_TYPE_NAME)
+        .addCol("beta", ColumnType.DATE_TYPE_NAME)
+        .build();
+    objectStore.addSchemaVersion(schemaVersion1_1);
+
+    SchemaVersion schemaVersion1_2 = new SchemaVersionBuilder()
+        .versionOf(schema1)
+        .setVersion(2)
+        .addCol("alpha", ColumnType.BIGINT_TYPE_NAME)
+        .addCol("beta", ColumnType.DATE_TYPE_NAME)
+        .addCol("gamma", ColumnType.BIGINT_TYPE_NAME, "namespace=x")
+        .build();
+    objectStore.addSchemaVersion(schemaVersion1_2);
+
+    SchemaVersion schemaVersion2_1 = new SchemaVersionBuilder()
+        .versionOf(schema2)
+        .setVersion(1)
+        .addCol("ALPHA", ColumnType.SMALLINT_TYPE_NAME)
+        .addCol("delta", ColumnType.DOUBLE_TYPE_NAME)
+        .build();
+    objectStore.addSchemaVersion(schemaVersion2_1);
+
+    SchemaVersion schemaVersion2_2 = new SchemaVersionBuilder()
+        .versionOf(schema2)
+        .setVersion(2)
+        .addCol("ALPHA", ColumnType.SMALLINT_TYPE_NAME)
+        .addCol("delta", ColumnType.DOUBLE_TYPE_NAME)
+        .addCol("epsilon", ColumnType.STRING_TYPE_NAME, "namespace=x")
+        .build();
+    objectStore.addSchemaVersion(schemaVersion2_2);
+
+    // Query that should return nothing
+    List<SchemaVersion> results = objectStore.getSchemaVersionsByColumns("x", "y", "z");
+    Assert.assertEquals(0, results.size());
+
+    // Query that should fetch one column
+    results = objectStore.getSchemaVersionsByColumns("gamma", null, null);
+    Assert.assertEquals(1, results.size());
+    Assert.assertEquals(schemaName1, results.get(0).getSchema().getSchemaName());
+    Assert.assertEquals(dbName, results.get(0).getSchema().getDbName());
+    Assert.assertEquals(2, results.get(0).getVersion());
+
+    // fetch 2 in same schema
+    results = objectStore.getSchemaVersionsByColumns("beta", null, null);
+    Assert.assertEquals(2, results.size());
+    Collections.sort(results);
+    Assert.assertEquals(schemaName1, results.get(0).getSchema().getSchemaName());
+    Assert.assertEquals(dbName, results.get(0).getSchema().getDbName());
+    Assert.assertEquals(1, results.get(0).getVersion());
+    Assert.assertEquals(schemaName1, results.get(1).getSchema().getSchemaName());
+    Assert.assertEquals(dbName, results.get(1).getSchema().getDbName());
+    Assert.assertEquals(2, results.get(1).getVersion());
+
+    // fetch across schemas
+    results = objectStore.getSchemaVersionsByColumns("alpha", null, null);
+    Assert.assertEquals(4, results.size());
+    Collections.sort(results);
+    Assert.assertEquals(schemaName1, results.get(0).getSchema().getSchemaName());
+    Assert.assertEquals(dbName, results.get(0).getSchema().getDbName());
+    Assert.assertEquals(1, results.get(0).getVersion());
+    Assert.assertEquals(schemaName1, results.get(1).getSchema().getSchemaName());
+    Assert.assertEquals(dbName, results.get(1).getSchema().getDbName());
+    Assert.assertEquals(2, results.get(1).getVersion());
+    Assert.assertEquals(schemaName2, results.get(2).getSchema().getSchemaName());
+    Assert.assertEquals(dbName, results.get(2).getSchema().getDbName());
+    Assert.assertEquals(1, results.get(2).getVersion());
+    Assert.assertEquals(schemaName2, results.get(3).getSchema().getSchemaName());
+    Assert.assertEquals(dbName, results.get(3).getSchema().getDbName());
+    Assert.assertEquals(2, results.get(3).getVersion());
+
+    // fetch by namespace
+    results = objectStore.getSchemaVersionsByColumns(null, "namespace=x", null);
+    Assert.assertEquals(2, results.size());
+    Collections.sort(results);
+    Assert.assertEquals(schemaName1, results.get(0).getSchema().getSchemaName());
+    Assert.assertEquals(dbName, results.get(0).getSchema().getDbName());
+    Assert.assertEquals(2, results.get(0).getVersion());
+    Assert.assertEquals(schemaName2, results.get(1).getSchema().getSchemaName());
+    Assert.assertEquals(dbName, results.get(1).getSchema().getDbName());
+    Assert.assertEquals(2, results.get(1).getVersion());
+
+    // fetch by name and type
+    results = objectStore.getSchemaVersionsByColumns("alpha", null, ColumnType.SMALLINT_TYPE_NAME);
+    Assert.assertEquals(2, results.size());
+    Collections.sort(results);
+    Assert.assertEquals(schemaName2, results.get(0).getSchema().getSchemaName());
+    Assert.assertEquals(dbName, results.get(0).getSchema().getDbName());
+    Assert.assertEquals(1, results.get(0).getVersion());
+    Assert.assertEquals(schemaName2, results.get(1).getSchema().getSchemaName());
+    Assert.assertEquals(dbName, results.get(1).getSchema().getDbName());
+    Assert.assertEquals(2, results.get(1).getVersion());
+
+    // Make sure matching name but wrong type doesn't return
+    results = objectStore.getSchemaVersionsByColumns("alpha", null, ColumnType.STRING_TYPE_NAME); Assert.assertEquals(0, results.size());
+  }
+
+  @Test(expected = MetaException.class)
+  public void schemaVersionQueryNoNameOrNamespace() throws MetaException {
+    objectStore.getSchemaVersionsByColumns(null, null, ColumnType.STRING_TYPE_NAME);
+  }
+
+  private static int dbNum = 1;
+  private String createUniqueDatabaseForTest() throws MetaException, InvalidObjectException {
+    String dbName = "uniquedbfortest" + dbNum++;
+    Database db = new DatabaseBuilder()
+        .setName(dbName)
+        .setLocation("somewhere")
+        .setDescription("descriptive")
+        .build();
+    objectStore.createDatabase(db);
+    return dbName;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/cache/TestCachedStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/cache/TestCachedStore.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/cache/TestCachedStore.java
index ab6feb6..0006815 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/cache/TestCachedStore.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/cache/TestCachedStore.java
@@ -28,10 +28,8 @@ import org.apache.hadoop.hive.common.ndv.hll.HyperLogLog;
 import org.apache.hadoop.hive.metastore.MetaStoreTestUtils;
 import org.apache.hadoop.hive.metastore.ObjectStore;
 import org.apache.hadoop.hive.metastore.TableType;
-import org.apache.hadoop.hive.metastore.TestObjectStore.MockPartitionExpressionProxy;
 import org.apache.hadoop.hive.metastore.annotation.MetastoreCheckinTest;
 import org.apache.hadoop.hive.metastore.api.AggrStats;
-import org.apache.hadoop.hive.metastore.api.BasicTxnInfo;
 import org.apache.hadoop.hive.metastore.api.BooleanColumnStatsData;
 import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData;


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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
index 8f3b848..8c5ceaf 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
@@ -402,6 +402,34 @@ import org.slf4j.LoggerFactory;
 
     public WMCreateOrDropTriggerToPoolMappingResponse create_or_drop_wm_trigger_to_pool_mapping(WMCreateOrDropTriggerToPoolMappingRequest request) throws AlreadyExistsException, NoSuchObjectException, InvalidObjectException, MetaException, org.apache.thrift.TException;
 
+    public void create_ischema(ISchema schema) throws AlreadyExistsException, NoSuchObjectException, MetaException, org.apache.thrift.TException;
+
+    public void alter_ischema(AlterISchemaRequest rqst) throws NoSuchObjectException, MetaException, org.apache.thrift.TException;
+
+    public ISchema get_ischema(ISchemaName name) throws NoSuchObjectException, MetaException, org.apache.thrift.TException;
+
+    public void drop_ischema(ISchemaName name) throws NoSuchObjectException, InvalidOperationException, MetaException, org.apache.thrift.TException;
+
+    public void add_schema_version(SchemaVersion schemaVersion) throws AlreadyExistsException, NoSuchObjectException, MetaException, org.apache.thrift.TException;
+
+    public SchemaVersion get_schema_version(SchemaVersionDescriptor schemaVersion) throws NoSuchObjectException, MetaException, org.apache.thrift.TException;
+
+    public SchemaVersion get_schema_latest_version(ISchemaName schemaName) throws NoSuchObjectException, MetaException, org.apache.thrift.TException;
+
+    public List<SchemaVersion> get_schema_all_versions(ISchemaName schemaName) throws NoSuchObjectException, MetaException, org.apache.thrift.TException;
+
+    public void drop_schema_version(SchemaVersionDescriptor schemaVersion) throws NoSuchObjectException, MetaException, org.apache.thrift.TException;
+
+    public FindSchemasByColsResp get_schemas_by_cols(FindSchemasByColsRqst rqst) throws MetaException, org.apache.thrift.TException;
+
+    public void map_schema_version_to_serde(MapSchemaVersionToSerdeRequest rqst) throws NoSuchObjectException, MetaException, org.apache.thrift.TException;
+
+    public void set_schema_version_state(SetSchemaVersionStateRequest rqst) throws NoSuchObjectException, InvalidOperationException, MetaException, org.apache.thrift.TException;
+
+    public void add_serde(SerDeInfo serde) throws AlreadyExistsException, MetaException, org.apache.thrift.TException;
+
+    public SerDeInfo get_serde(GetSerdeRequest rqst) throws NoSuchObjectException, MetaException, org.apache.thrift.TException;
+
   }
 
   @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public interface AsyncIface extends com.facebook.fb303.FacebookService .AsyncIface {
@@ -766,6 +794,34 @@ import org.slf4j.LoggerFactory;
 
     public void create_or_drop_wm_trigger_to_pool_mapping(WMCreateOrDropTriggerToPoolMappingRequest request, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
+    public void create_ischema(ISchema schema, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void alter_ischema(AlterISchemaRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void get_ischema(ISchemaName name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void drop_ischema(ISchemaName name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void add_schema_version(SchemaVersion schemaVersion, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void get_schema_version(SchemaVersionDescriptor schemaVersion, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void get_schema_latest_version(ISchemaName schemaName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void get_schema_all_versions(ISchemaName schemaName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void drop_schema_version(SchemaVersionDescriptor schemaVersion, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void get_schemas_by_cols(FindSchemasByColsRqst rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void map_schema_version_to_serde(MapSchemaVersionToSerdeRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void set_schema_version_state(SetSchemaVersionStateRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void add_serde(SerDeInfo serde, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void get_serde(GetSerdeRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
   }
 
   @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class Client extends com.facebook.fb303.FacebookService.Client implements Iface {
@@ -5994,6 +6050,397 @@ import org.slf4j.LoggerFactory;
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "create_or_drop_wm_trigger_to_pool_mapping failed: unknown result");
     }
 
+    public void create_ischema(ISchema schema) throws AlreadyExistsException, NoSuchObjectException, MetaException, org.apache.thrift.TException
+    {
+      send_create_ischema(schema);
+      recv_create_ischema();
+    }
+
+    public void send_create_ischema(ISchema schema) throws org.apache.thrift.TException
+    {
+      create_ischema_args args = new create_ischema_args();
+      args.setSchema(schema);
+      sendBase("create_ischema", args);
+    }
+
+    public void recv_create_ischema() throws AlreadyExistsException, NoSuchObjectException, MetaException, org.apache.thrift.TException
+    {
+      create_ischema_result result = new create_ischema_result();
+      receiveBase(result, "create_ischema");
+      if (result.o1 != null) {
+        throw result.o1;
+      }
+      if (result.o2 != null) {
+        throw result.o2;
+      }
+      if (result.o3 != null) {
+        throw result.o3;
+      }
+      return;
+    }
+
+    public void alter_ischema(AlterISchemaRequest rqst) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
+    {
+      send_alter_ischema(rqst);
+      recv_alter_ischema();
+    }
+
+    public void send_alter_ischema(AlterISchemaRequest rqst) throws org.apache.thrift.TException
+    {
+      alter_ischema_args args = new alter_ischema_args();
+      args.setRqst(rqst);
+      sendBase("alter_ischema", args);
+    }
+
+    public void recv_alter_ischema() throws NoSuchObjectException, MetaException, org.apache.thrift.TException
+    {
+      alter_ischema_result result = new alter_ischema_result();
+      receiveBase(result, "alter_ischema");
+      if (result.o1 != null) {
+        throw result.o1;
+      }
+      if (result.o2 != null) {
+        throw result.o2;
+      }
+      return;
+    }
+
+    public ISchema get_ischema(ISchemaName name) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
+    {
+      send_get_ischema(name);
+      return recv_get_ischema();
+    }
+
+    public void send_get_ischema(ISchemaName name) throws org.apache.thrift.TException
+    {
+      get_ischema_args args = new get_ischema_args();
+      args.setName(name);
+      sendBase("get_ischema", args);
+    }
+
+    public ISchema recv_get_ischema() throws NoSuchObjectException, MetaException, org.apache.thrift.TException
+    {
+      get_ischema_result result = new get_ischema_result();
+      receiveBase(result, "get_ischema");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.o1 != null) {
+        throw result.o1;
+      }
+      if (result.o2 != null) {
+        throw result.o2;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "get_ischema failed: unknown result");
+    }
+
+    public void drop_ischema(ISchemaName name) throws NoSuchObjectException, InvalidOperationException, MetaException, org.apache.thrift.TException
+    {
+      send_drop_ischema(name);
+      recv_drop_ischema();
+    }
+
+    public void send_drop_ischema(ISchemaName name) throws org.apache.thrift.TException
+    {
+      drop_ischema_args args = new drop_ischema_args();
+      args.setName(name);
+      sendBase("drop_ischema", args);
+    }
+
+    public void recv_drop_ischema() throws NoSuchObjectException, InvalidOperationException, MetaException, org.apache.thrift.TException
+    {
+      drop_ischema_result result = new drop_ischema_result();
+      receiveBase(result, "drop_ischema");
+      if (result.o1 != null) {
+        throw result.o1;
+      }
+      if (result.o2 != null) {
+        throw result.o2;
+      }
+      if (result.o3 != null) {
+        throw result.o3;
+      }
+      return;
+    }
+
+    public void add_schema_version(SchemaVersion schemaVersion) throws AlreadyExistsException, NoSuchObjectException, MetaException, org.apache.thrift.TException
+    {
+      send_add_schema_version(schemaVersion);
+      recv_add_schema_version();
+    }
+
+    public void send_add_schema_version(SchemaVersion schemaVersion) throws org.apache.thrift.TException
+    {
+      add_schema_version_args args = new add_schema_version_args();
+      args.setSchemaVersion(schemaVersion);
+      sendBase("add_schema_version", args);
+    }
+
+    public void recv_add_schema_version() throws AlreadyExistsException, NoSuchObjectException, MetaException, org.apache.thrift.TException
+    {
+      add_schema_version_result result = new add_schema_version_result();
+      receiveBase(result, "add_schema_version");
+      if (result.o1 != null) {
+        throw result.o1;
+      }
+      if (result.o2 != null) {
+        throw result.o2;
+      }
+      if (result.o3 != null) {
+        throw result.o3;
+      }
+      return;
+    }
+
+    public SchemaVersion get_schema_version(SchemaVersionDescriptor schemaVersion) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
+    {
+      send_get_schema_version(schemaVersion);
+      return recv_get_schema_version();
+    }
+
+    public void send_get_schema_version(SchemaVersionDescriptor schemaVersion) throws org.apache.thrift.TException
+    {
+      get_schema_version_args args = new get_schema_version_args();
+      args.setSchemaVersion(schemaVersion);
+      sendBase("get_schema_version", args);
+    }
+
+    public SchemaVersion recv_get_schema_version() throws NoSuchObjectException, MetaException, org.apache.thrift.TException
+    {
+      get_schema_version_result result = new get_schema_version_result();
+      receiveBase(result, "get_schema_version");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.o1 != null) {
+        throw result.o1;
+      }
+      if (result.o2 != null) {
+        throw result.o2;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "get_schema_version failed: unknown result");
+    }
+
+    public SchemaVersion get_schema_latest_version(ISchemaName schemaName) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
+    {
+      send_get_schema_latest_version(schemaName);
+      return recv_get_schema_latest_version();
+    }
+
+    public void send_get_schema_latest_version(ISchemaName schemaName) throws org.apache.thrift.TException
+    {
+      get_schema_latest_version_args args = new get_schema_latest_version_args();
+      args.setSchemaName(schemaName);
+      sendBase("get_schema_latest_version", args);
+    }
+
+    public SchemaVersion recv_get_schema_latest_version() throws NoSuchObjectException, MetaException, org.apache.thrift.TException
+    {
+      get_schema_latest_version_result result = new get_schema_latest_version_result();
+      receiveBase(result, "get_schema_latest_version");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.o1 != null) {
+        throw result.o1;
+      }
+      if (result.o2 != null) {
+        throw result.o2;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "get_schema_latest_version failed: unknown result");
+    }
+
+    public List<SchemaVersion> get_schema_all_versions(ISchemaName schemaName) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
+    {
+      send_get_schema_all_versions(schemaName);
+      return recv_get_schema_all_versions();
+    }
+
+    public void send_get_schema_all_versions(ISchemaName schemaName) throws org.apache.thrift.TException
+    {
+      get_schema_all_versions_args args = new get_schema_all_versions_args();
+      args.setSchemaName(schemaName);
+      sendBase("get_schema_all_versions", args);
+    }
+
+    public List<SchemaVersion> recv_get_schema_all_versions() throws NoSuchObjectException, MetaException, org.apache.thrift.TException
+    {
+      get_schema_all_versions_result result = new get_schema_all_versions_result();
+      receiveBase(result, "get_schema_all_versions");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.o1 != null) {
+        throw result.o1;
+      }
+      if (result.o2 != null) {
+        throw result.o2;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "get_schema_all_versions failed: unknown result");
+    }
+
+    public void drop_schema_version(SchemaVersionDescriptor schemaVersion) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
+    {
+      send_drop_schema_version(schemaVersion);
+      recv_drop_schema_version();
+    }
+
+    public void send_drop_schema_version(SchemaVersionDescriptor schemaVersion) throws org.apache.thrift.TException
+    {
+      drop_schema_version_args args = new drop_schema_version_args();
+      args.setSchemaVersion(schemaVersion);
+      sendBase("drop_schema_version", args);
+    }
+
+    public void recv_drop_schema_version() throws NoSuchObjectException, MetaException, org.apache.thrift.TException
+    {
+      drop_schema_version_result result = new drop_schema_version_result();
+      receiveBase(result, "drop_schema_version");
+      if (result.o1 != null) {
+        throw result.o1;
+      }
+      if (result.o2 != null) {
+        throw result.o2;
+      }
+      return;
+    }
+
+    public FindSchemasByColsResp get_schemas_by_cols(FindSchemasByColsRqst rqst) throws MetaException, org.apache.thrift.TException
+    {
+      send_get_schemas_by_cols(rqst);
+      return recv_get_schemas_by_cols();
+    }
+
+    public void send_get_schemas_by_cols(FindSchemasByColsRqst rqst) throws org.apache.thrift.TException
+    {
+      get_schemas_by_cols_args args = new get_schemas_by_cols_args();
+      args.setRqst(rqst);
+      sendBase("get_schemas_by_cols", args);
+    }
+
+    public FindSchemasByColsResp recv_get_schemas_by_cols() throws MetaException, org.apache.thrift.TException
+    {
+      get_schemas_by_cols_result result = new get_schemas_by_cols_result();
+      receiveBase(result, "get_schemas_by_cols");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.o1 != null) {
+        throw result.o1;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "get_schemas_by_cols failed: unknown result");
+    }
+
+    public void map_schema_version_to_serde(MapSchemaVersionToSerdeRequest rqst) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
+    {
+      send_map_schema_version_to_serde(rqst);
+      recv_map_schema_version_to_serde();
+    }
+
+    public void send_map_schema_version_to_serde(MapSchemaVersionToSerdeRequest rqst) throws org.apache.thrift.TException
+    {
+      map_schema_version_to_serde_args args = new map_schema_version_to_serde_args();
+      args.setRqst(rqst);
+      sendBase("map_schema_version_to_serde", args);
+    }
+
+    public void recv_map_schema_version_to_serde() throws NoSuchObjectException, MetaException, org.apache.thrift.TException
+    {
+      map_schema_version_to_serde_result result = new map_schema_version_to_serde_result();
+      receiveBase(result, "map_schema_version_to_serde");
+      if (result.o1 != null) {
+        throw result.o1;
+      }
+      if (result.o2 != null) {
+        throw result.o2;
+      }
+      return;
+    }
+
+    public void set_schema_version_state(SetSchemaVersionStateRequest rqst) throws NoSuchObjectException, InvalidOperationException, MetaException, org.apache.thrift.TException
+    {
+      send_set_schema_version_state(rqst);
+      recv_set_schema_version_state();
+    }
+
+    public void send_set_schema_version_state(SetSchemaVersionStateRequest rqst) throws org.apache.thrift.TException
+    {
+      set_schema_version_state_args args = new set_schema_version_state_args();
+      args.setRqst(rqst);
+      sendBase("set_schema_version_state", args);
+    }
+
+    public void recv_set_schema_version_state() throws NoSuchObjectException, InvalidOperationException, MetaException, org.apache.thrift.TException
+    {
+      set_schema_version_state_result result = new set_schema_version_state_result();
+      receiveBase(result, "set_schema_version_state");
+      if (result.o1 != null) {
+        throw result.o1;
+      }
+      if (result.o2 != null) {
+        throw result.o2;
+      }
+      if (result.o3 != null) {
+        throw result.o3;
+      }
+      return;
+    }
+
+    public void add_serde(SerDeInfo serde) throws AlreadyExistsException, MetaException, org.apache.thrift.TException
+    {
+      send_add_serde(serde);
+      recv_add_serde();
+    }
+
+    public void send_add_serde(SerDeInfo serde) throws org.apache.thrift.TException
+    {
+      add_serde_args args = new add_serde_args();
+      args.setSerde(serde);
+      sendBase("add_serde", args);
+    }
+
+    public void recv_add_serde() throws AlreadyExistsException, MetaException, org.apache.thrift.TException
+    {
+      add_serde_result result = new add_serde_result();
+      receiveBase(result, "add_serde");
+      if (result.o1 != null) {
+        throw result.o1;
+      }
+      if (result.o2 != null) {
+        throw result.o2;
+      }
+      return;
+    }
+
+    public SerDeInfo get_serde(GetSerdeRequest rqst) throws NoSuchObjectException, MetaException, org.apache.thrift.TException
+    {
+      send_get_serde(rqst);
+      return recv_get_serde();
+    }
+
+    public void send_get_serde(GetSerdeRequest rqst) throws org.apache.thrift.TException
+    {
+      get_serde_args args = new get_serde_args();
+      args.setRqst(rqst);
+      sendBase("get_serde", args);
+    }
+
+    public SerDeInfo recv_get_serde() throws NoSuchObjectException, MetaException, org.apache.thrift.TException
+    {
+      get_serde_result result = new get_serde_result();
+      receiveBase(result, "get_serde");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.o1 != null) {
+        throw result.o1;
+      }
+      if (result.o2 != null) {
+        throw result.o2;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "get_serde failed: unknown result");
+    }
+
   }
   @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class AsyncClient extends com.facebook.fb303.FacebookService.AsyncClient implements AsyncIface {
     @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class Factory implements org.apache.thrift.async.TAsyncClientFactory<AsyncClient> {
@@ -12243,6 +12690,454 @@ import org.slf4j.LoggerFactory;
       }
     }
 
+    public void create_ischema(ISchema schema, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      create_ischema_call method_call = new create_ischema_call(schema, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class create_ischema_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private ISchema schema;
+      public create_ischema_call(ISchema schema, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.schema = schema;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("create_ischema", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        create_ischema_args args = new create_ischema_args();
+        args.setSchema(schema);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public void getResult() throws AlreadyExistsException, NoSuchObjectException, MetaException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        (new Client(prot)).recv_create_ischema();
+      }
+    }
+
+    public void alter_ischema(AlterISchemaRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      alter_ischema_call method_call = new alter_ischema_call(rqst, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class alter_ischema_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private AlterISchemaRequest rqst;
+      public alter_ischema_call(AlterISchemaRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.rqst = rqst;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("alter_ischema", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        alter_ischema_args args = new alter_ischema_args();
+        args.setRqst(rqst);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public void getResult() throws NoSuchObjectException, MetaException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        (new Client(prot)).recv_alter_ischema();
+      }
+    }
+
+    public void get_ischema(ISchemaName name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      get_ischema_call method_call = new get_ischema_call(name, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_ischema_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private ISchemaName name;
+      public get_ischema_call(ISchemaName name, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.name = name;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("get_ischema", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        get_ischema_args args = new get_ischema_args();
+        args.setName(name);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public ISchema getResult() throws NoSuchObjectException, MetaException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_get_ischema();
+      }
+    }
+
+    public void drop_ischema(ISchemaName name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      drop_ischema_call method_call = new drop_ischema_call(name, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class drop_ischema_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private ISchemaName name;
+      public drop_ischema_call(ISchemaName name, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.name = name;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("drop_ischema", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        drop_ischema_args args = new drop_ischema_args();
+        args.setName(name);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public void getResult() throws NoSuchObjectException, InvalidOperationException, MetaException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        (new Client(prot)).recv_drop_ischema();
+      }
+    }
+
+    public void add_schema_version(SchemaVersion schemaVersion, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      add_schema_version_call method_call = new add_schema_version_call(schemaVersion, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class add_schema_version_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private SchemaVersion schemaVersion;
+      public add_schema_version_call(SchemaVersion schemaVersion, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.schemaVersion = schemaVersion;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("add_schema_version", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        add_schema_version_args args = new add_schema_version_args();
+        args.setSchemaVersion(schemaVersion);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public void getResult() throws AlreadyExistsException, NoSuchObjectException, MetaException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        (new Client(prot)).recv_add_schema_version();
+      }
+    }
+
+    public void get_schema_version(SchemaVersionDescriptor schemaVersion, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      get_schema_version_call method_call = new get_schema_version_call(schemaVersion, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_schema_version_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private SchemaVersionDescriptor schemaVersion;
+      public get_schema_version_call(SchemaVersionDescriptor schemaVersion, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.schemaVersion = schemaVersion;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("get_schema_version", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        get_schema_version_args args = new get_schema_version_args();
+        args.setSchemaVersion(schemaVersion);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public SchemaVersion getResult() throws NoSuchObjectException, MetaException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_get_schema_version();
+      }
+    }
+
+    public void get_schema_latest_version(ISchemaName schemaName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      get_schema_latest_version_call method_call = new get_schema_latest_version_call(schemaName, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_schema_latest_version_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private ISchemaName schemaName;
+      public get_schema_latest_version_call(ISchemaName schemaName, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.schemaName = schemaName;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("get_schema_latest_version", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        get_schema_latest_version_args args = new get_schema_latest_version_args();
+        args.setSchemaName(schemaName);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public SchemaVersion getResult() throws NoSuchObjectException, MetaException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_get_schema_latest_version();
+      }
+    }
+
+    public void get_schema_all_versions(ISchemaName schemaName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      get_schema_all_versions_call method_call = new get_schema_all_versions_call(schemaName, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_schema_all_versions_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private ISchemaName schemaName;
+      public get_schema_all_versions_call(ISchemaName schemaName, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.schemaName = schemaName;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("get_schema_all_versions", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        get_schema_all_versions_args args = new get_schema_all_versions_args();
+        args.setSchemaName(schemaName);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public List<SchemaVersion> getResult() throws NoSuchObjectException, MetaException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_get_schema_all_versions();
+      }
+    }
+
+    public void drop_schema_version(SchemaVersionDescriptor schemaVersion, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      drop_schema_version_call method_call = new drop_schema_version_call(schemaVersion, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class drop_schema_version_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private SchemaVersionDescriptor schemaVersion;
+      public drop_schema_version_call(SchemaVersionDescriptor schemaVersion, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.schemaVersion = schemaVersion;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("drop_schema_version", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        drop_schema_version_args args = new drop_schema_version_args();
+        args.setSchemaVersion(schemaVersion);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public void getResult() throws NoSuchObjectException, MetaException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        (new Client(prot)).recv_drop_schema_version();
+      }
+    }
+
+    public void get_schemas_by_cols(FindSchemasByColsRqst rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      get_schemas_by_cols_call method_call = new get_schemas_by_cols_call(rqst, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_schemas_by_cols_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private FindSchemasByColsRqst rqst;
+      public get_schemas_by_cols_call(FindSchemasByColsRqst rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.rqst = rqst;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("get_schemas_by_cols", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        get_schemas_by_cols_args args = new get_schemas_by_cols_args();
+        args.setRqst(rqst);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public FindSchemasByColsResp getResult() throws MetaException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_get_schemas_by_cols();
+      }
+    }
+
+    public void map_schema_version_to_serde(MapSchemaVersionToSerdeRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      map_schema_version_to_serde_call method_call = new map_schema_version_to_serde_call(rqst, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class map_schema_version_to_serde_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private MapSchemaVersionToSerdeRequest rqst;
+      public map_schema_version_to_serde_call(MapSchemaVersionToSerdeRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.rqst = rqst;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("map_schema_version_to_serde", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        map_schema_version_to_serde_args args = new map_schema_version_to_serde_args();
+        args.setRqst(rqst);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public void getResult() throws NoSuchObjectException, MetaException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        (new Client(prot)).recv_map_schema_version_to_serde();
+      }
+    }
+
+    public void set_schema_version_state(SetSchemaVersionStateRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      set_schema_version_state_call method_call = new set_schema_version_state_call(rqst, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class set_schema_version_state_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private SetSchemaVersionStateRequest rqst;
+      public set_schema_version_state_call(SetSchemaVersionStateRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.rqst = rqst;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("set_schema_version_state", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        set_schema_version_state_args args = new set_schema_version_state_args();
+        args.setRqst(rqst);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public void getResult() throws NoSuchObjectException, InvalidOperationException, MetaException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        (new Client(prot)).recv_set_schema_version_state();
+      }
+    }
+
+    public void add_serde(SerDeInfo serde, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      add_serde_call method_call = new add_serde_call(serde, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class add_serde_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private SerDeInfo serde;
+      public add_serde_call(SerDeInfo serde, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.serde = serde;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("add_serde", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        add_serde_args args = new add_serde_args();
+        args.setSerde(serde);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public void getResult() throws AlreadyExistsException, MetaException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        (new Client(prot)).recv_add_serde();
+      }
+    }
+
+    public void get_serde(GetSerdeRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      get_serde_call method_call = new get_serde_call(rqst, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_serde_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private GetSerdeRequest rqst;
+      public get_serde_call(GetSerdeRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.rqst = rqst;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("get_serde", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        get_serde_args args = new get_serde_args();
+        args.setRqst(rqst);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public SerDeInfo getResult() throws NoSuchObjectException, MetaException, org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_get_serde();
+      }
+    }
+
   }
 
   @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class Processor<I extends Iface> extends com.facebook.fb303.FacebookService.Processor<I> implements org.apache.thrift.TProcessor {
@@ -12436,6 +13331,20 @@ import org.slf4j.LoggerFactory;
       processMap.put("create_or_update_wm_mapping", new create_or_update_wm_mapping());
       processMap.put("drop_wm_mapping", new drop_wm_mapping());
       processMap.put("create_or_drop_wm_trigger_to_pool_mapping", new create_or_drop_wm_trigger_to_pool_mapping());
+      processMap.put("create_ischema", new create_ischema());
+      processMap.put("alter_ischema", new alter_ischema());
+      processMap.put("get_ischema", new get_ischema());
+      processMap.put("drop_ischema", new drop_ischema());
+      processMap.put("add_schema_version", new add_schema_version());
+      processMap.put("get_schema_version", new get_schema_version());
+      processMap.put("get_schema_latest_version", new get_schema_latest_version());
+      processMap.put("get_schema_all_versions", new get_schema_all_versions());
+      processMap.put("drop_schema_version", new drop_schema_version());
+      processMap.put("get_schemas_by_cols", new get_schemas_by_cols());
+      processMap.put("map_schema_version_to_serde", new map_schema_version_to_serde());
+      processMap.put("set_schema_version_state", new set_schema_version_state());
+      processMap.put("add_serde", new add_serde());
+      processMap.put("get_serde", new get_serde());
       return processMap;
     }
 
@@ -17060,6 +17969,376 @@ import org.slf4j.LoggerFactory;
       }
     }
 
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class create_ischema<I extends Iface> extends org.apache.thrift.ProcessFunction<I, create_ischema_args> {
+      public create_ischema() {
+        super("create_ischema");
+      }
+
+      public create_ischema_args getEmptyArgsInstance() {
+        return new create_ischema_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public create_ischema_result getResult(I iface, create_ischema_args args) throws org.apache.thrift.TException {
+        create_ischema_result result = new create_ischema_result();
+        try {
+          iface.create_ischema(args.schema);
+        } catch (AlreadyExistsException o1) {
+          result.o1 = o1;
+        } catch (NoSuchObjectException o2) {
+          result.o2 = o2;
+        } catch (MetaException o3) {
+          result.o3 = o3;
+        }
+        return result;
+      }
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class alter_ischema<I extends Iface> extends org.apache.thrift.ProcessFunction<I, alter_ischema_args> {
+      public alter_ischema() {
+        super("alter_ischema");
+      }
+
+      public alter_ischema_args getEmptyArgsInstance() {
+        return new alter_ischema_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public alter_ischema_result getResult(I iface, alter_ischema_args args) throws org.apache.thrift.TException {
+        alter_ischema_result result = new alter_ischema_result();
+        try {
+          iface.alter_ischema(args.rqst);
+        } catch (NoSuchObjectException o1) {
+          result.o1 = o1;
+        } catch (MetaException o2) {
+          result.o2 = o2;
+        }
+        return result;
+      }
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_ischema<I extends Iface> extends org.apache.thrift.ProcessFunction<I, get_ischema_args> {
+      public get_ischema() {
+        super("get_ischema");
+      }
+
+      public get_ischema_args getEmptyArgsInstance() {
+        return new get_ischema_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public get_ischema_result getResult(I iface, get_ischema_args args) throws org.apache.thrift.TException {
+        get_ischema_result result = new get_ischema_result();
+        try {
+          result.success = iface.get_ischema(args.name);
+        } catch (NoSuchObjectException o1) {
+          result.o1 = o1;
+        } catch (MetaException o2) {
+          result.o2 = o2;
+        }
+        return result;
+      }
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class drop_ischema<I extends Iface> extends org.apache.thrift.ProcessFunction<I, drop_ischema_args> {
+      public drop_ischema() {
+        super("drop_ischema");
+      }
+
+      public drop_ischema_args getEmptyArgsInstance() {
+        return new drop_ischema_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public drop_ischema_result getResult(I iface, drop_ischema_args args) throws org.apache.thrift.TException {
+        drop_ischema_result result = new drop_ischema_result();
+        try {
+          iface.drop_ischema(args.name);
+        } catch (NoSuchObjectException o1) {
+          result.o1 = o1;
+        } catch (InvalidOperationException o2) {
+          result.o2 = o2;
+        } catch (MetaException o3) {
+          result.o3 = o3;
+        }
+        return result;
+      }
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class add_schema_version<I extends Iface> extends org.apache.thrift.ProcessFunction<I, add_schema_version_args> {
+      public add_schema_version() {
+        super("add_schema_version");
+      }
+
+      public add_schema_version_args getEmptyArgsInstance() {
+        return new add_schema_version_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public add_schema_version_result getResult(I iface, add_schema_version_args args) throws org.apache.thrift.TException {
+        add_schema_version_result result = new add_schema_version_result();
+        try {
+          iface.add_schema_version(args.schemaVersion);
+        } catch (AlreadyExistsException o1) {
+          result.o1 = o1;
+        } catch (NoSuchObjectException o2) {
+          result.o2 = o2;
+        } catch (MetaException o3) {
+          result.o3 = o3;
+        }
+        return result;
+      }
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_schema_version<I extends Iface> extends org.apache.thrift.ProcessFunction<I, get_schema_version_args> {
+      public get_schema_version() {
+        super("get_schema_version");
+      }
+
+      public get_schema_version_args getEmptyArgsInstance() {
+        return new get_schema_version_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public get_schema_version_result getResult(I iface, get_schema_version_args args) throws org.apache.thrift.TException {
+        get_schema_version_result result = new get_schema_version_result();
+        try {
+          result.success = iface.get_schema_version(args.schemaVersion);
+        } catch (NoSuchObjectException o1) {
+          result.o1 = o1;
+        } catch (MetaException o2) {
+          result.o2 = o2;
+        }
+        return result;
+      }
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_schema_latest_version<I extends Iface> extends org.apache.thrift.ProcessFunction<I, get_schema_latest_version_args> {
+      public get_schema_latest_version() {
+        super("get_schema_latest_version");
+      }
+
+      public get_schema_latest_version_args getEmptyArgsInstance() {
+        return new get_schema_latest_version_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public get_schema_latest_version_result getResult(I iface, get_schema_latest_version_args args) throws org.apache.thrift.TException {
+        get_schema_latest_version_result result = new get_schema_latest_version_result();
+        try {
+          result.success = iface.get_schema_latest_version(args.schemaName);
+        } catch (NoSuchObjectException o1) {
+          result.o1 = o1;
+        } catch (MetaException o2) {
+          result.o2 = o2;
+        }
+        return result;
+      }
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_schema_all_versions<I extends Iface> extends org.apache.thrift.ProcessFunction<I, get_schema_all_versions_args> {
+      public get_schema_all_versions() {
+        super("get_schema_all_versions");
+      }
+
+      public get_schema_all_versions_args getEmptyArgsInstance() {
+        return new get_schema_all_versions_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public get_schema_all_versions_result getResult(I iface, get_schema_all_versions_args args) throws org.apache.thrift.TException {
+        get_schema_all_versions_result result = new get_schema_all_versions_result();
+        try {
+          result.success = iface.get_schema_all_versions(args.schemaName);
+        } catch (NoSuchObjectException o1) {
+          result.o1 = o1;
+        } catch (MetaException o2) {
+          result.o2 = o2;
+        }
+        return result;
+      }
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class drop_schema_version<I extends Iface> extends org.apache.thrift.ProcessFunction<I, drop_schema_version_args> {
+      public drop_schema_version() {
+        super("drop_schema_version");
+      }
+
+      public drop_schema_version_args getEmptyArgsInstance() {
+        return new drop_schema_version_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public drop_schema_version_result getResult(I iface, drop_schema_version_args args) throws org.apache.thrift.TException {
+        drop_schema_version_result result = new drop_schema_version_result();
+        try {
+          iface.drop_schema_version(args.schemaVersion);
+        } catch (NoSuchObjectException o1) {
+          result.o1 = o1;
+        } catch (MetaException o2) {
+          result.o2 = o2;
+        }
+        return result;
+      }
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_schemas_by_cols<I extends Iface> extends org.apache.thrift.ProcessFunction<I, get_schemas_by_cols_args> {
+      public get_schemas_by_cols() {
+        super("get_schemas_by_cols");
+      }
+
+      public get_schemas_by_cols_args getEmptyArgsInstance() {
+        return new get_schemas_by_cols_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public get_schemas_by_cols_result getResult(I iface, get_schemas_by_cols_args args) throws org.apache.thrift.TException {
+        get_schemas_by_cols_result result = new get_schemas_by_cols_result();
+        try {
+          result.success = iface.get_schemas_by_cols(args.rqst);
+        } catch (MetaException o1) {
+          result.o1 = o1;
+        }
+        return result;
+      }
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class map_schema_version_to_serde<I extends Iface> extends org.apache.thrift.ProcessFunction<I, map_schema_version_to_serde_args> {
+      public map_schema_version_to_serde() {
+        super("map_schema_version_to_serde");
+      }
+
+      public map_schema_version_to_serde_args getEmptyArgsInstance() {
+        return new map_schema_version_to_serde_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public map_schema_version_to_serde_result getResult(I iface, map_schema_version_to_serde_args args) throws org.apache.thrift.TException {
+        map_schema_version_to_serde_result result = new map_schema_version_to_serde_result();
+        try {
+          iface.map_schema_version_to_serde(args.rqst);
+        } catch (NoSuchObjectException o1) {
+          result.o1 = o1;
+        } catch (MetaException o2) {
+          result.o2 = o2;
+        }
+        return result;
+      }
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class set_schema_version_state<I extends Iface> extends org.apache.thrift.ProcessFunction<I, set_schema_version_state_args> {
+      public set_schema_version_state() {
+        super("set_schema_version_state");
+      }
+
+      public set_schema_version_state_args getEmptyArgsInstance() {
+        return new set_schema_version_state_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public set_schema_version_state_result getResult(I iface, set_schema_version_state_args args) throws org.apache.thrift.TException {
+        set_schema_version_state_result result = new set_schema_version_state_result();
+        try {
+          iface.set_schema_version_state(args.rqst);
+        } catch (NoSuchObjectException o1) {
+          result.o1 = o1;
+        } catch (InvalidOperationException o2) {
+          result.o2 = o2;
+        } catch (MetaException o3) {
+          result.o3 = o3;
+        }
+        return result;
+      }
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class add_serde<I extends Iface> extends org.apache.thrift.ProcessFunction<I, add_serde_args> {
+      public add_serde() {
+        super("add_serde");
+      }
+
+      public add_serde_args getEmptyArgsInstance() {
+        return new add_serde_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public add_serde_result getResult(I iface, add_serde_args args) throws org.apache.thrift.TException {
+        add_serde_result result = new add_serde_result();
+        try {
+          iface.add_serde(args.serde);
+        } catch (AlreadyExistsException o1) {
+          result.o1 = o1;
+        } catch (MetaException o2) {
+          result.o2 = o2;
+        }
+        return result;
+      }
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_serde<I extends Iface> extends org.apache.thrift.ProcessFunction<I, get_serde_args> {
+      public get_serde() {
+        super("get_serde");
+      }
+
+      public get_serde_args getEmptyArgsInstance() {
+        return new get_serde_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public get_serde_result getResult(I iface, get_serde_args args) throws org.apache.thrift.TException {
+        get_serde_result result = new get_serde_result();
+        try {
+          result.success = iface.get_serde(args.rqst);
+        } catch (NoSuchObjectException o1) {
+          result.o1 = o1;
+        } catch (MetaException o2) {
+          result.o2 = o2;
+        }
+        return result;
+      }
+    }
+
   }
 
   @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class AsyncProcessor<I extends AsyncIface> extends com.facebook.fb303.FacebookService.AsyncProcessor<I> {
@@ -17253,6 +18532,20 @@ import org.slf4j.LoggerFactory;
       processMap.put("create_or_update_wm_mapping", new create_or_update_wm_mapping());
       processMap.put("drop_wm_mapping", new drop_wm_mapping());
       processMap.put("create_or_drop_wm_trigger_to_pool_mapping", new create_or_drop_wm_trigger_to_pool_mapping());
+      processMap.put("create_ischema", new create_ischema());
+      processMap.put("alter_ischema", new alter_ischema());
+      processMap.put("get_ischema", new get_ischema());
+      processMap.put("drop_ischema", new drop_ischema());
+      processMap.put("add_schema_version", new add_schema_version());
+      processMap.put("get_schema_version", new get_schema_version());
+      processMap.put("get_schema_latest_version", new get_schema_latest_version());
+      processMap.put("get_schema_all_versions", new get_schema_all_versions());
+      processMap.put("drop_schema_version", new drop_schema_version());
+      processMap.put("get_schemas_by_cols", new get_schemas_by_cols());
+      processMap.put("map_schema_version_to_serde", new map_schema_version_to_serde());
+      processMap.put("set_schema_version_state", new set_schema_version_state());
+      processMap.put("add_serde", new add_serde());
+      processMap.put("get_serde", new get_serde());
       return processMap;
     }
 
@@ -28289,6 +29582,881 @@ import org.slf4j.LoggerFactory;
       }
     }
 
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class create_ischema<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, create_ischema_args, Void> {
+      public create_ischema() {
+        super("create_ischema");
+      }
+
+      public create_ischema_args getEmptyArgsInstance() {
+        return new create_ischema_args();
+      }
+
+      public AsyncMethodCallback<Void> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<Void>() { 
+          public void onComplete(Void o) {
+            create_ischema_result result = new create_ischema_result();
+            try {
+              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+              return;
+            } catch (Exception e) {
+              LOGGER.error("Exception writing to internal frame buffer", e);
+            }
+            fb.close();
+          }
+          public void onError(Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TBase msg;
+            create_ischema_result result = new create_ischema_result();
+            if (e instanceof AlreadyExistsException) {
+                        result.o1 = (AlreadyExistsException) e;
+                        result.setO1IsSet(true);
+                        msg = result;
+            }
+            else             if (e instanceof NoSuchObjectException) {
+                        result.o2 = (NoSuchObjectException) e;
+                        result.setO2IsSet(true);
+                        msg = result;
+            }
+            else             if (e instanceof MetaException) {
+                        result.o3 = (MetaException) e;
+                        result.setO3IsSet(true);
+                        msg = result;
+            }
+             else 
+            {
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+              return;
+            } catch (Exception ex) {
+              LOGGER.error("Exception writing to internal frame buffer", ex);
+            }
+            fb.close();
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, create_ischema_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
+        iface.create_ischema(args.schema,resultHandler);
+      }
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class alter_ischema<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, alter_ischema_args, Void> {
+      public alter_ischema() {
+        super("alter_ischema");
+      }
+
+      public alter_ischema_args getEmptyArgsInstance() {
+        return new alter_ischema_args();
+      }
+
+      public AsyncMethodCallback<Void> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<Void>() { 
+          public void onComplete(Void o) {
+            alter_ischema_result result = new alter_ischema_result();
+            try {
+              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+              return;
+            } catch (Exception e) {
+              LOGGER.error("Exception writing to internal frame buffer", e);
+            }
+            fb.close();
+          }
+          public void onError(Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TBase msg;
+            alter_ischema_result result = new alter_ischema_result();
+            if (e instanceof NoSuchObjectException) {
+                        result.o1 = (NoSuchObjectException) e;
+                        result.setO1IsSet(true);
+                        msg = result;
+            }
+            else             if (e instanceof MetaException) {
+                        result.o2 = (MetaException) e;
+                        result.setO2IsSet(true);
+                        msg = result;
+            }
+             else 
+            {
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+              return;
+            } catch (Exception ex) {
+              LOGGER.error("Exception writing to internal frame buffer", ex);
+            }
+            fb.close();
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, alter_ischema_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
+        iface.alter_ischema(args.rqst,resultHandler);
+      }
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_ischema<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_ischema_args, ISchema> {
+      public get_ischema() {
+        super("get_ischema");
+      }
+
+      public get_ischema_args getEmptyArgsInstance() {
+        return new get_ischema_args();
+      }
+
+      public AsyncMethodCallback<ISchema> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<ISchema>() { 
+          public void onComplete(ISchema o) {
+            get_ischema_result result = new get_ischema_result();
+            result.success = o;
+            try {
+              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+              return;
+            } catch (Exception e) {
+              LOGGER.error("Exception writing to internal frame buffer", e);
+            }
+            fb.close();
+          }
+          public void onError(Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TBase msg;
+            get_ischema_result result = new get_ischema_result();
+            if (e instanceof NoSuchObjectException) {
+                        result.o1 = (NoSuchObjectException) e;
+                        result.setO1IsSet(true);
+                        msg = result;
+            }
+            else             if (e instanceof MetaException) {
+                        result.o2 = (MetaException) e;
+                        result.setO2IsSet(true);
+                        msg = result;
+            }
+             else 
+            {
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+              return;
+            } catch (Exception ex) {
+              LOGGER.error("Exception writing to internal frame buffer", ex);
+            }
+            fb.close();
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, get_ischema_args args, org.apache.thrift.async.AsyncMethodCallback<ISchema> resultHandler) throws TException {
+        iface.get_ischema(args.name,resultHandler);
+      }
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class drop_ischema<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, drop_ischema_args, Void> {
+      public drop_ischema() {
+        super("drop_ischema");
+      }
+
+      public drop_ischema_args getEmptyArgsInstance() {
+        return new drop_ischema_args();
+      }
+
+      public AsyncMethodCallback<Void> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<Void>() { 
+          public void onComplete(Void o) {
+            drop_ischema_result result = new drop_ischema_result();
+            try {
+              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+              return;
+            } catch (Exception e) {
+              LOGGER.error("Exception writing to internal frame buffer", e);
+            }
+            fb.close();
+          }
+          public void onError(Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TBase msg;
+            drop_ischema_result result = new drop_ischema_result();
+            if (e instanceof NoSuchObjectException) {
+                        result.o1 = (NoSuchObjectException) e;
+                        result.setO1IsSet(true);
+                        msg = result;
+            }
+            else             if (e instanceof InvalidOperationException) {
+                        result.o2 = (InvalidOperationException) e;
+                        result.setO2IsSet(true);
+                        msg = result;
+            }
+            else             if (e instanceof MetaException) {
+                        result.o3 = (MetaException) e;
+                        result.setO3IsSet(true);
+                        msg = result;
+            }
+             else 
+            {
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+              return;
+            } catch (Exception ex) {
+              LOGGER.error("Exception writing to internal frame buffer", ex);
+            }
+            fb.close();
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, drop_ischema_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
+        iface.drop_ischema(args.name,resultHandler);
+      }
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class add_schema_version<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, add_schema_version_args, Void> {
+      public add_schema_version() {
+        super("add_schema_version");
+      }
+
+      public add_schema_version_args getEmptyArgsInstance() {
+        return new add_schema_version_args();
+      }
+
+      public AsyncMethodCallback<Void> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<Void>() { 
+          public void onComplete(Void o) {
+            add_schema_version_result result = new add_schema_version_result();
+            try {
+              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+              return;
+            } catch (Exception e) {
+              LOGGER.error("Exception writing to internal frame buffer", e);
+            }
+            fb.close();
+          }
+          public void onError(Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TBase msg;
+            add_schema_version_result result = new add_schema_version_result();
+            if (e instanceof AlreadyExistsException) {
+                        result.o1 = (AlreadyExistsException) e;
+                        result.setO1IsSet(true);
+                        msg = result;
+            }
+            else             if (e instanceof NoSuchObjectException) {
+                        result.o2 = (NoSuchObjectException) e;
+                        result.setO2IsSet(true);
+                        msg = result;
+            }
+            else             if (e instanceof MetaException) {
+                        result.o3 = (MetaException) e;
+                        result.setO3IsSet(true);
+                        msg = result;
+            }
+             else 
+            {
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+              return;
+            } catch (Exception ex) {
+              LOGGER.error("Exception writing to internal frame buffer", ex);
+            }
+            fb.close();
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, add_schema_version_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
+        iface.add_schema_version(args.schemaVersion,resultHandler);
+      }
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_schema_version<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_schema_version_args, SchemaVersion> {
+      public get_schema_version() {
+        super("get_schema_version");
+      }
+
+      public get_schema_version_args getEmptyArgsInstance() {
+        return new get_schema_version_args();
+      }
+
+      public AsyncMethodCallback<SchemaVersion> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<SchemaVersion>() { 
+          public void onComplete(SchemaVersion o) {
+            get_schema_version_result result = new get_schema_version_result();
+            result.success = o;
+            try {
+              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+              return;
+            } catch (Exception e) {
+              LOGGER.error("Exception writing to internal frame buffer", e);
+            }
+            fb.close();
+          }
+          public void onError(Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TBase msg;
+            get_schema_version_result result = new get_schema_version_result();
+            if (e instanceof NoSuchObjectException) {
+                        result.o1 = (NoSuchObjectException) e;
+                        result.setO1IsSet(true);
+                        msg = result;
+            }
+            else             if (e instanceof MetaException) {
+                        result.o2 = (MetaException) e;
+                        result.setO2IsSet(true);
+                        msg = result;
+            }
+             else 
+            {
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+              return;
+            } catch (Exception ex) {
+              LOGGER.error("Exception writing to internal frame buffer", ex);
+            }
+            fb.close();
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, get_schema_version_args args, org.apache.thrift.async.AsyncMethodCallback<SchemaVersion> resultHandler) throws TException {
+        iface.get_schema_version(args.schemaVersion,resultHandler);
+      }
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_schema_latest_version<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_schema_latest_version_args, SchemaVersion> {
+      public get_schema_latest_version() {
+        super("get_schema_latest_version");
+      }
+
+      public get_schema_latest_version_args getEmptyArgsInstance() {
+        return new get_schema_latest_version_args();
+      }
+
+      public AsyncMethodCallback<SchemaVersion> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<SchemaVersion>() { 
+          public void onComplete(SchemaVersion o) {
+            get_schema_latest_version_result result = new get_schema_latest_version_result();
+            result.success = o;
+            try {
+              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+              return;
+            } catch (Exception e) {
+              LOGGER.error("Exception writing to internal frame buffer", e);
+            }
+            fb.close();
+          }
+          public void onError(Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TBase msg;
+            get_schema_latest_version_result result = new get_schema_latest_version_result();
+            if (e instanceof NoSuchObjectException) {
+                        result.o1 = (NoSuchObjectException) e;
+                        result.setO1IsSet(true);
+                        msg = result;
+            }
+            else             if (e instanceof MetaException) {
+                        result.o2 = (MetaException) e;
+                        result.setO2IsSet(true);
+                        msg = result;
+            }
+             else 
+            {
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+              return;
+            } catch (Exception ex) {
+              LOGGER.error("Exception writing to internal frame buffer", ex);
+            }
+            fb.close();
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, get_schema_latest_version_args args, org.apache.thrift.async.AsyncMethodCallback<SchemaVersion> resultHandler) throws TException {
+        iface.get_schema_latest_version(args.schemaName,resultHandler);
+      }
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_schema_all_versions<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_schema_all_versions_args, List<SchemaVersion>> {
+      public get_schema_all_versions() {
+        super("get_schema_all_versions");
+      }
+
+      public get_schema_all_versions_args getEmptyArgsInstance() {
+        return new get_schema_all_versions_args();
+      }
+
+      public AsyncMethodCallback<List<SchemaVersion>> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+ 

<TRUNCATED>

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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SetSchemaVersionStateRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SetSchemaVersionStateRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SetSchemaVersionStateRequest.java
new file mode 100644
index 0000000..23bf9a4
--- /dev/null
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SetSchemaVersionStateRequest.java
@@ -0,0 +1,516 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hive.metastore.api;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+@org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public class SetSchemaVersionStateRequest implements org.apache.thrift.TBase<SetSchemaVersionStateRequest, SetSchemaVersionStateRequest._Fields>, java.io.Serializable, Cloneable, Comparable<SetSchemaVersionStateRequest> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SetSchemaVersionStateRequest");
+
+  private static final org.apache.thrift.protocol.TField SCHEMA_VERSION_FIELD_DESC = new org.apache.thrift.protocol.TField("schemaVersion", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+  private static final org.apache.thrift.protocol.TField STATE_FIELD_DESC = new org.apache.thrift.protocol.TField("state", org.apache.thrift.protocol.TType.I32, (short)2);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new SetSchemaVersionStateRequestStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new SetSchemaVersionStateRequestTupleSchemeFactory());
+  }
+
+  private SchemaVersionDescriptor schemaVersion; // required
+  private SchemaVersionState state; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    SCHEMA_VERSION((short)1, "schemaVersion"),
+    /**
+     * 
+     * @see SchemaVersionState
+     */
+    STATE((short)2, "state");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // SCHEMA_VERSION
+          return SCHEMA_VERSION;
+        case 2: // STATE
+          return STATE;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.SCHEMA_VERSION, new org.apache.thrift.meta_data.FieldMetaData("schemaVersion", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, SchemaVersionDescriptor.class)));
+    tmpMap.put(_Fields.STATE, new org.apache.thrift.meta_data.FieldMetaData("state", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, SchemaVersionState.class)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(SetSchemaVersionStateRequest.class, metaDataMap);
+  }
+
+  public SetSchemaVersionStateRequest() {
+  }
+
+  public SetSchemaVersionStateRequest(
+    SchemaVersionDescriptor schemaVersion,
+    SchemaVersionState state)
+  {
+    this();
+    this.schemaVersion = schemaVersion;
+    this.state = state;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public SetSchemaVersionStateRequest(SetSchemaVersionStateRequest other) {
+    if (other.isSetSchemaVersion()) {
+      this.schemaVersion = new SchemaVersionDescriptor(other.schemaVersion);
+    }
+    if (other.isSetState()) {
+      this.state = other.state;
+    }
+  }
+
+  public SetSchemaVersionStateRequest deepCopy() {
+    return new SetSchemaVersionStateRequest(this);
+  }
+
+  @Override
+  public void clear() {
+    this.schemaVersion = null;
+    this.state = null;
+  }
+
+  public SchemaVersionDescriptor getSchemaVersion() {
+    return this.schemaVersion;
+  }
+
+  public void setSchemaVersion(SchemaVersionDescriptor schemaVersion) {
+    this.schemaVersion = schemaVersion;
+  }
+
+  public void unsetSchemaVersion() {
+    this.schemaVersion = null;
+  }
+
+  /** Returns true if field schemaVersion is set (has been assigned a value) and false otherwise */
+  public boolean isSetSchemaVersion() {
+    return this.schemaVersion != null;
+  }
+
+  public void setSchemaVersionIsSet(boolean value) {
+    if (!value) {
+      this.schemaVersion = null;
+    }
+  }
+
+  /**
+   * 
+   * @see SchemaVersionState
+   */
+  public SchemaVersionState getState() {
+    return this.state;
+  }
+
+  /**
+   * 
+   * @see SchemaVersionState
+   */
+  public void setState(SchemaVersionState state) {
+    this.state = state;
+  }
+
+  public void unsetState() {
+    this.state = null;
+  }
+
+  /** Returns true if field state is set (has been assigned a value) and false otherwise */
+  public boolean isSetState() {
+    return this.state != null;
+  }
+
+  public void setStateIsSet(boolean value) {
+    if (!value) {
+      this.state = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case SCHEMA_VERSION:
+      if (value == null) {
+        unsetSchemaVersion();
+      } else {
+        setSchemaVersion((SchemaVersionDescriptor)value);
+      }
+      break;
+
+    case STATE:
+      if (value == null) {
+        unsetState();
+      } else {
+        setState((SchemaVersionState)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case SCHEMA_VERSION:
+      return getSchemaVersion();
+
+    case STATE:
+      return getState();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case SCHEMA_VERSION:
+      return isSetSchemaVersion();
+    case STATE:
+      return isSetState();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof SetSchemaVersionStateRequest)
+      return this.equals((SetSchemaVersionStateRequest)that);
+    return false;
+  }
+
+  public boolean equals(SetSchemaVersionStateRequest that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_schemaVersion = true && this.isSetSchemaVersion();
+    boolean that_present_schemaVersion = true && that.isSetSchemaVersion();
+    if (this_present_schemaVersion || that_present_schemaVersion) {
+      if (!(this_present_schemaVersion && that_present_schemaVersion))
+        return false;
+      if (!this.schemaVersion.equals(that.schemaVersion))
+        return false;
+    }
+
+    boolean this_present_state = true && this.isSetState();
+    boolean that_present_state = true && that.isSetState();
+    if (this_present_state || that_present_state) {
+      if (!(this_present_state && that_present_state))
+        return false;
+      if (!this.state.equals(that.state))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_schemaVersion = true && (isSetSchemaVersion());
+    list.add(present_schemaVersion);
+    if (present_schemaVersion)
+      list.add(schemaVersion);
+
+    boolean present_state = true && (isSetState());
+    list.add(present_state);
+    if (present_state)
+      list.add(state.getValue());
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(SetSchemaVersionStateRequest other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetSchemaVersion()).compareTo(other.isSetSchemaVersion());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetSchemaVersion()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.schemaVersion, other.schemaVersion);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetState()).compareTo(other.isSetState());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetState()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.state, other.state);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("SetSchemaVersionStateRequest(");
+    boolean first = true;
+
+    sb.append("schemaVersion:");
+    if (this.schemaVersion == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.schemaVersion);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("state:");
+    if (this.state == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.state);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+    if (schemaVersion != null) {
+      schemaVersion.validate();
+    }
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class SetSchemaVersionStateRequestStandardSchemeFactory implements SchemeFactory {
+    public SetSchemaVersionStateRequestStandardScheme getScheme() {
+      return new SetSchemaVersionStateRequestStandardScheme();
+    }
+  }
+
+  private static class SetSchemaVersionStateRequestStandardScheme extends StandardScheme<SetSchemaVersionStateRequest> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, SetSchemaVersionStateRequest struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // SCHEMA_VERSION
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.schemaVersion = new SchemaVersionDescriptor();
+              struct.schemaVersion.read(iprot);
+              struct.setSchemaVersionIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // STATE
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.state = org.apache.hadoop.hive.metastore.api.SchemaVersionState.findByValue(iprot.readI32());
+              struct.setStateIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, SetSchemaVersionStateRequest struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.schemaVersion != null) {
+        oprot.writeFieldBegin(SCHEMA_VERSION_FIELD_DESC);
+        struct.schemaVersion.write(oprot);
+        oprot.writeFieldEnd();
+      }
+      if (struct.state != null) {
+        oprot.writeFieldBegin(STATE_FIELD_DESC);
+        oprot.writeI32(struct.state.getValue());
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class SetSchemaVersionStateRequestTupleSchemeFactory implements SchemeFactory {
+    public SetSchemaVersionStateRequestTupleScheme getScheme() {
+      return new SetSchemaVersionStateRequestTupleScheme();
+    }
+  }
+
+  private static class SetSchemaVersionStateRequestTupleScheme extends TupleScheme<SetSchemaVersionStateRequest> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, SetSchemaVersionStateRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.isSetSchemaVersion()) {
+        optionals.set(0);
+      }
+      if (struct.isSetState()) {
+        optionals.set(1);
+      }
+      oprot.writeBitSet(optionals, 2);
+      if (struct.isSetSchemaVersion()) {
+        struct.schemaVersion.write(oprot);
+      }
+      if (struct.isSetState()) {
+        oprot.writeI32(struct.state.getValue());
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, SetSchemaVersionStateRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(2);
+      if (incoming.get(0)) {
+        struct.schemaVersion = new SchemaVersionDescriptor();
+        struct.schemaVersion.read(iprot);
+        struct.setSchemaVersionIsSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.state = org.apache.hadoop.hive.metastore.api.SchemaVersionState.findByValue(iprot.readI32());
+        struct.setStateIsSet(true);
+      }
+    }
+  }
+
+}
+


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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h b/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
index cee1382..ef8b694 100644
--- a/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
+++ b/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
@@ -132,6 +132,59 @@ struct EventRequestType {
 
 extern const std::map<int, const char*> _EventRequestType_VALUES_TO_NAMES;
 
+struct SerdeType {
+  enum type {
+    HIVE = 1,
+    SCHEMA_REGISTRY = 2
+  };
+};
+
+extern const std::map<int, const char*> _SerdeType_VALUES_TO_NAMES;
+
+struct SchemaType {
+  enum type {
+    HIVE = 1,
+    AVRO = 2
+  };
+};
+
+extern const std::map<int, const char*> _SchemaType_VALUES_TO_NAMES;
+
+struct SchemaCompatibility {
+  enum type {
+    NONE = 1,
+    BACKWARD = 2,
+    FORWARD = 3,
+    BOTH = 4
+  };
+};
+
+extern const std::map<int, const char*> _SchemaCompatibility_VALUES_TO_NAMES;
+
+struct SchemaValidation {
+  enum type {
+    LATEST = 1,
+    ALL = 2
+  };
+};
+
+extern const std::map<int, const char*> _SchemaValidation_VALUES_TO_NAMES;
+
+struct SchemaVersionState {
+  enum type {
+    INITIATED = 1,
+    START_REVIEW = 2,
+    CHANGES_REQUIRED = 3,
+    REVIEWED = 4,
+    ENABLED = 5,
+    DISABLED = 6,
+    ARCHIVED = 7,
+    DELETED = 8
+  };
+};
+
+extern const std::map<int, const char*> _SchemaVersionState_VALUES_TO_NAMES;
+
 struct FunctionType {
   enum type {
     JAVA = 1
@@ -566,6 +619,26 @@ class WMCreateOrDropTriggerToPoolMappingRequest;
 
 class WMCreateOrDropTriggerToPoolMappingResponse;
 
+class ISchema;
+
+class ISchemaName;
+
+class AlterISchemaRequest;
+
+class SchemaVersion;
+
+class SchemaVersionDescriptor;
+
+class FindSchemasByColsRqst;
+
+class FindSchemasByColsResp;
+
+class MapSchemaVersionToSerdeRequest;
+
+class SetSchemaVersionStateRequest;
+
+class GetSerdeRequest;
+
 class MetaException;
 
 class UnknownTableException;
@@ -2192,10 +2265,14 @@ inline std::ostream& operator<<(std::ostream& out, const Database& obj)
 }
 
 typedef struct _SerDeInfo__isset {
-  _SerDeInfo__isset() : name(false), serializationLib(false), parameters(false) {}
+  _SerDeInfo__isset() : name(false), serializationLib(false), parameters(false), description(false), serializerClass(false), deserializerClass(false), serdeType(false) {}
   bool name :1;
   bool serializationLib :1;
   bool parameters :1;
+  bool description :1;
+  bool serializerClass :1;
+  bool deserializerClass :1;
+  bool serdeType :1;
 } _SerDeInfo__isset;
 
 class SerDeInfo {
@@ -2203,13 +2280,17 @@ class SerDeInfo {
 
   SerDeInfo(const SerDeInfo&);
   SerDeInfo& operator=(const SerDeInfo&);
-  SerDeInfo() : name(), serializationLib() {
+  SerDeInfo() : name(), serializationLib(), description(), serializerClass(), deserializerClass(), serdeType((SerdeType::type)0) {
   }
 
   virtual ~SerDeInfo() throw();
   std::string name;
   std::string serializationLib;
   std::map<std::string, std::string>  parameters;
+  std::string description;
+  std::string serializerClass;
+  std::string deserializerClass;
+  SerdeType::type serdeType;
 
   _SerDeInfo__isset __isset;
 
@@ -2219,6 +2300,14 @@ class SerDeInfo {
 
   void __set_parameters(const std::map<std::string, std::string> & val);
 
+  void __set_description(const std::string& val);
+
+  void __set_serializerClass(const std::string& val);
+
+  void __set_deserializerClass(const std::string& val);
+
+  void __set_serdeType(const SerdeType::type val);
+
   bool operator == (const SerDeInfo & rhs) const
   {
     if (!(name == rhs.name))
@@ -2227,6 +2316,22 @@ class SerDeInfo {
       return false;
     if (!(parameters == rhs.parameters))
       return false;
+    if (__isset.description != rhs.__isset.description)
+      return false;
+    else if (__isset.description && !(description == rhs.description))
+      return false;
+    if (__isset.serializerClass != rhs.__isset.serializerClass)
+      return false;
+    else if (__isset.serializerClass && !(serializerClass == rhs.serializerClass))
+      return false;
+    if (__isset.deserializerClass != rhs.__isset.deserializerClass)
+      return false;
+    else if (__isset.deserializerClass && !(deserializerClass == rhs.deserializerClass))
+      return false;
+    if (__isset.serdeType != rhs.__isset.serdeType)
+      return false;
+    else if (__isset.serdeType && !(serdeType == rhs.serdeType))
+      return false;
     return true;
   }
   bool operator != (const SerDeInfo &rhs) const {
@@ -11328,6 +11433,626 @@ inline std::ostream& operator<<(std::ostream& out, const WMCreateOrDropTriggerTo
   return out;
 }
 
+typedef struct _ISchema__isset {
+  _ISchema__isset() : schemaType(false), name(false), dbName(false), compatibility(false), validationLevel(false), canEvolve(false), schemaGroup(false), description(false) {}
+  bool schemaType :1;
+  bool name :1;
+  bool dbName :1;
+  bool compatibility :1;
+  bool validationLevel :1;
+  bool canEvolve :1;
+  bool schemaGroup :1;
+  bool description :1;
+} _ISchema__isset;
+
+class ISchema {
+ public:
+
+  ISchema(const ISchema&);
+  ISchema& operator=(const ISchema&);
+  ISchema() : schemaType((SchemaType::type)0), name(), dbName(), compatibility((SchemaCompatibility::type)0), validationLevel((SchemaValidation::type)0), canEvolve(0), schemaGroup(), description() {
+  }
+
+  virtual ~ISchema() throw();
+  SchemaType::type schemaType;
+  std::string name;
+  std::string dbName;
+  SchemaCompatibility::type compatibility;
+  SchemaValidation::type validationLevel;
+  bool canEvolve;
+  std::string schemaGroup;
+  std::string description;
+
+  _ISchema__isset __isset;
+
+  void __set_schemaType(const SchemaType::type val);
+
+  void __set_name(const std::string& val);
+
+  void __set_dbName(const std::string& val);
+
+  void __set_compatibility(const SchemaCompatibility::type val);
+
+  void __set_validationLevel(const SchemaValidation::type val);
+
+  void __set_canEvolve(const bool val);
+
+  void __set_schemaGroup(const std::string& val);
+
+  void __set_description(const std::string& val);
+
+  bool operator == (const ISchema & rhs) const
+  {
+    if (!(schemaType == rhs.schemaType))
+      return false;
+    if (!(name == rhs.name))
+      return false;
+    if (!(dbName == rhs.dbName))
+      return false;
+    if (!(compatibility == rhs.compatibility))
+      return false;
+    if (!(validationLevel == rhs.validationLevel))
+      return false;
+    if (!(canEvolve == rhs.canEvolve))
+      return false;
+    if (__isset.schemaGroup != rhs.__isset.schemaGroup)
+      return false;
+    else if (__isset.schemaGroup && !(schemaGroup == rhs.schemaGroup))
+      return false;
+    if (__isset.description != rhs.__isset.description)
+      return false;
+    else if (__isset.description && !(description == rhs.description))
+      return false;
+    return true;
+  }
+  bool operator != (const ISchema &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ISchema & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  virtual void printTo(std::ostream& out) const;
+};
+
+void swap(ISchema &a, ISchema &b);
+
+inline std::ostream& operator<<(std::ostream& out, const ISchema& obj)
+{
+  obj.printTo(out);
+  return out;
+}
+
+typedef struct _ISchemaName__isset {
+  _ISchemaName__isset() : dbName(false), schemaName(false) {}
+  bool dbName :1;
+  bool schemaName :1;
+} _ISchemaName__isset;
+
+class ISchemaName {
+ public:
+
+  ISchemaName(const ISchemaName&);
+  ISchemaName& operator=(const ISchemaName&);
+  ISchemaName() : dbName(), schemaName() {
+  }
+
+  virtual ~ISchemaName() throw();
+  std::string dbName;
+  std::string schemaName;
+
+  _ISchemaName__isset __isset;
+
+  void __set_dbName(const std::string& val);
+
+  void __set_schemaName(const std::string& val);
+
+  bool operator == (const ISchemaName & rhs) const
+  {
+    if (!(dbName == rhs.dbName))
+      return false;
+    if (!(schemaName == rhs.schemaName))
+      return false;
+    return true;
+  }
+  bool operator != (const ISchemaName &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ISchemaName & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  virtual void printTo(std::ostream& out) const;
+};
+
+void swap(ISchemaName &a, ISchemaName &b);
+
+inline std::ostream& operator<<(std::ostream& out, const ISchemaName& obj)
+{
+  obj.printTo(out);
+  return out;
+}
+
+typedef struct _AlterISchemaRequest__isset {
+  _AlterISchemaRequest__isset() : name(false), newSchema(false) {}
+  bool name :1;
+  bool newSchema :1;
+} _AlterISchemaRequest__isset;
+
+class AlterISchemaRequest {
+ public:
+
+  AlterISchemaRequest(const AlterISchemaRequest&);
+  AlterISchemaRequest& operator=(const AlterISchemaRequest&);
+  AlterISchemaRequest() {
+  }
+
+  virtual ~AlterISchemaRequest() throw();
+  ISchemaName name;
+  ISchema newSchema;
+
+  _AlterISchemaRequest__isset __isset;
+
+  void __set_name(const ISchemaName& val);
+
+  void __set_newSchema(const ISchema& val);
+
+  bool operator == (const AlterISchemaRequest & rhs) const
+  {
+    if (!(name == rhs.name))
+      return false;
+    if (!(newSchema == rhs.newSchema))
+      return false;
+    return true;
+  }
+  bool operator != (const AlterISchemaRequest &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const AlterISchemaRequest & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  virtual void printTo(std::ostream& out) const;
+};
+
+void swap(AlterISchemaRequest &a, AlterISchemaRequest &b);
+
+inline std::ostream& operator<<(std::ostream& out, const AlterISchemaRequest& obj)
+{
+  obj.printTo(out);
+  return out;
+}
+
+typedef struct _SchemaVersion__isset {
+  _SchemaVersion__isset() : schema(false), version(false), createdAt(false), cols(false), state(false), description(false), schemaText(false), fingerprint(false), name(false), serDe(false) {}
+  bool schema :1;
+  bool version :1;
+  bool createdAt :1;
+  bool cols :1;
+  bool state :1;
+  bool description :1;
+  bool schemaText :1;
+  bool fingerprint :1;
+  bool name :1;
+  bool serDe :1;
+} _SchemaVersion__isset;
+
+class SchemaVersion {
+ public:
+
+  SchemaVersion(const SchemaVersion&);
+  SchemaVersion& operator=(const SchemaVersion&);
+  SchemaVersion() : version(0), createdAt(0), state((SchemaVersionState::type)0), description(), schemaText(), fingerprint(), name() {
+  }
+
+  virtual ~SchemaVersion() throw();
+  ISchemaName schema;
+  int32_t version;
+  int64_t createdAt;
+  std::vector<FieldSchema>  cols;
+  SchemaVersionState::type state;
+  std::string description;
+  std::string schemaText;
+  std::string fingerprint;
+  std::string name;
+  SerDeInfo serDe;
+
+  _SchemaVersion__isset __isset;
+
+  void __set_schema(const ISchemaName& val);
+
+  void __set_version(const int32_t val);
+
+  void __set_createdAt(const int64_t val);
+
+  void __set_cols(const std::vector<FieldSchema> & val);
+
+  void __set_state(const SchemaVersionState::type val);
+
+  void __set_description(const std::string& val);
+
+  void __set_schemaText(const std::string& val);
+
+  void __set_fingerprint(const std::string& val);
+
+  void __set_name(const std::string& val);
+
+  void __set_serDe(const SerDeInfo& val);
+
+  bool operator == (const SchemaVersion & rhs) const
+  {
+    if (!(schema == rhs.schema))
+      return false;
+    if (!(version == rhs.version))
+      return false;
+    if (!(createdAt == rhs.createdAt))
+      return false;
+    if (!(cols == rhs.cols))
+      return false;
+    if (__isset.state != rhs.__isset.state)
+      return false;
+    else if (__isset.state && !(state == rhs.state))
+      return false;
+    if (__isset.description != rhs.__isset.description)
+      return false;
+    else if (__isset.description && !(description == rhs.description))
+      return false;
+    if (__isset.schemaText != rhs.__isset.schemaText)
+      return false;
+    else if (__isset.schemaText && !(schemaText == rhs.schemaText))
+      return false;
+    if (__isset.fingerprint != rhs.__isset.fingerprint)
+      return false;
+    else if (__isset.fingerprint && !(fingerprint == rhs.fingerprint))
+      return false;
+    if (__isset.name != rhs.__isset.name)
+      return false;
+    else if (__isset.name && !(name == rhs.name))
+      return false;
+    if (__isset.serDe != rhs.__isset.serDe)
+      return false;
+    else if (__isset.serDe && !(serDe == rhs.serDe))
+      return false;
+    return true;
+  }
+  bool operator != (const SchemaVersion &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const SchemaVersion & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  virtual void printTo(std::ostream& out) const;
+};
+
+void swap(SchemaVersion &a, SchemaVersion &b);
+
+inline std::ostream& operator<<(std::ostream& out, const SchemaVersion& obj)
+{
+  obj.printTo(out);
+  return out;
+}
+
+typedef struct _SchemaVersionDescriptor__isset {
+  _SchemaVersionDescriptor__isset() : schema(false), version(false) {}
+  bool schema :1;
+  bool version :1;
+} _SchemaVersionDescriptor__isset;
+
+class SchemaVersionDescriptor {
+ public:
+
+  SchemaVersionDescriptor(const SchemaVersionDescriptor&);
+  SchemaVersionDescriptor& operator=(const SchemaVersionDescriptor&);
+  SchemaVersionDescriptor() : version(0) {
+  }
+
+  virtual ~SchemaVersionDescriptor() throw();
+  ISchemaName schema;
+  int32_t version;
+
+  _SchemaVersionDescriptor__isset __isset;
+
+  void __set_schema(const ISchemaName& val);
+
+  void __set_version(const int32_t val);
+
+  bool operator == (const SchemaVersionDescriptor & rhs) const
+  {
+    if (!(schema == rhs.schema))
+      return false;
+    if (!(version == rhs.version))
+      return false;
+    return true;
+  }
+  bool operator != (const SchemaVersionDescriptor &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const SchemaVersionDescriptor & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  virtual void printTo(std::ostream& out) const;
+};
+
+void swap(SchemaVersionDescriptor &a, SchemaVersionDescriptor &b);
+
+inline std::ostream& operator<<(std::ostream& out, const SchemaVersionDescriptor& obj)
+{
+  obj.printTo(out);
+  return out;
+}
+
+typedef struct _FindSchemasByColsRqst__isset {
+  _FindSchemasByColsRqst__isset() : colName(false), colNamespace(false), type(false) {}
+  bool colName :1;
+  bool colNamespace :1;
+  bool type :1;
+} _FindSchemasByColsRqst__isset;
+
+class FindSchemasByColsRqst {
+ public:
+
+  FindSchemasByColsRqst(const FindSchemasByColsRqst&);
+  FindSchemasByColsRqst& operator=(const FindSchemasByColsRqst&);
+  FindSchemasByColsRqst() : colName(), colNamespace(), type() {
+  }
+
+  virtual ~FindSchemasByColsRqst() throw();
+  std::string colName;
+  std::string colNamespace;
+  std::string type;
+
+  _FindSchemasByColsRqst__isset __isset;
+
+  void __set_colName(const std::string& val);
+
+  void __set_colNamespace(const std::string& val);
+
+  void __set_type(const std::string& val);
+
+  bool operator == (const FindSchemasByColsRqst & rhs) const
+  {
+    if (__isset.colName != rhs.__isset.colName)
+      return false;
+    else if (__isset.colName && !(colName == rhs.colName))
+      return false;
+    if (__isset.colNamespace != rhs.__isset.colNamespace)
+      return false;
+    else if (__isset.colNamespace && !(colNamespace == rhs.colNamespace))
+      return false;
+    if (__isset.type != rhs.__isset.type)
+      return false;
+    else if (__isset.type && !(type == rhs.type))
+      return false;
+    return true;
+  }
+  bool operator != (const FindSchemasByColsRqst &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const FindSchemasByColsRqst & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  virtual void printTo(std::ostream& out) const;
+};
+
+void swap(FindSchemasByColsRqst &a, FindSchemasByColsRqst &b);
+
+inline std::ostream& operator<<(std::ostream& out, const FindSchemasByColsRqst& obj)
+{
+  obj.printTo(out);
+  return out;
+}
+
+typedef struct _FindSchemasByColsResp__isset {
+  _FindSchemasByColsResp__isset() : schemaVersions(false) {}
+  bool schemaVersions :1;
+} _FindSchemasByColsResp__isset;
+
+class FindSchemasByColsResp {
+ public:
+
+  FindSchemasByColsResp(const FindSchemasByColsResp&);
+  FindSchemasByColsResp& operator=(const FindSchemasByColsResp&);
+  FindSchemasByColsResp() {
+  }
+
+  virtual ~FindSchemasByColsResp() throw();
+  std::vector<SchemaVersionDescriptor>  schemaVersions;
+
+  _FindSchemasByColsResp__isset __isset;
+
+  void __set_schemaVersions(const std::vector<SchemaVersionDescriptor> & val);
+
+  bool operator == (const FindSchemasByColsResp & rhs) const
+  {
+    if (!(schemaVersions == rhs.schemaVersions))
+      return false;
+    return true;
+  }
+  bool operator != (const FindSchemasByColsResp &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const FindSchemasByColsResp & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  virtual void printTo(std::ostream& out) const;
+};
+
+void swap(FindSchemasByColsResp &a, FindSchemasByColsResp &b);
+
+inline std::ostream& operator<<(std::ostream& out, const FindSchemasByColsResp& obj)
+{
+  obj.printTo(out);
+  return out;
+}
+
+typedef struct _MapSchemaVersionToSerdeRequest__isset {
+  _MapSchemaVersionToSerdeRequest__isset() : schemaVersion(false), serdeName(false) {}
+  bool schemaVersion :1;
+  bool serdeName :1;
+} _MapSchemaVersionToSerdeRequest__isset;
+
+class MapSchemaVersionToSerdeRequest {
+ public:
+
+  MapSchemaVersionToSerdeRequest(const MapSchemaVersionToSerdeRequest&);
+  MapSchemaVersionToSerdeRequest& operator=(const MapSchemaVersionToSerdeRequest&);
+  MapSchemaVersionToSerdeRequest() : serdeName() {
+  }
+
+  virtual ~MapSchemaVersionToSerdeRequest() throw();
+  SchemaVersionDescriptor schemaVersion;
+  std::string serdeName;
+
+  _MapSchemaVersionToSerdeRequest__isset __isset;
+
+  void __set_schemaVersion(const SchemaVersionDescriptor& val);
+
+  void __set_serdeName(const std::string& val);
+
+  bool operator == (const MapSchemaVersionToSerdeRequest & rhs) const
+  {
+    if (!(schemaVersion == rhs.schemaVersion))
+      return false;
+    if (!(serdeName == rhs.serdeName))
+      return false;
+    return true;
+  }
+  bool operator != (const MapSchemaVersionToSerdeRequest &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const MapSchemaVersionToSerdeRequest & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  virtual void printTo(std::ostream& out) const;
+};
+
+void swap(MapSchemaVersionToSerdeRequest &a, MapSchemaVersionToSerdeRequest &b);
+
+inline std::ostream& operator<<(std::ostream& out, const MapSchemaVersionToSerdeRequest& obj)
+{
+  obj.printTo(out);
+  return out;
+}
+
+typedef struct _SetSchemaVersionStateRequest__isset {
+  _SetSchemaVersionStateRequest__isset() : schemaVersion(false), state(false) {}
+  bool schemaVersion :1;
+  bool state :1;
+} _SetSchemaVersionStateRequest__isset;
+
+class SetSchemaVersionStateRequest {
+ public:
+
+  SetSchemaVersionStateRequest(const SetSchemaVersionStateRequest&);
+  SetSchemaVersionStateRequest& operator=(const SetSchemaVersionStateRequest&);
+  SetSchemaVersionStateRequest() : state((SchemaVersionState::type)0) {
+  }
+
+  virtual ~SetSchemaVersionStateRequest() throw();
+  SchemaVersionDescriptor schemaVersion;
+  SchemaVersionState::type state;
+
+  _SetSchemaVersionStateRequest__isset __isset;
+
+  void __set_schemaVersion(const SchemaVersionDescriptor& val);
+
+  void __set_state(const SchemaVersionState::type val);
+
+  bool operator == (const SetSchemaVersionStateRequest & rhs) const
+  {
+    if (!(schemaVersion == rhs.schemaVersion))
+      return false;
+    if (!(state == rhs.state))
+      return false;
+    return true;
+  }
+  bool operator != (const SetSchemaVersionStateRequest &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const SetSchemaVersionStateRequest & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  virtual void printTo(std::ostream& out) const;
+};
+
+void swap(SetSchemaVersionStateRequest &a, SetSchemaVersionStateRequest &b);
+
+inline std::ostream& operator<<(std::ostream& out, const SetSchemaVersionStateRequest& obj)
+{
+  obj.printTo(out);
+  return out;
+}
+
+typedef struct _GetSerdeRequest__isset {
+  _GetSerdeRequest__isset() : serdeName(false) {}
+  bool serdeName :1;
+} _GetSerdeRequest__isset;
+
+class GetSerdeRequest {
+ public:
+
+  GetSerdeRequest(const GetSerdeRequest&);
+  GetSerdeRequest& operator=(const GetSerdeRequest&);
+  GetSerdeRequest() : serdeName() {
+  }
+
+  virtual ~GetSerdeRequest() throw();
+  std::string serdeName;
+
+  _GetSerdeRequest__isset __isset;
+
+  void __set_serdeName(const std::string& val);
+
+  bool operator == (const GetSerdeRequest & rhs) const
+  {
+    if (!(serdeName == rhs.serdeName))
+      return false;
+    return true;
+  }
+  bool operator != (const GetSerdeRequest &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const GetSerdeRequest & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  virtual void printTo(std::ostream& out) const;
+};
+
+void swap(GetSerdeRequest &a, GetSerdeRequest &b);
+
+inline std::ostream& operator<<(std::ostream& out, const GetSerdeRequest& obj)
+{
+  obj.printTo(out);
+  return out;
+}
+
 typedef struct _MetaException__isset {
   _MetaException__isset() : message(false) {}
   bool message :1;

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AlterISchemaRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AlterISchemaRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AlterISchemaRequest.java
new file mode 100644
index 0000000..c079fab
--- /dev/null
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AlterISchemaRequest.java
@@ -0,0 +1,509 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hive.metastore.api;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+@org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public class AlterISchemaRequest implements org.apache.thrift.TBase<AlterISchemaRequest, AlterISchemaRequest._Fields>, java.io.Serializable, Cloneable, Comparable<AlterISchemaRequest> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("AlterISchemaRequest");
+
+  private static final org.apache.thrift.protocol.TField NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("name", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+  private static final org.apache.thrift.protocol.TField NEW_SCHEMA_FIELD_DESC = new org.apache.thrift.protocol.TField("newSchema", org.apache.thrift.protocol.TType.STRUCT, (short)3);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new AlterISchemaRequestStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new AlterISchemaRequestTupleSchemeFactory());
+  }
+
+  private ISchemaName name; // required
+  private ISchema newSchema; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    NAME((short)1, "name"),
+    NEW_SCHEMA((short)3, "newSchema");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // NAME
+          return NAME;
+        case 3: // NEW_SCHEMA
+          return NEW_SCHEMA;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.NAME, new org.apache.thrift.meta_data.FieldMetaData("name", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ISchemaName.class)));
+    tmpMap.put(_Fields.NEW_SCHEMA, new org.apache.thrift.meta_data.FieldMetaData("newSchema", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ISchema.class)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(AlterISchemaRequest.class, metaDataMap);
+  }
+
+  public AlterISchemaRequest() {
+  }
+
+  public AlterISchemaRequest(
+    ISchemaName name,
+    ISchema newSchema)
+  {
+    this();
+    this.name = name;
+    this.newSchema = newSchema;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public AlterISchemaRequest(AlterISchemaRequest other) {
+    if (other.isSetName()) {
+      this.name = new ISchemaName(other.name);
+    }
+    if (other.isSetNewSchema()) {
+      this.newSchema = new ISchema(other.newSchema);
+    }
+  }
+
+  public AlterISchemaRequest deepCopy() {
+    return new AlterISchemaRequest(this);
+  }
+
+  @Override
+  public void clear() {
+    this.name = null;
+    this.newSchema = null;
+  }
+
+  public ISchemaName getName() {
+    return this.name;
+  }
+
+  public void setName(ISchemaName name) {
+    this.name = name;
+  }
+
+  public void unsetName() {
+    this.name = null;
+  }
+
+  /** Returns true if field name is set (has been assigned a value) and false otherwise */
+  public boolean isSetName() {
+    return this.name != null;
+  }
+
+  public void setNameIsSet(boolean value) {
+    if (!value) {
+      this.name = null;
+    }
+  }
+
+  public ISchema getNewSchema() {
+    return this.newSchema;
+  }
+
+  public void setNewSchema(ISchema newSchema) {
+    this.newSchema = newSchema;
+  }
+
+  public void unsetNewSchema() {
+    this.newSchema = null;
+  }
+
+  /** Returns true if field newSchema is set (has been assigned a value) and false otherwise */
+  public boolean isSetNewSchema() {
+    return this.newSchema != null;
+  }
+
+  public void setNewSchemaIsSet(boolean value) {
+    if (!value) {
+      this.newSchema = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case NAME:
+      if (value == null) {
+        unsetName();
+      } else {
+        setName((ISchemaName)value);
+      }
+      break;
+
+    case NEW_SCHEMA:
+      if (value == null) {
+        unsetNewSchema();
+      } else {
+        setNewSchema((ISchema)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case NAME:
+      return getName();
+
+    case NEW_SCHEMA:
+      return getNewSchema();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case NAME:
+      return isSetName();
+    case NEW_SCHEMA:
+      return isSetNewSchema();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof AlterISchemaRequest)
+      return this.equals((AlterISchemaRequest)that);
+    return false;
+  }
+
+  public boolean equals(AlterISchemaRequest that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_name = true && this.isSetName();
+    boolean that_present_name = true && that.isSetName();
+    if (this_present_name || that_present_name) {
+      if (!(this_present_name && that_present_name))
+        return false;
+      if (!this.name.equals(that.name))
+        return false;
+    }
+
+    boolean this_present_newSchema = true && this.isSetNewSchema();
+    boolean that_present_newSchema = true && that.isSetNewSchema();
+    if (this_present_newSchema || that_present_newSchema) {
+      if (!(this_present_newSchema && that_present_newSchema))
+        return false;
+      if (!this.newSchema.equals(that.newSchema))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_name = true && (isSetName());
+    list.add(present_name);
+    if (present_name)
+      list.add(name);
+
+    boolean present_newSchema = true && (isSetNewSchema());
+    list.add(present_newSchema);
+    if (present_newSchema)
+      list.add(newSchema);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(AlterISchemaRequest other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetName()).compareTo(other.isSetName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.name, other.name);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetNewSchema()).compareTo(other.isSetNewSchema());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetNewSchema()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.newSchema, other.newSchema);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("AlterISchemaRequest(");
+    boolean first = true;
+
+    sb.append("name:");
+    if (this.name == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.name);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("newSchema:");
+    if (this.newSchema == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.newSchema);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+    if (name != null) {
+      name.validate();
+    }
+    if (newSchema != null) {
+      newSchema.validate();
+    }
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class AlterISchemaRequestStandardSchemeFactory implements SchemeFactory {
+    public AlterISchemaRequestStandardScheme getScheme() {
+      return new AlterISchemaRequestStandardScheme();
+    }
+  }
+
+  private static class AlterISchemaRequestStandardScheme extends StandardScheme<AlterISchemaRequest> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, AlterISchemaRequest struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.name = new ISchemaName();
+              struct.name.read(iprot);
+              struct.setNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // NEW_SCHEMA
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.newSchema = new ISchema();
+              struct.newSchema.read(iprot);
+              struct.setNewSchemaIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, AlterISchemaRequest struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.name != null) {
+        oprot.writeFieldBegin(NAME_FIELD_DESC);
+        struct.name.write(oprot);
+        oprot.writeFieldEnd();
+      }
+      if (struct.newSchema != null) {
+        oprot.writeFieldBegin(NEW_SCHEMA_FIELD_DESC);
+        struct.newSchema.write(oprot);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class AlterISchemaRequestTupleSchemeFactory implements SchemeFactory {
+    public AlterISchemaRequestTupleScheme getScheme() {
+      return new AlterISchemaRequestTupleScheme();
+    }
+  }
+
+  private static class AlterISchemaRequestTupleScheme extends TupleScheme<AlterISchemaRequest> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, AlterISchemaRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.isSetName()) {
+        optionals.set(0);
+      }
+      if (struct.isSetNewSchema()) {
+        optionals.set(1);
+      }
+      oprot.writeBitSet(optionals, 2);
+      if (struct.isSetName()) {
+        struct.name.write(oprot);
+      }
+      if (struct.isSetNewSchema()) {
+        struct.newSchema.write(oprot);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, AlterISchemaRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(2);
+      if (incoming.get(0)) {
+        struct.name = new ISchemaName();
+        struct.name.read(iprot);
+        struct.setNameIsSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.newSchema = new ISchema();
+        struct.newSchema.read(iprot);
+        struct.setNewSchemaIsSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FindSchemasByColsResp.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FindSchemasByColsResp.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FindSchemasByColsResp.java
new file mode 100644
index 0000000..01bc32d
--- /dev/null
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FindSchemasByColsResp.java
@@ -0,0 +1,449 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hive.metastore.api;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+@org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public class FindSchemasByColsResp implements org.apache.thrift.TBase<FindSchemasByColsResp, FindSchemasByColsResp._Fields>, java.io.Serializable, Cloneable, Comparable<FindSchemasByColsResp> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("FindSchemasByColsResp");
+
+  private static final org.apache.thrift.protocol.TField SCHEMA_VERSIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("schemaVersions", org.apache.thrift.protocol.TType.LIST, (short)1);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new FindSchemasByColsRespStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new FindSchemasByColsRespTupleSchemeFactory());
+  }
+
+  private List<SchemaVersionDescriptor> schemaVersions; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    SCHEMA_VERSIONS((short)1, "schemaVersions");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // SCHEMA_VERSIONS
+          return SCHEMA_VERSIONS;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.SCHEMA_VERSIONS, new org.apache.thrift.meta_data.FieldMetaData("schemaVersions", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, SchemaVersionDescriptor.class))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(FindSchemasByColsResp.class, metaDataMap);
+  }
+
+  public FindSchemasByColsResp() {
+  }
+
+  public FindSchemasByColsResp(
+    List<SchemaVersionDescriptor> schemaVersions)
+  {
+    this();
+    this.schemaVersions = schemaVersions;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public FindSchemasByColsResp(FindSchemasByColsResp other) {
+    if (other.isSetSchemaVersions()) {
+      List<SchemaVersionDescriptor> __this__schemaVersions = new ArrayList<SchemaVersionDescriptor>(other.schemaVersions.size());
+      for (SchemaVersionDescriptor other_element : other.schemaVersions) {
+        __this__schemaVersions.add(new SchemaVersionDescriptor(other_element));
+      }
+      this.schemaVersions = __this__schemaVersions;
+    }
+  }
+
+  public FindSchemasByColsResp deepCopy() {
+    return new FindSchemasByColsResp(this);
+  }
+
+  @Override
+  public void clear() {
+    this.schemaVersions = null;
+  }
+
+  public int getSchemaVersionsSize() {
+    return (this.schemaVersions == null) ? 0 : this.schemaVersions.size();
+  }
+
+  public java.util.Iterator<SchemaVersionDescriptor> getSchemaVersionsIterator() {
+    return (this.schemaVersions == null) ? null : this.schemaVersions.iterator();
+  }
+
+  public void addToSchemaVersions(SchemaVersionDescriptor elem) {
+    if (this.schemaVersions == null) {
+      this.schemaVersions = new ArrayList<SchemaVersionDescriptor>();
+    }
+    this.schemaVersions.add(elem);
+  }
+
+  public List<SchemaVersionDescriptor> getSchemaVersions() {
+    return this.schemaVersions;
+  }
+
+  public void setSchemaVersions(List<SchemaVersionDescriptor> schemaVersions) {
+    this.schemaVersions = schemaVersions;
+  }
+
+  public void unsetSchemaVersions() {
+    this.schemaVersions = null;
+  }
+
+  /** Returns true if field schemaVersions is set (has been assigned a value) and false otherwise */
+  public boolean isSetSchemaVersions() {
+    return this.schemaVersions != null;
+  }
+
+  public void setSchemaVersionsIsSet(boolean value) {
+    if (!value) {
+      this.schemaVersions = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case SCHEMA_VERSIONS:
+      if (value == null) {
+        unsetSchemaVersions();
+      } else {
+        setSchemaVersions((List<SchemaVersionDescriptor>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case SCHEMA_VERSIONS:
+      return getSchemaVersions();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case SCHEMA_VERSIONS:
+      return isSetSchemaVersions();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof FindSchemasByColsResp)
+      return this.equals((FindSchemasByColsResp)that);
+    return false;
+  }
+
+  public boolean equals(FindSchemasByColsResp that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_schemaVersions = true && this.isSetSchemaVersions();
+    boolean that_present_schemaVersions = true && that.isSetSchemaVersions();
+    if (this_present_schemaVersions || that_present_schemaVersions) {
+      if (!(this_present_schemaVersions && that_present_schemaVersions))
+        return false;
+      if (!this.schemaVersions.equals(that.schemaVersions))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_schemaVersions = true && (isSetSchemaVersions());
+    list.add(present_schemaVersions);
+    if (present_schemaVersions)
+      list.add(schemaVersions);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(FindSchemasByColsResp other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetSchemaVersions()).compareTo(other.isSetSchemaVersions());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetSchemaVersions()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.schemaVersions, other.schemaVersions);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("FindSchemasByColsResp(");
+    boolean first = true;
+
+    sb.append("schemaVersions:");
+    if (this.schemaVersions == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.schemaVersions);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class FindSchemasByColsRespStandardSchemeFactory implements SchemeFactory {
+    public FindSchemasByColsRespStandardScheme getScheme() {
+      return new FindSchemasByColsRespStandardScheme();
+    }
+  }
+
+  private static class FindSchemasByColsRespStandardScheme extends StandardScheme<FindSchemasByColsResp> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, FindSchemasByColsResp struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // SCHEMA_VERSIONS
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list872 = iprot.readListBegin();
+                struct.schemaVersions = new ArrayList<SchemaVersionDescriptor>(_list872.size);
+                SchemaVersionDescriptor _elem873;
+                for (int _i874 = 0; _i874 < _list872.size; ++_i874)
+                {
+                  _elem873 = new SchemaVersionDescriptor();
+                  _elem873.read(iprot);
+                  struct.schemaVersions.add(_elem873);
+                }
+                iprot.readListEnd();
+              }
+              struct.setSchemaVersionsIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, FindSchemasByColsResp struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.schemaVersions != null) {
+        oprot.writeFieldBegin(SCHEMA_VERSIONS_FIELD_DESC);
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.schemaVersions.size()));
+          for (SchemaVersionDescriptor _iter875 : struct.schemaVersions)
+          {
+            _iter875.write(oprot);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class FindSchemasByColsRespTupleSchemeFactory implements SchemeFactory {
+    public FindSchemasByColsRespTupleScheme getScheme() {
+      return new FindSchemasByColsRespTupleScheme();
+    }
+  }
+
+  private static class FindSchemasByColsRespTupleScheme extends TupleScheme<FindSchemasByColsResp> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, FindSchemasByColsResp struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.isSetSchemaVersions()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.isSetSchemaVersions()) {
+        {
+          oprot.writeI32(struct.schemaVersions.size());
+          for (SchemaVersionDescriptor _iter876 : struct.schemaVersions)
+          {
+            _iter876.write(oprot);
+          }
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, FindSchemasByColsResp struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        {
+          org.apache.thrift.protocol.TList _list877 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.schemaVersions = new ArrayList<SchemaVersionDescriptor>(_list877.size);
+          SchemaVersionDescriptor _elem878;
+          for (int _i879 = 0; _i879 < _list877.size; ++_i879)
+          {
+            _elem878 = new SchemaVersionDescriptor();
+            _elem878.read(iprot);
+            struct.schemaVersions.add(_elem878);
+          }
+        }
+        struct.setSchemaVersionsIsSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FindSchemasByColsRqst.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FindSchemasByColsRqst.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FindSchemasByColsRqst.java
new file mode 100644
index 0000000..d5230c9
--- /dev/null
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FindSchemasByColsRqst.java
@@ -0,0 +1,605 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hive.metastore.api;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+@org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public class FindSchemasByColsRqst implements org.apache.thrift.TBase<FindSchemasByColsRqst, FindSchemasByColsRqst._Fields>, java.io.Serializable, Cloneable, Comparable<FindSchemasByColsRqst> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("FindSchemasByColsRqst");
+
+  private static final org.apache.thrift.protocol.TField COL_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("colName", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField COL_NAMESPACE_FIELD_DESC = new org.apache.thrift.protocol.TField("colNamespace", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("type", org.apache.thrift.protocol.TType.STRING, (short)3);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new FindSchemasByColsRqstStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new FindSchemasByColsRqstTupleSchemeFactory());
+  }
+
+  private String colName; // optional
+  private String colNamespace; // optional
+  private String type; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    COL_NAME((short)1, "colName"),
+    COL_NAMESPACE((short)2, "colNamespace"),
+    TYPE((short)3, "type");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // COL_NAME
+          return COL_NAME;
+        case 2: // COL_NAMESPACE
+          return COL_NAMESPACE;
+        case 3: // TYPE
+          return TYPE;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final _Fields optionals[] = {_Fields.COL_NAME,_Fields.COL_NAMESPACE,_Fields.TYPE};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.COL_NAME, new org.apache.thrift.meta_data.FieldMetaData("colName", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.COL_NAMESPACE, new org.apache.thrift.meta_data.FieldMetaData("colNamespace", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.TYPE, new org.apache.thrift.meta_data.FieldMetaData("type", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(FindSchemasByColsRqst.class, metaDataMap);
+  }
+
+  public FindSchemasByColsRqst() {
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public FindSchemasByColsRqst(FindSchemasByColsRqst other) {
+    if (other.isSetColName()) {
+      this.colName = other.colName;
+    }
+    if (other.isSetColNamespace()) {
+      this.colNamespace = other.colNamespace;
+    }
+    if (other.isSetType()) {
+      this.type = other.type;
+    }
+  }
+
+  public FindSchemasByColsRqst deepCopy() {
+    return new FindSchemasByColsRqst(this);
+  }
+
+  @Override
+  public void clear() {
+    this.colName = null;
+    this.colNamespace = null;
+    this.type = null;
+  }
+
+  public String getColName() {
+    return this.colName;
+  }
+
+  public void setColName(String colName) {
+    this.colName = colName;
+  }
+
+  public void unsetColName() {
+    this.colName = null;
+  }
+
+  /** Returns true if field colName is set (has been assigned a value) and false otherwise */
+  public boolean isSetColName() {
+    return this.colName != null;
+  }
+
+  public void setColNameIsSet(boolean value) {
+    if (!value) {
+      this.colName = null;
+    }
+  }
+
+  public String getColNamespace() {
+    return this.colNamespace;
+  }
+
+  public void setColNamespace(String colNamespace) {
+    this.colNamespace = colNamespace;
+  }
+
+  public void unsetColNamespace() {
+    this.colNamespace = null;
+  }
+
+  /** Returns true if field colNamespace is set (has been assigned a value) and false otherwise */
+  public boolean isSetColNamespace() {
+    return this.colNamespace != null;
+  }
+
+  public void setColNamespaceIsSet(boolean value) {
+    if (!value) {
+      this.colNamespace = null;
+    }
+  }
+
+  public String getType() {
+    return this.type;
+  }
+
+  public void setType(String type) {
+    this.type = type;
+  }
+
+  public void unsetType() {
+    this.type = null;
+  }
+
+  /** Returns true if field type is set (has been assigned a value) and false otherwise */
+  public boolean isSetType() {
+    return this.type != null;
+  }
+
+  public void setTypeIsSet(boolean value) {
+    if (!value) {
+      this.type = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case COL_NAME:
+      if (value == null) {
+        unsetColName();
+      } else {
+        setColName((String)value);
+      }
+      break;
+
+    case COL_NAMESPACE:
+      if (value == null) {
+        unsetColNamespace();
+      } else {
+        setColNamespace((String)value);
+      }
+      break;
+
+    case TYPE:
+      if (value == null) {
+        unsetType();
+      } else {
+        setType((String)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case COL_NAME:
+      return getColName();
+
+    case COL_NAMESPACE:
+      return getColNamespace();
+
+    case TYPE:
+      return getType();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case COL_NAME:
+      return isSetColName();
+    case COL_NAMESPACE:
+      return isSetColNamespace();
+    case TYPE:
+      return isSetType();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof FindSchemasByColsRqst)
+      return this.equals((FindSchemasByColsRqst)that);
+    return false;
+  }
+
+  public boolean equals(FindSchemasByColsRqst that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_colName = true && this.isSetColName();
+    boolean that_present_colName = true && that.isSetColName();
+    if (this_present_colName || that_present_colName) {
+      if (!(this_present_colName && that_present_colName))
+        return false;
+      if (!this.colName.equals(that.colName))
+        return false;
+    }
+
+    boolean this_present_colNamespace = true && this.isSetColNamespace();
+    boolean that_present_colNamespace = true && that.isSetColNamespace();
+    if (this_present_colNamespace || that_present_colNamespace) {
+      if (!(this_present_colNamespace && that_present_colNamespace))
+        return false;
+      if (!this.colNamespace.equals(that.colNamespace))
+        return false;
+    }
+
+    boolean this_present_type = true && this.isSetType();
+    boolean that_present_type = true && that.isSetType();
+    if (this_present_type || that_present_type) {
+      if (!(this_present_type && that_present_type))
+        return false;
+      if (!this.type.equals(that.type))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_colName = true && (isSetColName());
+    list.add(present_colName);
+    if (present_colName)
+      list.add(colName);
+
+    boolean present_colNamespace = true && (isSetColNamespace());
+    list.add(present_colNamespace);
+    if (present_colNamespace)
+      list.add(colNamespace);
+
+    boolean present_type = true && (isSetType());
+    list.add(present_type);
+    if (present_type)
+      list.add(type);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(FindSchemasByColsRqst other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetColName()).compareTo(other.isSetColName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetColName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.colName, other.colName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetColNamespace()).compareTo(other.isSetColNamespace());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetColNamespace()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.colNamespace, other.colNamespace);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetType()).compareTo(other.isSetType());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetType()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.type, other.type);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("FindSchemasByColsRqst(");
+    boolean first = true;
+
+    if (isSetColName()) {
+      sb.append("colName:");
+      if (this.colName == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.colName);
+      }
+      first = false;
+    }
+    if (isSetColNamespace()) {
+      if (!first) sb.append(", ");
+      sb.append("colNamespace:");
+      if (this.colNamespace == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.colNamespace);
+      }
+      first = false;
+    }
+    if (isSetType()) {
+      if (!first) sb.append(", ");
+      sb.append("type:");
+      if (this.type == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.type);
+      }
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class FindSchemasByColsRqstStandardSchemeFactory implements SchemeFactory {
+    public FindSchemasByColsRqstStandardScheme getScheme() {
+      return new FindSchemasByColsRqstStandardScheme();
+    }
+  }
+
+  private static class FindSchemasByColsRqstStandardScheme extends StandardScheme<FindSchemasByColsRqst> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, FindSchemasByColsRqst struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // COL_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.colName = iprot.readString();
+              struct.setColNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // COL_NAMESPACE
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.colNamespace = iprot.readString();
+              struct.setColNamespaceIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // TYPE
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.type = iprot.readString();
+              struct.setTypeIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, FindSchemasByColsRqst struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.colName != null) {
+        if (struct.isSetColName()) {
+          oprot.writeFieldBegin(COL_NAME_FIELD_DESC);
+          oprot.writeString(struct.colName);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.colNamespace != null) {
+        if (struct.isSetColNamespace()) {
+          oprot.writeFieldBegin(COL_NAMESPACE_FIELD_DESC);
+          oprot.writeString(struct.colNamespace);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.type != null) {
+        if (struct.isSetType()) {
+          oprot.writeFieldBegin(TYPE_FIELD_DESC);
+          oprot.writeString(struct.type);
+          oprot.writeFieldEnd();
+        }
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class FindSchemasByColsRqstTupleSchemeFactory implements SchemeFactory {
+    public FindSchemasByColsRqstTupleScheme getScheme() {
+      return new FindSchemasByColsRqstTupleScheme();
+    }
+  }
+
+  private static class FindSchemasByColsRqstTupleScheme extends TupleScheme<FindSchemasByColsRqst> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, FindSchemasByColsRqst struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.isSetColName()) {
+        optionals.set(0);
+      }
+      if (struct.isSetColNamespace()) {
+        optionals.set(1);
+      }
+      if (struct.isSetType()) {
+        optionals.set(2);
+      }
+      oprot.writeBitSet(optionals, 3);
+      if (struct.isSetColName()) {
+        oprot.writeString(struct.colName);
+      }
+      if (struct.isSetColNamespace()) {
+        oprot.writeString(struct.colNamespace);
+      }
+      if (struct.isSetType()) {
+        oprot.writeString(struct.type);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, FindSchemasByColsRqst struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(3);
+      if (incoming.get(0)) {
+        struct.colName = iprot.readString();
+        struct.setColNameIsSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.colNamespace = iprot.readString();
+        struct.setColNamespaceIsSet(true);
+      }
+      if (incoming.get(2)) {
+        struct.type = iprot.readString();
+        struct.setTypeIsSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetSerdeRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetSerdeRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetSerdeRequest.java
new file mode 100644
index 0000000..bf0a595
--- /dev/null
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetSerdeRequest.java
@@ -0,0 +1,395 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hive.metastore.api;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+@org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public class GetSerdeRequest implements org.apache.thrift.TBase<GetSerdeRequest, GetSerdeRequest._Fields>, java.io.Serializable, Cloneable, Comparable<GetSerdeRequest> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetSerdeRequest");
+
+  private static final org.apache.thrift.protocol.TField SERDE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("serdeName", org.apache.thrift.protocol.TType.STRING, (short)1);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new GetSerdeRequestStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new GetSerdeRequestTupleSchemeFactory());
+  }
+
+  private String serdeName; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    SERDE_NAME((short)1, "serdeName");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // SERDE_NAME
+          return SERDE_NAME;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.SERDE_NAME, new org.apache.thrift.meta_data.FieldMetaData("serdeName", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetSerdeRequest.class, metaDataMap);
+  }
+
+  public GetSerdeRequest() {
+  }
+
+  public GetSerdeRequest(
+    String serdeName)
+  {
+    this();
+    this.serdeName = serdeName;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public GetSerdeRequest(GetSerdeRequest other) {
+    if (other.isSetSerdeName()) {
+      this.serdeName = other.serdeName;
+    }
+  }
+
+  public GetSerdeRequest deepCopy() {
+    return new GetSerdeRequest(this);
+  }
+
+  @Override
+  public void clear() {
+    this.serdeName = null;
+  }
+
+  public String getSerdeName() {
+    return this.serdeName;
+  }
+
+  public void setSerdeName(String serdeName) {
+    this.serdeName = serdeName;
+  }
+
+  public void unsetSerdeName() {
+    this.serdeName = null;
+  }
+
+  /** Returns true if field serdeName is set (has been assigned a value) and false otherwise */
+  public boolean isSetSerdeName() {
+    return this.serdeName != null;
+  }
+
+  public void setSerdeNameIsSet(boolean value) {
+    if (!value) {
+      this.serdeName = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case SERDE_NAME:
+      if (value == null) {
+        unsetSerdeName();
+      } else {
+        setSerdeName((String)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case SERDE_NAME:
+      return getSerdeName();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case SERDE_NAME:
+      return isSetSerdeName();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof GetSerdeRequest)
+      return this.equals((GetSerdeRequest)that);
+    return false;
+  }
+
+  public boolean equals(GetSerdeRequest that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_serdeName = true && this.isSetSerdeName();
+    boolean that_present_serdeName = true && that.isSetSerdeName();
+    if (this_present_serdeName || that_present_serdeName) {
+      if (!(this_present_serdeName && that_present_serdeName))
+        return false;
+      if (!this.serdeName.equals(that.serdeName))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_serdeName = true && (isSetSerdeName());
+    list.add(present_serdeName);
+    if (present_serdeName)
+      list.add(serdeName);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(GetSerdeRequest other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetSerdeName()).compareTo(other.isSetSerdeName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetSerdeName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.serdeName, other.serdeName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("GetSerdeRequest(");
+    boolean first = true;
+
+    sb.append("serdeName:");
+    if (this.serdeName == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.serdeName);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class GetSerdeRequestStandardSchemeFactory implements SchemeFactory {
+    public GetSerdeRequestStandardScheme getScheme() {
+      return new GetSerdeRequestStandardScheme();
+    }
+  }
+
+  private static class GetSerdeRequestStandardScheme extends StandardScheme<GetSerdeRequest> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, GetSerdeRequest struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // SERDE_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.serdeName = iprot.readString();
+              struct.setSerdeNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, GetSerdeRequest struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.serdeName != null) {
+        oprot.writeFieldBegin(SERDE_NAME_FIELD_DESC);
+        oprot.writeString(struct.serdeName);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class GetSerdeRequestTupleSchemeFactory implements SchemeFactory {
+    public GetSerdeRequestTupleScheme getScheme() {
+      return new GetSerdeRequestTupleScheme();
+    }
+  }
+
+  private static class GetSerdeRequestTupleScheme extends TupleScheme<GetSerdeRequest> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, GetSerdeRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.isSetSerdeName()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.isSetSerdeName()) {
+        oprot.writeString(struct.serdeName);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, GetSerdeRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        struct.serdeName = iprot.readString();
+        struct.setSerdeNameIsSet(true);
+      }
+    }
+  }
+
+}
+


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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py b/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
index b24c5f9..4d4429f 100644
--- a/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
+++ b/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
@@ -211,6 +211,100 @@ class EventRequestType:
     "DELETE": 3,
   }
 
+class SerdeType:
+  HIVE = 1
+  SCHEMA_REGISTRY = 2
+
+  _VALUES_TO_NAMES = {
+    1: "HIVE",
+    2: "SCHEMA_REGISTRY",
+  }
+
+  _NAMES_TO_VALUES = {
+    "HIVE": 1,
+    "SCHEMA_REGISTRY": 2,
+  }
+
+class SchemaType:
+  HIVE = 1
+  AVRO = 2
+
+  _VALUES_TO_NAMES = {
+    1: "HIVE",
+    2: "AVRO",
+  }
+
+  _NAMES_TO_VALUES = {
+    "HIVE": 1,
+    "AVRO": 2,
+  }
+
+class SchemaCompatibility:
+  NONE = 1
+  BACKWARD = 2
+  FORWARD = 3
+  BOTH = 4
+
+  _VALUES_TO_NAMES = {
+    1: "NONE",
+    2: "BACKWARD",
+    3: "FORWARD",
+    4: "BOTH",
+  }
+
+  _NAMES_TO_VALUES = {
+    "NONE": 1,
+    "BACKWARD": 2,
+    "FORWARD": 3,
+    "BOTH": 4,
+  }
+
+class SchemaValidation:
+  LATEST = 1
+  ALL = 2
+
+  _VALUES_TO_NAMES = {
+    1: "LATEST",
+    2: "ALL",
+  }
+
+  _NAMES_TO_VALUES = {
+    "LATEST": 1,
+    "ALL": 2,
+  }
+
+class SchemaVersionState:
+  INITIATED = 1
+  START_REVIEW = 2
+  CHANGES_REQUIRED = 3
+  REVIEWED = 4
+  ENABLED = 5
+  DISABLED = 6
+  ARCHIVED = 7
+  DELETED = 8
+
+  _VALUES_TO_NAMES = {
+    1: "INITIATED",
+    2: "START_REVIEW",
+    3: "CHANGES_REQUIRED",
+    4: "REVIEWED",
+    5: "ENABLED",
+    6: "DISABLED",
+    7: "ARCHIVED",
+    8: "DELETED",
+  }
+
+  _NAMES_TO_VALUES = {
+    "INITIATED": 1,
+    "START_REVIEW": 2,
+    "CHANGES_REQUIRED": 3,
+    "REVIEWED": 4,
+    "ENABLED": 5,
+    "DISABLED": 6,
+    "ARCHIVED": 7,
+    "DELETED": 8,
+  }
+
 class FunctionType:
   JAVA = 1
 
@@ -3053,6 +3147,10 @@ class SerDeInfo:
    - name
    - serializationLib
    - parameters
+   - description
+   - serializerClass
+   - deserializerClass
+   - serdeType
   """
 
   thrift_spec = (
@@ -3060,12 +3158,20 @@ class SerDeInfo:
     (1, TType.STRING, 'name', None, None, ), # 1
     (2, TType.STRING, 'serializationLib', None, None, ), # 2
     (3, TType.MAP, 'parameters', (TType.STRING,None,TType.STRING,None), None, ), # 3
+    (4, TType.STRING, 'description', None, None, ), # 4
+    (5, TType.STRING, 'serializerClass', None, None, ), # 5
+    (6, TType.STRING, 'deserializerClass', None, None, ), # 6
+    (7, TType.I32, 'serdeType', None, None, ), # 7
   )
 
-  def __init__(self, name=None, serializationLib=None, parameters=None,):
+  def __init__(self, name=None, serializationLib=None, parameters=None, description=None, serializerClass=None, deserializerClass=None, serdeType=None,):
     self.name = name
     self.serializationLib = serializationLib
     self.parameters = parameters
+    self.description = description
+    self.serializerClass = serializerClass
+    self.deserializerClass = deserializerClass
+    self.serdeType = serdeType
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -3097,6 +3203,26 @@ class SerDeInfo:
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.STRING:
+          self.description = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 5:
+        if ftype == TType.STRING:
+          self.serializerClass = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 6:
+        if ftype == TType.STRING:
+          self.deserializerClass = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 7:
+        if ftype == TType.I32:
+          self.serdeType = iprot.readI32()
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -3123,6 +3249,22 @@ class SerDeInfo:
         oprot.writeString(viter100)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
+    if self.description is not None:
+      oprot.writeFieldBegin('description', TType.STRING, 4)
+      oprot.writeString(self.description)
+      oprot.writeFieldEnd()
+    if self.serializerClass is not None:
+      oprot.writeFieldBegin('serializerClass', TType.STRING, 5)
+      oprot.writeString(self.serializerClass)
+      oprot.writeFieldEnd()
+    if self.deserializerClass is not None:
+      oprot.writeFieldBegin('deserializerClass', TType.STRING, 6)
+      oprot.writeString(self.deserializerClass)
+      oprot.writeFieldEnd()
+    if self.serdeType is not None:
+      oprot.writeFieldBegin('serdeType', TType.I32, 7)
+      oprot.writeI32(self.serdeType)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -3135,6 +3277,10 @@ class SerDeInfo:
     value = (value * 31) ^ hash(self.name)
     value = (value * 31) ^ hash(self.serializationLib)
     value = (value * 31) ^ hash(self.parameters)
+    value = (value * 31) ^ hash(self.description)
+    value = (value * 31) ^ hash(self.serializerClass)
+    value = (value * 31) ^ hash(self.deserializerClass)
+    value = (value * 31) ^ hash(self.serdeType)
     return value
 
   def __repr__(self):
@@ -18732,6 +18878,981 @@ class WMCreateOrDropTriggerToPoolMappingResponse:
   def __ne__(self, other):
     return not (self == other)
 
+class ISchema:
+  """
+  Attributes:
+   - schemaType
+   - name
+   - dbName
+   - compatibility
+   - validationLevel
+   - canEvolve
+   - schemaGroup
+   - description
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.I32, 'schemaType', None, None, ), # 1
+    (2, TType.STRING, 'name', None, None, ), # 2
+    (3, TType.STRING, 'dbName', None, None, ), # 3
+    (4, TType.I32, 'compatibility', None, None, ), # 4
+    (5, TType.I32, 'validationLevel', None, None, ), # 5
+    (6, TType.BOOL, 'canEvolve', None, None, ), # 6
+    (7, TType.STRING, 'schemaGroup', None, None, ), # 7
+    (8, TType.STRING, 'description', None, None, ), # 8
+  )
+
+  def __init__(self, schemaType=None, name=None, dbName=None, compatibility=None, validationLevel=None, canEvolve=None, schemaGroup=None, description=None,):
+    self.schemaType = schemaType
+    self.name = name
+    self.dbName = dbName
+    self.compatibility = compatibility
+    self.validationLevel = validationLevel
+    self.canEvolve = canEvolve
+    self.schemaGroup = schemaGroup
+    self.description = description
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.I32:
+          self.schemaType = iprot.readI32()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRING:
+          self.name = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.STRING:
+          self.dbName = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.I32:
+          self.compatibility = iprot.readI32()
+        else:
+          iprot.skip(ftype)
+      elif fid == 5:
+        if ftype == TType.I32:
+          self.validationLevel = iprot.readI32()
+        else:
+          iprot.skip(ftype)
+      elif fid == 6:
+        if ftype == TType.BOOL:
+          self.canEvolve = iprot.readBool()
+        else:
+          iprot.skip(ftype)
+      elif fid == 7:
+        if ftype == TType.STRING:
+          self.schemaGroup = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 8:
+        if ftype == TType.STRING:
+          self.description = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('ISchema')
+    if self.schemaType is not None:
+      oprot.writeFieldBegin('schemaType', TType.I32, 1)
+      oprot.writeI32(self.schemaType)
+      oprot.writeFieldEnd()
+    if self.name is not None:
+      oprot.writeFieldBegin('name', TType.STRING, 2)
+      oprot.writeString(self.name)
+      oprot.writeFieldEnd()
+    if self.dbName is not None:
+      oprot.writeFieldBegin('dbName', TType.STRING, 3)
+      oprot.writeString(self.dbName)
+      oprot.writeFieldEnd()
+    if self.compatibility is not None:
+      oprot.writeFieldBegin('compatibility', TType.I32, 4)
+      oprot.writeI32(self.compatibility)
+      oprot.writeFieldEnd()
+    if self.validationLevel is not None:
+      oprot.writeFieldBegin('validationLevel', TType.I32, 5)
+      oprot.writeI32(self.validationLevel)
+      oprot.writeFieldEnd()
+    if self.canEvolve is not None:
+      oprot.writeFieldBegin('canEvolve', TType.BOOL, 6)
+      oprot.writeBool(self.canEvolve)
+      oprot.writeFieldEnd()
+    if self.schemaGroup is not None:
+      oprot.writeFieldBegin('schemaGroup', TType.STRING, 7)
+      oprot.writeString(self.schemaGroup)
+      oprot.writeFieldEnd()
+    if self.description is not None:
+      oprot.writeFieldBegin('description', TType.STRING, 8)
+      oprot.writeString(self.description)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.schemaType)
+    value = (value * 31) ^ hash(self.name)
+    value = (value * 31) ^ hash(self.dbName)
+    value = (value * 31) ^ hash(self.compatibility)
+    value = (value * 31) ^ hash(self.validationLevel)
+    value = (value * 31) ^ hash(self.canEvolve)
+    value = (value * 31) ^ hash(self.schemaGroup)
+    value = (value * 31) ^ hash(self.description)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class ISchemaName:
+  """
+  Attributes:
+   - dbName
+   - schemaName
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'dbName', None, None, ), # 1
+    (2, TType.STRING, 'schemaName', None, None, ), # 2
+  )
+
+  def __init__(self, dbName=None, schemaName=None,):
+    self.dbName = dbName
+    self.schemaName = schemaName
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.dbName = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRING:
+          self.schemaName = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('ISchemaName')
+    if self.dbName is not None:
+      oprot.writeFieldBegin('dbName', TType.STRING, 1)
+      oprot.writeString(self.dbName)
+      oprot.writeFieldEnd()
+    if self.schemaName is not None:
+      oprot.writeFieldBegin('schemaName', TType.STRING, 2)
+      oprot.writeString(self.schemaName)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.dbName)
+    value = (value * 31) ^ hash(self.schemaName)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class AlterISchemaRequest:
+  """
+  Attributes:
+   - name
+   - newSchema
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRUCT, 'name', (ISchemaName, ISchemaName.thrift_spec), None, ), # 1
+    None, # 2
+    (3, TType.STRUCT, 'newSchema', (ISchema, ISchema.thrift_spec), None, ), # 3
+  )
+
+  def __init__(self, name=None, newSchema=None,):
+    self.name = name
+    self.newSchema = newSchema
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRUCT:
+          self.name = ISchemaName()
+          self.name.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.STRUCT:
+          self.newSchema = ISchema()
+          self.newSchema.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('AlterISchemaRequest')
+    if self.name is not None:
+      oprot.writeFieldBegin('name', TType.STRUCT, 1)
+      self.name.write(oprot)
+      oprot.writeFieldEnd()
+    if self.newSchema is not None:
+      oprot.writeFieldBegin('newSchema', TType.STRUCT, 3)
+      self.newSchema.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.name)
+    value = (value * 31) ^ hash(self.newSchema)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class SchemaVersion:
+  """
+  Attributes:
+   - schema
+   - version
+   - createdAt
+   - cols
+   - state
+   - description
+   - schemaText
+   - fingerprint
+   - name
+   - serDe
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRUCT, 'schema', (ISchemaName, ISchemaName.thrift_spec), None, ), # 1
+    (2, TType.I32, 'version', None, None, ), # 2
+    (3, TType.I64, 'createdAt', None, None, ), # 3
+    (4, TType.LIST, 'cols', (TType.STRUCT,(FieldSchema, FieldSchema.thrift_spec)), None, ), # 4
+    (5, TType.I32, 'state', None, None, ), # 5
+    (6, TType.STRING, 'description', None, None, ), # 6
+    (7, TType.STRING, 'schemaText', None, None, ), # 7
+    (8, TType.STRING, 'fingerprint', None, None, ), # 8
+    (9, TType.STRING, 'name', None, None, ), # 9
+    (10, TType.STRUCT, 'serDe', (SerDeInfo, SerDeInfo.thrift_spec), None, ), # 10
+  )
+
+  def __init__(self, schema=None, version=None, createdAt=None, cols=None, state=None, description=None, schemaText=None, fingerprint=None, name=None, serDe=None,):
+    self.schema = schema
+    self.version = version
+    self.createdAt = createdAt
+    self.cols = cols
+    self.state = state
+    self.description = description
+    self.schemaText = schemaText
+    self.fingerprint = fingerprint
+    self.name = name
+    self.serDe = serDe
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRUCT:
+          self.schema = ISchemaName()
+          self.schema.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.I32:
+          self.version = iprot.readI32()
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.I64:
+          self.createdAt = iprot.readI64()
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.LIST:
+          self.cols = []
+          (_etype763, _size760) = iprot.readListBegin()
+          for _i764 in xrange(_size760):
+            _elem765 = FieldSchema()
+            _elem765.read(iprot)
+            self.cols.append(_elem765)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 5:
+        if ftype == TType.I32:
+          self.state = iprot.readI32()
+        else:
+          iprot.skip(ftype)
+      elif fid == 6:
+        if ftype == TType.STRING:
+          self.description = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 7:
+        if ftype == TType.STRING:
+          self.schemaText = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 8:
+        if ftype == TType.STRING:
+          self.fingerprint = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 9:
+        if ftype == TType.STRING:
+          self.name = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 10:
+        if ftype == TType.STRUCT:
+          self.serDe = SerDeInfo()
+          self.serDe.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('SchemaVersion')
+    if self.schema is not None:
+      oprot.writeFieldBegin('schema', TType.STRUCT, 1)
+      self.schema.write(oprot)
+      oprot.writeFieldEnd()
+    if self.version is not None:
+      oprot.writeFieldBegin('version', TType.I32, 2)
+      oprot.writeI32(self.version)
+      oprot.writeFieldEnd()
+    if self.createdAt is not None:
+      oprot.writeFieldBegin('createdAt', TType.I64, 3)
+      oprot.writeI64(self.createdAt)
+      oprot.writeFieldEnd()
+    if self.cols is not None:
+      oprot.writeFieldBegin('cols', TType.LIST, 4)
+      oprot.writeListBegin(TType.STRUCT, len(self.cols))
+      for iter766 in self.cols:
+        iter766.write(oprot)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    if self.state is not None:
+      oprot.writeFieldBegin('state', TType.I32, 5)
+      oprot.writeI32(self.state)
+      oprot.writeFieldEnd()
+    if self.description is not None:
+      oprot.writeFieldBegin('description', TType.STRING, 6)
+      oprot.writeString(self.description)
+      oprot.writeFieldEnd()
+    if self.schemaText is not None:
+      oprot.writeFieldBegin('schemaText', TType.STRING, 7)
+      oprot.writeString(self.schemaText)
+      oprot.writeFieldEnd()
+    if self.fingerprint is not None:
+      oprot.writeFieldBegin('fingerprint', TType.STRING, 8)
+      oprot.writeString(self.fingerprint)
+      oprot.writeFieldEnd()
+    if self.name is not None:
+      oprot.writeFieldBegin('name', TType.STRING, 9)
+      oprot.writeString(self.name)
+      oprot.writeFieldEnd()
+    if self.serDe is not None:
+      oprot.writeFieldBegin('serDe', TType.STRUCT, 10)
+      self.serDe.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.schema)
+    value = (value * 31) ^ hash(self.version)
+    value = (value * 31) ^ hash(self.createdAt)
+    value = (value * 31) ^ hash(self.cols)
+    value = (value * 31) ^ hash(self.state)
+    value = (value * 31) ^ hash(self.description)
+    value = (value * 31) ^ hash(self.schemaText)
+    value = (value * 31) ^ hash(self.fingerprint)
+    value = (value * 31) ^ hash(self.name)
+    value = (value * 31) ^ hash(self.serDe)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class SchemaVersionDescriptor:
+  """
+  Attributes:
+   - schema
+   - version
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRUCT, 'schema', (ISchemaName, ISchemaName.thrift_spec), None, ), # 1
+    (2, TType.I32, 'version', None, None, ), # 2
+  )
+
+  def __init__(self, schema=None, version=None,):
+    self.schema = schema
+    self.version = version
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRUCT:
+          self.schema = ISchemaName()
+          self.schema.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.I32:
+          self.version = iprot.readI32()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('SchemaVersionDescriptor')
+    if self.schema is not None:
+      oprot.writeFieldBegin('schema', TType.STRUCT, 1)
+      self.schema.write(oprot)
+      oprot.writeFieldEnd()
+    if self.version is not None:
+      oprot.writeFieldBegin('version', TType.I32, 2)
+      oprot.writeI32(self.version)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.schema)
+    value = (value * 31) ^ hash(self.version)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class FindSchemasByColsRqst:
+  """
+  Attributes:
+   - colName
+   - colNamespace
+   - type
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'colName', None, None, ), # 1
+    (2, TType.STRING, 'colNamespace', None, None, ), # 2
+    (3, TType.STRING, 'type', None, None, ), # 3
+  )
+
+  def __init__(self, colName=None, colNamespace=None, type=None,):
+    self.colName = colName
+    self.colNamespace = colNamespace
+    self.type = type
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.colName = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRING:
+          self.colNamespace = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.STRING:
+          self.type = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('FindSchemasByColsRqst')
+    if self.colName is not None:
+      oprot.writeFieldBegin('colName', TType.STRING, 1)
+      oprot.writeString(self.colName)
+      oprot.writeFieldEnd()
+    if self.colNamespace is not None:
+      oprot.writeFieldBegin('colNamespace', TType.STRING, 2)
+      oprot.writeString(self.colNamespace)
+      oprot.writeFieldEnd()
+    if self.type is not None:
+      oprot.writeFieldBegin('type', TType.STRING, 3)
+      oprot.writeString(self.type)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.colName)
+    value = (value * 31) ^ hash(self.colNamespace)
+    value = (value * 31) ^ hash(self.type)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class FindSchemasByColsResp:
+  """
+  Attributes:
+   - schemaVersions
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.LIST, 'schemaVersions', (TType.STRUCT,(SchemaVersionDescriptor, SchemaVersionDescriptor.thrift_spec)), None, ), # 1
+  )
+
+  def __init__(self, schemaVersions=None,):
+    self.schemaVersions = schemaVersions
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.LIST:
+          self.schemaVersions = []
+          (_etype770, _size767) = iprot.readListBegin()
+          for _i771 in xrange(_size767):
+            _elem772 = SchemaVersionDescriptor()
+            _elem772.read(iprot)
+            self.schemaVersions.append(_elem772)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('FindSchemasByColsResp')
+    if self.schemaVersions is not None:
+      oprot.writeFieldBegin('schemaVersions', TType.LIST, 1)
+      oprot.writeListBegin(TType.STRUCT, len(self.schemaVersions))
+      for iter773 in self.schemaVersions:
+        iter773.write(oprot)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.schemaVersions)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class MapSchemaVersionToSerdeRequest:
+  """
+  Attributes:
+   - schemaVersion
+   - serdeName
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRUCT, 'schemaVersion', (SchemaVersionDescriptor, SchemaVersionDescriptor.thrift_spec), None, ), # 1
+    (2, TType.STRING, 'serdeName', None, None, ), # 2
+  )
+
+  def __init__(self, schemaVersion=None, serdeName=None,):
+    self.schemaVersion = schemaVersion
+    self.serdeName = serdeName
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRUCT:
+          self.schemaVersion = SchemaVersionDescriptor()
+          self.schemaVersion.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRING:
+          self.serdeName = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('MapSchemaVersionToSerdeRequest')
+    if self.schemaVersion is not None:
+      oprot.writeFieldBegin('schemaVersion', TType.STRUCT, 1)
+      self.schemaVersion.write(oprot)
+      oprot.writeFieldEnd()
+    if self.serdeName is not None:
+      oprot.writeFieldBegin('serdeName', TType.STRING, 2)
+      oprot.writeString(self.serdeName)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.schemaVersion)
+    value = (value * 31) ^ hash(self.serdeName)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class SetSchemaVersionStateRequest:
+  """
+  Attributes:
+   - schemaVersion
+   - state
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRUCT, 'schemaVersion', (SchemaVersionDescriptor, SchemaVersionDescriptor.thrift_spec), None, ), # 1
+    (2, TType.I32, 'state', None, None, ), # 2
+  )
+
+  def __init__(self, schemaVersion=None, state=None,):
+    self.schemaVersion = schemaVersion
+    self.state = state
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRUCT:
+          self.schemaVersion = SchemaVersionDescriptor()
+          self.schemaVersion.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.I32:
+          self.state = iprot.readI32()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('SetSchemaVersionStateRequest')
+    if self.schemaVersion is not None:
+      oprot.writeFieldBegin('schemaVersion', TType.STRUCT, 1)
+      self.schemaVersion.write(oprot)
+      oprot.writeFieldEnd()
+    if self.state is not None:
+      oprot.writeFieldBegin('state', TType.I32, 2)
+      oprot.writeI32(self.state)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.schemaVersion)
+    value = (value * 31) ^ hash(self.state)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class GetSerdeRequest:
+  """
+  Attributes:
+   - serdeName
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'serdeName', None, None, ), # 1
+  )
+
+  def __init__(self, serdeName=None,):
+    self.serdeName = serdeName
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.serdeName = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('GetSerdeRequest')
+    if self.serdeName is not None:
+      oprot.writeFieldBegin('serdeName', TType.STRING, 1)
+      oprot.writeString(self.serdeName)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.serdeName)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
 class MetaException(TException):
   """
   Attributes:

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb b/standalone-metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
index 29ad1bc..6b6746f 100644
--- a/standalone-metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
+++ b/standalone-metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
@@ -98,6 +98,49 @@ module EventRequestType
   VALID_VALUES = Set.new([INSERT, UPDATE, DELETE]).freeze
 end
 
+module SerdeType
+  HIVE = 1
+  SCHEMA_REGISTRY = 2
+  VALUE_MAP = {1 => "HIVE", 2 => "SCHEMA_REGISTRY"}
+  VALID_VALUES = Set.new([HIVE, SCHEMA_REGISTRY]).freeze
+end
+
+module SchemaType
+  HIVE = 1
+  AVRO = 2
+  VALUE_MAP = {1 => "HIVE", 2 => "AVRO"}
+  VALID_VALUES = Set.new([HIVE, AVRO]).freeze
+end
+
+module SchemaCompatibility
+  NONE = 1
+  BACKWARD = 2
+  FORWARD = 3
+  BOTH = 4
+  VALUE_MAP = {1 => "NONE", 2 => "BACKWARD", 3 => "FORWARD", 4 => "BOTH"}
+  VALID_VALUES = Set.new([NONE, BACKWARD, FORWARD, BOTH]).freeze
+end
+
+module SchemaValidation
+  LATEST = 1
+  ALL = 2
+  VALUE_MAP = {1 => "LATEST", 2 => "ALL"}
+  VALID_VALUES = Set.new([LATEST, ALL]).freeze
+end
+
+module SchemaVersionState
+  INITIATED = 1
+  START_REVIEW = 2
+  CHANGES_REQUIRED = 3
+  REVIEWED = 4
+  ENABLED = 5
+  DISABLED = 6
+  ARCHIVED = 7
+  DELETED = 8
+  VALUE_MAP = {1 => "INITIATED", 2 => "START_REVIEW", 3 => "CHANGES_REQUIRED", 4 => "REVIEWED", 5 => "ENABLED", 6 => "DISABLED", 7 => "ARCHIVED", 8 => "DELETED"}
+  VALID_VALUES = Set.new([INITIATED, START_REVIEW, CHANGES_REQUIRED, REVIEWED, ENABLED, DISABLED, ARCHIVED, DELETED]).freeze
+end
+
 module FunctionType
   JAVA = 1
   VALUE_MAP = {1 => "JAVA"}
@@ -731,16 +774,27 @@ class SerDeInfo
   NAME = 1
   SERIALIZATIONLIB = 2
   PARAMETERS = 3
+  DESCRIPTION = 4
+  SERIALIZERCLASS = 5
+  DESERIALIZERCLASS = 6
+  SERDETYPE = 7
 
   FIELDS = {
     NAME => {:type => ::Thrift::Types::STRING, :name => 'name'},
     SERIALIZATIONLIB => {:type => ::Thrift::Types::STRING, :name => 'serializationLib'},
-    PARAMETERS => {:type => ::Thrift::Types::MAP, :name => 'parameters', :key => {:type => ::Thrift::Types::STRING}, :value => {:type => ::Thrift::Types::STRING}}
+    PARAMETERS => {:type => ::Thrift::Types::MAP, :name => 'parameters', :key => {:type => ::Thrift::Types::STRING}, :value => {:type => ::Thrift::Types::STRING}},
+    DESCRIPTION => {:type => ::Thrift::Types::STRING, :name => 'description', :optional => true},
+    SERIALIZERCLASS => {:type => ::Thrift::Types::STRING, :name => 'serializerClass', :optional => true},
+    DESERIALIZERCLASS => {:type => ::Thrift::Types::STRING, :name => 'deserializerClass', :optional => true},
+    SERDETYPE => {:type => ::Thrift::Types::I32, :name => 'serdeType', :optional => true, :enum_class => ::SerdeType}
   }
 
   def struct_fields; FIELDS; end
 
   def validate
+    unless @serdeType.nil? || ::SerdeType::VALID_VALUES.include?(@serdeType)
+      raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Invalid value of field serdeType!')
+    end
   end
 
   ::Thrift::Struct.generate_accessors self
@@ -4280,6 +4334,227 @@ class WMCreateOrDropTriggerToPoolMappingResponse
   ::Thrift::Struct.generate_accessors self
 end
 
+class ISchema
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  SCHEMATYPE = 1
+  NAME = 2
+  DBNAME = 3
+  COMPATIBILITY = 4
+  VALIDATIONLEVEL = 5
+  CANEVOLVE = 6
+  SCHEMAGROUP = 7
+  DESCRIPTION = 8
+
+  FIELDS = {
+    SCHEMATYPE => {:type => ::Thrift::Types::I32, :name => 'schemaType', :enum_class => ::SchemaType},
+    NAME => {:type => ::Thrift::Types::STRING, :name => 'name'},
+    DBNAME => {:type => ::Thrift::Types::STRING, :name => 'dbName'},
+    COMPATIBILITY => {:type => ::Thrift::Types::I32, :name => 'compatibility', :enum_class => ::SchemaCompatibility},
+    VALIDATIONLEVEL => {:type => ::Thrift::Types::I32, :name => 'validationLevel', :enum_class => ::SchemaValidation},
+    CANEVOLVE => {:type => ::Thrift::Types::BOOL, :name => 'canEvolve'},
+    SCHEMAGROUP => {:type => ::Thrift::Types::STRING, :name => 'schemaGroup', :optional => true},
+    DESCRIPTION => {:type => ::Thrift::Types::STRING, :name => 'description', :optional => true}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+    unless @schemaType.nil? || ::SchemaType::VALID_VALUES.include?(@schemaType)
+      raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Invalid value of field schemaType!')
+    end
+    unless @compatibility.nil? || ::SchemaCompatibility::VALID_VALUES.include?(@compatibility)
+      raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Invalid value of field compatibility!')
+    end
+    unless @validationLevel.nil? || ::SchemaValidation::VALID_VALUES.include?(@validationLevel)
+      raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Invalid value of field validationLevel!')
+    end
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
+class ISchemaName
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  DBNAME = 1
+  SCHEMANAME = 2
+
+  FIELDS = {
+    DBNAME => {:type => ::Thrift::Types::STRING, :name => 'dbName'},
+    SCHEMANAME => {:type => ::Thrift::Types::STRING, :name => 'schemaName'}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
+class AlterISchemaRequest
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  NAME = 1
+  NEWSCHEMA = 3
+
+  FIELDS = {
+    NAME => {:type => ::Thrift::Types::STRUCT, :name => 'name', :class => ::ISchemaName},
+    NEWSCHEMA => {:type => ::Thrift::Types::STRUCT, :name => 'newSchema', :class => ::ISchema}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
+class SchemaVersion
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  SCHEMA = 1
+  VERSION = 2
+  CREATEDAT = 3
+  COLS = 4
+  STATE = 5
+  DESCRIPTION = 6
+  SCHEMATEXT = 7
+  FINGERPRINT = 8
+  NAME = 9
+  SERDE = 10
+
+  FIELDS = {
+    SCHEMA => {:type => ::Thrift::Types::STRUCT, :name => 'schema', :class => ::ISchemaName},
+    VERSION => {:type => ::Thrift::Types::I32, :name => 'version'},
+    CREATEDAT => {:type => ::Thrift::Types::I64, :name => 'createdAt'},
+    COLS => {:type => ::Thrift::Types::LIST, :name => 'cols', :element => {:type => ::Thrift::Types::STRUCT, :class => ::FieldSchema}},
+    STATE => {:type => ::Thrift::Types::I32, :name => 'state', :optional => true, :enum_class => ::SchemaVersionState},
+    DESCRIPTION => {:type => ::Thrift::Types::STRING, :name => 'description', :optional => true},
+    SCHEMATEXT => {:type => ::Thrift::Types::STRING, :name => 'schemaText', :optional => true},
+    FINGERPRINT => {:type => ::Thrift::Types::STRING, :name => 'fingerprint', :optional => true},
+    NAME => {:type => ::Thrift::Types::STRING, :name => 'name', :optional => true},
+    SERDE => {:type => ::Thrift::Types::STRUCT, :name => 'serDe', :class => ::SerDeInfo, :optional => true}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+    unless @state.nil? || ::SchemaVersionState::VALID_VALUES.include?(@state)
+      raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Invalid value of field state!')
+    end
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
+class SchemaVersionDescriptor
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  SCHEMA = 1
+  VERSION = 2
+
+  FIELDS = {
+    SCHEMA => {:type => ::Thrift::Types::STRUCT, :name => 'schema', :class => ::ISchemaName},
+    VERSION => {:type => ::Thrift::Types::I32, :name => 'version'}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
+class FindSchemasByColsRqst
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  COLNAME = 1
+  COLNAMESPACE = 2
+  TYPE = 3
+
+  FIELDS = {
+    COLNAME => {:type => ::Thrift::Types::STRING, :name => 'colName', :optional => true},
+    COLNAMESPACE => {:type => ::Thrift::Types::STRING, :name => 'colNamespace', :optional => true},
+    TYPE => {:type => ::Thrift::Types::STRING, :name => 'type', :optional => true}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
+class FindSchemasByColsResp
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  SCHEMAVERSIONS = 1
+
+  FIELDS = {
+    SCHEMAVERSIONS => {:type => ::Thrift::Types::LIST, :name => 'schemaVersions', :element => {:type => ::Thrift::Types::STRUCT, :class => ::SchemaVersionDescriptor}}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
+class MapSchemaVersionToSerdeRequest
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  SCHEMAVERSION = 1
+  SERDENAME = 2
+
+  FIELDS = {
+    SCHEMAVERSION => {:type => ::Thrift::Types::STRUCT, :name => 'schemaVersion', :class => ::SchemaVersionDescriptor},
+    SERDENAME => {:type => ::Thrift::Types::STRING, :name => 'serdeName'}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
+class SetSchemaVersionStateRequest
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  SCHEMAVERSION = 1
+  STATE = 2
+
+  FIELDS = {
+    SCHEMAVERSION => {:type => ::Thrift::Types::STRUCT, :name => 'schemaVersion', :class => ::SchemaVersionDescriptor},
+    STATE => {:type => ::Thrift::Types::I32, :name => 'state', :enum_class => ::SchemaVersionState}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+    unless @state.nil? || ::SchemaVersionState::VALID_VALUES.include?(@state)
+      raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Invalid value of field state!')
+    end
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
+class GetSerdeRequest
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  SERDENAME = 1
+
+  FIELDS = {
+    SERDENAME => {:type => ::Thrift::Types::STRING, :name => 'serdeName'}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
 class MetaException < ::Thrift::Exception
   include ::Thrift::Struct, ::Thrift::Struct_Union
   def initialize(message=nil)

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb b/standalone-metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
index 4990fed..e6787c1 100644
--- a/standalone-metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
+++ b/standalone-metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
@@ -3015,6 +3015,239 @@ module ThriftHiveMetastore
       raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'create_or_drop_wm_trigger_to_pool_mapping failed: unknown result')
     end
 
+    def create_ischema(schema)
+      send_create_ischema(schema)
+      recv_create_ischema()
+    end
+
+    def send_create_ischema(schema)
+      send_message('create_ischema', Create_ischema_args, :schema => schema)
+    end
+
+    def recv_create_ischema()
+      result = receive_message(Create_ischema_result)
+      raise result.o1 unless result.o1.nil?
+      raise result.o2 unless result.o2.nil?
+      raise result.o3 unless result.o3.nil?
+      return
+    end
+
+    def alter_ischema(rqst)
+      send_alter_ischema(rqst)
+      recv_alter_ischema()
+    end
+
+    def send_alter_ischema(rqst)
+      send_message('alter_ischema', Alter_ischema_args, :rqst => rqst)
+    end
+
+    def recv_alter_ischema()
+      result = receive_message(Alter_ischema_result)
+      raise result.o1 unless result.o1.nil?
+      raise result.o2 unless result.o2.nil?
+      return
+    end
+
+    def get_ischema(name)
+      send_get_ischema(name)
+      return recv_get_ischema()
+    end
+
+    def send_get_ischema(name)
+      send_message('get_ischema', Get_ischema_args, :name => name)
+    end
+
+    def recv_get_ischema()
+      result = receive_message(Get_ischema_result)
+      return result.success unless result.success.nil?
+      raise result.o1 unless result.o1.nil?
+      raise result.o2 unless result.o2.nil?
+      raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'get_ischema failed: unknown result')
+    end
+
+    def drop_ischema(name)
+      send_drop_ischema(name)
+      recv_drop_ischema()
+    end
+
+    def send_drop_ischema(name)
+      send_message('drop_ischema', Drop_ischema_args, :name => name)
+    end
+
+    def recv_drop_ischema()
+      result = receive_message(Drop_ischema_result)
+      raise result.o1 unless result.o1.nil?
+      raise result.o2 unless result.o2.nil?
+      raise result.o3 unless result.o3.nil?
+      return
+    end
+
+    def add_schema_version(schemaVersion)
+      send_add_schema_version(schemaVersion)
+      recv_add_schema_version()
+    end
+
+    def send_add_schema_version(schemaVersion)
+      send_message('add_schema_version', Add_schema_version_args, :schemaVersion => schemaVersion)
+    end
+
+    def recv_add_schema_version()
+      result = receive_message(Add_schema_version_result)
+      raise result.o1 unless result.o1.nil?
+      raise result.o2 unless result.o2.nil?
+      raise result.o3 unless result.o3.nil?
+      return
+    end
+
+    def get_schema_version(schemaVersion)
+      send_get_schema_version(schemaVersion)
+      return recv_get_schema_version()
+    end
+
+    def send_get_schema_version(schemaVersion)
+      send_message('get_schema_version', Get_schema_version_args, :schemaVersion => schemaVersion)
+    end
+
+    def recv_get_schema_version()
+      result = receive_message(Get_schema_version_result)
+      return result.success unless result.success.nil?
+      raise result.o1 unless result.o1.nil?
+      raise result.o2 unless result.o2.nil?
+      raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'get_schema_version failed: unknown result')
+    end
+
+    def get_schema_latest_version(schemaName)
+      send_get_schema_latest_version(schemaName)
+      return recv_get_schema_latest_version()
+    end
+
+    def send_get_schema_latest_version(schemaName)
+      send_message('get_schema_latest_version', Get_schema_latest_version_args, :schemaName => schemaName)
+    end
+
+    def recv_get_schema_latest_version()
+      result = receive_message(Get_schema_latest_version_result)
+      return result.success unless result.success.nil?
+      raise result.o1 unless result.o1.nil?
+      raise result.o2 unless result.o2.nil?
+      raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'get_schema_latest_version failed: unknown result')
+    end
+
+    def get_schema_all_versions(schemaName)
+      send_get_schema_all_versions(schemaName)
+      return recv_get_schema_all_versions()
+    end
+
+    def send_get_schema_all_versions(schemaName)
+      send_message('get_schema_all_versions', Get_schema_all_versions_args, :schemaName => schemaName)
+    end
+
+    def recv_get_schema_all_versions()
+      result = receive_message(Get_schema_all_versions_result)
+      return result.success unless result.success.nil?
+      raise result.o1 unless result.o1.nil?
+      raise result.o2 unless result.o2.nil?
+      raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'get_schema_all_versions failed: unknown result')
+    end
+
+    def drop_schema_version(schemaVersion)
+      send_drop_schema_version(schemaVersion)
+      recv_drop_schema_version()
+    end
+
+    def send_drop_schema_version(schemaVersion)
+      send_message('drop_schema_version', Drop_schema_version_args, :schemaVersion => schemaVersion)
+    end
+
+    def recv_drop_schema_version()
+      result = receive_message(Drop_schema_version_result)
+      raise result.o1 unless result.o1.nil?
+      raise result.o2 unless result.o2.nil?
+      return
+    end
+
+    def get_schemas_by_cols(rqst)
+      send_get_schemas_by_cols(rqst)
+      return recv_get_schemas_by_cols()
+    end
+
+    def send_get_schemas_by_cols(rqst)
+      send_message('get_schemas_by_cols', Get_schemas_by_cols_args, :rqst => rqst)
+    end
+
+    def recv_get_schemas_by_cols()
+      result = receive_message(Get_schemas_by_cols_result)
+      return result.success unless result.success.nil?
+      raise result.o1 unless result.o1.nil?
+      raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'get_schemas_by_cols failed: unknown result')
+    end
+
+    def map_schema_version_to_serde(rqst)
+      send_map_schema_version_to_serde(rqst)
+      recv_map_schema_version_to_serde()
+    end
+
+    def send_map_schema_version_to_serde(rqst)
+      send_message('map_schema_version_to_serde', Map_schema_version_to_serde_args, :rqst => rqst)
+    end
+
+    def recv_map_schema_version_to_serde()
+      result = receive_message(Map_schema_version_to_serde_result)
+      raise result.o1 unless result.o1.nil?
+      raise result.o2 unless result.o2.nil?
+      return
+    end
+
+    def set_schema_version_state(rqst)
+      send_set_schema_version_state(rqst)
+      recv_set_schema_version_state()
+    end
+
+    def send_set_schema_version_state(rqst)
+      send_message('set_schema_version_state', Set_schema_version_state_args, :rqst => rqst)
+    end
+
+    def recv_set_schema_version_state()
+      result = receive_message(Set_schema_version_state_result)
+      raise result.o1 unless result.o1.nil?
+      raise result.o2 unless result.o2.nil?
+      raise result.o3 unless result.o3.nil?
+      return
+    end
+
+    def add_serde(serde)
+      send_add_serde(serde)
+      recv_add_serde()
+    end
+
+    def send_add_serde(serde)
+      send_message('add_serde', Add_serde_args, :serde => serde)
+    end
+
+    def recv_add_serde()
+      result = receive_message(Add_serde_result)
+      raise result.o1 unless result.o1.nil?
+      raise result.o2 unless result.o2.nil?
+      return
+    end
+
+    def get_serde(rqst)
+      send_get_serde(rqst)
+      return recv_get_serde()
+    end
+
+    def send_get_serde(rqst)
+      send_message('get_serde', Get_serde_args, :rqst => rqst)
+    end
+
+    def recv_get_serde()
+      result = receive_message(Get_serde_result)
+      return result.success unless result.success.nil?
+      raise result.o1 unless result.o1.nil?
+      raise result.o2 unless result.o2.nil?
+      raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'get_serde failed: unknown result')
+    end
+
   end
 
   class Processor < ::FacebookService::Processor 
@@ -5274,6 +5507,194 @@ module ThriftHiveMetastore
       write_result(result, oprot, 'create_or_drop_wm_trigger_to_pool_mapping', seqid)
     end
 
+    def process_create_ischema(seqid, iprot, oprot)
+      args = read_args(iprot, Create_ischema_args)
+      result = Create_ischema_result.new()
+      begin
+        @handler.create_ischema(args.schema)
+      rescue ::AlreadyExistsException => o1
+        result.o1 = o1
+      rescue ::NoSuchObjectException => o2
+        result.o2 = o2
+      rescue ::MetaException => o3
+        result.o3 = o3
+      end
+      write_result(result, oprot, 'create_ischema', seqid)
+    end
+
+    def process_alter_ischema(seqid, iprot, oprot)
+      args = read_args(iprot, Alter_ischema_args)
+      result = Alter_ischema_result.new()
+      begin
+        @handler.alter_ischema(args.rqst)
+      rescue ::NoSuchObjectException => o1
+        result.o1 = o1
+      rescue ::MetaException => o2
+        result.o2 = o2
+      end
+      write_result(result, oprot, 'alter_ischema', seqid)
+    end
+
+    def process_get_ischema(seqid, iprot, oprot)
+      args = read_args(iprot, Get_ischema_args)
+      result = Get_ischema_result.new()
+      begin
+        result.success = @handler.get_ischema(args.name)
+      rescue ::NoSuchObjectException => o1
+        result.o1 = o1
+      rescue ::MetaException => o2
+        result.o2 = o2
+      end
+      write_result(result, oprot, 'get_ischema', seqid)
+    end
+
+    def process_drop_ischema(seqid, iprot, oprot)
+      args = read_args(iprot, Drop_ischema_args)
+      result = Drop_ischema_result.new()
+      begin
+        @handler.drop_ischema(args.name)
+      rescue ::NoSuchObjectException => o1
+        result.o1 = o1
+      rescue ::InvalidOperationException => o2
+        result.o2 = o2
+      rescue ::MetaException => o3
+        result.o3 = o3
+      end
+      write_result(result, oprot, 'drop_ischema', seqid)
+    end
+
+    def process_add_schema_version(seqid, iprot, oprot)
+      args = read_args(iprot, Add_schema_version_args)
+      result = Add_schema_version_result.new()
+      begin
+        @handler.add_schema_version(args.schemaVersion)
+      rescue ::AlreadyExistsException => o1
+        result.o1 = o1
+      rescue ::NoSuchObjectException => o2
+        result.o2 = o2
+      rescue ::MetaException => o3
+        result.o3 = o3
+      end
+      write_result(result, oprot, 'add_schema_version', seqid)
+    end
+
+    def process_get_schema_version(seqid, iprot, oprot)
+      args = read_args(iprot, Get_schema_version_args)
+      result = Get_schema_version_result.new()
+      begin
+        result.success = @handler.get_schema_version(args.schemaVersion)
+      rescue ::NoSuchObjectException => o1
+        result.o1 = o1
+      rescue ::MetaException => o2
+        result.o2 = o2
+      end
+      write_result(result, oprot, 'get_schema_version', seqid)
+    end
+
+    def process_get_schema_latest_version(seqid, iprot, oprot)
+      args = read_args(iprot, Get_schema_latest_version_args)
+      result = Get_schema_latest_version_result.new()
+      begin
+        result.success = @handler.get_schema_latest_version(args.schemaName)
+      rescue ::NoSuchObjectException => o1
+        result.o1 = o1
+      rescue ::MetaException => o2
+        result.o2 = o2
+      end
+      write_result(result, oprot, 'get_schema_latest_version', seqid)
+    end
+
+    def process_get_schema_all_versions(seqid, iprot, oprot)
+      args = read_args(iprot, Get_schema_all_versions_args)
+      result = Get_schema_all_versions_result.new()
+      begin
+        result.success = @handler.get_schema_all_versions(args.schemaName)
+      rescue ::NoSuchObjectException => o1
+        result.o1 = o1
+      rescue ::MetaException => o2
+        result.o2 = o2
+      end
+      write_result(result, oprot, 'get_schema_all_versions', seqid)
+    end
+
+    def process_drop_schema_version(seqid, iprot, oprot)
+      args = read_args(iprot, Drop_schema_version_args)
+      result = Drop_schema_version_result.new()
+      begin
+        @handler.drop_schema_version(args.schemaVersion)
+      rescue ::NoSuchObjectException => o1
+        result.o1 = o1
+      rescue ::MetaException => o2
+        result.o2 = o2
+      end
+      write_result(result, oprot, 'drop_schema_version', seqid)
+    end
+
+    def process_get_schemas_by_cols(seqid, iprot, oprot)
+      args = read_args(iprot, Get_schemas_by_cols_args)
+      result = Get_schemas_by_cols_result.new()
+      begin
+        result.success = @handler.get_schemas_by_cols(args.rqst)
+      rescue ::MetaException => o1
+        result.o1 = o1
+      end
+      write_result(result, oprot, 'get_schemas_by_cols', seqid)
+    end
+
+    def process_map_schema_version_to_serde(seqid, iprot, oprot)
+      args = read_args(iprot, Map_schema_version_to_serde_args)
+      result = Map_schema_version_to_serde_result.new()
+      begin
+        @handler.map_schema_version_to_serde(args.rqst)
+      rescue ::NoSuchObjectException => o1
+        result.o1 = o1
+      rescue ::MetaException => o2
+        result.o2 = o2
+      end
+      write_result(result, oprot, 'map_schema_version_to_serde', seqid)
+    end
+
+    def process_set_schema_version_state(seqid, iprot, oprot)
+      args = read_args(iprot, Set_schema_version_state_args)
+      result = Set_schema_version_state_result.new()
+      begin
+        @handler.set_schema_version_state(args.rqst)
+      rescue ::NoSuchObjectException => o1
+        result.o1 = o1
+      rescue ::InvalidOperationException => o2
+        result.o2 = o2
+      rescue ::MetaException => o3
+        result.o3 = o3
+      end
+      write_result(result, oprot, 'set_schema_version_state', seqid)
+    end
+
+    def process_add_serde(seqid, iprot, oprot)
+      args = read_args(iprot, Add_serde_args)
+      result = Add_serde_result.new()
+      begin
+        @handler.add_serde(args.serde)
+      rescue ::AlreadyExistsException => o1
+        result.o1 = o1
+      rescue ::MetaException => o2
+        result.o2 = o2
+      end
+      write_result(result, oprot, 'add_serde', seqid)
+    end
+
+    def process_get_serde(seqid, iprot, oprot)
+      args = read_args(iprot, Get_serde_args)
+      result = Get_serde_result.new()
+      begin
+        result.success = @handler.get_serde(args.rqst)
+      rescue ::NoSuchObjectException => o1
+        result.o1 = o1
+      rescue ::MetaException => o2
+        result.o2 = o2
+      end
+      write_result(result, oprot, 'get_serde', seqid)
+    end
+
   end
 
   # HELPER FUNCTIONS AND STRUCTURES
@@ -11991,5 +12412,499 @@ module ThriftHiveMetastore
     ::Thrift::Struct.generate_accessors self
   end
 
+  class Create_ischema_args
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    SCHEMA = 1
+
+    FIELDS = {
+      SCHEMA => {:type => ::Thrift::Types::STRUCT, :name => 'schema', :class => ::ISchema}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Create_ischema_result
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    O1 = 1
+    O2 = -1
+    O3 = 3
+
+    FIELDS = {
+      O1 => {:type => ::Thrift::Types::STRUCT, :name => 'o1', :class => ::AlreadyExistsException},
+      O2 => {:type => ::Thrift::Types::STRUCT, :name => 'o2', :class => ::NoSuchObjectException},
+      O3 => {:type => ::Thrift::Types::STRUCT, :name => 'o3', :class => ::MetaException}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Alter_ischema_args
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    RQST = 1
+
+    FIELDS = {
+      RQST => {:type => ::Thrift::Types::STRUCT, :name => 'rqst', :class => ::AlterISchemaRequest}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Alter_ischema_result
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    O1 = 1
+    O2 = 2
+
+    FIELDS = {
+      O1 => {:type => ::Thrift::Types::STRUCT, :name => 'o1', :class => ::NoSuchObjectException},
+      O2 => {:type => ::Thrift::Types::STRUCT, :name => 'o2', :class => ::MetaException}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Get_ischema_args
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    NAME = 1
+
+    FIELDS = {
+      NAME => {:type => ::Thrift::Types::STRUCT, :name => 'name', :class => ::ISchemaName}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Get_ischema_result
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    SUCCESS = 0
+    O1 = 1
+    O2 = 2
+
+    FIELDS = {
+      SUCCESS => {:type => ::Thrift::Types::STRUCT, :name => 'success', :class => ::ISchema},
+      O1 => {:type => ::Thrift::Types::STRUCT, :name => 'o1', :class => ::NoSuchObjectException},
+      O2 => {:type => ::Thrift::Types::STRUCT, :name => 'o2', :class => ::MetaException}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Drop_ischema_args
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    NAME = 1
+
+    FIELDS = {
+      NAME => {:type => ::Thrift::Types::STRUCT, :name => 'name', :class => ::ISchemaName}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Drop_ischema_result
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    O1 = 1
+    O2 = 2
+    O3 = 3
+
+    FIELDS = {
+      O1 => {:type => ::Thrift::Types::STRUCT, :name => 'o1', :class => ::NoSuchObjectException},
+      O2 => {:type => ::Thrift::Types::STRUCT, :name => 'o2', :class => ::InvalidOperationException},
+      O3 => {:type => ::Thrift::Types::STRUCT, :name => 'o3', :class => ::MetaException}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Add_schema_version_args
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    SCHEMAVERSION = 1
+
+    FIELDS = {
+      SCHEMAVERSION => {:type => ::Thrift::Types::STRUCT, :name => 'schemaVersion', :class => ::SchemaVersion}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Add_schema_version_result
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    O1 = 1
+    O2 = 2
+    O3 = 3
+
+    FIELDS = {
+      O1 => {:type => ::Thrift::Types::STRUCT, :name => 'o1', :class => ::AlreadyExistsException},
+      O2 => {:type => ::Thrift::Types::STRUCT, :name => 'o2', :class => ::NoSuchObjectException},
+      O3 => {:type => ::Thrift::Types::STRUCT, :name => 'o3', :class => ::MetaException}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Get_schema_version_args
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    SCHEMAVERSION = 1
+
+    FIELDS = {
+      SCHEMAVERSION => {:type => ::Thrift::Types::STRUCT, :name => 'schemaVersion', :class => ::SchemaVersionDescriptor}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Get_schema_version_result
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    SUCCESS = 0
+    O1 = 1
+    O2 = 2
+
+    FIELDS = {
+      SUCCESS => {:type => ::Thrift::Types::STRUCT, :name => 'success', :class => ::SchemaVersion},
+      O1 => {:type => ::Thrift::Types::STRUCT, :name => 'o1', :class => ::NoSuchObjectException},
+      O2 => {:type => ::Thrift::Types::STRUCT, :name => 'o2', :class => ::MetaException}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Get_schema_latest_version_args
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    SCHEMANAME = 1
+
+    FIELDS = {
+      SCHEMANAME => {:type => ::Thrift::Types::STRUCT, :name => 'schemaName', :class => ::ISchemaName}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Get_schema_latest_version_result
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    SUCCESS = 0
+    O1 = 1
+    O2 = 2
+
+    FIELDS = {
+      SUCCESS => {:type => ::Thrift::Types::STRUCT, :name => 'success', :class => ::SchemaVersion},
+      O1 => {:type => ::Thrift::Types::STRUCT, :name => 'o1', :class => ::NoSuchObjectException},
+      O2 => {:type => ::Thrift::Types::STRUCT, :name => 'o2', :class => ::MetaException}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Get_schema_all_versions_args
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    SCHEMANAME = 1
+
+    FIELDS = {
+      SCHEMANAME => {:type => ::Thrift::Types::STRUCT, :name => 'schemaName', :class => ::ISchemaName}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Get_schema_all_versions_result
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    SUCCESS = 0
+    O1 = 1
+    O2 = 2
+
+    FIELDS = {
+      SUCCESS => {:type => ::Thrift::Types::LIST, :name => 'success', :element => {:type => ::Thrift::Types::STRUCT, :class => ::SchemaVersion}},
+      O1 => {:type => ::Thrift::Types::STRUCT, :name => 'o1', :class => ::NoSuchObjectException},
+      O2 => {:type => ::Thrift::Types::STRUCT, :name => 'o2', :class => ::MetaException}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Drop_schema_version_args
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    SCHEMAVERSION = 1
+
+    FIELDS = {
+      SCHEMAVERSION => {:type => ::Thrift::Types::STRUCT, :name => 'schemaVersion', :class => ::SchemaVersionDescriptor}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Drop_schema_version_result
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    O1 = 1
+    O2 = 2
+
+    FIELDS = {
+      O1 => {:type => ::Thrift::Types::STRUCT, :name => 'o1', :class => ::NoSuchObjectException},
+      O2 => {:type => ::Thrift::Types::STRUCT, :name => 'o2', :class => ::MetaException}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Get_schemas_by_cols_args
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    RQST = 1
+
+    FIELDS = {
+      RQST => {:type => ::Thrift::Types::STRUCT, :name => 'rqst', :class => ::FindSchemasByColsRqst}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Get_schemas_by_cols_result
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    SUCCESS = 0
+    O1 = 1
+
+    FIELDS = {
+      SUCCESS => {:type => ::Thrift::Types::STRUCT, :name => 'success', :class => ::FindSchemasByColsResp},
+      O1 => {:type => ::Thrift::Types::STRUCT, :name => 'o1', :class => ::MetaException}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Map_schema_version_to_serde_args
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    RQST = 1
+
+    FIELDS = {
+      RQST => {:type => ::Thrift::Types::STRUCT, :name => 'rqst', :class => ::MapSchemaVersionToSerdeRequest}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Map_schema_version_to_serde_result
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    O1 = 1
+    O2 = 2
+
+    FIELDS = {
+      O1 => {:type => ::Thrift::Types::STRUCT, :name => 'o1', :class => ::NoSuchObjectException},
+      O2 => {:type => ::Thrift::Types::STRUCT, :name => 'o2', :class => ::MetaException}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Set_schema_version_state_args
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    RQST = 1
+
+    FIELDS = {
+      RQST => {:type => ::Thrift::Types::STRUCT, :name => 'rqst', :class => ::SetSchemaVersionStateRequest}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Set_schema_version_state_result
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    O1 = 1
+    O2 = 2
+    O3 = 3
+
+    FIELDS = {
+      O1 => {:type => ::Thrift::Types::STRUCT, :name => 'o1', :class => ::NoSuchObjectException},
+      O2 => {:type => ::Thrift::Types::STRUCT, :name => 'o2', :class => ::InvalidOperationException},
+      O3 => {:type => ::Thrift::Types::STRUCT, :name => 'o3', :class => ::MetaException}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Add_serde_args
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    SERDE = 1
+
+    FIELDS = {
+      SERDE => {:type => ::Thrift::Types::STRUCT, :name => 'serde', :class => ::SerDeInfo}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Add_serde_result
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    O1 = 1
+    O2 = 2
+
+    FIELDS = {
+      O1 => {:type => ::Thrift::Types::STRUCT, :name => 'o1', :class => ::AlreadyExistsException},
+      O2 => {:type => ::Thrift::Types::STRUCT, :name => 'o2', :class => ::MetaException}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Get_serde_args
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    RQST = 1
+
+    FIELDS = {
+      RQST => {:type => ::Thrift::Types::STRUCT, :name => 'rqst', :class => ::GetSerdeRequest}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Get_serde_result
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    SUCCESS = 0
+    O1 = 1
+    O2 = 2
+
+    FIELDS = {
+      SUCCESS => {:type => ::Thrift::Types::STRUCT, :name => 'success', :class => ::SerDeInfo},
+      O1 => {:type => ::Thrift::Types::STRUCT, :name => 'o1', :class => ::NoSuchObjectException},
+      O2 => {:type => ::Thrift::Types::STRUCT, :name => 'o2', :class => ::MetaException}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
 end
 


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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
index 0f9ba5d..7206e29 100644
--- a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
+++ b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
@@ -202,6 +202,20 @@ class ThriftHiveMetastoreIf : virtual public  ::facebook::fb303::FacebookService
   virtual void create_or_update_wm_mapping(WMCreateOrUpdateMappingResponse& _return, const WMCreateOrUpdateMappingRequest& request) = 0;
   virtual void drop_wm_mapping(WMDropMappingResponse& _return, const WMDropMappingRequest& request) = 0;
   virtual void create_or_drop_wm_trigger_to_pool_mapping(WMCreateOrDropTriggerToPoolMappingResponse& _return, const WMCreateOrDropTriggerToPoolMappingRequest& request) = 0;
+  virtual void create_ischema(const ISchema& schema) = 0;
+  virtual void alter_ischema(const AlterISchemaRequest& rqst) = 0;
+  virtual void get_ischema(ISchema& _return, const ISchemaName& name) = 0;
+  virtual void drop_ischema(const ISchemaName& name) = 0;
+  virtual void add_schema_version(const SchemaVersion& schemaVersion) = 0;
+  virtual void get_schema_version(SchemaVersion& _return, const SchemaVersionDescriptor& schemaVersion) = 0;
+  virtual void get_schema_latest_version(SchemaVersion& _return, const ISchemaName& schemaName) = 0;
+  virtual void get_schema_all_versions(std::vector<SchemaVersion> & _return, const ISchemaName& schemaName) = 0;
+  virtual void drop_schema_version(const SchemaVersionDescriptor& schemaVersion) = 0;
+  virtual void get_schemas_by_cols(FindSchemasByColsResp& _return, const FindSchemasByColsRqst& rqst) = 0;
+  virtual void map_schema_version_to_serde(const MapSchemaVersionToSerdeRequest& rqst) = 0;
+  virtual void set_schema_version_state(const SetSchemaVersionStateRequest& rqst) = 0;
+  virtual void add_serde(const SerDeInfo& serde) = 0;
+  virtual void get_serde(SerDeInfo& _return, const GetSerdeRequest& rqst) = 0;
 };
 
 class ThriftHiveMetastoreIfFactory : virtual public  ::facebook::fb303::FacebookServiceIfFactory {
@@ -798,6 +812,48 @@ class ThriftHiveMetastoreNull : virtual public ThriftHiveMetastoreIf , virtual p
   void create_or_drop_wm_trigger_to_pool_mapping(WMCreateOrDropTriggerToPoolMappingResponse& /* _return */, const WMCreateOrDropTriggerToPoolMappingRequest& /* request */) {
     return;
   }
+  void create_ischema(const ISchema& /* schema */) {
+    return;
+  }
+  void alter_ischema(const AlterISchemaRequest& /* rqst */) {
+    return;
+  }
+  void get_ischema(ISchema& /* _return */, const ISchemaName& /* name */) {
+    return;
+  }
+  void drop_ischema(const ISchemaName& /* name */) {
+    return;
+  }
+  void add_schema_version(const SchemaVersion& /* schemaVersion */) {
+    return;
+  }
+  void get_schema_version(SchemaVersion& /* _return */, const SchemaVersionDescriptor& /* schemaVersion */) {
+    return;
+  }
+  void get_schema_latest_version(SchemaVersion& /* _return */, const ISchemaName& /* schemaName */) {
+    return;
+  }
+  void get_schema_all_versions(std::vector<SchemaVersion> & /* _return */, const ISchemaName& /* schemaName */) {
+    return;
+  }
+  void drop_schema_version(const SchemaVersionDescriptor& /* schemaVersion */) {
+    return;
+  }
+  void get_schemas_by_cols(FindSchemasByColsResp& /* _return */, const FindSchemasByColsRqst& /* rqst */) {
+    return;
+  }
+  void map_schema_version_to_serde(const MapSchemaVersionToSerdeRequest& /* rqst */) {
+    return;
+  }
+  void set_schema_version_state(const SetSchemaVersionStateRequest& /* rqst */) {
+    return;
+  }
+  void add_serde(const SerDeInfo& /* serde */) {
+    return;
+  }
+  void get_serde(SerDeInfo& /* _return */, const GetSerdeRequest& /* rqst */) {
+    return;
+  }
 };
 
 typedef struct _ThriftHiveMetastore_getMetaConf_args__isset {
@@ -22973,228 +23029,1868 @@ class ThriftHiveMetastore_create_or_drop_wm_trigger_to_pool_mapping_presult {
 
 };
 
-class ThriftHiveMetastoreClient : virtual public ThriftHiveMetastoreIf, public  ::facebook::fb303::FacebookServiceClient {
+typedef struct _ThriftHiveMetastore_create_ischema_args__isset {
+  _ThriftHiveMetastore_create_ischema_args__isset() : schema(false) {}
+  bool schema :1;
+} _ThriftHiveMetastore_create_ischema_args__isset;
+
+class ThriftHiveMetastore_create_ischema_args {
  public:
-  ThriftHiveMetastoreClient(boost::shared_ptr< ::apache::thrift::protocol::TProtocol> prot) :
-     ::facebook::fb303::FacebookServiceClient(prot, prot) {}
-  ThriftHiveMetastoreClient(boost::shared_ptr< ::apache::thrift::protocol::TProtocol> iprot, boost::shared_ptr< ::apache::thrift::protocol::TProtocol> oprot) :     ::facebook::fb303::FacebookServiceClient(iprot, oprot) {}
-  boost::shared_ptr< ::apache::thrift::protocol::TProtocol> getInputProtocol() {
-    return piprot_;
+
+  ThriftHiveMetastore_create_ischema_args(const ThriftHiveMetastore_create_ischema_args&);
+  ThriftHiveMetastore_create_ischema_args& operator=(const ThriftHiveMetastore_create_ischema_args&);
+  ThriftHiveMetastore_create_ischema_args() {
   }
-  boost::shared_ptr< ::apache::thrift::protocol::TProtocol> getOutputProtocol() {
-    return poprot_;
+
+  virtual ~ThriftHiveMetastore_create_ischema_args() throw();
+  ISchema schema;
+
+  _ThriftHiveMetastore_create_ischema_args__isset __isset;
+
+  void __set_schema(const ISchema& val);
+
+  bool operator == (const ThriftHiveMetastore_create_ischema_args & rhs) const
+  {
+    if (!(schema == rhs.schema))
+      return false;
+    return true;
   }
-  void getMetaConf(std::string& _return, const std::string& key);
-  void send_getMetaConf(const std::string& key);
-  void recv_getMetaConf(std::string& _return);
-  void setMetaConf(const std::string& key, const std::string& value);
-  void send_setMetaConf(const std::string& key, const std::string& value);
-  void recv_setMetaConf();
-  void create_database(const Database& database);
-  void send_create_database(const Database& database);
-  void recv_create_database();
-  void get_database(Database& _return, const std::string& name);
-  void send_get_database(const std::string& name);
-  void recv_get_database(Database& _return);
-  void drop_database(const std::string& name, const bool deleteData, const bool cascade);
-  void send_drop_database(const std::string& name, const bool deleteData, const bool cascade);
-  void recv_drop_database();
-  void get_databases(std::vector<std::string> & _return, const std::string& pattern);
-  void send_get_databases(const std::string& pattern);
-  void recv_get_databases(std::vector<std::string> & _return);
-  void get_all_databases(std::vector<std::string> & _return);
-  void send_get_all_databases();
-  void recv_get_all_databases(std::vector<std::string> & _return);
-  void alter_database(const std::string& dbname, const Database& db);
-  void send_alter_database(const std::string& dbname, const Database& db);
-  void recv_alter_database();
-  void get_type(Type& _return, const std::string& name);
-  void send_get_type(const std::string& name);
-  void recv_get_type(Type& _return);
-  bool create_type(const Type& type);
-  void send_create_type(const Type& type);
-  bool recv_create_type();
-  bool drop_type(const std::string& type);
-  void send_drop_type(const std::string& type);
-  bool recv_drop_type();
-  void get_type_all(std::map<std::string, Type> & _return, const std::string& name);
-  void send_get_type_all(const std::string& name);
-  void recv_get_type_all(std::map<std::string, Type> & _return);
-  void get_fields(std::vector<FieldSchema> & _return, const std::string& db_name, const std::string& table_name);
-  void send_get_fields(const std::string& db_name, const std::string& table_name);
-  void recv_get_fields(std::vector<FieldSchema> & _return);
-  void get_fields_with_environment_context(std::vector<FieldSchema> & _return, const std::string& db_name, const std::string& table_name, const EnvironmentContext& environment_context);
-  void send_get_fields_with_environment_context(const std::string& db_name, const std::string& table_name, const EnvironmentContext& environment_context);
-  void recv_get_fields_with_environment_context(std::vector<FieldSchema> & _return);
-  void get_schema(std::vector<FieldSchema> & _return, const std::string& db_name, const std::string& table_name);
-  void send_get_schema(const std::string& db_name, const std::string& table_name);
-  void recv_get_schema(std::vector<FieldSchema> & _return);
-  void get_schema_with_environment_context(std::vector<FieldSchema> & _return, const std::string& db_name, const std::string& table_name, const EnvironmentContext& environment_context);
-  void send_get_schema_with_environment_context(const std::string& db_name, const std::string& table_name, const EnvironmentContext& environment_context);
-  void recv_get_schema_with_environment_context(std::vector<FieldSchema> & _return);
-  void create_table(const Table& tbl);
-  void send_create_table(const Table& tbl);
-  void recv_create_table();
-  void create_table_with_environment_context(const Table& tbl, const EnvironmentContext& environment_context);
-  void send_create_table_with_environment_context(const Table& tbl, const EnvironmentContext& environment_context);
-  void recv_create_table_with_environment_context();
-  void create_table_with_constraints(const Table& tbl, const std::vector<SQLPrimaryKey> & primaryKeys, const std::vector<SQLForeignKey> & foreignKeys, const std::vector<SQLUniqueConstraint> & uniqueConstraints, const std::vector<SQLNotNullConstraint> & notNullConstraints, const std::vector<SQLDefaultConstraint> & defaultConstraints);
-  void send_create_table_with_constraints(const Table& tbl, const std::vector<SQLPrimaryKey> & primaryKeys, const std::vector<SQLForeignKey> & foreignKeys, const std::vector<SQLUniqueConstraint> & uniqueConstraints, const std::vector<SQLNotNullConstraint> & notNullConstraints, const std::vector<SQLDefaultConstraint> & defaultConstraints);
-  void recv_create_table_with_constraints();
-  void drop_constraint(const DropConstraintRequest& req);
-  void send_drop_constraint(const DropConstraintRequest& req);
-  void recv_drop_constraint();
-  void add_primary_key(const AddPrimaryKeyRequest& req);
-  void send_add_primary_key(const AddPrimaryKeyRequest& req);
-  void recv_add_primary_key();
-  void add_foreign_key(const AddForeignKeyRequest& req);
-  void send_add_foreign_key(const AddForeignKeyRequest& req);
-  void recv_add_foreign_key();
-  void add_unique_constraint(const AddUniqueConstraintRequest& req);
-  void send_add_unique_constraint(const AddUniqueConstraintRequest& req);
-  void recv_add_unique_constraint();
-  void add_not_null_constraint(const AddNotNullConstraintRequest& req);
-  void send_add_not_null_constraint(const AddNotNullConstraintRequest& req);
-  void recv_add_not_null_constraint();
-  void add_default_constraint(const AddDefaultConstraintRequest& req);
-  void send_add_default_constraint(const AddDefaultConstraintRequest& req);
-  void recv_add_default_constraint();
-  void drop_table(const std::string& dbname, const std::string& name, const bool deleteData);
-  void send_drop_table(const std::string& dbname, const std::string& name, const bool deleteData);
-  void recv_drop_table();
-  void drop_table_with_environment_context(const std::string& dbname, const std::string& name, const bool deleteData, const EnvironmentContext& environment_context);
-  void send_drop_table_with_environment_context(const std::string& dbname, const std::string& name, const bool deleteData, const EnvironmentContext& environment_context);
-  void recv_drop_table_with_environment_context();
-  void truncate_table(const std::string& dbName, const std::string& tableName, const std::vector<std::string> & partNames);
-  void send_truncate_table(const std::string& dbName, const std::string& tableName, const std::vector<std::string> & partNames);
-  void recv_truncate_table();
-  void get_tables(std::vector<std::string> & _return, const std::string& db_name, const std::string& pattern);
-  void send_get_tables(const std::string& db_name, const std::string& pattern);
-  void recv_get_tables(std::vector<std::string> & _return);
-  void get_tables_by_type(std::vector<std::string> & _return, const std::string& db_name, const std::string& pattern, const std::string& tableType);
-  void send_get_tables_by_type(const std::string& db_name, const std::string& pattern, const std::string& tableType);
-  void recv_get_tables_by_type(std::vector<std::string> & _return);
-  void get_materialized_views_for_rewriting(std::vector<std::string> & _return, const std::string& db_name);
-  void send_get_materialized_views_for_rewriting(const std::string& db_name);
-  void recv_get_materialized_views_for_rewriting(std::vector<std::string> & _return);
-  void get_table_meta(std::vector<TableMeta> & _return, const std::string& db_patterns, const std::string& tbl_patterns, const std::vector<std::string> & tbl_types);
-  void send_get_table_meta(const std::string& db_patterns, const std::string& tbl_patterns, const std::vector<std::string> & tbl_types);
-  void recv_get_table_meta(std::vector<TableMeta> & _return);
-  void get_all_tables(std::vector<std::string> & _return, const std::string& db_name);
-  void send_get_all_tables(const std::string& db_name);
-  void recv_get_all_tables(std::vector<std::string> & _return);
-  void get_table(Table& _return, const std::string& dbname, const std::string& tbl_name);
-  void send_get_table(const std::string& dbname, const std::string& tbl_name);
-  void recv_get_table(Table& _return);
-  void get_table_objects_by_name(std::vector<Table> & _return, const std::string& dbname, const std::vector<std::string> & tbl_names);
-  void send_get_table_objects_by_name(const std::string& dbname, const std::vector<std::string> & tbl_names);
-  void recv_get_table_objects_by_name(std::vector<Table> & _return);
-  void get_table_req(GetTableResult& _return, const GetTableRequest& req);
-  void send_get_table_req(const GetTableRequest& req);
-  void recv_get_table_req(GetTableResult& _return);
-  void get_table_objects_by_name_req(GetTablesResult& _return, const GetTablesRequest& req);
-  void send_get_table_objects_by_name_req(const GetTablesRequest& req);
-  void recv_get_table_objects_by_name_req(GetTablesResult& _return);
-  void get_materialization_invalidation_info(std::map<std::string, Materialization> & _return, const std::string& dbname, const std::vector<std::string> & tbl_names);
-  void send_get_materialization_invalidation_info(const std::string& dbname, const std::vector<std::string> & tbl_names);
-  void recv_get_materialization_invalidation_info(std::map<std::string, Materialization> & _return);
-  void update_creation_metadata(const std::string& dbname, const std::string& tbl_name, const CreationMetadata& creation_metadata);
-  void send_update_creation_metadata(const std::string& dbname, const std::string& tbl_name, const CreationMetadata& creation_metadata);
-  void recv_update_creation_metadata();
-  void get_table_names_by_filter(std::vector<std::string> & _return, const std::string& dbname, const std::string& filter, const int16_t max_tables);
-  void send_get_table_names_by_filter(const std::string& dbname, const std::string& filter, const int16_t max_tables);
-  void recv_get_table_names_by_filter(std::vector<std::string> & _return);
-  void alter_table(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl);
-  void send_alter_table(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl);
-  void recv_alter_table();
-  void alter_table_with_environment_context(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl, const EnvironmentContext& environment_context);
-  void send_alter_table_with_environment_context(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl, const EnvironmentContext& environment_context);
-  void recv_alter_table_with_environment_context();
-  void alter_table_with_cascade(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl, const bool cascade);
-  void send_alter_table_with_cascade(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl, const bool cascade);
-  void recv_alter_table_with_cascade();
-  void add_partition(Partition& _return, const Partition& new_part);
-  void send_add_partition(const Partition& new_part);
-  void recv_add_partition(Partition& _return);
-  void add_partition_with_environment_context(Partition& _return, const Partition& new_part, const EnvironmentContext& environment_context);
-  void send_add_partition_with_environment_context(const Partition& new_part, const EnvironmentContext& environment_context);
-  void recv_add_partition_with_environment_context(Partition& _return);
-  int32_t add_partitions(const std::vector<Partition> & new_parts);
-  void send_add_partitions(const std::vector<Partition> & new_parts);
-  int32_t recv_add_partitions();
-  int32_t add_partitions_pspec(const std::vector<PartitionSpec> & new_parts);
-  void send_add_partitions_pspec(const std::vector<PartitionSpec> & new_parts);
-  int32_t recv_add_partitions_pspec();
-  void append_partition(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector<std::string> & part_vals);
-  void send_append_partition(const std::string& db_name, const std::string& tbl_name, const std::vector<std::string> & part_vals);
-  void recv_append_partition(Partition& _return);
-  void add_partitions_req(AddPartitionsResult& _return, const AddPartitionsRequest& request);
-  void send_add_partitions_req(const AddPartitionsRequest& request);
-  void recv_add_partitions_req(AddPartitionsResult& _return);
-  void append_partition_with_environment_context(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector<std::string> & part_vals, const EnvironmentContext& environment_context);
-  void send_append_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector<std::string> & part_vals, const EnvironmentContext& environment_context);
-  void recv_append_partition_with_environment_context(Partition& _return);
-  void append_partition_by_name(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name);
-  void send_append_partition_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& part_name);
-  void recv_append_partition_by_name(Partition& _return);
-  void append_partition_by_name_with_environment_context(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const EnvironmentContext& environment_context);
-  void send_append_partition_by_name_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const EnvironmentContext& environment_context);
-  void recv_append_partition_by_name_with_environment_context(Partition& _return);
-  bool drop_partition(const std::string& db_name, const std::string& tbl_name, const std::vector<std::string> & part_vals, const bool deleteData);
-  void send_drop_partition(const std::string& db_name, const std::string& tbl_name, const std::vector<std::string> & part_vals, const bool deleteData);
-  bool recv_drop_partition();
-  bool drop_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector<std::string> & part_vals, const bool deleteData, const EnvironmentContext& environment_context);
-  void send_drop_partition_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::vector<std::string> & part_vals, const bool deleteData, const EnvironmentContext& environment_context);
-  bool recv_drop_partition_with_environment_context();
-  bool drop_partition_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const bool deleteData);
-  void send_drop_partition_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const bool deleteData);
-  bool recv_drop_partition_by_name();
-  bool drop_partition_by_name_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const bool deleteData, const EnvironmentContext& environment_context);
-  void send_drop_partition_by_name_with_environment_context(const std::string& db_name, const std::string& tbl_name, const std::string& part_name, const bool deleteData, const EnvironmentContext& environment_context);
-  bool recv_drop_partition_by_name_with_environment_context();
-  void drop_partitions_req(DropPartitionsResult& _return, const DropPartitionsRequest& req);
-  void send_drop_partitions_req(const DropPartitionsRequest& req);
-  void recv_drop_partitions_req(DropPartitionsResult& _return);
-  void get_partition(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector<std::string> & part_vals);
-  void send_get_partition(const std::string& db_name, const std::string& tbl_name, const std::vector<std::string> & part_vals);
-  void recv_get_partition(Partition& _return);
-  void exchange_partition(Partition& _return, const std::map<std::string, std::string> & partitionSpecs, const std::string& source_db, const std::string& source_table_name, const std::string& dest_db, const std::string& dest_table_name);
-  void send_exchange_partition(const std::map<std::string, std::string> & partitionSpecs, const std::string& source_db, const std::string& source_table_name, const std::string& dest_db, const std::string& dest_table_name);
-  void recv_exchange_partition(Partition& _return);
-  void exchange_partitions(std::vector<Partition> & _return, const std::map<std::string, std::string> & partitionSpecs, const std::string& source_db, const std::string& source_table_name, const std::string& dest_db, const std::string& dest_table_name);
-  void send_exchange_partitions(const std::map<std::string, std::string> & partitionSpecs, const std::string& source_db, const std::string& source_table_name, const std::string& dest_db, const std::string& dest_table_name);
-  void recv_exchange_partitions(std::vector<Partition> & _return);
-  void get_partition_with_auth(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::vector<std::string> & part_vals, const std::string& user_name, const std::vector<std::string> & group_names);
-  void send_get_partition_with_auth(const std::string& db_name, const std::string& tbl_name, const std::vector<std::string> & part_vals, const std::string& user_name, const std::vector<std::string> & group_names);
-  void recv_get_partition_with_auth(Partition& _return);
-  void get_partition_by_name(Partition& _return, const std::string& db_name, const std::string& tbl_name, const std::string& part_name);
-  void send_get_partition_by_name(const std::string& db_name, const std::string& tbl_name, const std::string& part_name);
-  void recv_get_partition_by_name(Partition& _return);
-  void get_partitions(std::vector<Partition> & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_parts);
-  void send_get_partitions(const std::string& db_name, const std::string& tbl_name, const int16_t max_parts);
-  void recv_get_partitions(std::vector<Partition> & _return);
-  void get_partitions_with_auth(std::vector<Partition> & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_parts, const std::string& user_name, const std::vector<std::string> & group_names);
-  void send_get_partitions_with_auth(const std::string& db_name, const std::string& tbl_name, const int16_t max_parts, const std::string& user_name, const std::vector<std::string> & group_names);
-  void recv_get_partitions_with_auth(std::vector<Partition> & _return);
-  void get_partitions_pspec(std::vector<PartitionSpec> & _return, const std::string& db_name, const std::string& tbl_name, const int32_t max_parts);
-  void send_get_partitions_pspec(const std::string& db_name, const std::string& tbl_name, const int32_t max_parts);
-  void recv_get_partitions_pspec(std::vector<PartitionSpec> & _return);
-  void get_partition_names(std::vector<std::string> & _return, const std::string& db_name, const std::string& tbl_name, const int16_t max_parts);
-  void send_get_partition_names(const std::string& db_name, const std::string& tbl_name, const int16_t max_parts);
-  void recv_get_partition_names(std::vector<std::string> & _return);
-  void get_partition_values(PartitionValuesResponse& _return, const PartitionValuesRequest& request);
-  void send_get_partition_values(const PartitionValuesRequest& request);
-  void recv_get_partition_values(PartitionValuesResponse& _return);
-  void get_partitions_ps(std::vector<Partition> & _return, const std::string& db_name, const std::string& tbl_name, const std::vector<std::string> & part_vals, const int16_t max_parts);
-  void send_get_partitions_ps(const std::string& db_name, const std::string& tbl_name, const std::vector<std::string> & part_vals, const int16_t max_parts);
-  void recv_get_partitions_ps(std::vector<Partition> & _return);
-  void get_partitions_ps_with_auth(std::vector<Partition> & _return, const std::string& db_name, const std::string& tbl_name, const std::vector<std::string> & part_vals, const int16_t max_parts, const std::string& user_name, const std::vector<std::string> & group_names);
-  void send_get_partitions_ps_with_auth(const std::string& db_name, const std::string& tbl_name, const std::vector<std::string> & part_vals, const int16_t max_parts, const std::string& user_name, const std::vector<std::string> & group_names);
-  void recv_get_partitions_ps_with_auth(std::vector<Partition> & _return);
-  void get_partition_names_ps(std::vector<std::string> & _return, const std::string& db_name, const std::string& tbl_name, const std::vector<std::string> & part_vals, const int16_t max_parts);
-  void send_get_partition_names_ps(const std::string& db_name, const std::string& tbl_name, const std::vector<std::string> & part_vals, const int16_t max_parts);
-  void recv_get_partition_names_ps(std::vector<std::string> & _return);
-  void get_partitions_by_filter(std::vector<Partition> & _return, const std::string& db_name, const std::string& tbl_name, const std::string& filter, const int16_t max_parts);
+  bool operator != (const ThriftHiveMetastore_create_ischema_args &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_create_ischema_args & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+
+class ThriftHiveMetastore_create_ischema_pargs {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_create_ischema_pargs() throw();
+  const ISchema* schema;
+
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_create_ischema_result__isset {
+  _ThriftHiveMetastore_create_ischema_result__isset() : o1(false), o2(false), o3(false) {}
+  bool o1 :1;
+  bool o2 :1;
+  bool o3 :1;
+} _ThriftHiveMetastore_create_ischema_result__isset;
+
+class ThriftHiveMetastore_create_ischema_result {
+ public:
+
+  ThriftHiveMetastore_create_ischema_result(const ThriftHiveMetastore_create_ischema_result&);
+  ThriftHiveMetastore_create_ischema_result& operator=(const ThriftHiveMetastore_create_ischema_result&);
+  ThriftHiveMetastore_create_ischema_result() {
+  }
+
+  virtual ~ThriftHiveMetastore_create_ischema_result() throw();
+  AlreadyExistsException o1;
+  NoSuchObjectException o2;
+  MetaException o3;
+
+  _ThriftHiveMetastore_create_ischema_result__isset __isset;
+
+  void __set_o1(const AlreadyExistsException& val);
+
+  void __set_o2(const NoSuchObjectException& val);
+
+  void __set_o3(const MetaException& val);
+
+  bool operator == (const ThriftHiveMetastore_create_ischema_result & rhs) const
+  {
+    if (!(o1 == rhs.o1))
+      return false;
+    if (!(o2 == rhs.o2))
+      return false;
+    if (!(o3 == rhs.o3))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_create_ischema_result &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_create_ischema_result & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_create_ischema_presult__isset {
+  _ThriftHiveMetastore_create_ischema_presult__isset() : o1(false), o2(false), o3(false) {}
+  bool o1 :1;
+  bool o2 :1;
+  bool o3 :1;
+} _ThriftHiveMetastore_create_ischema_presult__isset;
+
+class ThriftHiveMetastore_create_ischema_presult {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_create_ischema_presult() throw();
+  AlreadyExistsException o1;
+  NoSuchObjectException o2;
+  MetaException o3;
+
+  _ThriftHiveMetastore_create_ischema_presult__isset __isset;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+
+};
+
+typedef struct _ThriftHiveMetastore_alter_ischema_args__isset {
+  _ThriftHiveMetastore_alter_ischema_args__isset() : rqst(false) {}
+  bool rqst :1;
+} _ThriftHiveMetastore_alter_ischema_args__isset;
+
+class ThriftHiveMetastore_alter_ischema_args {
+ public:
+
+  ThriftHiveMetastore_alter_ischema_args(const ThriftHiveMetastore_alter_ischema_args&);
+  ThriftHiveMetastore_alter_ischema_args& operator=(const ThriftHiveMetastore_alter_ischema_args&);
+  ThriftHiveMetastore_alter_ischema_args() {
+  }
+
+  virtual ~ThriftHiveMetastore_alter_ischema_args() throw();
+  AlterISchemaRequest rqst;
+
+  _ThriftHiveMetastore_alter_ischema_args__isset __isset;
+
+  void __set_rqst(const AlterISchemaRequest& val);
+
+  bool operator == (const ThriftHiveMetastore_alter_ischema_args & rhs) const
+  {
+    if (!(rqst == rhs.rqst))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_alter_ischema_args &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_alter_ischema_args & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+
+class ThriftHiveMetastore_alter_ischema_pargs {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_alter_ischema_pargs() throw();
+  const AlterISchemaRequest* rqst;
+
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_alter_ischema_result__isset {
+  _ThriftHiveMetastore_alter_ischema_result__isset() : o1(false), o2(false) {}
+  bool o1 :1;
+  bool o2 :1;
+} _ThriftHiveMetastore_alter_ischema_result__isset;
+
+class ThriftHiveMetastore_alter_ischema_result {
+ public:
+
+  ThriftHiveMetastore_alter_ischema_result(const ThriftHiveMetastore_alter_ischema_result&);
+  ThriftHiveMetastore_alter_ischema_result& operator=(const ThriftHiveMetastore_alter_ischema_result&);
+  ThriftHiveMetastore_alter_ischema_result() {
+  }
+
+  virtual ~ThriftHiveMetastore_alter_ischema_result() throw();
+  NoSuchObjectException o1;
+  MetaException o2;
+
+  _ThriftHiveMetastore_alter_ischema_result__isset __isset;
+
+  void __set_o1(const NoSuchObjectException& val);
+
+  void __set_o2(const MetaException& val);
+
+  bool operator == (const ThriftHiveMetastore_alter_ischema_result & rhs) const
+  {
+    if (!(o1 == rhs.o1))
+      return false;
+    if (!(o2 == rhs.o2))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_alter_ischema_result &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_alter_ischema_result & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_alter_ischema_presult__isset {
+  _ThriftHiveMetastore_alter_ischema_presult__isset() : o1(false), o2(false) {}
+  bool o1 :1;
+  bool o2 :1;
+} _ThriftHiveMetastore_alter_ischema_presult__isset;
+
+class ThriftHiveMetastore_alter_ischema_presult {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_alter_ischema_presult() throw();
+  NoSuchObjectException o1;
+  MetaException o2;
+
+  _ThriftHiveMetastore_alter_ischema_presult__isset __isset;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+
+};
+
+typedef struct _ThriftHiveMetastore_get_ischema_args__isset {
+  _ThriftHiveMetastore_get_ischema_args__isset() : name(false) {}
+  bool name :1;
+} _ThriftHiveMetastore_get_ischema_args__isset;
+
+class ThriftHiveMetastore_get_ischema_args {
+ public:
+
+  ThriftHiveMetastore_get_ischema_args(const ThriftHiveMetastore_get_ischema_args&);
+  ThriftHiveMetastore_get_ischema_args& operator=(const ThriftHiveMetastore_get_ischema_args&);
+  ThriftHiveMetastore_get_ischema_args() {
+  }
+
+  virtual ~ThriftHiveMetastore_get_ischema_args() throw();
+  ISchemaName name;
+
+  _ThriftHiveMetastore_get_ischema_args__isset __isset;
+
+  void __set_name(const ISchemaName& val);
+
+  bool operator == (const ThriftHiveMetastore_get_ischema_args & rhs) const
+  {
+    if (!(name == rhs.name))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_get_ischema_args &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_get_ischema_args & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+
+class ThriftHiveMetastore_get_ischema_pargs {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_get_ischema_pargs() throw();
+  const ISchemaName* name;
+
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_get_ischema_result__isset {
+  _ThriftHiveMetastore_get_ischema_result__isset() : success(false), o1(false), o2(false) {}
+  bool success :1;
+  bool o1 :1;
+  bool o2 :1;
+} _ThriftHiveMetastore_get_ischema_result__isset;
+
+class ThriftHiveMetastore_get_ischema_result {
+ public:
+
+  ThriftHiveMetastore_get_ischema_result(const ThriftHiveMetastore_get_ischema_result&);
+  ThriftHiveMetastore_get_ischema_result& operator=(const ThriftHiveMetastore_get_ischema_result&);
+  ThriftHiveMetastore_get_ischema_result() {
+  }
+
+  virtual ~ThriftHiveMetastore_get_ischema_result() throw();
+  ISchema success;
+  NoSuchObjectException o1;
+  MetaException o2;
+
+  _ThriftHiveMetastore_get_ischema_result__isset __isset;
+
+  void __set_success(const ISchema& val);
+
+  void __set_o1(const NoSuchObjectException& val);
+
+  void __set_o2(const MetaException& val);
+
+  bool operator == (const ThriftHiveMetastore_get_ischema_result & rhs) const
+  {
+    if (!(success == rhs.success))
+      return false;
+    if (!(o1 == rhs.o1))
+      return false;
+    if (!(o2 == rhs.o2))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_get_ischema_result &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_get_ischema_result & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_get_ischema_presult__isset {
+  _ThriftHiveMetastore_get_ischema_presult__isset() : success(false), o1(false), o2(false) {}
+  bool success :1;
+  bool o1 :1;
+  bool o2 :1;
+} _ThriftHiveMetastore_get_ischema_presult__isset;
+
+class ThriftHiveMetastore_get_ischema_presult {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_get_ischema_presult() throw();
+  ISchema* success;
+  NoSuchObjectException o1;
+  MetaException o2;
+
+  _ThriftHiveMetastore_get_ischema_presult__isset __isset;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+
+};
+
+typedef struct _ThriftHiveMetastore_drop_ischema_args__isset {
+  _ThriftHiveMetastore_drop_ischema_args__isset() : name(false) {}
+  bool name :1;
+} _ThriftHiveMetastore_drop_ischema_args__isset;
+
+class ThriftHiveMetastore_drop_ischema_args {
+ public:
+
+  ThriftHiveMetastore_drop_ischema_args(const ThriftHiveMetastore_drop_ischema_args&);
+  ThriftHiveMetastore_drop_ischema_args& operator=(const ThriftHiveMetastore_drop_ischema_args&);
+  ThriftHiveMetastore_drop_ischema_args() {
+  }
+
+  virtual ~ThriftHiveMetastore_drop_ischema_args() throw();
+  ISchemaName name;
+
+  _ThriftHiveMetastore_drop_ischema_args__isset __isset;
+
+  void __set_name(const ISchemaName& val);
+
+  bool operator == (const ThriftHiveMetastore_drop_ischema_args & rhs) const
+  {
+    if (!(name == rhs.name))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_drop_ischema_args &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_drop_ischema_args & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+
+class ThriftHiveMetastore_drop_ischema_pargs {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_drop_ischema_pargs() throw();
+  const ISchemaName* name;
+
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_drop_ischema_result__isset {
+  _ThriftHiveMetastore_drop_ischema_result__isset() : o1(false), o2(false), o3(false) {}
+  bool o1 :1;
+  bool o2 :1;
+  bool o3 :1;
+} _ThriftHiveMetastore_drop_ischema_result__isset;
+
+class ThriftHiveMetastore_drop_ischema_result {
+ public:
+
+  ThriftHiveMetastore_drop_ischema_result(const ThriftHiveMetastore_drop_ischema_result&);
+  ThriftHiveMetastore_drop_ischema_result& operator=(const ThriftHiveMetastore_drop_ischema_result&);
+  ThriftHiveMetastore_drop_ischema_result() {
+  }
+
+  virtual ~ThriftHiveMetastore_drop_ischema_result() throw();
+  NoSuchObjectException o1;
+  InvalidOperationException o2;
+  MetaException o3;
+
+  _ThriftHiveMetastore_drop_ischema_result__isset __isset;
+
+  void __set_o1(const NoSuchObjectException& val);
+
+  void __set_o2(const InvalidOperationException& val);
+
+  void __set_o3(const MetaException& val);
+
+  bool operator == (const ThriftHiveMetastore_drop_ischema_result & rhs) const
+  {
+    if (!(o1 == rhs.o1))
+      return false;
+    if (!(o2 == rhs.o2))
+      return false;
+    if (!(o3 == rhs.o3))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_drop_ischema_result &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_drop_ischema_result & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_drop_ischema_presult__isset {
+  _ThriftHiveMetastore_drop_ischema_presult__isset() : o1(false), o2(false), o3(false) {}
+  bool o1 :1;
+  bool o2 :1;
+  bool o3 :1;
+} _ThriftHiveMetastore_drop_ischema_presult__isset;
+
+class ThriftHiveMetastore_drop_ischema_presult {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_drop_ischema_presult() throw();
+  NoSuchObjectException o1;
+  InvalidOperationException o2;
+  MetaException o3;
+
+  _ThriftHiveMetastore_drop_ischema_presult__isset __isset;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+
+};
+
+typedef struct _ThriftHiveMetastore_add_schema_version_args__isset {
+  _ThriftHiveMetastore_add_schema_version_args__isset() : schemaVersion(false) {}
+  bool schemaVersion :1;
+} _ThriftHiveMetastore_add_schema_version_args__isset;
+
+class ThriftHiveMetastore_add_schema_version_args {
+ public:
+
+  ThriftHiveMetastore_add_schema_version_args(const ThriftHiveMetastore_add_schema_version_args&);
+  ThriftHiveMetastore_add_schema_version_args& operator=(const ThriftHiveMetastore_add_schema_version_args&);
+  ThriftHiveMetastore_add_schema_version_args() {
+  }
+
+  virtual ~ThriftHiveMetastore_add_schema_version_args() throw();
+  SchemaVersion schemaVersion;
+
+  _ThriftHiveMetastore_add_schema_version_args__isset __isset;
+
+  void __set_schemaVersion(const SchemaVersion& val);
+
+  bool operator == (const ThriftHiveMetastore_add_schema_version_args & rhs) const
+  {
+    if (!(schemaVersion == rhs.schemaVersion))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_add_schema_version_args &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_add_schema_version_args & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+
+class ThriftHiveMetastore_add_schema_version_pargs {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_add_schema_version_pargs() throw();
+  const SchemaVersion* schemaVersion;
+
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_add_schema_version_result__isset {
+  _ThriftHiveMetastore_add_schema_version_result__isset() : o1(false), o2(false), o3(false) {}
+  bool o1 :1;
+  bool o2 :1;
+  bool o3 :1;
+} _ThriftHiveMetastore_add_schema_version_result__isset;
+
+class ThriftHiveMetastore_add_schema_version_result {
+ public:
+
+  ThriftHiveMetastore_add_schema_version_result(const ThriftHiveMetastore_add_schema_version_result&);
+  ThriftHiveMetastore_add_schema_version_result& operator=(const ThriftHiveMetastore_add_schema_version_result&);
+  ThriftHiveMetastore_add_schema_version_result() {
+  }
+
+  virtual ~ThriftHiveMetastore_add_schema_version_result() throw();
+  AlreadyExistsException o1;
+  NoSuchObjectException o2;
+  MetaException o3;
+
+  _ThriftHiveMetastore_add_schema_version_result__isset __isset;
+
+  void __set_o1(const AlreadyExistsException& val);
+
+  void __set_o2(const NoSuchObjectException& val);
+
+  void __set_o3(const MetaException& val);
+
+  bool operator == (const ThriftHiveMetastore_add_schema_version_result & rhs) const
+  {
+    if (!(o1 == rhs.o1))
+      return false;
+    if (!(o2 == rhs.o2))
+      return false;
+    if (!(o3 == rhs.o3))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_add_schema_version_result &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_add_schema_version_result & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_add_schema_version_presult__isset {
+  _ThriftHiveMetastore_add_schema_version_presult__isset() : o1(false), o2(false), o3(false) {}
+  bool o1 :1;
+  bool o2 :1;
+  bool o3 :1;
+} _ThriftHiveMetastore_add_schema_version_presult__isset;
+
+class ThriftHiveMetastore_add_schema_version_presult {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_add_schema_version_presult() throw();
+  AlreadyExistsException o1;
+  NoSuchObjectException o2;
+  MetaException o3;
+
+  _ThriftHiveMetastore_add_schema_version_presult__isset __isset;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+
+};
+
+typedef struct _ThriftHiveMetastore_get_schema_version_args__isset {
+  _ThriftHiveMetastore_get_schema_version_args__isset() : schemaVersion(false) {}
+  bool schemaVersion :1;
+} _ThriftHiveMetastore_get_schema_version_args__isset;
+
+class ThriftHiveMetastore_get_schema_version_args {
+ public:
+
+  ThriftHiveMetastore_get_schema_version_args(const ThriftHiveMetastore_get_schema_version_args&);
+  ThriftHiveMetastore_get_schema_version_args& operator=(const ThriftHiveMetastore_get_schema_version_args&);
+  ThriftHiveMetastore_get_schema_version_args() {
+  }
+
+  virtual ~ThriftHiveMetastore_get_schema_version_args() throw();
+  SchemaVersionDescriptor schemaVersion;
+
+  _ThriftHiveMetastore_get_schema_version_args__isset __isset;
+
+  void __set_schemaVersion(const SchemaVersionDescriptor& val);
+
+  bool operator == (const ThriftHiveMetastore_get_schema_version_args & rhs) const
+  {
+    if (!(schemaVersion == rhs.schemaVersion))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_get_schema_version_args &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_get_schema_version_args & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+
+class ThriftHiveMetastore_get_schema_version_pargs {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_get_schema_version_pargs() throw();
+  const SchemaVersionDescriptor* schemaVersion;
+
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_get_schema_version_result__isset {
+  _ThriftHiveMetastore_get_schema_version_result__isset() : success(false), o1(false), o2(false) {}
+  bool success :1;
+  bool o1 :1;
+  bool o2 :1;
+} _ThriftHiveMetastore_get_schema_version_result__isset;
+
+class ThriftHiveMetastore_get_schema_version_result {
+ public:
+
+  ThriftHiveMetastore_get_schema_version_result(const ThriftHiveMetastore_get_schema_version_result&);
+  ThriftHiveMetastore_get_schema_version_result& operator=(const ThriftHiveMetastore_get_schema_version_result&);
+  ThriftHiveMetastore_get_schema_version_result() {
+  }
+
+  virtual ~ThriftHiveMetastore_get_schema_version_result() throw();
+  SchemaVersion success;
+  NoSuchObjectException o1;
+  MetaException o2;
+
+  _ThriftHiveMetastore_get_schema_version_result__isset __isset;
+
+  void __set_success(const SchemaVersion& val);
+
+  void __set_o1(const NoSuchObjectException& val);
+
+  void __set_o2(const MetaException& val);
+
+  bool operator == (const ThriftHiveMetastore_get_schema_version_result & rhs) const
+  {
+    if (!(success == rhs.success))
+      return false;
+    if (!(o1 == rhs.o1))
+      return false;
+    if (!(o2 == rhs.o2))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_get_schema_version_result &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_get_schema_version_result & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_get_schema_version_presult__isset {
+  _ThriftHiveMetastore_get_schema_version_presult__isset() : success(false), o1(false), o2(false) {}
+  bool success :1;
+  bool o1 :1;
+  bool o2 :1;
+} _ThriftHiveMetastore_get_schema_version_presult__isset;
+
+class ThriftHiveMetastore_get_schema_version_presult {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_get_schema_version_presult() throw();
+  SchemaVersion* success;
+  NoSuchObjectException o1;
+  MetaException o2;
+
+  _ThriftHiveMetastore_get_schema_version_presult__isset __isset;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+
+};
+
+typedef struct _ThriftHiveMetastore_get_schema_latest_version_args__isset {
+  _ThriftHiveMetastore_get_schema_latest_version_args__isset() : schemaName(false) {}
+  bool schemaName :1;
+} _ThriftHiveMetastore_get_schema_latest_version_args__isset;
+
+class ThriftHiveMetastore_get_schema_latest_version_args {
+ public:
+
+  ThriftHiveMetastore_get_schema_latest_version_args(const ThriftHiveMetastore_get_schema_latest_version_args&);
+  ThriftHiveMetastore_get_schema_latest_version_args& operator=(const ThriftHiveMetastore_get_schema_latest_version_args&);
+  ThriftHiveMetastore_get_schema_latest_version_args() {
+  }
+
+  virtual ~ThriftHiveMetastore_get_schema_latest_version_args() throw();
+  ISchemaName schemaName;
+
+  _ThriftHiveMetastore_get_schema_latest_version_args__isset __isset;
+
+  void __set_schemaName(const ISchemaName& val);
+
+  bool operator == (const ThriftHiveMetastore_get_schema_latest_version_args & rhs) const
+  {
+    if (!(schemaName == rhs.schemaName))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_get_schema_latest_version_args &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_get_schema_latest_version_args & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+
+class ThriftHiveMetastore_get_schema_latest_version_pargs {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_get_schema_latest_version_pargs() throw();
+  const ISchemaName* schemaName;
+
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_get_schema_latest_version_result__isset {
+  _ThriftHiveMetastore_get_schema_latest_version_result__isset() : success(false), o1(false), o2(false) {}
+  bool success :1;
+  bool o1 :1;
+  bool o2 :1;
+} _ThriftHiveMetastore_get_schema_latest_version_result__isset;
+
+class ThriftHiveMetastore_get_schema_latest_version_result {
+ public:
+
+  ThriftHiveMetastore_get_schema_latest_version_result(const ThriftHiveMetastore_get_schema_latest_version_result&);
+  ThriftHiveMetastore_get_schema_latest_version_result& operator=(const ThriftHiveMetastore_get_schema_latest_version_result&);
+  ThriftHiveMetastore_get_schema_latest_version_result() {
+  }
+
+  virtual ~ThriftHiveMetastore_get_schema_latest_version_result() throw();
+  SchemaVersion success;
+  NoSuchObjectException o1;
+  MetaException o2;
+
+  _ThriftHiveMetastore_get_schema_latest_version_result__isset __isset;
+
+  void __set_success(const SchemaVersion& val);
+
+  void __set_o1(const NoSuchObjectException& val);
+
+  void __set_o2(const MetaException& val);
+
+  bool operator == (const ThriftHiveMetastore_get_schema_latest_version_result & rhs) const
+  {
+    if (!(success == rhs.success))
+      return false;
+    if (!(o1 == rhs.o1))
+      return false;
+    if (!(o2 == rhs.o2))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_get_schema_latest_version_result &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_get_schema_latest_version_result & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_get_schema_latest_version_presult__isset {
+  _ThriftHiveMetastore_get_schema_latest_version_presult__isset() : success(false), o1(false), o2(false) {}
+  bool success :1;
+  bool o1 :1;
+  bool o2 :1;
+} _ThriftHiveMetastore_get_schema_latest_version_presult__isset;
+
+class ThriftHiveMetastore_get_schema_latest_version_presult {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_get_schema_latest_version_presult() throw();
+  SchemaVersion* success;
+  NoSuchObjectException o1;
+  MetaException o2;
+
+  _ThriftHiveMetastore_get_schema_latest_version_presult__isset __isset;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+
+};
+
+typedef struct _ThriftHiveMetastore_get_schema_all_versions_args__isset {
+  _ThriftHiveMetastore_get_schema_all_versions_args__isset() : schemaName(false) {}
+  bool schemaName :1;
+} _ThriftHiveMetastore_get_schema_all_versions_args__isset;
+
+class ThriftHiveMetastore_get_schema_all_versions_args {
+ public:
+
+  ThriftHiveMetastore_get_schema_all_versions_args(const ThriftHiveMetastore_get_schema_all_versions_args&);
+  ThriftHiveMetastore_get_schema_all_versions_args& operator=(const ThriftHiveMetastore_get_schema_all_versions_args&);
+  ThriftHiveMetastore_get_schema_all_versions_args() {
+  }
+
+  virtual ~ThriftHiveMetastore_get_schema_all_versions_args() throw();
+  ISchemaName schemaName;
+
+  _ThriftHiveMetastore_get_schema_all_versions_args__isset __isset;
+
+  void __set_schemaName(const ISchemaName& val);
+
+  bool operator == (const ThriftHiveMetastore_get_schema_all_versions_args & rhs) const
+  {
+    if (!(schemaName == rhs.schemaName))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_get_schema_all_versions_args &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_get_schema_all_versions_args & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+
+class ThriftHiveMetastore_get_schema_all_versions_pargs {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_get_schema_all_versions_pargs() throw();
+  const ISchemaName* schemaName;
+
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_get_schema_all_versions_result__isset {
+  _ThriftHiveMetastore_get_schema_all_versions_result__isset() : success(false), o1(false), o2(false) {}
+  bool success :1;
+  bool o1 :1;
+  bool o2 :1;
+} _ThriftHiveMetastore_get_schema_all_versions_result__isset;
+
+class ThriftHiveMetastore_get_schema_all_versions_result {
+ public:
+
+  ThriftHiveMetastore_get_schema_all_versions_result(const ThriftHiveMetastore_get_schema_all_versions_result&);
+  ThriftHiveMetastore_get_schema_all_versions_result& operator=(const ThriftHiveMetastore_get_schema_all_versions_result&);
+  ThriftHiveMetastore_get_schema_all_versions_result() {
+  }
+
+  virtual ~ThriftHiveMetastore_get_schema_all_versions_result() throw();
+  std::vector<SchemaVersion>  success;
+  NoSuchObjectException o1;
+  MetaException o2;
+
+  _ThriftHiveMetastore_get_schema_all_versions_result__isset __isset;
+
+  void __set_success(const std::vector<SchemaVersion> & val);
+
+  void __set_o1(const NoSuchObjectException& val);
+
+  void __set_o2(const MetaException& val);
+
+  bool operator == (const ThriftHiveMetastore_get_schema_all_versions_result & rhs) const
+  {
+    if (!(success == rhs.success))
+      return false;
+    if (!(o1 == rhs.o1))
+      return false;
+    if (!(o2 == rhs.o2))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_get_schema_all_versions_result &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_get_schema_all_versions_result & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_get_schema_all_versions_presult__isset {
+  _ThriftHiveMetastore_get_schema_all_versions_presult__isset() : success(false), o1(false), o2(false) {}
+  bool success :1;
+  bool o1 :1;
+  bool o2 :1;
+} _ThriftHiveMetastore_get_schema_all_versions_presult__isset;
+
+class ThriftHiveMetastore_get_schema_all_versions_presult {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_get_schema_all_versions_presult() throw();
+  std::vector<SchemaVersion> * success;
+  NoSuchObjectException o1;
+  MetaException o2;
+
+  _ThriftHiveMetastore_get_schema_all_versions_presult__isset __isset;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+
+};
+
+typedef struct _ThriftHiveMetastore_drop_schema_version_args__isset {
+  _ThriftHiveMetastore_drop_schema_version_args__isset() : schemaVersion(false) {}
+  bool schemaVersion :1;
+} _ThriftHiveMetastore_drop_schema_version_args__isset;
+
+class ThriftHiveMetastore_drop_schema_version_args {
+ public:
+
+  ThriftHiveMetastore_drop_schema_version_args(const ThriftHiveMetastore_drop_schema_version_args&);
+  ThriftHiveMetastore_drop_schema_version_args& operator=(const ThriftHiveMetastore_drop_schema_version_args&);
+  ThriftHiveMetastore_drop_schema_version_args() {
+  }
+
+  virtual ~ThriftHiveMetastore_drop_schema_version_args() throw();
+  SchemaVersionDescriptor schemaVersion;
+
+  _ThriftHiveMetastore_drop_schema_version_args__isset __isset;
+
+  void __set_schemaVersion(const SchemaVersionDescriptor& val);
+
+  bool operator == (const ThriftHiveMetastore_drop_schema_version_args & rhs) const
+  {
+    if (!(schemaVersion == rhs.schemaVersion))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_drop_schema_version_args &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_drop_schema_version_args & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+
+class ThriftHiveMetastore_drop_schema_version_pargs {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_drop_schema_version_pargs() throw();
+  const SchemaVersionDescriptor* schemaVersion;
+
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_drop_schema_version_result__isset {
+  _ThriftHiveMetastore_drop_schema_version_result__isset() : o1(false), o2(false) {}
+  bool o1 :1;
+  bool o2 :1;
+} _ThriftHiveMetastore_drop_schema_version_result__isset;
+
+class ThriftHiveMetastore_drop_schema_version_result {
+ public:
+
+  ThriftHiveMetastore_drop_schema_version_result(const ThriftHiveMetastore_drop_schema_version_result&);
+  ThriftHiveMetastore_drop_schema_version_result& operator=(const ThriftHiveMetastore_drop_schema_version_result&);
+  ThriftHiveMetastore_drop_schema_version_result() {
+  }
+
+  virtual ~ThriftHiveMetastore_drop_schema_version_result() throw();
+  NoSuchObjectException o1;
+  MetaException o2;
+
+  _ThriftHiveMetastore_drop_schema_version_result__isset __isset;
+
+  void __set_o1(const NoSuchObjectException& val);
+
+  void __set_o2(const MetaException& val);
+
+  bool operator == (const ThriftHiveMetastore_drop_schema_version_result & rhs) const
+  {
+    if (!(o1 == rhs.o1))
+      return false;
+    if (!(o2 == rhs.o2))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_drop_schema_version_result &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_drop_schema_version_result & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_drop_schema_version_presult__isset {
+  _ThriftHiveMetastore_drop_schema_version_presult__isset() : o1(false), o2(false) {}
+  bool o1 :1;
+  bool o2 :1;
+} _ThriftHiveMetastore_drop_schema_version_presult__isset;
+
+class ThriftHiveMetastore_drop_schema_version_presult {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_drop_schema_version_presult() throw();
+  NoSuchObjectException o1;
+  MetaException o2;
+
+  _ThriftHiveMetastore_drop_schema_version_presult__isset __isset;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+
+};
+
+typedef struct _ThriftHiveMetastore_get_schemas_by_cols_args__isset {
+  _ThriftHiveMetastore_get_schemas_by_cols_args__isset() : rqst(false) {}
+  bool rqst :1;
+} _ThriftHiveMetastore_get_schemas_by_cols_args__isset;
+
+class ThriftHiveMetastore_get_schemas_by_cols_args {
+ public:
+
+  ThriftHiveMetastore_get_schemas_by_cols_args(const ThriftHiveMetastore_get_schemas_by_cols_args&);
+  ThriftHiveMetastore_get_schemas_by_cols_args& operator=(const ThriftHiveMetastore_get_schemas_by_cols_args&);
+  ThriftHiveMetastore_get_schemas_by_cols_args() {
+  }
+
+  virtual ~ThriftHiveMetastore_get_schemas_by_cols_args() throw();
+  FindSchemasByColsRqst rqst;
+
+  _ThriftHiveMetastore_get_schemas_by_cols_args__isset __isset;
+
+  void __set_rqst(const FindSchemasByColsRqst& val);
+
+  bool operator == (const ThriftHiveMetastore_get_schemas_by_cols_args & rhs) const
+  {
+    if (!(rqst == rhs.rqst))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_get_schemas_by_cols_args &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_get_schemas_by_cols_args & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+
+class ThriftHiveMetastore_get_schemas_by_cols_pargs {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_get_schemas_by_cols_pargs() throw();
+  const FindSchemasByColsRqst* rqst;
+
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_get_schemas_by_cols_result__isset {
+  _ThriftHiveMetastore_get_schemas_by_cols_result__isset() : success(false), o1(false) {}
+  bool success :1;
+  bool o1 :1;
+} _ThriftHiveMetastore_get_schemas_by_cols_result__isset;
+
+class ThriftHiveMetastore_get_schemas_by_cols_result {
+ public:
+
+  ThriftHiveMetastore_get_schemas_by_cols_result(const ThriftHiveMetastore_get_schemas_by_cols_result&);
+  ThriftHiveMetastore_get_schemas_by_cols_result& operator=(const ThriftHiveMetastore_get_schemas_by_cols_result&);
+  ThriftHiveMetastore_get_schemas_by_cols_result() {
+  }
+
+  virtual ~ThriftHiveMetastore_get_schemas_by_cols_result() throw();
+  FindSchemasByColsResp success;
+  MetaException o1;
+
+  _ThriftHiveMetastore_get_schemas_by_cols_result__isset __isset;
+
+  void __set_success(const FindSchemasByColsResp& val);
+
+  void __set_o1(const MetaException& val);
+
+  bool operator == (const ThriftHiveMetastore_get_schemas_by_cols_result & rhs) const
+  {
+    if (!(success == rhs.success))
+      return false;
+    if (!(o1 == rhs.o1))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_get_schemas_by_cols_result &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_get_schemas_by_cols_result & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_get_schemas_by_cols_presult__isset {
+  _ThriftHiveMetastore_get_schemas_by_cols_presult__isset() : success(false), o1(false) {}
+  bool success :1;
+  bool o1 :1;
+} _ThriftHiveMetastore_get_schemas_by_cols_presult__isset;
+
+class ThriftHiveMetastore_get_schemas_by_cols_presult {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_get_schemas_by_cols_presult() throw();
+  FindSchemasByColsResp* success;
+  MetaException o1;
+
+  _ThriftHiveMetastore_get_schemas_by_cols_presult__isset __isset;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+
+};
+
+typedef struct _ThriftHiveMetastore_map_schema_version_to_serde_args__isset {
+  _ThriftHiveMetastore_map_schema_version_to_serde_args__isset() : rqst(false) {}
+  bool rqst :1;
+} _ThriftHiveMetastore_map_schema_version_to_serde_args__isset;
+
+class ThriftHiveMetastore_map_schema_version_to_serde_args {
+ public:
+
+  ThriftHiveMetastore_map_schema_version_to_serde_args(const ThriftHiveMetastore_map_schema_version_to_serde_args&);
+  ThriftHiveMetastore_map_schema_version_to_serde_args& operator=(const ThriftHiveMetastore_map_schema_version_to_serde_args&);
+  ThriftHiveMetastore_map_schema_version_to_serde_args() {
+  }
+
+  virtual ~ThriftHiveMetastore_map_schema_version_to_serde_args() throw();
+  MapSchemaVersionToSerdeRequest rqst;
+
+  _ThriftHiveMetastore_map_schema_version_to_serde_args__isset __isset;
+
+  void __set_rqst(const MapSchemaVersionToSerdeRequest& val);
+
+  bool operator == (const ThriftHiveMetastore_map_schema_version_to_serde_args & rhs) const
+  {
+    if (!(rqst == rhs.rqst))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_map_schema_version_to_serde_args &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_map_schema_version_to_serde_args & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+
+class ThriftHiveMetastore_map_schema_version_to_serde_pargs {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_map_schema_version_to_serde_pargs() throw();
+  const MapSchemaVersionToSerdeRequest* rqst;
+
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_map_schema_version_to_serde_result__isset {
+  _ThriftHiveMetastore_map_schema_version_to_serde_result__isset() : o1(false), o2(false) {}
+  bool o1 :1;
+  bool o2 :1;
+} _ThriftHiveMetastore_map_schema_version_to_serde_result__isset;
+
+class ThriftHiveMetastore_map_schema_version_to_serde_result {
+ public:
+
+  ThriftHiveMetastore_map_schema_version_to_serde_result(const ThriftHiveMetastore_map_schema_version_to_serde_result&);
+  ThriftHiveMetastore_map_schema_version_to_serde_result& operator=(const ThriftHiveMetastore_map_schema_version_to_serde_result&);
+  ThriftHiveMetastore_map_schema_version_to_serde_result() {
+  }
+
+  virtual ~ThriftHiveMetastore_map_schema_version_to_serde_result() throw();
+  NoSuchObjectException o1;
+  MetaException o2;
+
+  _ThriftHiveMetastore_map_schema_version_to_serde_result__isset __isset;
+
+  void __set_o1(const NoSuchObjectException& val);
+
+  void __set_o2(const MetaException& val);
+
+  bool operator == (const ThriftHiveMetastore_map_schema_version_to_serde_result & rhs) const
+  {
+    if (!(o1 == rhs.o1))
+      return false;
+    if (!(o2 == rhs.o2))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_map_schema_version_to_serde_result &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_map_schema_version_to_serde_result & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_map_schema_version_to_serde_presult__isset {
+  _ThriftHiveMetastore_map_schema_version_to_serde_presult__isset() : o1(false), o2(false) {}
+  bool o1 :1;
+  bool o2 :1;
+} _ThriftHiveMetastore_map_schema_version_to_serde_presult__isset;
+
+class ThriftHiveMetastore_map_schema_version_to_serde_presult {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_map_schema_version_to_serde_presult() throw();
+  NoSuchObjectException o1;
+  MetaException o2;
+
+  _ThriftHiveMetastore_map_schema_version_to_serde_presult__isset __isset;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+
+};
+
+typedef struct _ThriftHiveMetastore_set_schema_version_state_args__isset {
+  _ThriftHiveMetastore_set_schema_version_state_args__isset() : rqst(false) {}
+  bool rqst :1;
+} _ThriftHiveMetastore_set_schema_version_state_args__isset;
+
+class ThriftHiveMetastore_set_schema_version_state_args {
+ public:
+
+  ThriftHiveMetastore_set_schema_version_state_args(const ThriftHiveMetastore_set_schema_version_state_args&);
+  ThriftHiveMetastore_set_schema_version_state_args& operator=(const ThriftHiveMetastore_set_schema_version_state_args&);
+  ThriftHiveMetastore_set_schema_version_state_args() {
+  }
+
+  virtual ~ThriftHiveMetastore_set_schema_version_state_args() throw();
+  SetSchemaVersionStateRequest rqst;
+
+  _ThriftHiveMetastore_set_schema_version_state_args__isset __isset;
+
+  void __set_rqst(const SetSchemaVersionStateRequest& val);
+
+  bool operator == (const ThriftHiveMetastore_set_schema_version_state_args & rhs) const
+  {
+    if (!(rqst == rhs.rqst))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_set_schema_version_state_args &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_set_schema_version_state_args & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+
+class ThriftHiveMetastore_set_schema_version_state_pargs {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_set_schema_version_state_pargs() throw();
+  const SetSchemaVersionStateRequest* rqst;
+
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_set_schema_version_state_result__isset {
+  _ThriftHiveMetastore_set_schema_version_state_result__isset() : o1(false), o2(false), o3(false) {}
+  bool o1 :1;
+  bool o2 :1;
+  bool o3 :1;
+} _ThriftHiveMetastore_set_schema_version_state_result__isset;
+
+class ThriftHiveMetastore_set_schema_version_state_result {
+ public:
+
+  ThriftHiveMetastore_set_schema_version_state_result(const ThriftHiveMetastore_set_schema_version_state_result&);
+  ThriftHiveMetastore_set_schema_version_state_result& operator=(const ThriftHiveMetastore_set_schema_version_state_result&);
+  ThriftHiveMetastore_set_schema_version_state_result() {
+  }
+
+  virtual ~ThriftHiveMetastore_set_schema_version_state_result() throw();
+  NoSuchObjectException o1;
+  InvalidOperationException o2;
+  MetaException o3;
+
+  _ThriftHiveMetastore_set_schema_version_state_result__isset __isset;
+
+  void __set_o1(const NoSuchObjectException& val);
+
+  void __set_o2(const InvalidOperationException& val);
+
+  void __set_o3(const MetaException& val);
+
+  bool operator == (const ThriftHiveMetastore_set_schema_version_state_result & rhs) const
+  {
+    if (!(o1 == rhs.o1))
+      return false;
+    if (!(o2 == rhs.o2))
+      return false;
+    if (!(o3 == rhs.o3))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_set_schema_version_state_result &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_set_schema_version_state_result & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_set_schema_version_state_presult__isset {
+  _ThriftHiveMetastore_set_schema_version_state_presult__isset() : o1(false), o2(false), o3(false) {}
+  bool o1 :1;
+  bool o2 :1;
+  bool o3 :1;
+} _ThriftHiveMetastore_set_schema_version_state_presult__isset;
+
+class ThriftHiveMetastore_set_schema_version_state_presult {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_set_schema_version_state_presult() throw();
+  NoSuchObjectException o1;
+  InvalidOperationException o2;
+  MetaException o3;
+
+  _ThriftHiveMetastore_set_schema_version_state_presult__isset __isset;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+
+};
+
+typedef struct _ThriftHiveMetastore_add_serde_args__isset {
+  _ThriftHiveMetastore_add_serde_args__isset() : serde(false) {}
+  bool serde :1;
+} _ThriftHiveMetastore_add_serde_args__isset;
+
+class ThriftHiveMetastore_add_serde_args {
+ public:
+
+  ThriftHiveMetastore_add_serde_args(const ThriftHiveMetastore_add_serde_args&);
+  ThriftHiveMetastore_add_serde_args& operator=(const ThriftHiveMetastore_add_serde_args&);
+  ThriftHiveMetastore_add_serde_args() {
+  }
+
+  virtual ~ThriftHiveMetastore_add_serde_args() throw();
+  SerDeInfo serde;
+
+  _ThriftHiveMetastore_add_serde_args__isset __isset;
+
+  void __set_serde(const SerDeInfo& val);
+
+  bool operator == (const ThriftHiveMetastore_add_serde_args & rhs) const
+  {
+    if (!(serde == rhs.serde))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_add_serde_args &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_add_serde_args & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+
+class ThriftHiveMetastore_add_serde_pargs {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_add_serde_pargs() throw();
+  const SerDeInfo* serde;
+
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_add_serde_result__isset {
+  _ThriftHiveMetastore_add_serde_result__isset() : o1(false), o2(false) {}
+  bool o1 :1;
+  bool o2 :1;
+} _ThriftHiveMetastore_add_serde_result__isset;
+
+class ThriftHiveMetastore_add_serde_result {
+ public:
+
+  ThriftHiveMetastore_add_serde_result(const ThriftHiveMetastore_add_serde_result&);
+  ThriftHiveMetastore_add_serde_result& operator=(const ThriftHiveMetastore_add_serde_result&);
+  ThriftHiveMetastore_add_serde_result() {
+  }
+
+  virtual ~ThriftHiveMetastore_add_serde_result() throw();
+  AlreadyExistsException o1;
+  MetaException o2;
+
+  _ThriftHiveMetastore_add_serde_result__isset __isset;
+
+  void __set_o1(const AlreadyExistsException& val);
+
+  void __set_o2(const MetaException& val);
+
+  bool operator == (const ThriftHiveMetastore_add_serde_result & rhs) const
+  {
+    if (!(o1 == rhs.o1))
+      return false;
+    if (!(o2 == rhs.o2))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_add_serde_result &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_add_serde_result & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_add_serde_presult__isset {
+  _ThriftHiveMetastore_add_serde_presult__isset() : o1(false), o2(false) {}
+  bool o1 :1;
+  bool o2 :1;
+} _ThriftHiveMetastore_add_serde_presult__isset;
+
+class ThriftHiveMetastore_add_serde_presult {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_add_serde_presult() throw();
+  AlreadyExistsException o1;
+  MetaException o2;
+
+  _ThriftHiveMetastore_add_serde_presult__isset __isset;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+
+};
+
+typedef struct _ThriftHiveMetastore_get_serde_args__isset {
+  _ThriftHiveMetastore_get_serde_args__isset() : rqst(false) {}
+  bool rqst :1;
+} _ThriftHiveMetastore_get_serde_args__isset;
+
+class ThriftHiveMetastore_get_serde_args {
+ public:
+
+  ThriftHiveMetastore_get_serde_args(const ThriftHiveMetastore_get_serde_args&);
+  ThriftHiveMetastore_get_serde_args& operator=(const ThriftHiveMetastore_get_serde_args&);
+  ThriftHiveMetastore_get_serde_args() {
+  }
+
+  virtual ~ThriftHiveMetastore_get_serde_args() throw();
+  GetSerdeRequest rqst;
+
+  _ThriftHiveMetastore_get_serde_args__isset __isset;
+
+  void __set_rqst(const GetSerdeRequest& val);
+
+  bool operator == (const ThriftHiveMetastore_get_serde_args & rhs) const
+  {
+    if (!(rqst == rhs.rqst))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_get_serde_args &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_get_serde_args & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+
+class ThriftHiveMetastore_get_serde_pargs {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_get_serde_pargs() throw();
+  const GetSerdeRequest* rqst;
+
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_get_serde_result__isset {
+  _ThriftHiveMetastore_get_serde_result__isset() : success(false), o1(false), o2(false) {}
+  bool success :1;
+  bool o1 :1;
+  bool o2 :1;
+} _ThriftHiveMetastore_get_serde_result__isset;
+
+class ThriftHiveMetastore_get_serde_result {
+ public:
+
+  ThriftHiveMetastore_get_serde_result(const ThriftHiveMetastore_get_serde_result&);
+  ThriftHiveMetastore_get_serde_result& operator=(const ThriftHiveMetastore_get_serde_result&);
+  ThriftHiveMetastore_get_serde_result() {
+  }
+
+  virtual ~ThriftHiveMetastore_get_serde_result() throw();
+  SerDeInfo success;
+  NoSuchObjectException o1;
+  MetaException o2;
+
+  _ThriftHiveMetastore_get_serde_result__isset __isset;
+
+  void __set_success(const SerDeInfo& val);
+
+  void __set_o1(const NoSuchObjectException& val);
+
+  void __set_o2(const MetaException& val);
+
+  bool operator == (const ThriftHiveMetastore_get_serde_result & rhs) const
+  {
+    if (!(success == rhs.success))
+      return false;
+    if (!(o1 == rhs.o1))
+      return false;
+    if (!(o2 == rhs.o2))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_get_serde_result &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_get_serde_result & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_get_serde_presult__isset {
+  _ThriftHiveMetastore_get_serde_presult__isset() : success(false), o1(false), o2(false) {}
+  bool success :1;
+  bool o1 :1;
+  bool o2 :1;
+} _ThriftHiveMetastore_get_serde_presult__isset;
+
+class ThriftHiveMetastore_get_serde_presult {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_get_serde_presult() throw();
+  SerDeInfo* success;
+  NoSuchObjectException o1;
+  MetaException o2;
+
+  _ThriftHiveMetastore_get_serde_presult__isset __isset;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+
+};
+
+class ThriftHiveMetastoreClient : virtual public ThriftHiveMetastoreIf, public  ::facebook::fb303::FacebookServiceClient {
+ public:
+  ThriftHiveMetastoreClient(boost::shared_ptr< ::apache::thrift::protocol::TProtocol> prot) :
+     ::facebook::fb303::FacebookServiceClient(prot, prot) {}
+  ThriftHiveMetastoreClient(boost::shared_ptr< ::apache::thrift::protocol::TProtocol> iprot, boost::shared_ptr< ::apache::thrift::protocol::TProtocol> oprot) :     ::facebook::fb303::FacebookServiceClient(iprot, oprot) {}
+  boost::shared_ptr< ::apache::thrift::protocol::TProtocol> getInputProtocol() {
+    return piprot_;
+  }
+  boost::shared_ptr< ::apache::thrift::protocol::TProtocol> getOutputProtocol() {
+    return poprot_;
+  }
+  void getMetaConf(std::string& _return, const std::string& key);
+  void send_getMetaConf(const std::string& key);
+  void recv_getMetaConf(std::string& _return);
+  void setMetaConf(const std::string& key, const std::string& value);
+  void send_setMetaConf(const std::string& key, const std::string& value);
+  void recv_setMetaConf();
+  void create_database(const Database& database);
+  void send_create_database(const Database& database);
+  void recv_create_database();
+  void get_database(Database& _return, const std::string& name);
+  void send_get_database(const std::string& name);
+  void recv_get_database(Database& _return);
+  void drop_database(const std::string& name, const bool deleteData, const bool cascade);
+  void send_drop_database(const std::string& name, const bool deleteData, const bool cascade);
+  void recv_drop_database();
+  void get_databases(std::vector<std::string> & _return, const std::string& pattern);
+  void send_get_databases(const std::string& pattern);
+  void recv_get_databases(std::vector<std::string> & _return);
+  void get_all_databases(std::vector<std::string> & _return);
+  void send_get_all_databases();
+  void recv_get_all_databases(std::vector<std::string> & _return);
+  void alter_database(const std::string& dbname, const Database& db);
+  void send_alter_database(const std::string& dbname, const Database& db);
+  void recv_alter_database();
+  void get_type(Type& _return, const std::string& name);
+  void send_get_type(const std::string& name);
+  void recv_get_type(Type& _return);
+  bool create_type(const Type& type);
+  void send_create_type(const Type& type);
+  bool recv_create_type();
+  bool drop_type(const std::string& type);
+  void send_drop_type(const std::string& type);
+  bool recv_drop_type();
+  void get_type_all(std::map<std::string, Type> & _return, const std::string& name);
+  void send_get_type_all(const std::string& name);
+  void recv_get_type_all(std::map<std::string, Type> & _return);
+  void get_fields(std::vector<FieldSchema> & _return, const std::string& db_name, const std::string& table_name);
+  void send_get_fields(const std::string& db_name, const std::string& table_name);
+  void recv_get_fields(std::vector<FieldSchema> & _return);
+  void get_fields_with_environment_context(std::vector<FieldSchema> & _return, const std::string& db_name, const std::string& table_name, const EnvironmentContext& environment_context);
+  void send_get_fields_with_environment_context(const std::string& db_name, const std::string& table_name, const EnvironmentContext& environment_context);
+  void recv_get_fields_with_environment_context(std::vector<FieldSchema> & _return);
+  void get_schema(std::vector<FieldSchema> & _return, const std::string& db_name, const std::string& table_name);
+  void send_get_schema(const std::string& db_name, const std::string& table_name);
+  void recv_get_schema(std::vector<FieldSchema> & _return);
+  void get_schema_with_environment_context(std::vector<FieldSchema> & _return, const std::string& db_name, const std::string& table_name, const EnvironmentContext& environment_context);
+  void send_get_schema_with_environment_context(const std::string& db_name, const std::string& table_name, const EnvironmentContext& environment_context);
+  void recv_get_schema_with_environment_context(std::vector<FieldSchema> & _return);
+  void create_table(const Table& tbl);
+  void send_create_table(const Table& tbl);
+  void recv_create_table();
+  void create_table_with_environment_context(const Table& tbl, const EnvironmentContext& environment_context);
+  void send_create_table_with_environment_context(const Table& tbl, const EnvironmentContext& environment_context);
+  void recv_create_table_with_environment_context();
+  void create_table_with_constraints(const Table& tbl, const std::vector<SQLPrimaryKey> & primaryKeys, const std::vector<SQLForeignKey> & foreignKeys, const std::vector<SQLUniqueConstraint> & uniqueConstraints, const std::vector<SQLNotNullConstraint> & notNullConstraints, const std::vector<SQLDefaultConstraint> & defaultConstraints);
+  void send_create_table_with_constraints(const Table& tbl, const std::vector<SQLPrimaryKey> & primaryKeys, const std::vector<SQLForeignKey> & foreignKeys, const std::vector<SQLUniqueConstraint> & uniqueConstraints, const std::vector<SQLNotNullConstraint> & notNullConstraints, const std::vector<SQLDefaultConstraint> & defaultConstraints);
+  void recv_create_table_with_constraints();
+  void drop_constraint(const DropConstraintRequest& req);
+  void send_drop_constraint(const DropConstraintRequest& req);
+  void recv_drop_constraint();
+  void add_primary_key(const AddPrimaryKeyRequest& req);
+  void send_add_primary_key(const AddPrimaryKeyRequest& req);
+  void recv_add_primary_key();
+  void add_foreign_key(const AddForeignKeyRequest& req);
+  void send_add_foreign_key(const AddForeignKeyRequest& req);
+  void recv_add_foreign_key();
+  void add_unique_constraint(const AddUniqueConstraintRequest& req);
+  void send_add_unique_constraint(const AddUniqueConstraintRequest& req);
+  void recv_add_unique_constraint();
+  void add_not_null_constraint(const AddNotNullConstraintRequest& req);
+  void send_add_not_null_constraint(const AddNotNullConstraintRequest& req);
+  void recv_add_not_null_constraint();
+  void add_default_constraint(const AddDefaultConstraintRequest& req);
+  void send_add_default_constraint(const AddDefaultConstraintRequest& req);
+  void recv_add_default_constraint();
+  void drop_table(const std::string& dbname, const std::string& name, const bool deleteData);
+  void send_drop_table(const std::string& dbname, const std::string& name, const bool deleteData);
+  void recv_drop_table();
+  void drop_table_with_environment_context(const std::string& dbname, const std::string& name, const bool deleteData, const EnvironmentContext& environment_context);
+  void send_drop_table_with_environment_context(const std::string& dbname, const std::string& name, const bool deleteData, const EnvironmentContext& environment_context);
+  void recv_drop_table_with_environment_context();
+  void truncate_table(const std::string& dbName, const std::string& tableName, const std::vector<std::string> & partNames);
+  void send_truncate_table(const std::string& dbName, const std::string& tableName, const std::vector<std::string> & partNames);
+  void recv_truncate_table();
+  void get_tables(std::vector<std::string> & _return, const std::string& db_name, const std::string& pattern);
+  void send_get_tables(const std::string& db_name, const std::string& pattern);
+  void recv_get_tables(std::vector<std::string> & _return);
+  void get_tables_by_type(std::vector<std::string> & _return, const std::string& db_name, const std::string& pattern, const std::string& tableType);
+  void send_get_tables_by_type(const std::string& db_name, const std::string& pattern, const std::string& tableType);
+  void recv_get_tables_by_type(std::vector<std::string> & _return);
+  void get_materialized_views_for_rewriting(std::vector<std::string> & _return, const std::string& db_name);
+  void send_get_materialized_views_for_rewriting(const std::string& db_name);
+  void recv_get_materialized_views_for_rewriting(std::vector<std::string> & _return);
+  void get_table_meta(std::vector<TableMeta> & _return, const std::string& db_patterns, const std::string& tbl_patterns, const std::vector<std::string> & tbl_types);
+  void send_get_table_meta(const std::string& db_patterns, const std::string& tbl_patterns, const std::vector<std::string> & tbl_types);
+  void recv_get_table_meta(std::vector<TableMeta> & _return);
+  void get_all_tables(std::vector<std::string> & _return, const std::string& db_name);
+  void send_get_all_tables(const std::string& db_name);
+  void recv_get_all_tables(std::vector<std::string> & _return);
+  void get_table(Table& _return, const std::string& dbname, const std::string& tbl_name);
+  void send_get_table(const std::string& dbname, const std::string& tbl_name);
+  void recv_get_table(Table& _return);
+  void get_table_objects_by_name(std::vector<Table> & _return, const std::string& dbname, const std::vector<std::string> & tbl_names);
+  void send_get_table_objects_by_name(const std::string& dbname, const std::vector<std::string> & tbl_names);
+  void recv_get_table_objects_by_name(std::vector<Table> & _return);
+  void get_table_req(GetTableResult& _return, const GetTableRequest& req);
+  void send_get_table_req(const GetTableRequest& req);
+  void recv_get_table_req(GetTableResult& _return);
+  void get_table_objects_by_name_req(GetTablesResult& _return, const GetTablesRequest& req);
+  void send_get_table_objects_by_name_req(const GetTablesRequest& req);
+  void recv_get_table_objects_by_name_req(GetTablesResult& _return);
+  void get_materialization_invalidation_info(std::map<std::string, Materialization> & _return, const std::string& dbname, const std::vector<std::string> & tbl_names);
+  void send_get_materialization_invalidation_info(const std::string& dbname, const std::vector<std::string> & tbl_names);
+  void recv_get_materialization_invalidation_info(std::map<std::string, Materialization> & _return);
+  void update_creation_metadata(const std::string& dbname, const std::string& tbl_name, const CreationMetadata& creation_metadata);
+  void send_update_creation_metadata(const std::string& dbname, const std::string& tbl_name, const CreationMetadata& creation_metadata);
+  void recv_update_creation_metadata();
+  void get_table_names_by_filter(std::vector<std::string> & _return, const std::string& dbname, const std::string& filter, const int16_t max_tables);
+  void send_get_table_names_by_filter(const std::string& dbname, const std::string& filter, const int16_t max_tables);
+  void recv_get_table_names_by_filter(std::vector<std::string> & _return);
+  void alter_table(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl);
+  void send_alter_table(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl);
+  void recv_alter_table();
+  void alter_table_with_environment_context(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl, const EnvironmentContext& environment_context);
+  void send_alter_table_with_environment_context(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl, const EnvironmentContext& environment_context);
+  void recv_alter_table_with_environment_context();
+  void alter_table_with_cascade(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl, const bool cascade);
+  void send_alter_table_with_cascade(const std::string& dbname, const std::string& tbl_name, const Table& new_tbl, const bool cascade);
+  void recv_alter_table_with_cascade();
+  void add_partition(Partition& _return, const Partition& new_part);
+  void send_add_partition(const Partition& new_part);
+  void recv_add_partition(Partition& _return);
+  void add_partition_with_environment_context(Partition& _return, const Partition& new_part, const EnvironmentContext& environment_context);
+  void send

<TRUNCATED>

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

Posted by ga...@apache.org.
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';
 


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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaVersion.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaVersion.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaVersion.java
new file mode 100644
index 0000000..fb0be40
--- /dev/null
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaVersion.java
@@ -0,0 +1,1412 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hive.metastore.api;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+@org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public class SchemaVersion implements org.apache.thrift.TBase<SchemaVersion, SchemaVersion._Fields>, java.io.Serializable, Cloneable, Comparable<SchemaVersion> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SchemaVersion");
+
+  private static final org.apache.thrift.protocol.TField SCHEMA_FIELD_DESC = new org.apache.thrift.protocol.TField("schema", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+  private static final org.apache.thrift.protocol.TField VERSION_FIELD_DESC = new org.apache.thrift.protocol.TField("version", org.apache.thrift.protocol.TType.I32, (short)2);
+  private static final org.apache.thrift.protocol.TField CREATED_AT_FIELD_DESC = new org.apache.thrift.protocol.TField("createdAt", org.apache.thrift.protocol.TType.I64, (short)3);
+  private static final org.apache.thrift.protocol.TField COLS_FIELD_DESC = new org.apache.thrift.protocol.TField("cols", org.apache.thrift.protocol.TType.LIST, (short)4);
+  private static final org.apache.thrift.protocol.TField STATE_FIELD_DESC = new org.apache.thrift.protocol.TField("state", org.apache.thrift.protocol.TType.I32, (short)5);
+  private static final org.apache.thrift.protocol.TField DESCRIPTION_FIELD_DESC = new org.apache.thrift.protocol.TField("description", org.apache.thrift.protocol.TType.STRING, (short)6);
+  private static final org.apache.thrift.protocol.TField SCHEMA_TEXT_FIELD_DESC = new org.apache.thrift.protocol.TField("schemaText", org.apache.thrift.protocol.TType.STRING, (short)7);
+  private static final org.apache.thrift.protocol.TField FINGERPRINT_FIELD_DESC = new org.apache.thrift.protocol.TField("fingerprint", org.apache.thrift.protocol.TType.STRING, (short)8);
+  private static final org.apache.thrift.protocol.TField NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("name", org.apache.thrift.protocol.TType.STRING, (short)9);
+  private static final org.apache.thrift.protocol.TField SER_DE_FIELD_DESC = new org.apache.thrift.protocol.TField("serDe", org.apache.thrift.protocol.TType.STRUCT, (short)10);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new SchemaVersionStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new SchemaVersionTupleSchemeFactory());
+  }
+
+  private ISchemaName schema; // required
+  private int version; // required
+  private long createdAt; // required
+  private List<FieldSchema> cols; // required
+  private SchemaVersionState state; // optional
+  private String description; // optional
+  private String schemaText; // optional
+  private String fingerprint; // optional
+  private String name; // optional
+  private SerDeInfo serDe; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    SCHEMA((short)1, "schema"),
+    VERSION((short)2, "version"),
+    CREATED_AT((short)3, "createdAt"),
+    COLS((short)4, "cols"),
+    /**
+     * 
+     * @see SchemaVersionState
+     */
+    STATE((short)5, "state"),
+    DESCRIPTION((short)6, "description"),
+    SCHEMA_TEXT((short)7, "schemaText"),
+    FINGERPRINT((short)8, "fingerprint"),
+    NAME((short)9, "name"),
+    SER_DE((short)10, "serDe");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // SCHEMA
+          return SCHEMA;
+        case 2: // VERSION
+          return VERSION;
+        case 3: // CREATED_AT
+          return CREATED_AT;
+        case 4: // COLS
+          return COLS;
+        case 5: // STATE
+          return STATE;
+        case 6: // DESCRIPTION
+          return DESCRIPTION;
+        case 7: // SCHEMA_TEXT
+          return SCHEMA_TEXT;
+        case 8: // FINGERPRINT
+          return FINGERPRINT;
+        case 9: // NAME
+          return NAME;
+        case 10: // SER_DE
+          return SER_DE;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __VERSION_ISSET_ID = 0;
+  private static final int __CREATEDAT_ISSET_ID = 1;
+  private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.STATE,_Fields.DESCRIPTION,_Fields.SCHEMA_TEXT,_Fields.FINGERPRINT,_Fields.NAME,_Fields.SER_DE};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.SCHEMA, new org.apache.thrift.meta_data.FieldMetaData("schema", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ISchemaName.class)));
+    tmpMap.put(_Fields.VERSION, new org.apache.thrift.meta_data.FieldMetaData("version", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.CREATED_AT, new org.apache.thrift.meta_data.FieldMetaData("createdAt", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    tmpMap.put(_Fields.COLS, new org.apache.thrift.meta_data.FieldMetaData("cols", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, FieldSchema.class))));
+    tmpMap.put(_Fields.STATE, new org.apache.thrift.meta_data.FieldMetaData("state", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, SchemaVersionState.class)));
+    tmpMap.put(_Fields.DESCRIPTION, new org.apache.thrift.meta_data.FieldMetaData("description", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.SCHEMA_TEXT, new org.apache.thrift.meta_data.FieldMetaData("schemaText", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.FINGERPRINT, new org.apache.thrift.meta_data.FieldMetaData("fingerprint", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.NAME, new org.apache.thrift.meta_data.FieldMetaData("name", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.SER_DE, new org.apache.thrift.meta_data.FieldMetaData("serDe", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, SerDeInfo.class)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(SchemaVersion.class, metaDataMap);
+  }
+
+  public SchemaVersion() {
+  }
+
+  public SchemaVersion(
+    ISchemaName schema,
+    int version,
+    long createdAt,
+    List<FieldSchema> cols)
+  {
+    this();
+    this.schema = schema;
+    this.version = version;
+    setVersionIsSet(true);
+    this.createdAt = createdAt;
+    setCreatedAtIsSet(true);
+    this.cols = cols;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public SchemaVersion(SchemaVersion other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.isSetSchema()) {
+      this.schema = new ISchemaName(other.schema);
+    }
+    this.version = other.version;
+    this.createdAt = other.createdAt;
+    if (other.isSetCols()) {
+      List<FieldSchema> __this__cols = new ArrayList<FieldSchema>(other.cols.size());
+      for (FieldSchema other_element : other.cols) {
+        __this__cols.add(new FieldSchema(other_element));
+      }
+      this.cols = __this__cols;
+    }
+    if (other.isSetState()) {
+      this.state = other.state;
+    }
+    if (other.isSetDescription()) {
+      this.description = other.description;
+    }
+    if (other.isSetSchemaText()) {
+      this.schemaText = other.schemaText;
+    }
+    if (other.isSetFingerprint()) {
+      this.fingerprint = other.fingerprint;
+    }
+    if (other.isSetName()) {
+      this.name = other.name;
+    }
+    if (other.isSetSerDe()) {
+      this.serDe = new SerDeInfo(other.serDe);
+    }
+  }
+
+  public SchemaVersion deepCopy() {
+    return new SchemaVersion(this);
+  }
+
+  @Override
+  public void clear() {
+    this.schema = null;
+    setVersionIsSet(false);
+    this.version = 0;
+    setCreatedAtIsSet(false);
+    this.createdAt = 0;
+    this.cols = null;
+    this.state = null;
+    this.description = null;
+    this.schemaText = null;
+    this.fingerprint = null;
+    this.name = null;
+    this.serDe = null;
+  }
+
+  public ISchemaName getSchema() {
+    return this.schema;
+  }
+
+  public void setSchema(ISchemaName schema) {
+    this.schema = schema;
+  }
+
+  public void unsetSchema() {
+    this.schema = null;
+  }
+
+  /** Returns true if field schema is set (has been assigned a value) and false otherwise */
+  public boolean isSetSchema() {
+    return this.schema != null;
+  }
+
+  public void setSchemaIsSet(boolean value) {
+    if (!value) {
+      this.schema = null;
+    }
+  }
+
+  public int getVersion() {
+    return this.version;
+  }
+
+  public void setVersion(int version) {
+    this.version = version;
+    setVersionIsSet(true);
+  }
+
+  public void unsetVersion() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __VERSION_ISSET_ID);
+  }
+
+  /** Returns true if field version is set (has been assigned a value) and false otherwise */
+  public boolean isSetVersion() {
+    return EncodingUtils.testBit(__isset_bitfield, __VERSION_ISSET_ID);
+  }
+
+  public void setVersionIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __VERSION_ISSET_ID, value);
+  }
+
+  public long getCreatedAt() {
+    return this.createdAt;
+  }
+
+  public void setCreatedAt(long createdAt) {
+    this.createdAt = createdAt;
+    setCreatedAtIsSet(true);
+  }
+
+  public void unsetCreatedAt() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __CREATEDAT_ISSET_ID);
+  }
+
+  /** Returns true if field createdAt is set (has been assigned a value) and false otherwise */
+  public boolean isSetCreatedAt() {
+    return EncodingUtils.testBit(__isset_bitfield, __CREATEDAT_ISSET_ID);
+  }
+
+  public void setCreatedAtIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __CREATEDAT_ISSET_ID, value);
+  }
+
+  public int getColsSize() {
+    return (this.cols == null) ? 0 : this.cols.size();
+  }
+
+  public java.util.Iterator<FieldSchema> getColsIterator() {
+    return (this.cols == null) ? null : this.cols.iterator();
+  }
+
+  public void addToCols(FieldSchema elem) {
+    if (this.cols == null) {
+      this.cols = new ArrayList<FieldSchema>();
+    }
+    this.cols.add(elem);
+  }
+
+  public List<FieldSchema> getCols() {
+    return this.cols;
+  }
+
+  public void setCols(List<FieldSchema> cols) {
+    this.cols = cols;
+  }
+
+  public void unsetCols() {
+    this.cols = null;
+  }
+
+  /** Returns true if field cols is set (has been assigned a value) and false otherwise */
+  public boolean isSetCols() {
+    return this.cols != null;
+  }
+
+  public void setColsIsSet(boolean value) {
+    if (!value) {
+      this.cols = null;
+    }
+  }
+
+  /**
+   * 
+   * @see SchemaVersionState
+   */
+  public SchemaVersionState getState() {
+    return this.state;
+  }
+
+  /**
+   * 
+   * @see SchemaVersionState
+   */
+  public void setState(SchemaVersionState state) {
+    this.state = state;
+  }
+
+  public void unsetState() {
+    this.state = null;
+  }
+
+  /** Returns true if field state is set (has been assigned a value) and false otherwise */
+  public boolean isSetState() {
+    return this.state != null;
+  }
+
+  public void setStateIsSet(boolean value) {
+    if (!value) {
+      this.state = null;
+    }
+  }
+
+  public String getDescription() {
+    return this.description;
+  }
+
+  public void setDescription(String description) {
+    this.description = description;
+  }
+
+  public void unsetDescription() {
+    this.description = null;
+  }
+
+  /** Returns true if field description is set (has been assigned a value) and false otherwise */
+  public boolean isSetDescription() {
+    return this.description != null;
+  }
+
+  public void setDescriptionIsSet(boolean value) {
+    if (!value) {
+      this.description = null;
+    }
+  }
+
+  public String getSchemaText() {
+    return this.schemaText;
+  }
+
+  public void setSchemaText(String schemaText) {
+    this.schemaText = schemaText;
+  }
+
+  public void unsetSchemaText() {
+    this.schemaText = null;
+  }
+
+  /** Returns true if field schemaText is set (has been assigned a value) and false otherwise */
+  public boolean isSetSchemaText() {
+    return this.schemaText != null;
+  }
+
+  public void setSchemaTextIsSet(boolean value) {
+    if (!value) {
+      this.schemaText = null;
+    }
+  }
+
+  public String getFingerprint() {
+    return this.fingerprint;
+  }
+
+  public void setFingerprint(String fingerprint) {
+    this.fingerprint = fingerprint;
+  }
+
+  public void unsetFingerprint() {
+    this.fingerprint = null;
+  }
+
+  /** Returns true if field fingerprint is set (has been assigned a value) and false otherwise */
+  public boolean isSetFingerprint() {
+    return this.fingerprint != null;
+  }
+
+  public void setFingerprintIsSet(boolean value) {
+    if (!value) {
+      this.fingerprint = null;
+    }
+  }
+
+  public String getName() {
+    return this.name;
+  }
+
+  public void setName(String name) {
+    this.name = name;
+  }
+
+  public void unsetName() {
+    this.name = null;
+  }
+
+  /** Returns true if field name is set (has been assigned a value) and false otherwise */
+  public boolean isSetName() {
+    return this.name != null;
+  }
+
+  public void setNameIsSet(boolean value) {
+    if (!value) {
+      this.name = null;
+    }
+  }
+
+  public SerDeInfo getSerDe() {
+    return this.serDe;
+  }
+
+  public void setSerDe(SerDeInfo serDe) {
+    this.serDe = serDe;
+  }
+
+  public void unsetSerDe() {
+    this.serDe = null;
+  }
+
+  /** Returns true if field serDe is set (has been assigned a value) and false otherwise */
+  public boolean isSetSerDe() {
+    return this.serDe != null;
+  }
+
+  public void setSerDeIsSet(boolean value) {
+    if (!value) {
+      this.serDe = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case SCHEMA:
+      if (value == null) {
+        unsetSchema();
+      } else {
+        setSchema((ISchemaName)value);
+      }
+      break;
+
+    case VERSION:
+      if (value == null) {
+        unsetVersion();
+      } else {
+        setVersion((Integer)value);
+      }
+      break;
+
+    case CREATED_AT:
+      if (value == null) {
+        unsetCreatedAt();
+      } else {
+        setCreatedAt((Long)value);
+      }
+      break;
+
+    case COLS:
+      if (value == null) {
+        unsetCols();
+      } else {
+        setCols((List<FieldSchema>)value);
+      }
+      break;
+
+    case STATE:
+      if (value == null) {
+        unsetState();
+      } else {
+        setState((SchemaVersionState)value);
+      }
+      break;
+
+    case DESCRIPTION:
+      if (value == null) {
+        unsetDescription();
+      } else {
+        setDescription((String)value);
+      }
+      break;
+
+    case SCHEMA_TEXT:
+      if (value == null) {
+        unsetSchemaText();
+      } else {
+        setSchemaText((String)value);
+      }
+      break;
+
+    case FINGERPRINT:
+      if (value == null) {
+        unsetFingerprint();
+      } else {
+        setFingerprint((String)value);
+      }
+      break;
+
+    case NAME:
+      if (value == null) {
+        unsetName();
+      } else {
+        setName((String)value);
+      }
+      break;
+
+    case SER_DE:
+      if (value == null) {
+        unsetSerDe();
+      } else {
+        setSerDe((SerDeInfo)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case SCHEMA:
+      return getSchema();
+
+    case VERSION:
+      return getVersion();
+
+    case CREATED_AT:
+      return getCreatedAt();
+
+    case COLS:
+      return getCols();
+
+    case STATE:
+      return getState();
+
+    case DESCRIPTION:
+      return getDescription();
+
+    case SCHEMA_TEXT:
+      return getSchemaText();
+
+    case FINGERPRINT:
+      return getFingerprint();
+
+    case NAME:
+      return getName();
+
+    case SER_DE:
+      return getSerDe();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case SCHEMA:
+      return isSetSchema();
+    case VERSION:
+      return isSetVersion();
+    case CREATED_AT:
+      return isSetCreatedAt();
+    case COLS:
+      return isSetCols();
+    case STATE:
+      return isSetState();
+    case DESCRIPTION:
+      return isSetDescription();
+    case SCHEMA_TEXT:
+      return isSetSchemaText();
+    case FINGERPRINT:
+      return isSetFingerprint();
+    case NAME:
+      return isSetName();
+    case SER_DE:
+      return isSetSerDe();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof SchemaVersion)
+      return this.equals((SchemaVersion)that);
+    return false;
+  }
+
+  public boolean equals(SchemaVersion that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_schema = true && this.isSetSchema();
+    boolean that_present_schema = true && that.isSetSchema();
+    if (this_present_schema || that_present_schema) {
+      if (!(this_present_schema && that_present_schema))
+        return false;
+      if (!this.schema.equals(that.schema))
+        return false;
+    }
+
+    boolean this_present_version = true;
+    boolean that_present_version = true;
+    if (this_present_version || that_present_version) {
+      if (!(this_present_version && that_present_version))
+        return false;
+      if (this.version != that.version)
+        return false;
+    }
+
+    boolean this_present_createdAt = true;
+    boolean that_present_createdAt = true;
+    if (this_present_createdAt || that_present_createdAt) {
+      if (!(this_present_createdAt && that_present_createdAt))
+        return false;
+      if (this.createdAt != that.createdAt)
+        return false;
+    }
+
+    boolean this_present_cols = true && this.isSetCols();
+    boolean that_present_cols = true && that.isSetCols();
+    if (this_present_cols || that_present_cols) {
+      if (!(this_present_cols && that_present_cols))
+        return false;
+      if (!this.cols.equals(that.cols))
+        return false;
+    }
+
+    boolean this_present_state = true && this.isSetState();
+    boolean that_present_state = true && that.isSetState();
+    if (this_present_state || that_present_state) {
+      if (!(this_present_state && that_present_state))
+        return false;
+      if (!this.state.equals(that.state))
+        return false;
+    }
+
+    boolean this_present_description = true && this.isSetDescription();
+    boolean that_present_description = true && that.isSetDescription();
+    if (this_present_description || that_present_description) {
+      if (!(this_present_description && that_present_description))
+        return false;
+      if (!this.description.equals(that.description))
+        return false;
+    }
+
+    boolean this_present_schemaText = true && this.isSetSchemaText();
+    boolean that_present_schemaText = true && that.isSetSchemaText();
+    if (this_present_schemaText || that_present_schemaText) {
+      if (!(this_present_schemaText && that_present_schemaText))
+        return false;
+      if (!this.schemaText.equals(that.schemaText))
+        return false;
+    }
+
+    boolean this_present_fingerprint = true && this.isSetFingerprint();
+    boolean that_present_fingerprint = true && that.isSetFingerprint();
+    if (this_present_fingerprint || that_present_fingerprint) {
+      if (!(this_present_fingerprint && that_present_fingerprint))
+        return false;
+      if (!this.fingerprint.equals(that.fingerprint))
+        return false;
+    }
+
+    boolean this_present_name = true && this.isSetName();
+    boolean that_present_name = true && that.isSetName();
+    if (this_present_name || that_present_name) {
+      if (!(this_present_name && that_present_name))
+        return false;
+      if (!this.name.equals(that.name))
+        return false;
+    }
+
+    boolean this_present_serDe = true && this.isSetSerDe();
+    boolean that_present_serDe = true && that.isSetSerDe();
+    if (this_present_serDe || that_present_serDe) {
+      if (!(this_present_serDe && that_present_serDe))
+        return false;
+      if (!this.serDe.equals(that.serDe))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_schema = true && (isSetSchema());
+    list.add(present_schema);
+    if (present_schema)
+      list.add(schema);
+
+    boolean present_version = true;
+    list.add(present_version);
+    if (present_version)
+      list.add(version);
+
+    boolean present_createdAt = true;
+    list.add(present_createdAt);
+    if (present_createdAt)
+      list.add(createdAt);
+
+    boolean present_cols = true && (isSetCols());
+    list.add(present_cols);
+    if (present_cols)
+      list.add(cols);
+
+    boolean present_state = true && (isSetState());
+    list.add(present_state);
+    if (present_state)
+      list.add(state.getValue());
+
+    boolean present_description = true && (isSetDescription());
+    list.add(present_description);
+    if (present_description)
+      list.add(description);
+
+    boolean present_schemaText = true && (isSetSchemaText());
+    list.add(present_schemaText);
+    if (present_schemaText)
+      list.add(schemaText);
+
+    boolean present_fingerprint = true && (isSetFingerprint());
+    list.add(present_fingerprint);
+    if (present_fingerprint)
+      list.add(fingerprint);
+
+    boolean present_name = true && (isSetName());
+    list.add(present_name);
+    if (present_name)
+      list.add(name);
+
+    boolean present_serDe = true && (isSetSerDe());
+    list.add(present_serDe);
+    if (present_serDe)
+      list.add(serDe);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(SchemaVersion other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetSchema()).compareTo(other.isSetSchema());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetSchema()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.schema, other.schema);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetVersion()).compareTo(other.isSetVersion());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetVersion()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.version, other.version);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetCreatedAt()).compareTo(other.isSetCreatedAt());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCreatedAt()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.createdAt, other.createdAt);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetCols()).compareTo(other.isSetCols());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCols()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.cols, other.cols);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetState()).compareTo(other.isSetState());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetState()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.state, other.state);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetDescription()).compareTo(other.isSetDescription());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetDescription()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.description, other.description);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetSchemaText()).compareTo(other.isSetSchemaText());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetSchemaText()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.schemaText, other.schemaText);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetFingerprint()).compareTo(other.isSetFingerprint());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetFingerprint()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.fingerprint, other.fingerprint);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetName()).compareTo(other.isSetName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.name, other.name);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetSerDe()).compareTo(other.isSetSerDe());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetSerDe()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.serDe, other.serDe);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("SchemaVersion(");
+    boolean first = true;
+
+    sb.append("schema:");
+    if (this.schema == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.schema);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("version:");
+    sb.append(this.version);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("createdAt:");
+    sb.append(this.createdAt);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("cols:");
+    if (this.cols == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.cols);
+    }
+    first = false;
+    if (isSetState()) {
+      if (!first) sb.append(", ");
+      sb.append("state:");
+      if (this.state == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.state);
+      }
+      first = false;
+    }
+    if (isSetDescription()) {
+      if (!first) sb.append(", ");
+      sb.append("description:");
+      if (this.description == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.description);
+      }
+      first = false;
+    }
+    if (isSetSchemaText()) {
+      if (!first) sb.append(", ");
+      sb.append("schemaText:");
+      if (this.schemaText == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.schemaText);
+      }
+      first = false;
+    }
+    if (isSetFingerprint()) {
+      if (!first) sb.append(", ");
+      sb.append("fingerprint:");
+      if (this.fingerprint == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.fingerprint);
+      }
+      first = false;
+    }
+    if (isSetName()) {
+      if (!first) sb.append(", ");
+      sb.append("name:");
+      if (this.name == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.name);
+      }
+      first = false;
+    }
+    if (isSetSerDe()) {
+      if (!first) sb.append(", ");
+      sb.append("serDe:");
+      if (this.serDe == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.serDe);
+      }
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+    if (schema != null) {
+      schema.validate();
+    }
+    if (serDe != null) {
+      serDe.validate();
+    }
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class SchemaVersionStandardSchemeFactory implements SchemeFactory {
+    public SchemaVersionStandardScheme getScheme() {
+      return new SchemaVersionStandardScheme();
+    }
+  }
+
+  private static class SchemaVersionStandardScheme extends StandardScheme<SchemaVersion> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, SchemaVersion struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // SCHEMA
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.schema = new ISchemaName();
+              struct.schema.read(iprot);
+              struct.setSchemaIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // VERSION
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.version = iprot.readI32();
+              struct.setVersionIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // CREATED_AT
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.createdAt = iprot.readI64();
+              struct.setCreatedAtIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // COLS
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list864 = iprot.readListBegin();
+                struct.cols = new ArrayList<FieldSchema>(_list864.size);
+                FieldSchema _elem865;
+                for (int _i866 = 0; _i866 < _list864.size; ++_i866)
+                {
+                  _elem865 = new FieldSchema();
+                  _elem865.read(iprot);
+                  struct.cols.add(_elem865);
+                }
+                iprot.readListEnd();
+              }
+              struct.setColsIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 5: // STATE
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.state = org.apache.hadoop.hive.metastore.api.SchemaVersionState.findByValue(iprot.readI32());
+              struct.setStateIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 6: // DESCRIPTION
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.description = iprot.readString();
+              struct.setDescriptionIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 7: // SCHEMA_TEXT
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.schemaText = iprot.readString();
+              struct.setSchemaTextIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 8: // FINGERPRINT
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.fingerprint = iprot.readString();
+              struct.setFingerprintIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 9: // NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.name = iprot.readString();
+              struct.setNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 10: // SER_DE
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.serDe = new SerDeInfo();
+              struct.serDe.read(iprot);
+              struct.setSerDeIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, SchemaVersion struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.schema != null) {
+        oprot.writeFieldBegin(SCHEMA_FIELD_DESC);
+        struct.schema.write(oprot);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldBegin(VERSION_FIELD_DESC);
+      oprot.writeI32(struct.version);
+      oprot.writeFieldEnd();
+      oprot.writeFieldBegin(CREATED_AT_FIELD_DESC);
+      oprot.writeI64(struct.createdAt);
+      oprot.writeFieldEnd();
+      if (struct.cols != null) {
+        oprot.writeFieldBegin(COLS_FIELD_DESC);
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.cols.size()));
+          for (FieldSchema _iter867 : struct.cols)
+          {
+            _iter867.write(oprot);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      if (struct.state != null) {
+        if (struct.isSetState()) {
+          oprot.writeFieldBegin(STATE_FIELD_DESC);
+          oprot.writeI32(struct.state.getValue());
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.description != null) {
+        if (struct.isSetDescription()) {
+          oprot.writeFieldBegin(DESCRIPTION_FIELD_DESC);
+          oprot.writeString(struct.description);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.schemaText != null) {
+        if (struct.isSetSchemaText()) {
+          oprot.writeFieldBegin(SCHEMA_TEXT_FIELD_DESC);
+          oprot.writeString(struct.schemaText);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.fingerprint != null) {
+        if (struct.isSetFingerprint()) {
+          oprot.writeFieldBegin(FINGERPRINT_FIELD_DESC);
+          oprot.writeString(struct.fingerprint);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.name != null) {
+        if (struct.isSetName()) {
+          oprot.writeFieldBegin(NAME_FIELD_DESC);
+          oprot.writeString(struct.name);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.serDe != null) {
+        if (struct.isSetSerDe()) {
+          oprot.writeFieldBegin(SER_DE_FIELD_DESC);
+          struct.serDe.write(oprot);
+          oprot.writeFieldEnd();
+        }
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class SchemaVersionTupleSchemeFactory implements SchemeFactory {
+    public SchemaVersionTupleScheme getScheme() {
+      return new SchemaVersionTupleScheme();
+    }
+  }
+
+  private static class SchemaVersionTupleScheme extends TupleScheme<SchemaVersion> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, SchemaVersion struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.isSetSchema()) {
+        optionals.set(0);
+      }
+      if (struct.isSetVersion()) {
+        optionals.set(1);
+      }
+      if (struct.isSetCreatedAt()) {
+        optionals.set(2);
+      }
+      if (struct.isSetCols()) {
+        optionals.set(3);
+      }
+      if (struct.isSetState()) {
+        optionals.set(4);
+      }
+      if (struct.isSetDescription()) {
+        optionals.set(5);
+      }
+      if (struct.isSetSchemaText()) {
+        optionals.set(6);
+      }
+      if (struct.isSetFingerprint()) {
+        optionals.set(7);
+      }
+      if (struct.isSetName()) {
+        optionals.set(8);
+      }
+      if (struct.isSetSerDe()) {
+        optionals.set(9);
+      }
+      oprot.writeBitSet(optionals, 10);
+      if (struct.isSetSchema()) {
+        struct.schema.write(oprot);
+      }
+      if (struct.isSetVersion()) {
+        oprot.writeI32(struct.version);
+      }
+      if (struct.isSetCreatedAt()) {
+        oprot.writeI64(struct.createdAt);
+      }
+      if (struct.isSetCols()) {
+        {
+          oprot.writeI32(struct.cols.size());
+          for (FieldSchema _iter868 : struct.cols)
+          {
+            _iter868.write(oprot);
+          }
+        }
+      }
+      if (struct.isSetState()) {
+        oprot.writeI32(struct.state.getValue());
+      }
+      if (struct.isSetDescription()) {
+        oprot.writeString(struct.description);
+      }
+      if (struct.isSetSchemaText()) {
+        oprot.writeString(struct.schemaText);
+      }
+      if (struct.isSetFingerprint()) {
+        oprot.writeString(struct.fingerprint);
+      }
+      if (struct.isSetName()) {
+        oprot.writeString(struct.name);
+      }
+      if (struct.isSetSerDe()) {
+        struct.serDe.write(oprot);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, SchemaVersion struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(10);
+      if (incoming.get(0)) {
+        struct.schema = new ISchemaName();
+        struct.schema.read(iprot);
+        struct.setSchemaIsSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.version = iprot.readI32();
+        struct.setVersionIsSet(true);
+      }
+      if (incoming.get(2)) {
+        struct.createdAt = iprot.readI64();
+        struct.setCreatedAtIsSet(true);
+      }
+      if (incoming.get(3)) {
+        {
+          org.apache.thrift.protocol.TList _list869 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.cols = new ArrayList<FieldSchema>(_list869.size);
+          FieldSchema _elem870;
+          for (int _i871 = 0; _i871 < _list869.size; ++_i871)
+          {
+            _elem870 = new FieldSchema();
+            _elem870.read(iprot);
+            struct.cols.add(_elem870);
+          }
+        }
+        struct.setColsIsSet(true);
+      }
+      if (incoming.get(4)) {
+        struct.state = org.apache.hadoop.hive.metastore.api.SchemaVersionState.findByValue(iprot.readI32());
+        struct.setStateIsSet(true);
+      }
+      if (incoming.get(5)) {
+        struct.description = iprot.readString();
+        struct.setDescriptionIsSet(true);
+      }
+      if (incoming.get(6)) {
+        struct.schemaText = iprot.readString();
+        struct.setSchemaTextIsSet(true);
+      }
+      if (incoming.get(7)) {
+        struct.fingerprint = iprot.readString();
+        struct.setFingerprintIsSet(true);
+      }
+      if (incoming.get(8)) {
+        struct.name = iprot.readString();
+        struct.setNameIsSet(true);
+      }
+      if (incoming.get(9)) {
+        struct.serDe = new SerDeInfo();
+        struct.serDe.read(iprot);
+        struct.setSerDeIsSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaVersionDescriptor.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaVersionDescriptor.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaVersionDescriptor.java
new file mode 100644
index 0000000..1df8b41
--- /dev/null
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaVersionDescriptor.java
@@ -0,0 +1,502 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hive.metastore.api;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+@org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public class SchemaVersionDescriptor implements org.apache.thrift.TBase<SchemaVersionDescriptor, SchemaVersionDescriptor._Fields>, java.io.Serializable, Cloneable, Comparable<SchemaVersionDescriptor> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SchemaVersionDescriptor");
+
+  private static final org.apache.thrift.protocol.TField SCHEMA_FIELD_DESC = new org.apache.thrift.protocol.TField("schema", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+  private static final org.apache.thrift.protocol.TField VERSION_FIELD_DESC = new org.apache.thrift.protocol.TField("version", org.apache.thrift.protocol.TType.I32, (short)2);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new SchemaVersionDescriptorStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new SchemaVersionDescriptorTupleSchemeFactory());
+  }
+
+  private ISchemaName schema; // required
+  private int version; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    SCHEMA((short)1, "schema"),
+    VERSION((short)2, "version");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // SCHEMA
+          return SCHEMA;
+        case 2: // VERSION
+          return VERSION;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __VERSION_ISSET_ID = 0;
+  private byte __isset_bitfield = 0;
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.SCHEMA, new org.apache.thrift.meta_data.FieldMetaData("schema", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ISchemaName.class)));
+    tmpMap.put(_Fields.VERSION, new org.apache.thrift.meta_data.FieldMetaData("version", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(SchemaVersionDescriptor.class, metaDataMap);
+  }
+
+  public SchemaVersionDescriptor() {
+  }
+
+  public SchemaVersionDescriptor(
+    ISchemaName schema,
+    int version)
+  {
+    this();
+    this.schema = schema;
+    this.version = version;
+    setVersionIsSet(true);
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public SchemaVersionDescriptor(SchemaVersionDescriptor other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.isSetSchema()) {
+      this.schema = new ISchemaName(other.schema);
+    }
+    this.version = other.version;
+  }
+
+  public SchemaVersionDescriptor deepCopy() {
+    return new SchemaVersionDescriptor(this);
+  }
+
+  @Override
+  public void clear() {
+    this.schema = null;
+    setVersionIsSet(false);
+    this.version = 0;
+  }
+
+  public ISchemaName getSchema() {
+    return this.schema;
+  }
+
+  public void setSchema(ISchemaName schema) {
+    this.schema = schema;
+  }
+
+  public void unsetSchema() {
+    this.schema = null;
+  }
+
+  /** Returns true if field schema is set (has been assigned a value) and false otherwise */
+  public boolean isSetSchema() {
+    return this.schema != null;
+  }
+
+  public void setSchemaIsSet(boolean value) {
+    if (!value) {
+      this.schema = null;
+    }
+  }
+
+  public int getVersion() {
+    return this.version;
+  }
+
+  public void setVersion(int version) {
+    this.version = version;
+    setVersionIsSet(true);
+  }
+
+  public void unsetVersion() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __VERSION_ISSET_ID);
+  }
+
+  /** Returns true if field version is set (has been assigned a value) and false otherwise */
+  public boolean isSetVersion() {
+    return EncodingUtils.testBit(__isset_bitfield, __VERSION_ISSET_ID);
+  }
+
+  public void setVersionIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __VERSION_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case SCHEMA:
+      if (value == null) {
+        unsetSchema();
+      } else {
+        setSchema((ISchemaName)value);
+      }
+      break;
+
+    case VERSION:
+      if (value == null) {
+        unsetVersion();
+      } else {
+        setVersion((Integer)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case SCHEMA:
+      return getSchema();
+
+    case VERSION:
+      return getVersion();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case SCHEMA:
+      return isSetSchema();
+    case VERSION:
+      return isSetVersion();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof SchemaVersionDescriptor)
+      return this.equals((SchemaVersionDescriptor)that);
+    return false;
+  }
+
+  public boolean equals(SchemaVersionDescriptor that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_schema = true && this.isSetSchema();
+    boolean that_present_schema = true && that.isSetSchema();
+    if (this_present_schema || that_present_schema) {
+      if (!(this_present_schema && that_present_schema))
+        return false;
+      if (!this.schema.equals(that.schema))
+        return false;
+    }
+
+    boolean this_present_version = true;
+    boolean that_present_version = true;
+    if (this_present_version || that_present_version) {
+      if (!(this_present_version && that_present_version))
+        return false;
+      if (this.version != that.version)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_schema = true && (isSetSchema());
+    list.add(present_schema);
+    if (present_schema)
+      list.add(schema);
+
+    boolean present_version = true;
+    list.add(present_version);
+    if (present_version)
+      list.add(version);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(SchemaVersionDescriptor other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetSchema()).compareTo(other.isSetSchema());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetSchema()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.schema, other.schema);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetVersion()).compareTo(other.isSetVersion());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetVersion()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.version, other.version);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("SchemaVersionDescriptor(");
+    boolean first = true;
+
+    sb.append("schema:");
+    if (this.schema == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.schema);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("version:");
+    sb.append(this.version);
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+    if (schema != null) {
+      schema.validate();
+    }
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class SchemaVersionDescriptorStandardSchemeFactory implements SchemeFactory {
+    public SchemaVersionDescriptorStandardScheme getScheme() {
+      return new SchemaVersionDescriptorStandardScheme();
+    }
+  }
+
+  private static class SchemaVersionDescriptorStandardScheme extends StandardScheme<SchemaVersionDescriptor> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, SchemaVersionDescriptor struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // SCHEMA
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.schema = new ISchemaName();
+              struct.schema.read(iprot);
+              struct.setSchemaIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // VERSION
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.version = iprot.readI32();
+              struct.setVersionIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, SchemaVersionDescriptor struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.schema != null) {
+        oprot.writeFieldBegin(SCHEMA_FIELD_DESC);
+        struct.schema.write(oprot);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldBegin(VERSION_FIELD_DESC);
+      oprot.writeI32(struct.version);
+      oprot.writeFieldEnd();
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class SchemaVersionDescriptorTupleSchemeFactory implements SchemeFactory {
+    public SchemaVersionDescriptorTupleScheme getScheme() {
+      return new SchemaVersionDescriptorTupleScheme();
+    }
+  }
+
+  private static class SchemaVersionDescriptorTupleScheme extends TupleScheme<SchemaVersionDescriptor> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, SchemaVersionDescriptor struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.isSetSchema()) {
+        optionals.set(0);
+      }
+      if (struct.isSetVersion()) {
+        optionals.set(1);
+      }
+      oprot.writeBitSet(optionals, 2);
+      if (struct.isSetSchema()) {
+        struct.schema.write(oprot);
+      }
+      if (struct.isSetVersion()) {
+        oprot.writeI32(struct.version);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, SchemaVersionDescriptor struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(2);
+      if (incoming.get(0)) {
+        struct.schema = new ISchemaName();
+        struct.schema.read(iprot);
+        struct.setSchemaIsSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.version = iprot.readI32();
+        struct.setVersionIsSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaVersionState.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaVersionState.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaVersionState.java
new file mode 100644
index 0000000..9dd96dc
--- /dev/null
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaVersionState.java
@@ -0,0 +1,63 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hive.metastore.api;
+
+
+import java.util.Map;
+import java.util.HashMap;
+import org.apache.thrift.TEnum;
+
+public enum SchemaVersionState implements org.apache.thrift.TEnum {
+  INITIATED(1),
+  START_REVIEW(2),
+  CHANGES_REQUIRED(3),
+  REVIEWED(4),
+  ENABLED(5),
+  DISABLED(6),
+  ARCHIVED(7),
+  DELETED(8);
+
+  private final int value;
+
+  private SchemaVersionState(int value) {
+    this.value = value;
+  }
+
+  /**
+   * Get the integer value of this enum value, as defined in the Thrift IDL.
+   */
+  public int getValue() {
+    return value;
+  }
+
+  /**
+   * Find a the enum type by its integer value, as defined in the Thrift IDL.
+   * @return null if the value is not found.
+   */
+  public static SchemaVersionState findByValue(int value) { 
+    switch (value) {
+      case 1:
+        return INITIATED;
+      case 2:
+        return START_REVIEW;
+      case 3:
+        return CHANGES_REQUIRED;
+      case 4:
+        return REVIEWED;
+      case 5:
+        return ENABLED;
+      case 6:
+        return DISABLED;
+      case 7:
+        return ARCHIVED;
+      case 8:
+        return DELETED;
+      default:
+        return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SerDeInfo.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SerDeInfo.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SerDeInfo.java
index b744f44..a7aba9f 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SerDeInfo.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SerDeInfo.java
@@ -41,6 +41,10 @@ import org.slf4j.LoggerFactory;
   private static final org.apache.thrift.protocol.TField NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("name", org.apache.thrift.protocol.TType.STRING, (short)1);
   private static final org.apache.thrift.protocol.TField SERIALIZATION_LIB_FIELD_DESC = new org.apache.thrift.protocol.TField("serializationLib", org.apache.thrift.protocol.TType.STRING, (short)2);
   private static final org.apache.thrift.protocol.TField PARAMETERS_FIELD_DESC = new org.apache.thrift.protocol.TField("parameters", org.apache.thrift.protocol.TType.MAP, (short)3);
+  private static final org.apache.thrift.protocol.TField DESCRIPTION_FIELD_DESC = new org.apache.thrift.protocol.TField("description", org.apache.thrift.protocol.TType.STRING, (short)4);
+  private static final org.apache.thrift.protocol.TField SERIALIZER_CLASS_FIELD_DESC = new org.apache.thrift.protocol.TField("serializerClass", org.apache.thrift.protocol.TType.STRING, (short)5);
+  private static final org.apache.thrift.protocol.TField DESERIALIZER_CLASS_FIELD_DESC = new org.apache.thrift.protocol.TField("deserializerClass", org.apache.thrift.protocol.TType.STRING, (short)6);
+  private static final org.apache.thrift.protocol.TField SERDE_TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("serdeType", org.apache.thrift.protocol.TType.I32, (short)7);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -51,12 +55,24 @@ import org.slf4j.LoggerFactory;
   private String name; // required
   private String serializationLib; // required
   private Map<String,String> parameters; // required
+  private String description; // optional
+  private String serializerClass; // optional
+  private String deserializerClass; // optional
+  private SerdeType serdeType; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
     NAME((short)1, "name"),
     SERIALIZATION_LIB((short)2, "serializationLib"),
-    PARAMETERS((short)3, "parameters");
+    PARAMETERS((short)3, "parameters"),
+    DESCRIPTION((short)4, "description"),
+    SERIALIZER_CLASS((short)5, "serializerClass"),
+    DESERIALIZER_CLASS((short)6, "deserializerClass"),
+    /**
+     * 
+     * @see SerdeType
+     */
+    SERDE_TYPE((short)7, "serdeType");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -77,6 +93,14 @@ import org.slf4j.LoggerFactory;
           return SERIALIZATION_LIB;
         case 3: // PARAMETERS
           return PARAMETERS;
+        case 4: // DESCRIPTION
+          return DESCRIPTION;
+        case 5: // SERIALIZER_CLASS
+          return SERIALIZER_CLASS;
+        case 6: // DESERIALIZER_CLASS
+          return DESERIALIZER_CLASS;
+        case 7: // SERDE_TYPE
+          return SERDE_TYPE;
         default:
           return null;
       }
@@ -117,6 +141,7 @@ import org.slf4j.LoggerFactory;
   }
 
   // isset id assignments
+  private static final _Fields optionals[] = {_Fields.DESCRIPTION,_Fields.SERIALIZER_CLASS,_Fields.DESERIALIZER_CLASS,_Fields.SERDE_TYPE};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -128,6 +153,14 @@ import org.slf4j.LoggerFactory;
         new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, 
             new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), 
             new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+    tmpMap.put(_Fields.DESCRIPTION, new org.apache.thrift.meta_data.FieldMetaData("description", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.SERIALIZER_CLASS, new org.apache.thrift.meta_data.FieldMetaData("serializerClass", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.DESERIALIZER_CLASS, new org.apache.thrift.meta_data.FieldMetaData("deserializerClass", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.SERDE_TYPE, new org.apache.thrift.meta_data.FieldMetaData("serdeType", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, SerdeType.class)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(SerDeInfo.class, metaDataMap);
   }
@@ -160,6 +193,18 @@ import org.slf4j.LoggerFactory;
       Map<String,String> __this__parameters = new HashMap<String,String>(other.parameters);
       this.parameters = __this__parameters;
     }
+    if (other.isSetDescription()) {
+      this.description = other.description;
+    }
+    if (other.isSetSerializerClass()) {
+      this.serializerClass = other.serializerClass;
+    }
+    if (other.isSetDeserializerClass()) {
+      this.deserializerClass = other.deserializerClass;
+    }
+    if (other.isSetSerdeType()) {
+      this.serdeType = other.serdeType;
+    }
   }
 
   public SerDeInfo deepCopy() {
@@ -171,6 +216,10 @@ import org.slf4j.LoggerFactory;
     this.name = null;
     this.serializationLib = null;
     this.parameters = null;
+    this.description = null;
+    this.serializerClass = null;
+    this.deserializerClass = null;
+    this.serdeType = null;
   }
 
   public String getName() {
@@ -253,6 +302,106 @@ import org.slf4j.LoggerFactory;
     }
   }
 
+  public String getDescription() {
+    return this.description;
+  }
+
+  public void setDescription(String description) {
+    this.description = description;
+  }
+
+  public void unsetDescription() {
+    this.description = null;
+  }
+
+  /** Returns true if field description is set (has been assigned a value) and false otherwise */
+  public boolean isSetDescription() {
+    return this.description != null;
+  }
+
+  public void setDescriptionIsSet(boolean value) {
+    if (!value) {
+      this.description = null;
+    }
+  }
+
+  public String getSerializerClass() {
+    return this.serializerClass;
+  }
+
+  public void setSerializerClass(String serializerClass) {
+    this.serializerClass = serializerClass;
+  }
+
+  public void unsetSerializerClass() {
+    this.serializerClass = null;
+  }
+
+  /** Returns true if field serializerClass is set (has been assigned a value) and false otherwise */
+  public boolean isSetSerializerClass() {
+    return this.serializerClass != null;
+  }
+
+  public void setSerializerClassIsSet(boolean value) {
+    if (!value) {
+      this.serializerClass = null;
+    }
+  }
+
+  public String getDeserializerClass() {
+    return this.deserializerClass;
+  }
+
+  public void setDeserializerClass(String deserializerClass) {
+    this.deserializerClass = deserializerClass;
+  }
+
+  public void unsetDeserializerClass() {
+    this.deserializerClass = null;
+  }
+
+  /** Returns true if field deserializerClass is set (has been assigned a value) and false otherwise */
+  public boolean isSetDeserializerClass() {
+    return this.deserializerClass != null;
+  }
+
+  public void setDeserializerClassIsSet(boolean value) {
+    if (!value) {
+      this.deserializerClass = null;
+    }
+  }
+
+  /**
+   * 
+   * @see SerdeType
+   */
+  public SerdeType getSerdeType() {
+    return this.serdeType;
+  }
+
+  /**
+   * 
+   * @see SerdeType
+   */
+  public void setSerdeType(SerdeType serdeType) {
+    this.serdeType = serdeType;
+  }
+
+  public void unsetSerdeType() {
+    this.serdeType = null;
+  }
+
+  /** Returns true if field serdeType is set (has been assigned a value) and false otherwise */
+  public boolean isSetSerdeType() {
+    return this.serdeType != null;
+  }
+
+  public void setSerdeTypeIsSet(boolean value) {
+    if (!value) {
+      this.serdeType = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case NAME:
@@ -279,6 +428,38 @@ import org.slf4j.LoggerFactory;
       }
       break;
 
+    case DESCRIPTION:
+      if (value == null) {
+        unsetDescription();
+      } else {
+        setDescription((String)value);
+      }
+      break;
+
+    case SERIALIZER_CLASS:
+      if (value == null) {
+        unsetSerializerClass();
+      } else {
+        setSerializerClass((String)value);
+      }
+      break;
+
+    case DESERIALIZER_CLASS:
+      if (value == null) {
+        unsetDeserializerClass();
+      } else {
+        setDeserializerClass((String)value);
+      }
+      break;
+
+    case SERDE_TYPE:
+      if (value == null) {
+        unsetSerdeType();
+      } else {
+        setSerdeType((SerdeType)value);
+      }
+      break;
+
     }
   }
 
@@ -293,6 +474,18 @@ import org.slf4j.LoggerFactory;
     case PARAMETERS:
       return getParameters();
 
+    case DESCRIPTION:
+      return getDescription();
+
+    case SERIALIZER_CLASS:
+      return getSerializerClass();
+
+    case DESERIALIZER_CLASS:
+      return getDeserializerClass();
+
+    case SERDE_TYPE:
+      return getSerdeType();
+
     }
     throw new IllegalStateException();
   }
@@ -310,6 +503,14 @@ import org.slf4j.LoggerFactory;
       return isSetSerializationLib();
     case PARAMETERS:
       return isSetParameters();
+    case DESCRIPTION:
+      return isSetDescription();
+    case SERIALIZER_CLASS:
+      return isSetSerializerClass();
+    case DESERIALIZER_CLASS:
+      return isSetDeserializerClass();
+    case SERDE_TYPE:
+      return isSetSerdeType();
     }
     throw new IllegalStateException();
   }
@@ -354,6 +555,42 @@ import org.slf4j.LoggerFactory;
         return false;
     }
 
+    boolean this_present_description = true && this.isSetDescription();
+    boolean that_present_description = true && that.isSetDescription();
+    if (this_present_description || that_present_description) {
+      if (!(this_present_description && that_present_description))
+        return false;
+      if (!this.description.equals(that.description))
+        return false;
+    }
+
+    boolean this_present_serializerClass = true && this.isSetSerializerClass();
+    boolean that_present_serializerClass = true && that.isSetSerializerClass();
+    if (this_present_serializerClass || that_present_serializerClass) {
+      if (!(this_present_serializerClass && that_present_serializerClass))
+        return false;
+      if (!this.serializerClass.equals(that.serializerClass))
+        return false;
+    }
+
+    boolean this_present_deserializerClass = true && this.isSetDeserializerClass();
+    boolean that_present_deserializerClass = true && that.isSetDeserializerClass();
+    if (this_present_deserializerClass || that_present_deserializerClass) {
+      if (!(this_present_deserializerClass && that_present_deserializerClass))
+        return false;
+      if (!this.deserializerClass.equals(that.deserializerClass))
+        return false;
+    }
+
+    boolean this_present_serdeType = true && this.isSetSerdeType();
+    boolean that_present_serdeType = true && that.isSetSerdeType();
+    if (this_present_serdeType || that_present_serdeType) {
+      if (!(this_present_serdeType && that_present_serdeType))
+        return false;
+      if (!this.serdeType.equals(that.serdeType))
+        return false;
+    }
+
     return true;
   }
 
@@ -376,6 +613,26 @@ import org.slf4j.LoggerFactory;
     if (present_parameters)
       list.add(parameters);
 
+    boolean present_description = true && (isSetDescription());
+    list.add(present_description);
+    if (present_description)
+      list.add(description);
+
+    boolean present_serializerClass = true && (isSetSerializerClass());
+    list.add(present_serializerClass);
+    if (present_serializerClass)
+      list.add(serializerClass);
+
+    boolean present_deserializerClass = true && (isSetDeserializerClass());
+    list.add(present_deserializerClass);
+    if (present_deserializerClass)
+      list.add(deserializerClass);
+
+    boolean present_serdeType = true && (isSetSerdeType());
+    list.add(present_serdeType);
+    if (present_serdeType)
+      list.add(serdeType.getValue());
+
     return list.hashCode();
   }
 
@@ -417,6 +674,46 @@ import org.slf4j.LoggerFactory;
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetDescription()).compareTo(other.isSetDescription());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetDescription()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.description, other.description);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetSerializerClass()).compareTo(other.isSetSerializerClass());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetSerializerClass()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.serializerClass, other.serializerClass);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetDeserializerClass()).compareTo(other.isSetDeserializerClass());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetDeserializerClass()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.deserializerClass, other.deserializerClass);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetSerdeType()).compareTo(other.isSetSerdeType());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetSerdeType()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.serdeType, other.serdeType);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -460,6 +757,46 @@ import org.slf4j.LoggerFactory;
       sb.append(this.parameters);
     }
     first = false;
+    if (isSetDescription()) {
+      if (!first) sb.append(", ");
+      sb.append("description:");
+      if (this.description == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.description);
+      }
+      first = false;
+    }
+    if (isSetSerializerClass()) {
+      if (!first) sb.append(", ");
+      sb.append("serializerClass:");
+      if (this.serializerClass == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.serializerClass);
+      }
+      first = false;
+    }
+    if (isSetDeserializerClass()) {
+      if (!first) sb.append(", ");
+      sb.append("deserializerClass:");
+      if (this.deserializerClass == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.deserializerClass);
+      }
+      first = false;
+    }
+    if (isSetSerdeType()) {
+      if (!first) sb.append(", ");
+      sb.append("serdeType:");
+      if (this.serdeType == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.serdeType);
+      }
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -539,6 +876,38 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 4: // DESCRIPTION
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.description = iprot.readString();
+              struct.setDescriptionIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 5: // SERIALIZER_CLASS
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.serializerClass = iprot.readString();
+              struct.setSerializerClassIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 6: // DESERIALIZER_CLASS
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.deserializerClass = iprot.readString();
+              struct.setDeserializerClassIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 7: // SERDE_TYPE
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.serdeType = org.apache.hadoop.hive.metastore.api.SerdeType.findByValue(iprot.readI32());
+              struct.setSerdeTypeIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -575,6 +944,34 @@ import org.slf4j.LoggerFactory;
         }
         oprot.writeFieldEnd();
       }
+      if (struct.description != null) {
+        if (struct.isSetDescription()) {
+          oprot.writeFieldBegin(DESCRIPTION_FIELD_DESC);
+          oprot.writeString(struct.description);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.serializerClass != null) {
+        if (struct.isSetSerializerClass()) {
+          oprot.writeFieldBegin(SERIALIZER_CLASS_FIELD_DESC);
+          oprot.writeString(struct.serializerClass);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.deserializerClass != null) {
+        if (struct.isSetDeserializerClass()) {
+          oprot.writeFieldBegin(DESERIALIZER_CLASS_FIELD_DESC);
+          oprot.writeString(struct.deserializerClass);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.serdeType != null) {
+        if (struct.isSetSerdeType()) {
+          oprot.writeFieldBegin(SERDE_TYPE_FIELD_DESC);
+          oprot.writeI32(struct.serdeType.getValue());
+          oprot.writeFieldEnd();
+        }
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -602,7 +999,19 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetParameters()) {
         optionals.set(2);
       }
-      oprot.writeBitSet(optionals, 3);
+      if (struct.isSetDescription()) {
+        optionals.set(3);
+      }
+      if (struct.isSetSerializerClass()) {
+        optionals.set(4);
+      }
+      if (struct.isSetDeserializerClass()) {
+        optionals.set(5);
+      }
+      if (struct.isSetSerdeType()) {
+        optionals.set(6);
+      }
+      oprot.writeBitSet(optionals, 7);
       if (struct.isSetName()) {
         oprot.writeString(struct.name);
       }
@@ -619,12 +1028,24 @@ import org.slf4j.LoggerFactory;
           }
         }
       }
+      if (struct.isSetDescription()) {
+        oprot.writeString(struct.description);
+      }
+      if (struct.isSetSerializerClass()) {
+        oprot.writeString(struct.serializerClass);
+      }
+      if (struct.isSetDeserializerClass()) {
+        oprot.writeString(struct.deserializerClass);
+      }
+      if (struct.isSetSerdeType()) {
+        oprot.writeI32(struct.serdeType.getValue());
+      }
     }
 
     @Override
     public void read(org.apache.thrift.protocol.TProtocol prot, SerDeInfo struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
-      BitSet incoming = iprot.readBitSet(3);
+      BitSet incoming = iprot.readBitSet(7);
       if (incoming.get(0)) {
         struct.name = iprot.readString();
         struct.setNameIsSet(true);
@@ -648,6 +1069,22 @@ import org.slf4j.LoggerFactory;
         }
         struct.parameters = org.apache.hadoop.hive.metastore.utils.StringUtils.intern(struct.parameters); struct.setParametersIsSet(true);
       }
+      if (incoming.get(3)) {
+        struct.description = iprot.readString();
+        struct.setDescriptionIsSet(true);
+      }
+      if (incoming.get(4)) {
+        struct.serializerClass = iprot.readString();
+        struct.setSerializerClassIsSet(true);
+      }
+      if (incoming.get(5)) {
+        struct.deserializerClass = iprot.readString();
+        struct.setDeserializerClassIsSet(true);
+      }
+      if (incoming.get(6)) {
+        struct.serdeType = org.apache.hadoop.hive.metastore.api.SerdeType.findByValue(iprot.readI32());
+        struct.setSerdeTypeIsSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SerdeType.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SerdeType.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SerdeType.java
new file mode 100644
index 0000000..7daabed
--- /dev/null
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SerdeType.java
@@ -0,0 +1,45 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hive.metastore.api;
+
+
+import java.util.Map;
+import java.util.HashMap;
+import org.apache.thrift.TEnum;
+
+public enum SerdeType implements org.apache.thrift.TEnum {
+  HIVE(1),
+  SCHEMA_REGISTRY(2);
+
+  private final int value;
+
+  private SerdeType(int value) {
+    this.value = value;
+  }
+
+  /**
+   * Get the integer value of this enum value, as defined in the Thrift IDL.
+   */
+  public int getValue() {
+    return value;
+  }
+
+  /**
+   * Find a the enum type by its integer value, as defined in the Thrift IDL.
+   * @return null if the value is not found.
+   */
+  public static SerdeType findByValue(int value) { 
+    switch (value) {
+      case 1:
+        return HIVE;
+      case 2:
+        return SCHEMA_REGISTRY;
+      default:
+        return null;
+    }
+  }
+}


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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py b/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
index 516be77..30214d8 100644
--- a/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
+++ b/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
@@ -1410,6 +1410,104 @@ class Iface(fb303.FacebookService.Iface):
     """
     pass
 
+  def create_ischema(self, schema):
+    """
+    Parameters:
+     - schema
+    """
+    pass
+
+  def alter_ischema(self, rqst):
+    """
+    Parameters:
+     - rqst
+    """
+    pass
+
+  def get_ischema(self, name):
+    """
+    Parameters:
+     - name
+    """
+    pass
+
+  def drop_ischema(self, name):
+    """
+    Parameters:
+     - name
+    """
+    pass
+
+  def add_schema_version(self, schemaVersion):
+    """
+    Parameters:
+     - schemaVersion
+    """
+    pass
+
+  def get_schema_version(self, schemaVersion):
+    """
+    Parameters:
+     - schemaVersion
+    """
+    pass
+
+  def get_schema_latest_version(self, schemaName):
+    """
+    Parameters:
+     - schemaName
+    """
+    pass
+
+  def get_schema_all_versions(self, schemaName):
+    """
+    Parameters:
+     - schemaName
+    """
+    pass
+
+  def drop_schema_version(self, schemaVersion):
+    """
+    Parameters:
+     - schemaVersion
+    """
+    pass
+
+  def get_schemas_by_cols(self, rqst):
+    """
+    Parameters:
+     - rqst
+    """
+    pass
+
+  def map_schema_version_to_serde(self, rqst):
+    """
+    Parameters:
+     - rqst
+    """
+    pass
+
+  def set_schema_version_state(self, rqst):
+    """
+    Parameters:
+     - rqst
+    """
+    pass
+
+  def add_serde(self, serde):
+    """
+    Parameters:
+     - serde
+    """
+    pass
+
+  def get_serde(self, rqst):
+    """
+    Parameters:
+     - rqst
+    """
+    pass
+
 
 class Client(fb303.FacebookService.Client, Iface):
   """
@@ -7888,6 +7986,486 @@ class Client(fb303.FacebookService.Client, Iface):
       raise result.o4
     raise TApplicationException(TApplicationException.MISSING_RESULT, "create_or_drop_wm_trigger_to_pool_mapping failed: unknown result")
 
+  def create_ischema(self, schema):
+    """
+    Parameters:
+     - schema
+    """
+    self.send_create_ischema(schema)
+    self.recv_create_ischema()
+
+  def send_create_ischema(self, schema):
+    self._oprot.writeMessageBegin('create_ischema', TMessageType.CALL, self._seqid)
+    args = create_ischema_args()
+    args.schema = schema
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_create_ischema(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = create_ischema_result()
+    result.read(iprot)
+    iprot.readMessageEnd()
+    if result.o1 is not None:
+      raise result.o1
+    if result.o2 is not None:
+      raise result.o2
+    if result.o3 is not None:
+      raise result.o3
+    return
+
+  def alter_ischema(self, rqst):
+    """
+    Parameters:
+     - rqst
+    """
+    self.send_alter_ischema(rqst)
+    self.recv_alter_ischema()
+
+  def send_alter_ischema(self, rqst):
+    self._oprot.writeMessageBegin('alter_ischema', TMessageType.CALL, self._seqid)
+    args = alter_ischema_args()
+    args.rqst = rqst
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_alter_ischema(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = alter_ischema_result()
+    result.read(iprot)
+    iprot.readMessageEnd()
+    if result.o1 is not None:
+      raise result.o1
+    if result.o2 is not None:
+      raise result.o2
+    return
+
+  def get_ischema(self, name):
+    """
+    Parameters:
+     - name
+    """
+    self.send_get_ischema(name)
+    return self.recv_get_ischema()
+
+  def send_get_ischema(self, name):
+    self._oprot.writeMessageBegin('get_ischema', TMessageType.CALL, self._seqid)
+    args = get_ischema_args()
+    args.name = name
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_get_ischema(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = get_ischema_result()
+    result.read(iprot)
+    iprot.readMessageEnd()
+    if result.success is not None:
+      return result.success
+    if result.o1 is not None:
+      raise result.o1
+    if result.o2 is not None:
+      raise result.o2
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "get_ischema failed: unknown result")
+
+  def drop_ischema(self, name):
+    """
+    Parameters:
+     - name
+    """
+    self.send_drop_ischema(name)
+    self.recv_drop_ischema()
+
+  def send_drop_ischema(self, name):
+    self._oprot.writeMessageBegin('drop_ischema', TMessageType.CALL, self._seqid)
+    args = drop_ischema_args()
+    args.name = name
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_drop_ischema(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = drop_ischema_result()
+    result.read(iprot)
+    iprot.readMessageEnd()
+    if result.o1 is not None:
+      raise result.o1
+    if result.o2 is not None:
+      raise result.o2
+    if result.o3 is not None:
+      raise result.o3
+    return
+
+  def add_schema_version(self, schemaVersion):
+    """
+    Parameters:
+     - schemaVersion
+    """
+    self.send_add_schema_version(schemaVersion)
+    self.recv_add_schema_version()
+
+  def send_add_schema_version(self, schemaVersion):
+    self._oprot.writeMessageBegin('add_schema_version', TMessageType.CALL, self._seqid)
+    args = add_schema_version_args()
+    args.schemaVersion = schemaVersion
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_add_schema_version(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = add_schema_version_result()
+    result.read(iprot)
+    iprot.readMessageEnd()
+    if result.o1 is not None:
+      raise result.o1
+    if result.o2 is not None:
+      raise result.o2
+    if result.o3 is not None:
+      raise result.o3
+    return
+
+  def get_schema_version(self, schemaVersion):
+    """
+    Parameters:
+     - schemaVersion
+    """
+    self.send_get_schema_version(schemaVersion)
+    return self.recv_get_schema_version()
+
+  def send_get_schema_version(self, schemaVersion):
+    self._oprot.writeMessageBegin('get_schema_version', TMessageType.CALL, self._seqid)
+    args = get_schema_version_args()
+    args.schemaVersion = schemaVersion
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_get_schema_version(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = get_schema_version_result()
+    result.read(iprot)
+    iprot.readMessageEnd()
+    if result.success is not None:
+      return result.success
+    if result.o1 is not None:
+      raise result.o1
+    if result.o2 is not None:
+      raise result.o2
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "get_schema_version failed: unknown result")
+
+  def get_schema_latest_version(self, schemaName):
+    """
+    Parameters:
+     - schemaName
+    """
+    self.send_get_schema_latest_version(schemaName)
+    return self.recv_get_schema_latest_version()
+
+  def send_get_schema_latest_version(self, schemaName):
+    self._oprot.writeMessageBegin('get_schema_latest_version', TMessageType.CALL, self._seqid)
+    args = get_schema_latest_version_args()
+    args.schemaName = schemaName
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_get_schema_latest_version(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = get_schema_latest_version_result()
+    result.read(iprot)
+    iprot.readMessageEnd()
+    if result.success is not None:
+      return result.success
+    if result.o1 is not None:
+      raise result.o1
+    if result.o2 is not None:
+      raise result.o2
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "get_schema_latest_version failed: unknown result")
+
+  def get_schema_all_versions(self, schemaName):
+    """
+    Parameters:
+     - schemaName
+    """
+    self.send_get_schema_all_versions(schemaName)
+    return self.recv_get_schema_all_versions()
+
+  def send_get_schema_all_versions(self, schemaName):
+    self._oprot.writeMessageBegin('get_schema_all_versions', TMessageType.CALL, self._seqid)
+    args = get_schema_all_versions_args()
+    args.schemaName = schemaName
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_get_schema_all_versions(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = get_schema_all_versions_result()
+    result.read(iprot)
+    iprot.readMessageEnd()
+    if result.success is not None:
+      return result.success
+    if result.o1 is not None:
+      raise result.o1
+    if result.o2 is not None:
+      raise result.o2
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "get_schema_all_versions failed: unknown result")
+
+  def drop_schema_version(self, schemaVersion):
+    """
+    Parameters:
+     - schemaVersion
+    """
+    self.send_drop_schema_version(schemaVersion)
+    self.recv_drop_schema_version()
+
+  def send_drop_schema_version(self, schemaVersion):
+    self._oprot.writeMessageBegin('drop_schema_version', TMessageType.CALL, self._seqid)
+    args = drop_schema_version_args()
+    args.schemaVersion = schemaVersion
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_drop_schema_version(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = drop_schema_version_result()
+    result.read(iprot)
+    iprot.readMessageEnd()
+    if result.o1 is not None:
+      raise result.o1
+    if result.o2 is not None:
+      raise result.o2
+    return
+
+  def get_schemas_by_cols(self, rqst):
+    """
+    Parameters:
+     - rqst
+    """
+    self.send_get_schemas_by_cols(rqst)
+    return self.recv_get_schemas_by_cols()
+
+  def send_get_schemas_by_cols(self, rqst):
+    self._oprot.writeMessageBegin('get_schemas_by_cols', TMessageType.CALL, self._seqid)
+    args = get_schemas_by_cols_args()
+    args.rqst = rqst
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_get_schemas_by_cols(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = get_schemas_by_cols_result()
+    result.read(iprot)
+    iprot.readMessageEnd()
+    if result.success is not None:
+      return result.success
+    if result.o1 is not None:
+      raise result.o1
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "get_schemas_by_cols failed: unknown result")
+
+  def map_schema_version_to_serde(self, rqst):
+    """
+    Parameters:
+     - rqst
+    """
+    self.send_map_schema_version_to_serde(rqst)
+    self.recv_map_schema_version_to_serde()
+
+  def send_map_schema_version_to_serde(self, rqst):
+    self._oprot.writeMessageBegin('map_schema_version_to_serde', TMessageType.CALL, self._seqid)
+    args = map_schema_version_to_serde_args()
+    args.rqst = rqst
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_map_schema_version_to_serde(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = map_schema_version_to_serde_result()
+    result.read(iprot)
+    iprot.readMessageEnd()
+    if result.o1 is not None:
+      raise result.o1
+    if result.o2 is not None:
+      raise result.o2
+    return
+
+  def set_schema_version_state(self, rqst):
+    """
+    Parameters:
+     - rqst
+    """
+    self.send_set_schema_version_state(rqst)
+    self.recv_set_schema_version_state()
+
+  def send_set_schema_version_state(self, rqst):
+    self._oprot.writeMessageBegin('set_schema_version_state', TMessageType.CALL, self._seqid)
+    args = set_schema_version_state_args()
+    args.rqst = rqst
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_set_schema_version_state(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = set_schema_version_state_result()
+    result.read(iprot)
+    iprot.readMessageEnd()
+    if result.o1 is not None:
+      raise result.o1
+    if result.o2 is not None:
+      raise result.o2
+    if result.o3 is not None:
+      raise result.o3
+    return
+
+  def add_serde(self, serde):
+    """
+    Parameters:
+     - serde
+    """
+    self.send_add_serde(serde)
+    self.recv_add_serde()
+
+  def send_add_serde(self, serde):
+    self._oprot.writeMessageBegin('add_serde', TMessageType.CALL, self._seqid)
+    args = add_serde_args()
+    args.serde = serde
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_add_serde(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = add_serde_result()
+    result.read(iprot)
+    iprot.readMessageEnd()
+    if result.o1 is not None:
+      raise result.o1
+    if result.o2 is not None:
+      raise result.o2
+    return
+
+  def get_serde(self, rqst):
+    """
+    Parameters:
+     - rqst
+    """
+    self.send_get_serde(rqst)
+    return self.recv_get_serde()
+
+  def send_get_serde(self, rqst):
+    self._oprot.writeMessageBegin('get_serde', TMessageType.CALL, self._seqid)
+    args = get_serde_args()
+    args.rqst = rqst
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_get_serde(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = get_serde_result()
+    result.read(iprot)
+    iprot.readMessageEnd()
+    if result.success is not None:
+      return result.success
+    if result.o1 is not None:
+      raise result.o1
+    if result.o2 is not None:
+      raise result.o2
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "get_serde failed: unknown result")
+
 
 class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
   def __init__(self, handler):
@@ -8072,6 +8650,20 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
     self._processMap["create_or_update_wm_mapping"] = Processor.process_create_or_update_wm_mapping
     self._processMap["drop_wm_mapping"] = Processor.process_drop_wm_mapping
     self._processMap["create_or_drop_wm_trigger_to_pool_mapping"] = Processor.process_create_or_drop_wm_trigger_to_pool_mapping
+    self._processMap["create_ischema"] = Processor.process_create_ischema
+    self._processMap["alter_ischema"] = Processor.process_alter_ischema
+    self._processMap["get_ischema"] = Processor.process_get_ischema
+    self._processMap["drop_ischema"] = Processor.process_drop_ischema
+    self._processMap["add_schema_version"] = Processor.process_add_schema_version
+    self._processMap["get_schema_version"] = Processor.process_get_schema_version
+    self._processMap["get_schema_latest_version"] = Processor.process_get_schema_latest_version
+    self._processMap["get_schema_all_versions"] = Processor.process_get_schema_all_versions
+    self._processMap["drop_schema_version"] = Processor.process_drop_schema_version
+    self._processMap["get_schemas_by_cols"] = Processor.process_get_schemas_by_cols
+    self._processMap["map_schema_version_to_serde"] = Processor.process_map_schema_version_to_serde
+    self._processMap["set_schema_version_state"] = Processor.process_set_schema_version_state
+    self._processMap["add_serde"] = Processor.process_add_serde
+    self._processMap["get_serde"] = Processor.process_get_serde
 
   def process(self, iprot, oprot):
     (name, type, seqid) = iprot.readMessageBegin()
@@ -12534,6 +13126,365 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
     oprot.writeMessageEnd()
     oprot.trans.flush()
 
+  def process_create_ischema(self, seqid, iprot, oprot):
+    args = create_ischema_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = create_ischema_result()
+    try:
+      self._handler.create_ischema(args.schema)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except AlreadyExistsException as o1:
+      msg_type = TMessageType.REPLY
+      result.o1 = o1
+    except NoSuchObjectException as o2:
+      msg_type = TMessageType.REPLY
+      result.o2 = o2
+    except MetaException as o3:
+      msg_type = TMessageType.REPLY
+      result.o3 = o3
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("create_ischema", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_alter_ischema(self, seqid, iprot, oprot):
+    args = alter_ischema_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = alter_ischema_result()
+    try:
+      self._handler.alter_ischema(args.rqst)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except NoSuchObjectException as o1:
+      msg_type = TMessageType.REPLY
+      result.o1 = o1
+    except MetaException as o2:
+      msg_type = TMessageType.REPLY
+      result.o2 = o2
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("alter_ischema", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_get_ischema(self, seqid, iprot, oprot):
+    args = get_ischema_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = get_ischema_result()
+    try:
+      result.success = self._handler.get_ischema(args.name)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except NoSuchObjectException as o1:
+      msg_type = TMessageType.REPLY
+      result.o1 = o1
+    except MetaException as o2:
+      msg_type = TMessageType.REPLY
+      result.o2 = o2
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("get_ischema", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_drop_ischema(self, seqid, iprot, oprot):
+    args = drop_ischema_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = drop_ischema_result()
+    try:
+      self._handler.drop_ischema(args.name)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except NoSuchObjectException as o1:
+      msg_type = TMessageType.REPLY
+      result.o1 = o1
+    except InvalidOperationException as o2:
+      msg_type = TMessageType.REPLY
+      result.o2 = o2
+    except MetaException as o3:
+      msg_type = TMessageType.REPLY
+      result.o3 = o3
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("drop_ischema", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_add_schema_version(self, seqid, iprot, oprot):
+    args = add_schema_version_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = add_schema_version_result()
+    try:
+      self._handler.add_schema_version(args.schemaVersion)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except AlreadyExistsException as o1:
+      msg_type = TMessageType.REPLY
+      result.o1 = o1
+    except NoSuchObjectException as o2:
+      msg_type = TMessageType.REPLY
+      result.o2 = o2
+    except MetaException as o3:
+      msg_type = TMessageType.REPLY
+      result.o3 = o3
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("add_schema_version", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_get_schema_version(self, seqid, iprot, oprot):
+    args = get_schema_version_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = get_schema_version_result()
+    try:
+      result.success = self._handler.get_schema_version(args.schemaVersion)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except NoSuchObjectException as o1:
+      msg_type = TMessageType.REPLY
+      result.o1 = o1
+    except MetaException as o2:
+      msg_type = TMessageType.REPLY
+      result.o2 = o2
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("get_schema_version", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_get_schema_latest_version(self, seqid, iprot, oprot):
+    args = get_schema_latest_version_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = get_schema_latest_version_result()
+    try:
+      result.success = self._handler.get_schema_latest_version(args.schemaName)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except NoSuchObjectException as o1:
+      msg_type = TMessageType.REPLY
+      result.o1 = o1
+    except MetaException as o2:
+      msg_type = TMessageType.REPLY
+      result.o2 = o2
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("get_schema_latest_version", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_get_schema_all_versions(self, seqid, iprot, oprot):
+    args = get_schema_all_versions_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = get_schema_all_versions_result()
+    try:
+      result.success = self._handler.get_schema_all_versions(args.schemaName)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except NoSuchObjectException as o1:
+      msg_type = TMessageType.REPLY
+      result.o1 = o1
+    except MetaException as o2:
+      msg_type = TMessageType.REPLY
+      result.o2 = o2
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("get_schema_all_versions", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_drop_schema_version(self, seqid, iprot, oprot):
+    args = drop_schema_version_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = drop_schema_version_result()
+    try:
+      self._handler.drop_schema_version(args.schemaVersion)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except NoSuchObjectException as o1:
+      msg_type = TMessageType.REPLY
+      result.o1 = o1
+    except MetaException as o2:
+      msg_type = TMessageType.REPLY
+      result.o2 = o2
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("drop_schema_version", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_get_schemas_by_cols(self, seqid, iprot, oprot):
+    args = get_schemas_by_cols_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = get_schemas_by_cols_result()
+    try:
+      result.success = self._handler.get_schemas_by_cols(args.rqst)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except MetaException as o1:
+      msg_type = TMessageType.REPLY
+      result.o1 = o1
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("get_schemas_by_cols", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_map_schema_version_to_serde(self, seqid, iprot, oprot):
+    args = map_schema_version_to_serde_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = map_schema_version_to_serde_result()
+    try:
+      self._handler.map_schema_version_to_serde(args.rqst)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except NoSuchObjectException as o1:
+      msg_type = TMessageType.REPLY
+      result.o1 = o1
+    except MetaException as o2:
+      msg_type = TMessageType.REPLY
+      result.o2 = o2
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("map_schema_version_to_serde", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_set_schema_version_state(self, seqid, iprot, oprot):
+    args = set_schema_version_state_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = set_schema_version_state_result()
+    try:
+      self._handler.set_schema_version_state(args.rqst)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except NoSuchObjectException as o1:
+      msg_type = TMessageType.REPLY
+      result.o1 = o1
+    except InvalidOperationException as o2:
+      msg_type = TMessageType.REPLY
+      result.o2 = o2
+    except MetaException as o3:
+      msg_type = TMessageType.REPLY
+      result.o3 = o3
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("set_schema_version_state", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_add_serde(self, seqid, iprot, oprot):
+    args = add_serde_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = add_serde_result()
+    try:
+      self._handler.add_serde(args.serde)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except AlreadyExistsException as o1:
+      msg_type = TMessageType.REPLY
+      result.o1 = o1
+    except MetaException as o2:
+      msg_type = TMessageType.REPLY
+      result.o2 = o2
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("add_serde", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_get_serde(self, seqid, iprot, oprot):
+    args = get_serde_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = get_serde_result()
+    try:
+      result.success = self._handler.get_serde(args.rqst)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except NoSuchObjectException as o1:
+      msg_type = TMessageType.REPLY
+      result.o1 = o1
+    except MetaException as o2:
+      msg_type = TMessageType.REPLY
+      result.o2 = o2
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("get_serde", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
 
 # HELPER FUNCTIONS AND STRUCTURES
 
@@ -13420,10 +14371,10 @@ class get_databases_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype763, _size760) = iprot.readListBegin()
-          for _i764 in xrange(_size760):
-            _elem765 = iprot.readString()
-            self.success.append(_elem765)
+          (_etype777, _size774) = iprot.readListBegin()
+          for _i778 in xrange(_size774):
+            _elem779 = iprot.readString()
+            self.success.append(_elem779)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -13446,8 +14397,8 @@ class get_databases_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter766 in self.success:
-        oprot.writeString(iter766)
+      for iter780 in self.success:
+        oprot.writeString(iter780)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -13552,10 +14503,10 @@ class get_all_databases_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype770, _size767) = iprot.readListBegin()
-          for _i771 in xrange(_size767):
-            _elem772 = iprot.readString()
-            self.success.append(_elem772)
+          (_etype784, _size781) = iprot.readListBegin()
+          for _i785 in xrange(_size781):
+            _elem786 = iprot.readString()
+            self.success.append(_elem786)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -13578,8 +14529,8 @@ class get_all_databases_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter773 in self.success:
-        oprot.writeString(iter773)
+      for iter787 in self.success:
+        oprot.writeString(iter787)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -14349,12 +15300,12 @@ class get_type_all_result:
       if fid == 0:
         if ftype == TType.MAP:
           self.success = {}
-          (_ktype775, _vtype776, _size774 ) = iprot.readMapBegin()
-          for _i778 in xrange(_size774):
-            _key779 = iprot.readString()
-            _val780 = Type()
-            _val780.read(iprot)
-            self.success[_key779] = _val780
+          (_ktype789, _vtype790, _size788 ) = iprot.readMapBegin()
+          for _i792 in xrange(_size788):
+            _key793 = iprot.readString()
+            _val794 = Type()
+            _val794.read(iprot)
+            self.success[_key793] = _val794
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -14377,9 +15328,9 @@ class get_type_all_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.MAP, 0)
       oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.success))
-      for kiter781,viter782 in self.success.items():
-        oprot.writeString(kiter781)
-        viter782.write(oprot)
+      for kiter795,viter796 in self.success.items():
+        oprot.writeString(kiter795)
+        viter796.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.o2 is not None:
@@ -14522,11 +15473,11 @@ class get_fields_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype786, _size783) = iprot.readListBegin()
-          for _i787 in xrange(_size783):
-            _elem788 = FieldSchema()
-            _elem788.read(iprot)
-            self.success.append(_elem788)
+          (_etype800, _size797) = iprot.readListBegin()
+          for _i801 in xrange(_size797):
+            _elem802 = FieldSchema()
+            _elem802.read(iprot)
+            self.success.append(_elem802)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14561,8 +15512,8 @@ class get_fields_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter789 in self.success:
-        iter789.write(oprot)
+      for iter803 in self.success:
+        iter803.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -14729,11 +15680,11 @@ class get_fields_with_environment_context_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype793, _size790) = iprot.readListBegin()
-          for _i794 in xrange(_size790):
-            _elem795 = FieldSchema()
-            _elem795.read(iprot)
-            self.success.append(_elem795)
+          (_etype807, _size804) = iprot.readListBegin()
+          for _i808 in xrange(_size804):
+            _elem809 = FieldSchema()
+            _elem809.read(iprot)
+            self.success.append(_elem809)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14768,8 +15719,8 @@ class get_fields_with_environment_context_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter796 in self.success:
-        iter796.write(oprot)
+      for iter810 in self.success:
+        iter810.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -14922,11 +15873,11 @@ class get_schema_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype800, _size797) = iprot.readListBegin()
-          for _i801 in xrange(_size797):
-            _elem802 = FieldSchema()
-            _elem802.read(iprot)
-            self.success.append(_elem802)
+          (_etype814, _size811) = iprot.readListBegin()
+          for _i815 in xrange(_size811):
+            _elem816 = FieldSchema()
+            _elem816.read(iprot)
+            self.success.append(_elem816)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14961,8 +15912,8 @@ class get_schema_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter803 in self.success:
-        iter803.write(oprot)
+      for iter817 in self.success:
+        iter817.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -15129,11 +16080,11 @@ class get_schema_with_environment_context_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype807, _size804) = iprot.readListBegin()
-          for _i808 in xrange(_size804):
-            _elem809 = FieldSchema()
-            _elem809.read(iprot)
-            self.success.append(_elem809)
+          (_etype821, _size818) = iprot.readListBegin()
+          for _i822 in xrange(_size818):
+            _elem823 = FieldSchema()
+            _elem823.read(iprot)
+            self.success.append(_elem823)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15168,8 +16119,8 @@ class get_schema_with_environment_context_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter810 in self.success:
-        iter810.write(oprot)
+      for iter824 in self.success:
+        iter824.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -15619,55 +16570,55 @@ class create_table_with_constraints_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.primaryKeys = []
-          (_etype814, _size811) = iprot.readListBegin()
-          for _i815 in xrange(_size811):
-            _elem816 = SQLPrimaryKey()
-            _elem816.read(iprot)
-            self.primaryKeys.append(_elem816)
+          (_etype828, _size825) = iprot.readListBegin()
+          for _i829 in xrange(_size825):
+            _elem830 = SQLPrimaryKey()
+            _elem830.read(iprot)
+            self.primaryKeys.append(_elem830)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.LIST:
           self.foreignKeys = []
-          (_etype820, _size817) = iprot.readListBegin()
-          for _i821 in xrange(_size817):
-            _elem822 = SQLForeignKey()
-            _elem822.read(iprot)
-            self.foreignKeys.append(_elem822)
+          (_etype834, _size831) = iprot.readListBegin()
+          for _i835 in xrange(_size831):
+            _elem836 = SQLForeignKey()
+            _elem836.read(iprot)
+            self.foreignKeys.append(_elem836)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 4:
         if ftype == TType.LIST:
           self.uniqueConstraints = []
-          (_etype826, _size823) = iprot.readListBegin()
-          for _i827 in xrange(_size823):
-            _elem828 = SQLUniqueConstraint()
-            _elem828.read(iprot)
-            self.uniqueConstraints.append(_elem828)
+          (_etype840, _size837) = iprot.readListBegin()
+          for _i841 in xrange(_size837):
+            _elem842 = SQLUniqueConstraint()
+            _elem842.read(iprot)
+            self.uniqueConstraints.append(_elem842)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 5:
         if ftype == TType.LIST:
           self.notNullConstraints = []
-          (_etype832, _size829) = iprot.readListBegin()
-          for _i833 in xrange(_size829):
-            _elem834 = SQLNotNullConstraint()
-            _elem834.read(iprot)
-            self.notNullConstraints.append(_elem834)
+          (_etype846, _size843) = iprot.readListBegin()
+          for _i847 in xrange(_size843):
+            _elem848 = SQLNotNullConstraint()
+            _elem848.read(iprot)
+            self.notNullConstraints.append(_elem848)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 6:
         if ftype == TType.LIST:
           self.defaultConstraints = []
-          (_etype838, _size835) = iprot.readListBegin()
-          for _i839 in xrange(_size835):
-            _elem840 = SQLDefaultConstraint()
-            _elem840.read(iprot)
-            self.defaultConstraints.append(_elem840)
+          (_etype852, _size849) = iprot.readListBegin()
+          for _i853 in xrange(_size849):
+            _elem854 = SQLDefaultConstraint()
+            _elem854.read(iprot)
+            self.defaultConstraints.append(_elem854)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15688,36 +16639,36 @@ class create_table_with_constraints_args:
     if self.primaryKeys is not None:
       oprot.writeFieldBegin('primaryKeys', TType.LIST, 2)
       oprot.writeListBegin(TType.STRUCT, len(self.primaryKeys))
-      for iter841 in self.primaryKeys:
-        iter841.write(oprot)
+      for iter855 in self.primaryKeys:
+        iter855.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.foreignKeys is not None:
       oprot.writeFieldBegin('foreignKeys', TType.LIST, 3)
       oprot.writeListBegin(TType.STRUCT, len(self.foreignKeys))
-      for iter842 in self.foreignKeys:
-        iter842.write(oprot)
+      for iter856 in self.foreignKeys:
+        iter856.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.uniqueConstraints is not None:
       oprot.writeFieldBegin('uniqueConstraints', TType.LIST, 4)
       oprot.writeListBegin(TType.STRUCT, len(self.uniqueConstraints))
-      for iter843 in self.uniqueConstraints:
-        iter843.write(oprot)
+      for iter857 in self.uniqueConstraints:
+        iter857.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.notNullConstraints is not None:
       oprot.writeFieldBegin('notNullConstraints', TType.LIST, 5)
       oprot.writeListBegin(TType.STRUCT, len(self.notNullConstraints))
-      for iter844 in self.notNullConstraints:
-        iter844.write(oprot)
+      for iter858 in self.notNullConstraints:
+        iter858.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.defaultConstraints is not None:
       oprot.writeFieldBegin('defaultConstraints', TType.LIST, 6)
       oprot.writeListBegin(TType.STRUCT, len(self.defaultConstraints))
-      for iter845 in self.defaultConstraints:
-        iter845.write(oprot)
+      for iter859 in self.defaultConstraints:
+        iter859.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -17130,10 +18081,10 @@ class truncate_table_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.partNames = []
-          (_etype849, _size846) = iprot.readListBegin()
-          for _i850 in xrange(_size846):
-            _elem851 = iprot.readString()
-            self.partNames.append(_elem851)
+          (_etype863, _size860) = iprot.readListBegin()
+          for _i864 in xrange(_size860):
+            _elem865 = iprot.readString()
+            self.partNames.append(_elem865)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17158,8 +18109,8 @@ class truncate_table_args:
     if self.partNames is not None:
       oprot.writeFieldBegin('partNames', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.partNames))
-      for iter852 in self.partNames:
-        oprot.writeString(iter852)
+      for iter866 in self.partNames:
+        oprot.writeString(iter866)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -17359,10 +18310,10 @@ class get_tables_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype856, _size853) = iprot.readListBegin()
-          for _i857 in xrange(_size853):
-            _elem858 = iprot.readString()
-            self.success.append(_elem858)
+          (_etype870, _size867) = iprot.readListBegin()
+          for _i871 in xrange(_size867):
+            _elem872 = iprot.readString()
+            self.success.append(_elem872)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17385,8 +18336,8 @@ class get_tables_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter859 in self.success:
-        oprot.writeString(iter859)
+      for iter873 in self.success:
+        oprot.writeString(iter873)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -17536,10 +18487,10 @@ class get_tables_by_type_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype863, _size860) = iprot.readListBegin()
-          for _i864 in xrange(_size860):
-            _elem865 = iprot.readString()
-            self.success.append(_elem865)
+          (_etype877, _size874) = iprot.readListBegin()
+          for _i878 in xrange(_size874):
+            _elem879 = iprot.readString()
+            self.success.append(_elem879)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17562,8 +18513,8 @@ class get_tables_by_type_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter866 in self.success:
-        oprot.writeString(iter866)
+      for iter880 in self.success:
+        oprot.writeString(iter880)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -17687,10 +18638,10 @@ class get_materialized_views_for_rewriting_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype870, _size867) = iprot.readListBegin()
-          for _i871 in xrange(_size867):
-            _elem872 = iprot.readString()
-            self.success.append(_elem872)
+          (_etype884, _size881) = iprot.readListBegin()
+          for _i885 in xrange(_size881):
+            _elem886 = iprot.readString()
+            self.success.append(_elem886)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17713,8 +18664,8 @@ class get_materialized_views_for_rewriting_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter873 in self.success:
-        oprot.writeString(iter873)
+      for iter887 in self.success:
+        oprot.writeString(iter887)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -17787,10 +18738,10 @@ class get_table_meta_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.tbl_types = []
-          (_etype877, _size874) = iprot.readListBegin()
-          for _i878 in xrange(_size874):
-            _elem879 = iprot.readString()
-            self.tbl_types.append(_elem879)
+          (_etype891, _size888) = iprot.readListBegin()
+          for _i892 in xrange(_size888):
+            _elem893 = iprot.readString()
+            self.tbl_types.append(_elem893)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17815,8 +18766,8 @@ class get_table_meta_args:
     if self.tbl_types is not None:
       oprot.writeFieldBegin('tbl_types', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.tbl_types))
-      for iter880 in self.tbl_types:
-        oprot.writeString(iter880)
+      for iter894 in self.tbl_types:
+        oprot.writeString(iter894)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -17872,11 +18823,11 @@ class get_table_meta_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype884, _size881) = iprot.readListBegin()
-          for _i885 in xrange(_size881):
-            _elem886 = TableMeta()
-            _elem886.read(iprot)
-            self.success.append(_elem886)
+          (_etype898, _size895) = iprot.readListBegin()
+          for _i899 in xrange(_size895):
+            _elem900 = TableMeta()
+            _elem900.read(iprot)
+            self.success.append(_elem900)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17899,8 +18850,8 @@ class get_table_meta_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter887 in self.success:
-        iter887.write(oprot)
+      for iter901 in self.success:
+        iter901.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -18024,10 +18975,10 @@ class get_all_tables_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype891, _size888) = iprot.readListBegin()
-          for _i892 in xrange(_size888):
-            _elem893 = iprot.readString()
-            self.success.append(_elem893)
+          (_etype905, _size902) = iprot.readListBegin()
+          for _i906 in xrange(_size902):
+            _elem907 = iprot.readString()
+            self.success.append(_elem907)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18050,8 +19001,8 @@ class get_all_tables_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter894 in self.success:
-        oprot.writeString(iter894)
+      for iter908 in self.success:
+        oprot.writeString(iter908)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -18287,10 +19238,10 @@ class get_table_objects_by_name_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.tbl_names = []
-          (_etype898, _size895) = iprot.readListBegin()
-          for _i899 in xrange(_size895):
-            _elem900 = iprot.readString()
-            self.tbl_names.append(_elem900)
+          (_etype912, _size909) = iprot.readListBegin()
+          for _i913 in xrange(_size909):
+            _elem914 = iprot.readString()
+            self.tbl_names.append(_elem914)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18311,8 +19262,8 @@ class get_table_objects_by_name_args:
     if self.tbl_names is not None:
       oprot.writeFieldBegin('tbl_names', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.tbl_names))
-      for iter901 in self.tbl_names:
-        oprot.writeString(iter901)
+      for iter915 in self.tbl_names:
+        oprot.writeString(iter915)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -18364,11 +19315,11 @@ class get_table_objects_by_name_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype905, _size902) = iprot.readListBegin()
-          for _i906 in xrange(_size902):
-            _elem907 = Table()
-            _elem907.read(iprot)
-            self.success.append(_elem907)
+          (_etype919, _size916) = iprot.readListBegin()
+          for _i920 in xrange(_size916):
+            _elem921 = Table()
+            _elem921.read(iprot)
+            self.success.append(_elem921)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18385,8 +19336,8 @@ class get_table_objects_by_name_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter908 in self.success:
-        iter908.write(oprot)
+      for iter922 in self.success:
+        iter922.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -18778,10 +19729,10 @@ class get_materialization_invalidation_info_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.tbl_names = []
-          (_etype912, _size909) = iprot.readListBegin()
-          for _i913 in xrange(_size909):
-            _elem914 = iprot.readString()
-            self.tbl_names.append(_elem914)
+          (_etype926, _size923) = iprot.readListBegin()
+          for _i927 in xrange(_size923):
+            _elem928 = iprot.readString()
+            self.tbl_names.append(_elem928)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18802,8 +19753,8 @@ class get_materialization_invalidation_info_args:
     if self.tbl_names is not None:
       oprot.writeFieldBegin('tbl_names', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.tbl_names))
-      for iter915 in self.tbl_names:
-        oprot.writeString(iter915)
+      for iter929 in self.tbl_names:
+        oprot.writeString(iter929)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -18864,12 +19815,12 @@ class get_materialization_invalidation_info_result:
       if fid == 0:
         if ftype == TType.MAP:
           self.success = {}
-          (_ktype917, _vtype918, _size916 ) = iprot.readMapBegin()
-          for _i920 in xrange(_size916):
-            _key921 = iprot.readString()
-            _val922 = Materialization()
-            _val922.read(iprot)
-            self.success[_key921] = _val922
+          (_ktype931, _vtype932, _size930 ) = iprot.readMapBegin()
+          for _i934 in xrange(_size930):
+            _key935 = iprot.readString()
+            _val936 = Materialization()
+            _val936.read(iprot)
+            self.success[_key935] = _val936
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -18904,9 +19855,9 @@ class get_materialization_invalidation_info_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.MAP, 0)
       oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.success))
-      for kiter923,viter924 in self.success.items():
-        oprot.writeString(kiter923)
-        viter924.write(oprot)
+      for kiter937,viter938 in self.success.items():
+        oprot.writeString(kiter937)
+        viter938.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -19258,10 +20209,10 @@ class get_table_names_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype928, _size925) = iprot.readListBegin()
-          for _i929 in xrange(_size925):
-            _elem930 = iprot.readString()
-            self.success.append(_elem930)
+          (_etype942, _size939) = iprot.readListBegin()
+          for _i943 in xrange(_size939):
+            _elem944 = iprot.readString()
+            self.success.append(_elem944)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19296,8 +20247,8 @@ class get_table_names_by_filter_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter931 in self.success:
-        oprot.writeString(iter931)
+      for iter945 in self.success:
+        oprot.writeString(iter945)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -20267,11 +21218,11 @@ class add_partitions_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype935, _size932) = iprot.readListBegin()
-          for _i936 in xrange(_size932):
-            _elem937 = Partition()
-            _elem937.read(iprot)
-            self.new_parts.append(_elem937)
+          (_etype949, _size946) = iprot.readListBegin()
+          for _i950 in xrange(_size946):
+            _elem951 = Partition()
+            _elem951.read(iprot)
+            self.new_parts.append(_elem951)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20288,8 +21239,8 @@ class add_partitions_args:
     if self.new_parts is not None:
       oprot.writeFieldBegin('new_parts', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.new_parts))
-      for iter938 in self.new_parts:
-        iter938.write(oprot)
+      for iter952 in self.new_parts:
+        iter952.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -20447,11 +21398,11 @@ class add_partitions_pspec_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype942, _size939) = iprot.readListBegin()
-          for _i943 in xrange(_size939):
-            _elem944 = PartitionSpec()
-            _elem944.read(iprot)
-            self.new_parts.append(_elem944)
+          (_etype956, _size953) = iprot.readListBegin()
+          for _i957 in xrange(_size953):
+            _elem958 = PartitionSpec()
+            _elem958.read(iprot)
+            self.new_parts.append(_elem958)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20468,8 +21419,8 @@ class add_partitions_pspec_args:
     if self.new_parts is not None:
       oprot.writeFieldBegin('new_parts', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.new_parts))
-      for iter945 in self.new_parts:
-        iter945.write(oprot)
+      for iter959 in self.new_parts:
+        iter959.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -20643,10 +21594,10 @@ class append_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype949, _size946) = iprot.readListBegin()
-          for _i950 in xrange(_size946):
-            _elem951 = iprot.readString()
-            self.part_vals.append(_elem951)
+          (_etype963, _size960) = iprot.readListBegin()
+          for _i964 in xrange(_size960):
+            _elem965 = iprot.readString()
+            self.part_vals.append(_elem965)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20671,8 +21622,8 @@ class append_partition_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter952 in self.part_vals:
-        oprot.writeString(iter952)
+      for iter966 in self.part_vals:
+        oprot.writeString(iter966)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -21025,10 +21976,10 @@ class append_partition_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype956, _size953) = iprot.readListBegin()
-          for _i957 in xrange(_size953):
-            _elem958 = iprot.readString()
-            self.part_vals.append(_elem958)
+          (_etype970, _size967) = iprot.readListBegin()
+          for _i971 in xrange(_size967):
+            _elem972 = iprot.readString()
+            self.part_vals.append(_elem972)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21059,8 +22010,8 @@ class append_partition_with_environment_context_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter959 in self.part_vals:
-        oprot.writeString(iter959)
+      for iter973 in self.part_vals:
+        oprot.writeString(iter973)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.environment_context is not None:
@@ -21655,10 +22606,10 @@ class drop_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype963, _size960) = iprot.readListBegin()
-          for _i964 in xrange(_size960):
-            _elem965 = iprot.readString()
-            self.part_vals.append(_elem965)
+          (_etype977, _size974) = iprot.readListBegin()
+          for _i978 in xrange(_size974):
+            _elem979 = iprot.readString()
+            self.part_vals.append(_elem979)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21688,8 +22639,8 @@ class drop_partition_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter966 in self.part_vals:
-        oprot.writeString(iter966)
+      for iter980 in self.part_vals:
+        oprot.writeString(iter980)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.deleteData is not None:
@@ -21862,10 +22813,10 @@ class drop_partition_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype970, _size967) = iprot.readListBegin()
-          for _i971 in xrange(_size967):
-            _elem972 = iprot.readString()
-            self.part_vals.append(_elem972)
+          (_etype984, _size981) = iprot.readListBegin()
+          for _i985 in xrange(_size981):
+            _elem986 = iprot.readString()
+            self.part_vals.append(_elem986)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21901,8 +22852,8 @@ class drop_partition_with_environment_context_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter973 in self.part_vals:
-        oprot.writeString(iter973)
+      for iter987 in self.part_vals:
+        oprot.writeString(iter987)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.deleteData is not None:
@@ -22639,10 +23590,10 @@ class get_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype977, _size974) = iprot.readListBegin()
-          for _i978 in xrange(_size974):
-            _elem979 = iprot.readString()
-            self.part_vals.append(_elem979)
+          (_etype991, _size988) = iprot.readListBegin()
+          for _i992 in xrange(_size988):
+            _elem993 = iprot.readString()
+            self.part_vals.append(_elem993)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22667,8 +23618,8 @@ class get_partition_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter980 in self.part_vals:
-        oprot.writeString(iter980)
+      for iter994 in self.part_vals:
+        oprot.writeString(iter994)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -22827,11 +23778,11 @@ class exchange_partition_args:
       if fid == 1:
         if ftype == TType.MAP:
           self.partitionSpecs = {}
-          (_ktype982, _vtype983, _size981 ) = iprot.readMapBegin()
-          for _i985 in xrange(_size981):
-            _key986 = iprot.readString()
-            _val987 = iprot.readString()
-            self.partitionSpecs[_key986] = _val987
+          (_ktype996, _vtype997, _size995 ) = iprot.readMapBegin()
+          for _i999 in xrange(_size995):
+            _key1000 = iprot.readString()
+            _val1001 = iprot.readString()
+            self.partitionSpecs[_key1000] = _val1001
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -22868,9 +23819,9 @@ class exchange_partition_args:
     if self.partitionSpecs is not None:
       oprot.writeFieldBegin('partitionSpecs', TType.MAP, 1)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.partitionSpecs))
-      for kiter988,viter989 in self.partitionSpecs.items():
-        oprot.writeString(kiter988)
-        oprot.writeString(viter989)
+      for kiter1002,viter1003 in self.partitionSpecs.items():
+        oprot.writeString(kiter1002)
+        oprot.writeString(viter1003)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.source_db is not None:
@@ -23075,11 +24026,11 @@ class exchange_partitions_args:
       if fid == 1:
         if ftype == TType.MAP:
           self.partitionSpecs = {}
-          (_ktype991, _vtype992, _size990 ) = iprot.readMapBegin()
-          for _i994 in xrange(_size990):
-            _key995 = iprot.readString()
-            _val996 = iprot.readString()
-            self.partitionSpecs[_key995] = _val996
+          (_ktype1005, _vtype1006, _size1004 ) = iprot.readMapBegin()
+          for _i1008 in xrange(_size1004):
+            _key1009 = iprot.readString()
+            _val1010 = iprot.readString()
+            self.partitionSpecs[_key1009] = _val1010
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -23116,9 +24067,9 @@ class exchange_partitions_args:
     if self.partitionSpecs is not None:
       oprot.writeFieldBegin('partitionSpecs', TType.MAP, 1)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.partitionSpecs))
-      for kiter997,viter998 in self.partitionSpecs.items():
-        oprot.writeString(kiter997)
-        oprot.writeString(viter998)
+      for kiter1011,viter1012 in self.partitionSpecs.items():
+        oprot.writeString(kiter1011)
+        oprot.writeString(viter1012)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.source_db is not None:
@@ -23201,11 +24152,11 @@ class exchange_partitions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1002, _size999) = iprot.readListBegin()
-          for _i1003 in xrange(_size999):
-            _elem1004 = Partition()
-            _elem1004.read(iprot)
-            self.success.append(_elem1004)
+          (_etype1016, _size1013) = iprot.readListBegin()
+          for _i1017 in xrange(_size1013):
+            _elem1018 = Partition()
+            _elem1018.read(iprot)
+            self.success.append(_elem1018)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23246,8 +24197,8 @@ class exchange_partitions_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1005 in self.success:
-        iter1005.write(oprot)
+      for iter1019 in self.success:
+        iter1019.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -23341,10 +24292,10 @@ class get_partition_with_auth_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1009, _size1006) = iprot.readListBegin()
-          for _i1010 in xrange(_size1006):
-            _elem1011 = iprot.readString()
-            self.part_vals.append(_elem1011)
+          (_etype1023, _size1020) = iprot.readListBegin()
+          for _i1024 in xrange(_size1020):
+            _elem1025 = iprot.readString()
+            self.part_vals.append(_elem1025)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23356,10 +24307,10 @@ class get_partition_with_auth_args:
       elif fid == 5:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype1015, _size1012) = iprot.readListBegin()
-          for _i1016 in xrange(_size1012):
-            _elem1017 = iprot.readString()
-            self.group_names.append(_elem1017)
+          (_etype1029, _size1026) = iprot.readListBegin()
+          for _i1030 in xrange(_size1026):
+            _elem1031 = iprot.readString()
+            self.group_names.append(_elem1031)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23384,8 +24335,8 @@ class get_partition_with_auth_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter1018 in self.part_vals:
-        oprot.writeString(iter1018)
+      for iter1032 in self.part_vals:
+        oprot.writeString(iter1032)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.user_name is not None:
@@ -23395,8 +24346,8 @@ class get_partition_with_auth_args:
     if self.group_names is not None:
       oprot.writeFieldBegin('group_names', TType.LIST, 5)
       oprot.writeListBegin(TType.STRING, len(self.group_names))
-      for iter1019 in self.group_names:
-        oprot.writeString(iter1019)
+      for iter1033 in self.group_names:
+        oprot.writeString(iter1033)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -23825,11 +24776,11 @@ class get_partitions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1023, _size1020) = iprot.readListBegin()
-          for _i1024 in xrange(_size1020):
-            _elem1025 = Partition()
-            _elem1025.read(iprot)
-            self.success.append(_elem1025)
+          (_etype1037, _size1034) = iprot.readListBegin()
+          for _i1038 in xrange(_size1034):
+            _elem1039 = Partition()
+            _elem1039.read(iprot)
+            self.success.append(_elem1039)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23858,8 +24809,8 @@ class get_partitions_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1026 in self.success:
-        iter1026.write(oprot)
+      for iter1040 in self.success:
+        iter1040.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -23953,10 +24904,10 @@ class get_partitions_with_auth_args:
       elif fid == 5:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype1030, _size1027) = iprot.readListBegin()
-          for _i1031 in xrange(_size1027):
-            _elem1032 = iprot.readString()
-            self.group_names.append(_elem1032)
+          (_etype1044, _size1041) = iprot.readListBegin()
+          for _i1045 in xrange(_size1041):
+            _elem1046 = iprot.readString()
+            self.group_names.append(_elem1046)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23989,8 +24940,8 @@ class get_partitions_with_auth_args:
     if self.group_names is not None:
       oprot.writeFieldBegin('group_names', TType.LIST, 5)
       oprot.writeListBegin(TType.STRING, len(self.group_names))
-      for iter1033 in self.group_names:
-        oprot.writeString(iter1033)
+      for iter1047 in self.group_names:
+        oprot.writeString(iter1047)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -24051,11 +25002,11 @@ class get_partitions_with_auth_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1037, _size1034) = iprot.readListBegin()
-          for _i1038 in xrange(_size1034):
-            _elem1039 = Partition()
-            _elem1039.read(iprot)
-            self.success.append(_elem1039)
+          (_etype1051, _size1048) = iprot.readListBegin()
+          for _i1052 in xrange(_size1048):
+            _elem1053 = Partition()
+            _elem1053.read(iprot)
+            self.success.append(_elem1053)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -24084,8 +25035,8 @@ class get_partitions_with_auth_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1040 in self.success:
-        iter1040.write(oprot)
+      for iter1054 in self.success:
+        iter1054.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -24243,11 +25194,11 @@ class get_partitions_pspec_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1044, _size1041) = iprot.readListBegin()
-          for _i1045 in xrange(_size1041):
-            _elem1046 = PartitionSpec()
-            _elem1046.read(iprot)
-            self.success.append(_elem1046)
+          (_etype1058, _size1055) = iprot.readListBegin()
+          for _i1059 in xrange(_size1055):
+            _elem1060 = PartitionSpec()
+            _elem1060.read(iprot)
+            self.success.append(_elem1060)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -24276,8 +25227,8 @@ class get_partitions_pspec_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1047 in self.success:
-        iter1047.write(oprot)
+      for iter1061 in self.success:
+        iter1061.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -24435,10 +25386,10 @@ class get_partition_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1051, _size1048) = iprot.readListBegin()
-          for _i1052 in xrange(_size1048):
-            _elem1053 = iprot.readString()
-            self.success.append(_elem1053)
+          (_etype1065, _size1062) = iprot.readListBegin()
+          for _i1066 in xrange(_size1062):
+            _elem1067 = iprot.readString()
+            self.success.append(_elem1067)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -24467,8 +25418,8 @@ class get_partition_names_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1054 in self.success:
-        oprot.writeString(iter1054)
+      for iter1068 in self.success:
+        oprot.writeString(iter1068)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -24708,10 +25659,10 @@ class get_partitions_ps_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1058, _size1055) = iprot.readListBegin()
-          for _i1059 in xrange(_size1055):
-            _elem1060 = iprot.readString()
-            self.part_vals.append(_elem1060)
+          (_etype1072, _size1069) = iprot.readListBegin()
+          for _i1073 in xrange(_size1069):
+            _elem1074 = iprot.readString()
+            self.part_vals.append(_elem1074)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -24741,8 +25692,8 @@ class get_partitions_ps_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter1061 in self.part_vals:
-        oprot.writeString(iter1061)
+      for iter1075 in self.part_vals:
+        oprot.writeString(iter1075)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -24806,11 +25757,11 @@ class get_partitions_ps_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1065, _size1062) = iprot.readListBegin()
-          for _i1066 in xrange(_size1062):
-            _elem1067 = Partition()
-            _elem1067.read(iprot)
-            self.success.append(_elem1067)
+          (_etype1079, _size1076) = iprot.readListBegin()
+          for _i1080 in xrange(_size1076):
+            _elem1081 = Partition()
+            _elem1081.read(iprot)
+            self.success.append(_elem1081)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -24839,8 +25790,8 @@ class get_partitions_ps_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1068 in self.success:
-        iter1068.write(oprot)
+      for iter1082 in self.success:
+        iter1082.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -24927,10 +25878,10 @@ class get_partitions_ps_with_auth_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1072, _size1069) = iprot.readListBegin()
-          for _i1073 in xrange(_size1069):
-            _elem1074 = iprot.readString()
-            self.part_vals.append(_elem1074)
+          (_etype1086, _size1083) = iprot.readListBegin()
+          for _i1087 in xrange(_size1083):
+            _elem1088 = iprot.readString()
+            self.part_vals.append(_elem1088)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -24947,10 +25898,10 @@ class get_partitions_ps_with_auth_args:
       elif fid == 6:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype1078, _size1075) = iprot.readListBegin()
-          for _i1079 in xrange(_size1075):
-            _elem1080 = iprot.readString()
-            self.group_names.append(_elem1080)
+          (_etype1092, _size1089) = iprot.readListBegin()
+          for _i1093 in xrange(_size1089):
+            _elem1094 = iprot.readString()
+            self.group_names.append(_elem1094)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -24975,8 +25926,8 @@ class get_partitions_ps_with_auth_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter1081 in self.part_vals:
-        oprot.writeString(iter1081)
+      for iter1095 in self.part_vals:
+        oprot.writeString(iter1095)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -24990,8 +25941,8 @@ class get_partitions_ps_with_auth_args:
     if self.group_names is not None:
       oprot.writeFieldBegin('group_names', TType.LIST, 6)
       oprot.writeListBegin(TType.STRING, len(self.group_names))
-      for iter1082 in self.group_names:
-        oprot.writeString(iter1082)
+      for iter1096 in self.group_names:
+        oprot.writeString(iter1096)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -25053,11 +26004,11 @@ class get_partitions_ps_with_auth_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1086, _size1083) = iprot.readListBegin()
-          for _i1087 in xrange(_size1083):
-            _elem1088 = Partition()
-            _elem1088.read(iprot)
-            self.success.append(_elem1088)
+          (_etype1100, _size1097) = iprot.readListBegin()
+          for _i1101 in xrange(_size1097):
+            _elem1102 = Partition()
+            _elem1102.read(iprot)
+            self.success.append(_elem1102)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -25086,8 +26037,8 @@ class get_partitions_ps_with_auth_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1089 in self.success:
-        iter1089.write(oprot)
+      for iter1103 in self.success:
+        iter1103.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -25168,10 +26119,10 @@ class get_partition_names_ps_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1093, _size1090) = iprot.readListBegin()
-          for _i1094 in xrange(_size1090):
-            _elem1095 = iprot.readString()
-            self.part_vals.append(_elem1095)
+          (_etype1107, _size1104) = iprot.readListBegin()
+          for _i1108 in xrange(_size1104):
+            _elem1109 = iprot.readString()
+            self.part_vals.append(_elem1109)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -25201,8 +26152,8 @@ class get_partition_names_ps_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter1096 in self.part_vals:
-        oprot.writeString(iter1096)
+      for iter1110 in self.part_vals:
+        oprot.writeString(iter1110)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -25266,10 +26217,10 @@ class get_partition_names_ps_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1100, _size1097) = iprot.readListBegin()
-          for _i1101 in xrange(_size1097):
-            _elem1102 = iprot.readString()
-            self.success.append(_elem1102)
+          (_etype1114, _size1111) = iprot.readListBegin()
+          for _i1115 in xrange(_size1111):
+            _elem1116 = iprot.readString()
+            self.success.append(_elem1116)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -25298,8 +26249,8 @@ class get_partition_names_ps_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1103 in self.success:
-        oprot.writeString(iter1103)
+      for iter1117 in self.success:
+        oprot.writeString(iter1117)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -25470,11 +26421,11 @@ class get_partitions_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1107, _size1104) = iprot.readListBegin()
-          for _i1108 in xrange(_size1104):
-            _elem1109 = Partition()
-            _elem1109.read(iprot)
-            self.success.append(_elem1109)
+          (_etype1121, _size1118) = iprot.readListBegin()
+          for _i1122 in xrange(_size1118):
+            _elem1123 = Partition()
+            _elem1123.read(iprot)
+            self.success.append(_elem1123)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -25503,8 +26454,8 @@ class get_partitions_by_filter_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1110 in self.success:
-        iter1110.write(oprot)
+      for iter1124 in self.success:
+        iter1124.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -25675,11 +26626,11 @@ class get_part_specs_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1114, _size1111) = iprot.readListBegin()
-          for _i1115 in xrange(_size1111):
-            _elem1116 = PartitionSpec()
-            _elem1116.read(iprot)
-            self.success.append(_elem1116)
+          (_etype1128, _size1125) = iprot.readListBegin()
+          for _i1129 in xrange(_size1125):
+            _elem1130 = PartitionSpec()
+            _elem1130.read(iprot)
+            self.success.append(_elem1130)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -25708,8 +26659,8 @@ class get_part_specs_by_filter_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1117 in self.success:
-        iter1117.write(oprot)
+      for iter1131 in self.success:
+        iter1131.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -26129,10 +27080,10 @@ class get_partitions_by_names_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.names = []
-          (_etype1121, _size1118) = iprot.readListBegin()
-          for _i1122 in xrange(_size1118):
-            _elem1123 = iprot.readString()
-            self.names.append(_elem1123)
+          (_etype1135, _size1132) = iprot.readListBegin()
+          for _i1136 in xrange(_size1132):
+            _elem1137 = iprot.readString()
+            self.names.append(_elem1137)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26157,8 +27108,8 @@ class get_partitions_by_names_args:
     if self.names is not None:
       oprot.writeFieldBegin('names', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.names))
-      for iter1124 in self.names:
-        oprot.writeString(iter1124)
+      for iter1138 in self.names:
+        oprot.writeString(iter1138)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -26217,11 +27168,11 @@ class get_partitions_by_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1128, _size1125) = iprot.readListBegin()
-          for _i1129 in xrange(_size1125):
-            _elem1130 = Partition()
-            _elem1130.read(iprot)
-            self.success.append(_elem1130)
+          (_etype1142, _size1139) = iprot.readListBegin()
+          for _i1143 in xrange(_size1139):
+            _elem1144 = Partition()
+            _elem1144.read(iprot)
+            self.success.append(_elem1144)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26250,8 +27201,8 @@ class get_partitions_by_names_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1131 in self.success:
-        iter1131.write(oprot)
+      for iter1145 in self.success:
+        iter1145.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -26501,11 +27452,11 @@ class alter_partitions_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype1135, _size1132) = iprot.readListBegin()
-          for _i1136 in xrange(_size1132):
-            _elem1137 = Partition()
-            _elem1137.read(iprot)
-            self.new_parts.append(_elem1137)
+          (_etype1149, _size1146) = iprot.readListBegin()
+          for _i1150 in xrange(_size1146):
+            _elem1151 = Partition()
+            _elem1151.read(iprot)
+            self.new_parts.append(_elem1151)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26530,8 +27481,8 @@ class alter_partitions_args:
     if self.new_parts is not None:
       oprot.writeFieldBegin('new_parts', TType.LIST, 3)
       oprot.writeListBegin(TType.STRUCT, len(self.new_parts))
-      for iter1138 in self.new_parts:
-        iter1138.write(oprot)
+      for iter1152 in self.new_parts:
+        iter1152.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -26684,11 +27635,11 @@ class alter_partitions_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype1142, _size1139) = iprot.readListBegin()
-          for _i1143 in xrange(_size1139):
-            _elem1144 = Partition()
-

<TRUNCATED>

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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/main/thrift/hive_metastore.thrift
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/thrift/hive_metastore.thrift b/standalone-metastore/src/main/thrift/hive_metastore.thrift
index 65e2f78..b816eb6 100644
--- a/standalone-metastore/src/main/thrift/hive_metastore.thrift
+++ b/standalone-metastore/src/main/thrift/hive_metastore.thrift
@@ -183,6 +183,39 @@ enum EventRequestType {
     DELETE = 3,
 }
 
+enum SerdeType {
+  HIVE = 1,
+  SCHEMA_REGISTRY = 2,
+}
+
+enum SchemaType {
+  HIVE = 1,
+  AVRO = 2,
+}
+
+enum SchemaCompatibility {
+  NONE = 1,
+  BACKWARD = 2,
+  FORWARD = 3,
+  BOTH = 4
+}
+
+enum SchemaValidation {
+  LATEST = 1,
+  ALL = 2
+}
+
+enum SchemaVersionState {
+  INITIATED = 1,
+  START_REVIEW = 2,
+  CHANGES_REQUIRED = 3,
+  REVIEWED = 4,
+  ENABLED = 5,
+  DISABLED = 6,
+  ARCHIVED = 7,
+  DELETED = 8
+}
+
 struct HiveObjectRef{
   1: HiveObjectType objectType,
   2: string dbName,
@@ -289,7 +322,11 @@ struct Database {
 struct SerDeInfo {
   1: string name,                   // name of the serde, table name by default
   2: string serializationLib,       // usually the class that implements the extractor & loader
-  3: map<string, string> parameters // initialization parameters
+  3: map<string, string> parameters, // initialization parameters
+  4: optional string description,
+  5: optional string serializerClass,
+  6: optional string deserializerClass,
+  7: optional SerdeType serdeType
 }
 
 // sort order of a column (column name along with asc(1)/desc(0))
@@ -1318,6 +1355,71 @@ struct WMCreateOrDropTriggerToPoolMappingRequest {
 struct WMCreateOrDropTriggerToPoolMappingResponse {
 }
 
+// Schema objects
+// Schema is already taken, so for the moment I'm calling it an ISchema for Independent Schema
+struct ISchema {
+  1: SchemaType schemaType,
+  2: string name,
+  3: string dbName,
+  4: SchemaCompatibility compatibility,
+  5: SchemaValidation validationLevel,
+  6: bool canEvolve,
+  7: optional string schemaGroup,
+  8: optional string description
+}
+
+struct ISchemaName {
+  1: string dbName,
+  2: string schemaName
+}
+
+struct AlterISchemaRequest {
+  1: ISchemaName name,
+  3: ISchema newSchema
+}
+
+struct SchemaVersion {
+  1:  ISchemaName schema,
+  2:  i32 version,
+  3:  i64 createdAt,
+  4:  list<FieldSchema> cols,
+  5:  optional SchemaVersionState state,
+  6:  optional string description,
+  7:  optional string schemaText,
+  8:  optional string fingerprint,
+  9:  optional string name,
+  10: optional SerDeInfo serDe
+}
+
+struct SchemaVersionDescriptor {
+  1: ISchemaName schema,
+  2: i32 version
+}
+
+struct FindSchemasByColsRqst {
+  1: optional string colName,
+  2: optional string colNamespace,
+  3: optional string type
+}
+
+struct FindSchemasByColsResp {
+  1: list<SchemaVersionDescriptor> schemaVersions
+}
+
+struct MapSchemaVersionToSerdeRequest {
+  1: SchemaVersionDescriptor schemaVersion,
+  2: string serdeName
+}
+
+struct SetSchemaVersionStateRequest {
+  1: SchemaVersionDescriptor schemaVersion,
+  2: SchemaVersionState state
+}
+
+struct GetSerdeRequest {
+  1: string serdeName
+}
+
 // Exceptions.
 
 exception MetaException {
@@ -1929,6 +2031,38 @@ service ThriftHiveMetastore extends fb303.FacebookService
 
   WMCreateOrDropTriggerToPoolMappingResponse create_or_drop_wm_trigger_to_pool_mapping(1:WMCreateOrDropTriggerToPoolMappingRequest request)
       throws(1:AlreadyExistsException o1, 2:NoSuchObjectException o2, 3:InvalidObjectException o3, 4:MetaException o4)
+
+  // Schema calls
+  void create_ischema(1:ISchema schema) throws(1:AlreadyExistsException o1,
+        NoSuchObjectException o2, 3:MetaException o3)
+  void alter_ischema(1:AlterISchemaRequest rqst)
+        throws(1:NoSuchObjectException o1, 2:MetaException o2)
+  ISchema get_ischema(1:ISchemaName name) throws (1:NoSuchObjectException o1, 2:MetaException o2)
+  void drop_ischema(1:ISchemaName name)
+        throws(1:NoSuchObjectException o1, 2:InvalidOperationException o2, 3:MetaException o3)
+
+  void add_schema_version(1:SchemaVersion schemaVersion)
+        throws(1:AlreadyExistsException o1, 2:NoSuchObjectException o2, 3:MetaException o3)
+  SchemaVersion get_schema_version(1: SchemaVersionDescriptor schemaVersion)
+        throws (1:NoSuchObjectException o1, 2:MetaException o2)
+  SchemaVersion get_schema_latest_version(1: ISchemaName schemaName)
+        throws (1:NoSuchObjectException o1, 2:MetaException o2)
+  list<SchemaVersion> get_schema_all_versions(1: ISchemaName schemaName)
+        throws (1:NoSuchObjectException o1, 2:MetaException o2)
+  void drop_schema_version(1: SchemaVersionDescriptor schemaVersion)
+        throws(1:NoSuchObjectException o1, 2:MetaException o2)
+  FindSchemasByColsResp get_schemas_by_cols(1: FindSchemasByColsRqst rqst)
+        throws(1:MetaException o1)
+  // There is no blanket update of SchemaVersion since it is (mostly) immutable.  The only
+  // updates are the specific ones to associate a version with a serde and to change its state
+  void map_schema_version_to_serde(1: MapSchemaVersionToSerdeRequest rqst)
+        throws(1:NoSuchObjectException o1, 2:MetaException o2)
+  void set_schema_version_state(1: SetSchemaVersionStateRequest rqst)
+        throws(1:NoSuchObjectException o1, 2:InvalidOperationException o2, 3:MetaException o3)
+
+  void add_serde(1: SerDeInfo serde) throws(1:AlreadyExistsException o1, 2:MetaException o2)
+  SerDeInfo get_serde(1: GetSerdeRequest rqst) throws(1:NoSuchObjectException o1, 2:MetaException o2)
+
 }
 
 // * Note about the DDL_TIME: When creating or altering a table or a partition,

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
index 1c0eafd..5edf8b3 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
@@ -19,6 +19,8 @@
 package org.apache.hadoop.hive.metastore;
 
 import org.apache.hadoop.hive.metastore.api.CreationMetadata;
+import org.apache.hadoop.hive.metastore.api.ISchemaName;
+import org.apache.hadoop.hive.metastore.api.SchemaVersionDescriptor;
 import org.apache.hadoop.hive.metastore.api.WMFullResourcePlan;
 
 import java.nio.ByteBuffer;
@@ -38,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.InvalidInputException;
 import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
 import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
@@ -67,6 +70,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;
@@ -1047,4 +1052,74 @@ public class DummyRawStoreControlledCommit implements RawStore, Configurable {
     // TODO Auto-generated method stub
     return null;
   }
+
+  public void createISchema(ISchema schema) throws AlreadyExistsException, MetaException,
+      NoSuchObjectException {
+    objectStore.createISchema(schema);
+  }
+
+  @Override
+  public void alterISchema(ISchemaName schemaName, ISchema newSchema) throws NoSuchObjectException,
+      MetaException {
+    objectStore.alterISchema(schemaName, newSchema);
+  }
+
+  @Override
+  public ISchema getISchema(ISchemaName schemaName) throws MetaException {
+    return objectStore.getISchema(schemaName);
+  }
+
+  @Override
+  public void dropISchema(ISchemaName schemaName) throws NoSuchObjectException, MetaException {
+    objectStore.dropISchema(schemaName);
+  }
+
+  @Override
+  public void addSchemaVersion(SchemaVersion schemaVersion) throws
+      AlreadyExistsException, InvalidObjectException, NoSuchObjectException, MetaException {
+    objectStore.addSchemaVersion(schemaVersion);
+  }
+
+  @Override
+  public void alterSchemaVersion(SchemaVersionDescriptor version, SchemaVersion newVersion) throws
+      NoSuchObjectException, MetaException {
+    objectStore.alterSchemaVersion(version, newVersion);
+  }
+
+  @Override
+  public SchemaVersion getSchemaVersion(SchemaVersionDescriptor version) throws MetaException {
+    return objectStore.getSchemaVersion(version);
+  }
+
+  @Override
+  public SchemaVersion getLatestSchemaVersion(ISchemaName schemaName) throws MetaException {
+    return objectStore.getLatestSchemaVersion(schemaName);
+  }
+
+  @Override
+  public List<SchemaVersion> getAllSchemaVersion(ISchemaName schemaName) throws MetaException {
+    return objectStore.getAllSchemaVersion(schemaName);
+  }
+
+  @Override
+  public List<SchemaVersion> getSchemaVersionsByColumns(String colName, String colNamespace,
+                                                        String type) throws MetaException {
+    return objectStore.getSchemaVersionsByColumns(colName, colNamespace, type);
+  }
+
+  @Override
+  public void dropSchemaVersion(SchemaVersionDescriptor version) throws NoSuchObjectException,
+      MetaException {
+    objectStore.dropSchemaVersion(version);
+  }
+
+  @Override
+  public SerDeInfo getSerDeInfo(String serDeName) throws NoSuchObjectException, MetaException {
+    return objectStore.getSerDeInfo(serDeName);
+  }
+
+  @Override
+  public void addSerde(SerDeInfo serde) throws AlreadyExistsException, MetaException {
+    objectStore.addSerde(serde);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
index cbd8678..132cdc3 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
@@ -19,6 +19,8 @@
 package org.apache.hadoop.hive.metastore;
 
 import org.apache.hadoop.hive.metastore.api.CreationMetadata;
+import org.apache.hadoop.hive.metastore.api.ISchemaName;
+import org.apache.hadoop.hive.metastore.api.SchemaVersionDescriptor;
 import org.apache.hadoop.hive.metastore.api.WMFullResourcePlan;
 
 import java.nio.ByteBuffer;
@@ -36,6 +38,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.InvalidInputException;
 import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
 import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
@@ -65,6 +68,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;
@@ -1034,4 +1039,73 @@ public class DummyRawStoreForJdoConnection implements RawStore {
     // TODO Auto-generated method stub
     return null;
   }
+
+  public void createISchema(ISchema schema) throws AlreadyExistsException, MetaException {
+
+  }
+
+  @Override
+  public void alterISchema(ISchemaName schemaName, ISchema newSchema) throws NoSuchObjectException,
+      MetaException {
+
+  }
+
+  @Override
+  public ISchema getISchema(ISchemaName schemaName) throws MetaException {
+    return null;
+  }
+
+  @Override
+  public void dropISchema(ISchemaName schemaName) throws NoSuchObjectException, MetaException {
+
+  }
+
+  @Override
+  public void addSchemaVersion(SchemaVersion schemaVersion) throws
+      AlreadyExistsException, InvalidObjectException, NoSuchObjectException, MetaException {
+
+  }
+
+  @Override
+  public void alterSchemaVersion(SchemaVersionDescriptor version, SchemaVersion newVersion) throws
+      NoSuchObjectException, MetaException {
+
+  }
+
+  @Override
+  public SchemaVersion getSchemaVersion(SchemaVersionDescriptor version) throws MetaException {
+    return null;
+  }
+
+  @Override
+  public SchemaVersion getLatestSchemaVersion(ISchemaName schemaName) throws MetaException {
+    return null;
+  }
+
+  @Override
+  public List<SchemaVersion> getAllSchemaVersion(ISchemaName schemaName) throws MetaException {
+    return null;
+  }
+
+  @Override
+  public List<SchemaVersion> getSchemaVersionsByColumns(String colName, String colNamespace,
+                                                        String type) throws MetaException {
+    return null;
+  }
+
+  @Override
+  public void dropSchemaVersion(SchemaVersionDescriptor version) throws NoSuchObjectException,
+      MetaException {
+
+  }
+
+  @Override
+  public SerDeInfo getSerDeInfo(String serDeName) throws MetaException {
+    return null;
+  }
+
+  @Override
+  public void addSerde(SerDeInfo serde) throws AlreadyExistsException, MetaException {
+
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreSchemaMethods.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreSchemaMethods.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreSchemaMethods.java
new file mode 100644
index 0000000..1b30090
--- /dev/null
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreSchemaMethods.java
@@ -0,0 +1,1180 @@
+/*
+ * 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;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.annotation.MetastoreCheckinTest;
+import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
+import org.apache.hadoop.hive.metastore.api.Database;
+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.ISchema;
+import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+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.SchemaVersionDescriptor;
+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.client.builder.DatabaseBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.ISchemaBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.SchemaVersionBuilder;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
+import org.apache.hadoop.hive.metastore.events.AddSchemaVersionEvent;
+import org.apache.hadoop.hive.metastore.events.AlterISchemaEvent;
+import org.apache.hadoop.hive.metastore.events.AlterSchemaVersionEvent;
+import org.apache.hadoop.hive.metastore.events.CreateISchemaEvent;
+import org.apache.hadoop.hive.metastore.events.DropISchemaEvent;
+import org.apache.hadoop.hive.metastore.events.DropSchemaVersionEvent;
+import org.apache.hadoop.hive.metastore.events.PreEventContext;
+import org.apache.hadoop.hive.metastore.messaging.EventMessage;
+import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge;
+import org.apache.thrift.TException;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_DATABASE_NAME;
+
+// This does the testing using a remote metastore, as that finds more issues in thrift
+@Category(MetastoreCheckinTest.class)
+public class TestHiveMetaStoreSchemaMethods {
+  private static Map<EventMessage.EventType, Integer> events;
+  private static Map<EventMessage.EventType, Integer> transactionalEvents;
+  private static Map<PreEventContext.PreEventType, Integer> preEvents;
+
+  private static IMetaStoreClient client;
+
+
+  @BeforeClass
+  public static void startMetastore() throws Exception {
+    Configuration conf = MetastoreConf.newMetastoreConf();
+    MetaStoreTestUtils.setConfForStandloneMode(conf);
+    MetastoreConf.setClass(conf, ConfVars.EVENT_LISTENERS, SchemaEventListener.class,
+        MetaStoreEventListener.class);
+    MetastoreConf.setClass(conf, ConfVars.TRANSACTIONAL_EVENT_LISTENERS,
+        TransactionalSchemaEventListener.class, MetaStoreEventListener.class);
+    MetastoreConf.setClass(conf, ConfVars.PRE_EVENT_LISTENERS, SchemaPreEventListener.class,
+        MetaStorePreEventListener.class);
+    int port = MetaStoreTestUtils.startMetaStoreWithRetry(HadoopThriftAuthBridge.getBridge(),
+        conf);
+    MetastoreConf.setVar(conf, ConfVars.THRIFT_URIS, "thrift://localhost:" + port);
+    client = new HiveMetaStoreClient(conf);
+  }
+
+  @Before
+  public void newMaps() {
+    events = new HashMap<>();
+    transactionalEvents = new HashMap<>();
+    preEvents = new HashMap<>();
+  }
+
+  @Test(expected = NoSuchObjectException.class)
+  public void getNonExistentSchema() throws TException {
+    client.getISchema(DEFAULT_DATABASE_NAME, "no.such.schema");
+  }
+
+  @Test
+  public void iSchema() throws TException {
+    String schemaName = uniqueSchemaName();
+    String schemaGroup = "group1";
+    String description = "This is a description";
+    ISchema schema = new ISchemaBuilder()
+        .setSchemaType(SchemaType.AVRO)
+        .setName(schemaName)
+        .setCompatibility(SchemaCompatibility.FORWARD)
+        .setValidationLevel(SchemaValidation.LATEST)
+        .setCanEvolve(false)
+        .setSchemaGroup(schemaGroup)
+        .setDescription(description)
+        .build();
+    client.createISchema(schema);
+
+    Assert.assertEquals(1, (int)preEvents.get(PreEventContext.PreEventType.CREATE_ISCHEMA));
+    Assert.assertEquals(1, (int)events.get(EventMessage.EventType.CREATE_ISCHEMA));
+    Assert.assertEquals(1, (int)transactionalEvents.get(EventMessage.EventType.CREATE_ISCHEMA));
+
+    schema = client.getISchema(DEFAULT_DATABASE_NAME, schemaName);
+    Assert.assertEquals(1, (int)preEvents.get(PreEventContext.PreEventType.READ_ISCHEMA));
+
+    Assert.assertEquals(SchemaType.AVRO, schema.getSchemaType());
+    Assert.assertEquals(schemaName, schema.getName());
+    Assert.assertEquals(SchemaCompatibility.FORWARD, schema.getCompatibility());
+    Assert.assertEquals(SchemaValidation.LATEST, schema.getValidationLevel());
+    Assert.assertFalse(schema.isCanEvolve());
+    Assert.assertEquals(schemaGroup, schema.getSchemaGroup());
+    Assert.assertEquals(description, schema.getDescription());
+
+    schemaGroup = "new group";
+    description = "new description";
+    schema.setCompatibility(SchemaCompatibility.BOTH);
+    schema.setValidationLevel(SchemaValidation.ALL);
+    schema.setCanEvolve(true);
+    schema.setSchemaGroup(schemaGroup);
+    schema.setDescription(description);
+    client.alterISchema(DEFAULT_DATABASE_NAME, schemaName, schema);
+    Assert.assertEquals(1, (int)preEvents.get(PreEventContext.PreEventType.ALTER_ISCHEMA));
+    Assert.assertEquals(1, (int)events.get(EventMessage.EventType.ALTER_ISCHEMA));
+    Assert.assertEquals(1, (int)transactionalEvents.get(EventMessage.EventType.ALTER_ISCHEMA));
+
+    schema = client.getISchema(DEFAULT_DATABASE_NAME, schemaName);
+    Assert.assertEquals(2, (int)preEvents.get(PreEventContext.PreEventType.READ_ISCHEMA));
+
+    Assert.assertEquals(SchemaType.AVRO, schema.getSchemaType());
+    Assert.assertEquals(schemaName, schema.getName());
+    Assert.assertEquals(SchemaCompatibility.BOTH, schema.getCompatibility());
+    Assert.assertEquals(SchemaValidation.ALL, schema.getValidationLevel());
+    Assert.assertTrue(schema.isCanEvolve());
+    Assert.assertEquals(schemaGroup, schema.getSchemaGroup());
+    Assert.assertEquals(description, schema.getDescription());
+
+    client.dropISchema(DEFAULT_DATABASE_NAME, schemaName);
+    Assert.assertEquals(1, (int)preEvents.get(PreEventContext.PreEventType.DROP_ISCHEMA));
+    Assert.assertEquals(1, (int)events.get(EventMessage.EventType.DROP_ISCHEMA));
+    Assert.assertEquals(1, (int)transactionalEvents.get(EventMessage.EventType.DROP_ISCHEMA));
+    try {
+      client.getISchema(DEFAULT_DATABASE_NAME, schemaName);
+      Assert.fail();
+    } catch (NoSuchObjectException e) {
+      // all good
+    }
+  }
+
+  @Test
+  public void iSchemaOtherDatabase() throws TException {
+    String dbName = "other_db";
+    Database db = new DatabaseBuilder()
+        .setName(dbName)
+        .build();
+    client.createDatabase(db);
+
+    String schemaName = uniqueSchemaName();
+    String schemaGroup = "group1";
+    String description = "This is a description";
+    ISchema schema = new ISchemaBuilder()
+        .setSchemaType(SchemaType.AVRO)
+        .setName(schemaName)
+        .setDbName(dbName)
+        .setCompatibility(SchemaCompatibility.FORWARD)
+        .setValidationLevel(SchemaValidation.LATEST)
+        .setCanEvolve(false)
+        .setSchemaGroup(schemaGroup)
+        .setDescription(description)
+        .build();
+    client.createISchema(schema);
+
+    schema = client.getISchema(dbName, schemaName);
+
+    Assert.assertEquals(SchemaType.AVRO, schema.getSchemaType());
+    Assert.assertEquals(schemaName, schema.getName());
+    Assert.assertEquals(dbName, schema.getDbName());
+    Assert.assertEquals(SchemaCompatibility.FORWARD, schema.getCompatibility());
+    Assert.assertEquals(SchemaValidation.LATEST, schema.getValidationLevel());
+    Assert.assertFalse(schema.isCanEvolve());
+    Assert.assertEquals(schemaGroup, schema.getSchemaGroup());
+    Assert.assertEquals(description, schema.getDescription());
+
+    schemaGroup = "new group";
+    description = "new description";
+    schema.setCompatibility(SchemaCompatibility.BOTH);
+    schema.setValidationLevel(SchemaValidation.ALL);
+    schema.setCanEvolve(true);
+    schema.setSchemaGroup(schemaGroup);
+    schema.setDescription(description);
+    client.alterISchema(dbName, schemaName, schema);
+
+    schema = client.getISchema(dbName, schemaName);
+
+    Assert.assertEquals(SchemaType.AVRO, schema.getSchemaType());
+    Assert.assertEquals(schemaName, schema.getName());
+    Assert.assertEquals(dbName, schema.getDbName());
+    Assert.assertEquals(SchemaCompatibility.BOTH, schema.getCompatibility());
+    Assert.assertEquals(SchemaValidation.ALL, schema.getValidationLevel());
+    Assert.assertTrue(schema.isCanEvolve());
+    Assert.assertEquals(schemaGroup, schema.getSchemaGroup());
+    Assert.assertEquals(description, schema.getDescription());
+
+    client.dropISchema(dbName, schemaName);
+    try {
+      client.getISchema(dbName, schemaName);
+      Assert.fail();
+    } catch (NoSuchObjectException e) {
+      // all good
+    }
+  }
+
+  @Test(expected = NoSuchObjectException.class)
+  public void schemaWithInvalidDatabase() throws TException {
+    ISchema schema = new ISchemaBuilder()
+        .setName("thisSchemaDoesntHaveADb")
+        .setDbName("no.such.database")
+        .setSchemaType(SchemaType.AVRO)
+        .build();
+    client.createISchema(schema);
+  }
+
+  @Test(expected = AlreadyExistsException.class)
+  public void schemaAlreadyExists() throws TException {
+    String schemaName = uniqueSchemaName();
+    ISchema schema = new ISchemaBuilder()
+        .setSchemaType(SchemaType.HIVE)
+        .setName(schemaName)
+        .build();
+    client.createISchema(schema);
+
+    schema = client.getISchema(DEFAULT_DATABASE_NAME, schemaName);
+    Assert.assertNotNull(schema);
+
+    Assert.assertEquals(SchemaType.HIVE, schema.getSchemaType());
+    Assert.assertEquals(schemaName, schema.getName());
+    Assert.assertEquals(SchemaCompatibility.BACKWARD, schema.getCompatibility());
+    Assert.assertEquals(SchemaValidation.ALL, schema.getValidationLevel());
+    Assert.assertTrue(schema.isCanEvolve());
+
+    // This second attempt to create it should throw
+    client.createISchema(schema);
+  }
+
+  @Test(expected = NoSuchObjectException.class)
+  public void alterNonExistentSchema() throws TException {
+    String schemaName = uniqueSchemaName();
+    ISchema schema = new ISchemaBuilder()
+        .setSchemaType(SchemaType.HIVE)
+        .setName(schemaName)
+        .setDescription("a new description")
+        .build();
+    client.alterISchema(DEFAULT_DATABASE_NAME, schemaName, schema);
+  }
+
+  @Test(expected = NoSuchObjectException.class)
+  public void dropNonExistentSchema() throws TException {
+    client.dropISchema(DEFAULT_DATABASE_NAME, "no_such_schema");
+  }
+
+  @Test(expected = NoSuchObjectException.class)
+  public void createVersionOfNonExistentSchema() throws TException {
+    SchemaVersion schemaVersion = new SchemaVersionBuilder()
+        .setSchemaName("noSchemaOfThisNameExists")
+        .setDbName(DEFAULT_DATABASE_NAME)
+        .setVersion(1)
+        .addCol("a", ColumnType.STRING_TYPE_NAME)
+        .build();
+    client.addSchemaVersion(schemaVersion);
+  }
+
+  @Test
+  public void addSchemaVersion() throws TException {
+    String schemaName = uniqueSchemaName();
+    int version = 1;
+
+    ISchema schema = new ISchemaBuilder()
+        .setSchemaType(SchemaType.AVRO)
+        .setName(schemaName)
+        .build();
+    client.createISchema(schema);
+
+    String description = "very descriptive";
+    String schemaText = "this should look like json, but oh well";
+    String fingerprint = "this should be an md5 string";
+    String versionName = "why would I name a version?";
+    long creationTime = 10;
+    String serdeName = "serde_for_schema37";
+    String serializer = "org.apache.hadoop.hive.metastore.test.Serializer";
+    String deserializer = "org.apache.hadoop.hive.metastore.test.Deserializer";
+    String serdeDescription = "how do you describe a serde?";
+    SchemaVersion schemaVersion = new SchemaVersionBuilder()
+        .versionOf(schema)
+        .setVersion(version)
+        .addCol("a", ColumnType.INT_TYPE_NAME)
+        .addCol("b", ColumnType.FLOAT_TYPE_NAME)
+        .setCreatedAt(creationTime)
+        .setState(SchemaVersionState.INITIATED)
+        .setDescription(description)
+        .setSchemaText(schemaText)
+        .setFingerprint(fingerprint)
+        .setName(versionName)
+        .setSerdeName(serdeName)
+        .setSerdeSerializerClass(serializer)
+        .setSerdeDeserializerClass(deserializer)
+        .setSerdeDescription(serdeDescription)
+        .build();
+    client.addSchemaVersion(schemaVersion);
+    Assert.assertEquals(1, (int)preEvents.get(PreEventContext.PreEventType.ADD_SCHEMA_VERSION));
+    Assert.assertEquals(1, (int)events.get(EventMessage.EventType.ADD_SCHEMA_VERSION));
+    Assert.assertEquals(1, (int)transactionalEvents.get(EventMessage.EventType.ADD_SCHEMA_VERSION));
+
+    schemaVersion = client.getSchemaVersion(DEFAULT_DATABASE_NAME, schemaName, version);
+    Assert.assertNotNull(schemaVersion);
+    Assert.assertEquals(schemaName, schemaVersion.getSchema().getSchemaName());
+    Assert.assertEquals(DEFAULT_DATABASE_NAME, schemaVersion.getSchema().getDbName());
+    Assert.assertEquals(version, schemaVersion.getVersion());
+    Assert.assertEquals(creationTime, schemaVersion.getCreatedAt());
+    Assert.assertEquals(SchemaVersionState.INITIATED, schemaVersion.getState());
+    Assert.assertEquals(description, schemaVersion.getDescription());
+    Assert.assertEquals(schemaText, schemaVersion.getSchemaText());
+    Assert.assertEquals(fingerprint, schemaVersion.getFingerprint());
+    Assert.assertEquals(versionName, schemaVersion.getName());
+    Assert.assertEquals(serdeName, schemaVersion.getSerDe().getName());
+    Assert.assertEquals(serializer, schemaVersion.getSerDe().getSerializerClass());
+    Assert.assertEquals(deserializer, schemaVersion.getSerDe().getDeserializerClass());
+    Assert.assertEquals(serdeDescription, schemaVersion.getSerDe().getDescription());
+    Assert.assertEquals(2, schemaVersion.getColsSize());
+    List<FieldSchema> cols = schemaVersion.getCols();
+    Collections.sort(cols);
+    Assert.assertEquals("a", cols.get(0).getName());
+    Assert.assertEquals(ColumnType.INT_TYPE_NAME, cols.get(0).getType());
+    Assert.assertEquals("b", cols.get(1).getName());
+    Assert.assertEquals(ColumnType.FLOAT_TYPE_NAME, cols.get(1).getType());
+    Assert.assertEquals(1, (int)preEvents.get(PreEventContext.PreEventType.READ_SCHEMA_VERSION));
+
+    client.dropSchemaVersion(DEFAULT_DATABASE_NAME, schemaName, version);
+    Assert.assertEquals(1, (int)preEvents.get(PreEventContext.PreEventType.DROP_SCHEMA_VERSION));
+    Assert.assertEquals(1, (int)events.get(EventMessage.EventType.DROP_SCHEMA_VERSION));
+    Assert.assertEquals(1, (int)transactionalEvents.get(EventMessage.EventType.DROP_SCHEMA_VERSION));
+    try {
+      client.getSchemaVersion(DEFAULT_DATABASE_NAME, schemaName, version);
+      Assert.fail();
+    } catch (NoSuchObjectException e) {
+      // all good
+    }
+  }
+
+  @Test
+  public void addSchemaVersionOtherDb() throws TException {
+    String dbName = "other_db_for_schema_version";
+    Database db = new DatabaseBuilder()
+        .setName(dbName)
+        .build();
+    client.createDatabase(db);
+
+    String schemaName = uniqueSchemaName();
+    int version = 1;
+
+    ISchema schema = new ISchemaBuilder()
+        .setDbName(dbName)
+        .setSchemaType(SchemaType.AVRO)
+        .setName(schemaName)
+        .build();
+    client.createISchema(schema);
+
+    String description = "very descriptive";
+    String schemaText = "this should look like json, but oh well";
+    String fingerprint = "this should be an md5 string";
+    String versionName = "why would I name a version?";
+    long creationTime = 10;
+    String serdeName = "serde_for_schema37";
+    String serializer = "org.apache.hadoop.hive.metastore.test.Serializer";
+    String deserializer = "org.apache.hadoop.hive.metastore.test.Deserializer";
+    String serdeDescription = "how do you describe a serde?";
+    SchemaVersion schemaVersion = new SchemaVersionBuilder()
+        .versionOf(schema)
+        .setVersion(version)
+        .addCol("a", ColumnType.INT_TYPE_NAME)
+        .addCol("b", ColumnType.FLOAT_TYPE_NAME)
+        .setCreatedAt(creationTime)
+        .setState(SchemaVersionState.INITIATED)
+        .setDescription(description)
+        .setSchemaText(schemaText)
+        .setFingerprint(fingerprint)
+        .setName(versionName)
+        .setSerdeName(serdeName)
+        .setSerdeSerializerClass(serializer)
+        .setSerdeDeserializerClass(deserializer)
+        .setSerdeDescription(serdeDescription)
+        .build();
+    client.addSchemaVersion(schemaVersion);
+
+    schemaVersion = client.getSchemaVersion(dbName, schemaName, version);
+    Assert.assertNotNull(schemaVersion);
+    Assert.assertEquals(schemaName, schemaVersion.getSchema().getSchemaName());
+    Assert.assertEquals(dbName, schemaVersion.getSchema().getDbName());
+    Assert.assertEquals(version, schemaVersion.getVersion());
+    Assert.assertEquals(creationTime, schemaVersion.getCreatedAt());
+    Assert.assertEquals(SchemaVersionState.INITIATED, schemaVersion.getState());
+    Assert.assertEquals(description, schemaVersion.getDescription());
+    Assert.assertEquals(schemaText, schemaVersion.getSchemaText());
+    Assert.assertEquals(fingerprint, schemaVersion.getFingerprint());
+    Assert.assertEquals(versionName, schemaVersion.getName());
+    Assert.assertEquals(serdeName, schemaVersion.getSerDe().getName());
+    Assert.assertEquals(serializer, schemaVersion.getSerDe().getSerializerClass());
+    Assert.assertEquals(deserializer, schemaVersion.getSerDe().getDeserializerClass());
+    Assert.assertEquals(serdeDescription, schemaVersion.getSerDe().getDescription());
+    Assert.assertEquals(2, schemaVersion.getColsSize());
+    List<FieldSchema> cols = schemaVersion.getCols();
+    Collections.sort(cols);
+    Assert.assertEquals("a", cols.get(0).getName());
+    Assert.assertEquals(ColumnType.INT_TYPE_NAME, cols.get(0).getType());
+    Assert.assertEquals("b", cols.get(1).getName());
+    Assert.assertEquals(ColumnType.FLOAT_TYPE_NAME, cols.get(1).getType());
+    Assert.assertEquals(1, (int)preEvents.get(PreEventContext.PreEventType.READ_SCHEMA_VERSION));
+
+    client.dropSchemaVersion(dbName, schemaName, version);
+    try {
+      client.getSchemaVersion(dbName, schemaName, version);
+      Assert.fail();
+    } catch (NoSuchObjectException e) {
+      // all good
+    }
+  }
+
+  // Test that adding multiple versions of the same schema
+  @Test
+  public void multipleSchemaVersions() throws TException {
+    String schemaName = uniqueSchemaName();
+
+    ISchema schema = new ISchemaBuilder()
+        .setSchemaType(SchemaType.AVRO)
+        .setName(schemaName)
+        .build();
+    client.createISchema(schema);
+    SchemaVersion schemaVersion = new SchemaVersionBuilder()
+        .versionOf(schema)
+        .setVersion(1)
+        .addCol("a", ColumnType.BIGINT_TYPE_NAME)
+        .build();
+    client.addSchemaVersion(schemaVersion);
+
+    schemaVersion = new SchemaVersionBuilder()
+        .versionOf(schema)
+        .setVersion(2)
+        .addCol("a", ColumnType.BIGINT_TYPE_NAME)
+        .addCol("b", ColumnType.DATE_TYPE_NAME)
+        .build();
+    client.addSchemaVersion(schemaVersion);
+
+    schemaVersion = new SchemaVersionBuilder()
+        .versionOf(schema)
+        .setVersion(3)
+        .addCol("a", ColumnType.BIGINT_TYPE_NAME)
+        .addCol("b", ColumnType.DATE_TYPE_NAME)
+        .addCol("c", ColumnType.TIMESTAMP_TYPE_NAME)
+        .build();
+    client.addSchemaVersion(schemaVersion);
+    Assert.assertEquals(3, (int)preEvents.get(PreEventContext.PreEventType.ADD_SCHEMA_VERSION));
+    Assert.assertEquals(3, (int)events.get(EventMessage.EventType.ADD_SCHEMA_VERSION));
+    Assert.assertEquals(3, (int)transactionalEvents.get(EventMessage.EventType.ADD_SCHEMA_VERSION));
+
+    schemaVersion = client.getSchemaLatestVersion(DEFAULT_DATABASE_NAME, schemaName);
+    Assert.assertEquals(3, schemaVersion.getVersion());
+    Assert.assertEquals(3, schemaVersion.getColsSize());
+    List<FieldSchema> cols = schemaVersion.getCols();
+    Collections.sort(cols);
+    Assert.assertEquals("a", cols.get(0).getName());
+    Assert.assertEquals("b", cols.get(1).getName());
+    Assert.assertEquals("c", cols.get(2).getName());
+    Assert.assertEquals(ColumnType.BIGINT_TYPE_NAME, cols.get(0).getType());
+    Assert.assertEquals(ColumnType.DATE_TYPE_NAME, cols.get(1).getType());
+    Assert.assertEquals(ColumnType.TIMESTAMP_TYPE_NAME, cols.get(2).getType());
+    Assert.assertEquals(1, (int)preEvents.get(PreEventContext.PreEventType.READ_SCHEMA_VERSION));
+
+    List<SchemaVersion> versions = client.getSchemaAllVersions(DEFAULT_DATABASE_NAME, schemaName);
+    Assert.assertEquals(2, (int)preEvents.get(PreEventContext.PreEventType.READ_SCHEMA_VERSION));
+    Assert.assertEquals(3, versions.size());
+    versions.sort(Comparator.comparingInt(SchemaVersion::getVersion));
+    Assert.assertEquals(1, versions.get(0).getVersion());
+    Assert.assertEquals(1, versions.get(0).getColsSize());
+    Assert.assertEquals(ColumnType.BIGINT_TYPE_NAME, versions.get(0).getCols().get(0).getType());
+
+    Assert.assertEquals(2, versions.get(1).getVersion());
+    Assert.assertEquals(2, versions.get(1).getColsSize());
+    cols = versions.get(1).getCols();
+    Collections.sort(cols);
+    Assert.assertEquals("a", cols.get(0).getName());
+    Assert.assertEquals("b", cols.get(1).getName());
+    Assert.assertEquals(ColumnType.BIGINT_TYPE_NAME, cols.get(0).getType());
+    Assert.assertEquals(ColumnType.DATE_TYPE_NAME, cols.get(1).getType());
+
+    Assert.assertEquals(3, versions.get(2).getVersion());
+    Assert.assertEquals(3, versions.get(2).getColsSize());
+    cols = versions.get(2).getCols();
+    Collections.sort(cols);
+    Assert.assertEquals("a", cols.get(0).getName());
+    Assert.assertEquals("b", cols.get(1).getName());
+    Assert.assertEquals("c", cols.get(2).getName());
+    Assert.assertEquals(ColumnType.BIGINT_TYPE_NAME, cols.get(0).getType());
+    Assert.assertEquals(ColumnType.DATE_TYPE_NAME, cols.get(1).getType());
+    Assert.assertEquals(ColumnType.TIMESTAMP_TYPE_NAME, cols.get(2).getType());
+  }
+
+  @Test(expected = NoSuchObjectException.class)
+  public void nonExistentSchemaVersion() throws TException {
+    String schemaName = uniqueSchemaName();
+    ISchema schema = new ISchemaBuilder()
+        .setSchemaType(SchemaType.AVRO)
+        .setName(schemaName)
+        .build();
+    client.createISchema(schema);
+    client.getSchemaVersion(DEFAULT_DATABASE_NAME, schemaName, 1);
+  }
+
+  @Test(expected = NoSuchObjectException.class)
+  public void schemaVersionBogusDb() throws TException {
+    String schemaName = uniqueSchemaName();
+    ISchema schema = new ISchemaBuilder()
+        .setSchemaType(SchemaType.AVRO)
+        .setName(schemaName)
+        .build();
+    client.createISchema(schema);
+    client.getSchemaVersion("bogus", schemaName, 1);
+  }
+
+  @Test(expected = NoSuchObjectException.class)
+  public void nonExistentSchemaVersionButOtherVersionsExist() throws TException {
+    String schemaName = uniqueSchemaName();
+
+    ISchema schema = new ISchemaBuilder()
+        .setSchemaType(SchemaType.AVRO)
+        .setName(schemaName)
+        .build();
+    client.createISchema(schema);
+
+    SchemaVersion schemaVersion = new SchemaVersionBuilder()
+        .versionOf(schema)
+        .setVersion(1)
+        .addCol("a", ColumnType.INT_TYPE_NAME)
+        .addCol("b", ColumnType.FLOAT_TYPE_NAME)
+        .build();
+    client.addSchemaVersion(schemaVersion);
+
+    client.getSchemaVersion(DEFAULT_DATABASE_NAME, schemaName, 2);
+  }
+
+  @Test(expected = NoSuchObjectException.class)
+  public void getLatestSchemaButNoVersions() throws TException {
+    String schemaName = uniqueSchemaName();
+    ISchema schema = new ISchemaBuilder()
+        .setSchemaType(SchemaType.AVRO)
+        .setName(schemaName)
+        .build();
+    client.createISchema(schema);
+    client.getSchemaLatestVersion(DEFAULT_DATABASE_NAME, schemaName);
+  }
+
+  @Test(expected = NoSuchObjectException.class)
+  public void getLatestSchemaNoSuchSchema() throws TException {
+    client.getSchemaLatestVersion(DEFAULT_DATABASE_NAME, "no.such.schema.with.this.name");
+  }
+
+  @Test(expected = NoSuchObjectException.class)
+  public void latestSchemaVersionBogusDb() throws TException {
+    String schemaName = uniqueSchemaName();
+    ISchema schema = new ISchemaBuilder()
+        .setSchemaType(SchemaType.AVRO)
+        .setName(schemaName)
+        .build();
+    client.createISchema(schema);
+    client.getSchemaLatestVersion("bogus", schemaName);
+  }
+
+  @Test(expected = NoSuchObjectException.class)
+  public void getAllSchemaButNoVersions() throws TException {
+    String schemaName = uniqueSchemaName();
+    ISchema schema = new ISchemaBuilder()
+        .setSchemaType(SchemaType.AVRO)
+        .setName(schemaName)
+        .build();
+    client.createISchema(schema);
+    client.getSchemaAllVersions(DEFAULT_DATABASE_NAME, schemaName);
+  }
+
+  @Test(expected = NoSuchObjectException.class)
+  public void getAllSchemaNoSuchSchema() throws TException {
+    client.getSchemaAllVersions(DEFAULT_DATABASE_NAME, "no.such.schema.with.this.name");
+  }
+
+  @Test(expected = NoSuchObjectException.class)
+  public void allSchemaVersionBogusDb() throws TException {
+    String schemaName = uniqueSchemaName();
+    ISchema schema = new ISchemaBuilder()
+        .setSchemaType(SchemaType.AVRO)
+        .setName(schemaName)
+        .build();
+    client.createISchema(schema);
+    client.getSchemaAllVersions("bogus", schemaName);
+  }
+
+  @Test(expected = AlreadyExistsException.class)
+  public void addDuplicateSchemaVersion() throws TException {
+    String schemaName = uniqueSchemaName();
+    int version = 1;
+
+    ISchema schema = new ISchemaBuilder()
+        .setSchemaType(SchemaType.AVRO)
+        .setName(schemaName)
+        .build();
+    client.createISchema(schema);
+
+    SchemaVersion schemaVersion = new SchemaVersionBuilder()
+        .versionOf(schema)
+        .setVersion(version)
+        .addCol("a", ColumnType.INT_TYPE_NAME)
+        .addCol("b", ColumnType.FLOAT_TYPE_NAME)
+        .build();
+    client.addSchemaVersion(schemaVersion);
+
+    client.addSchemaVersion(schemaVersion);
+  }
+
+  @Test(expected = NoSuchObjectException.class)
+  public void mapSerDeNoSuchSchema() throws TException {
+    SerDeInfo serDeInfo = new SerDeInfo(uniqueSerdeName(), "lib", Collections.emptyMap());
+    client.mapSchemaVersionToSerde(DEFAULT_DATABASE_NAME, uniqueSchemaName(), 1, serDeInfo.getName());
+  }
+
+  @Test(expected = NoSuchObjectException.class)
+  public void mapSerDeNoSuchSchemaVersion() throws TException {
+    SerDeInfo serDeInfo = new SerDeInfo(uniqueSerdeName(), "lib", Collections.emptyMap());
+    ISchema schema = new ISchemaBuilder()
+        .setSchemaType(SchemaType.AVRO)
+        .setName(uniqueSchemaName())
+        .build();
+    client.createISchema(schema);
+    client.mapSchemaVersionToSerde(DEFAULT_DATABASE_NAME, schema.getName(), 3, serDeInfo.getName());
+  }
+
+  @Test(expected = NoSuchObjectException.class)
+  public void mapNonExistentSerdeToSchemaVersion() throws TException {
+    ISchema schema = new ISchemaBuilder()
+        .setSchemaType(SchemaType.AVRO)
+        .setName(uniqueSchemaName())
+        .build();
+    client.createISchema(schema);
+
+    SchemaVersion schemaVersion = new SchemaVersionBuilder()
+        .versionOf(schema)
+        .setVersion(1)
+        .addCol("x", ColumnType.BOOLEAN_TYPE_NAME)
+        .build();
+    client.addSchemaVersion(schemaVersion);
+    client.mapSchemaVersionToSerde(DEFAULT_DATABASE_NAME, schema.getName(), schemaVersion.getVersion(), uniqueSerdeName());
+  }
+
+  @Test
+  public void mapSerdeToSchemaVersion() throws TException {
+    ISchema schema = new ISchemaBuilder()
+        .setSchemaType(SchemaType.AVRO)
+        .setName(uniqueSchemaName())
+        .build();
+    client.createISchema(schema);
+
+    // Create schema with no serde, then map it
+    SchemaVersion schemaVersion = new SchemaVersionBuilder()
+        .versionOf(schema)
+        .setVersion(1)
+        .addCol("x", ColumnType.BOOLEAN_TYPE_NAME)
+        .build();
+    client.addSchemaVersion(schemaVersion);
+
+    SerDeInfo serDeInfo = new SerDeInfo(uniqueSerdeName(), "lib", Collections.emptyMap());
+    client.addSerDe(serDeInfo);
+
+    client.mapSchemaVersionToSerde(DEFAULT_DATABASE_NAME, schema.getName(), schemaVersion.getVersion(), serDeInfo.getName());
+    schemaVersion = client.getSchemaVersion(DEFAULT_DATABASE_NAME, schema.getName(), schemaVersion.getVersion());
+    Assert.assertEquals(serDeInfo.getName(), schemaVersion.getSerDe().getName());
+
+    // Create schema with a serde, then remap it
+    String serDeName = uniqueSerdeName();
+    schemaVersion = new SchemaVersionBuilder()
+        .versionOf(schema)
+        .setVersion(2)
+        .addCol("x", ColumnType.BOOLEAN_TYPE_NAME)
+        .setSerdeName(serDeName)
+        .setSerdeLib("x")
+        .build();
+    client.addSchemaVersion(schemaVersion);
+
+    schemaVersion = client.getSchemaVersion(DEFAULT_DATABASE_NAME, schema.getName(), 2);
+    Assert.assertEquals(serDeName, schemaVersion.getSerDe().getName());
+
+    serDeInfo = new SerDeInfo(uniqueSerdeName(), "y", Collections.emptyMap());
+    client.addSerDe(serDeInfo);
+    client.mapSchemaVersionToSerde(DEFAULT_DATABASE_NAME, schema.getName(), 2, serDeInfo.getName());
+    schemaVersion = client.getSchemaVersion(DEFAULT_DATABASE_NAME, schema.getName(), 2);
+    Assert.assertEquals(serDeInfo.getName(), schemaVersion.getSerDe().getName());
+  }
+
+  @Test
+  public void mapSerdeToSchemaVersionOtherDb() throws TException {
+    String dbName = "map_other_db";
+    Database db = new DatabaseBuilder()
+        .setName(dbName)
+        .build();
+    client.createDatabase(db);
+
+    ISchema schema = new ISchemaBuilder()
+        .setSchemaType(SchemaType.AVRO)
+        .setDbName(dbName)
+        .setName(uniqueSchemaName())
+        .build();
+    client.createISchema(schema);
+
+    // Create schema with no serde, then map it
+    SchemaVersion schemaVersion = new SchemaVersionBuilder()
+        .versionOf(schema)
+        .setVersion(1)
+        .addCol("x", ColumnType.BOOLEAN_TYPE_NAME)
+        .build();
+    client.addSchemaVersion(schemaVersion);
+
+    SerDeInfo serDeInfo = new SerDeInfo(uniqueSerdeName(), "lib", Collections.emptyMap());
+    client.addSerDe(serDeInfo);
+
+    client.mapSchemaVersionToSerde(dbName, schema.getName(), schemaVersion.getVersion(), serDeInfo.getName());
+    schemaVersion = client.getSchemaVersion(dbName, schema.getName(), schemaVersion.getVersion());
+    Assert.assertEquals(serDeInfo.getName(), schemaVersion.getSerDe().getName());
+
+    // Create schema with a serde, then remap it
+    String serDeName = uniqueSerdeName();
+    schemaVersion = new SchemaVersionBuilder()
+        .versionOf(schema)
+        .setVersion(2)
+        .addCol("x", ColumnType.BOOLEAN_TYPE_NAME)
+        .setSerdeName(serDeName)
+        .setSerdeLib("x")
+        .build();
+    client.addSchemaVersion(schemaVersion);
+
+    schemaVersion = client.getSchemaVersion(dbName, schema.getName(), 2);
+    Assert.assertEquals(serDeName, schemaVersion.getSerDe().getName());
+
+    serDeInfo = new SerDeInfo(uniqueSerdeName(), "y", Collections.emptyMap());
+    client.addSerDe(serDeInfo);
+    client.mapSchemaVersionToSerde(dbName, schema.getName(), 2, serDeInfo.getName());
+    schemaVersion = client.getSchemaVersion(dbName, schema.getName(), 2);
+    Assert.assertEquals(serDeInfo.getName(), schemaVersion.getSerDe().getName());
+
+  }
+
+  @Test
+  public void addSerde() throws TException {
+    String serdeName = uniqueSerdeName();
+    SerDeInfo serDeInfo = new SerDeInfo(serdeName, "serdeLib", Collections.singletonMap("a", "b"));
+    serDeInfo.setSerializerClass("serializer");
+    serDeInfo.setDeserializerClass("deserializer");
+    serDeInfo.setDescription("description");
+    serDeInfo.setSerdeType(SerdeType.SCHEMA_REGISTRY);
+    client.addSerDe(serDeInfo);
+
+    serDeInfo = client.getSerDe(serdeName);
+    Assert.assertEquals(serdeName, serDeInfo.getName());
+    Assert.assertEquals("serdeLib", serDeInfo.getSerializationLib());
+    Assert.assertEquals(1, serDeInfo.getParametersSize());
+    Assert.assertEquals("b", serDeInfo.getParameters().get("a"));
+    Assert.assertEquals("serializer", serDeInfo.getSerializerClass());
+    Assert.assertEquals("deserializer", serDeInfo.getDeserializerClass());
+    Assert.assertEquals("description", serDeInfo.getDescription());
+    Assert.assertEquals(SerdeType.SCHEMA_REGISTRY, serDeInfo.getSerdeType());
+  }
+
+  @Test(expected = AlreadyExistsException.class)
+  public void duplicateSerde() throws TException {
+    String serdeName = uniqueSerdeName();
+    SerDeInfo serDeInfo = new SerDeInfo(serdeName, "x", Collections.emptyMap());
+    client.addSerDe(serDeInfo);
+    client.addSerDe(serDeInfo);
+  }
+
+  @Test(expected = NoSuchObjectException.class)
+  public void noSuchSerDe() throws TException {
+    client.getSerDe(uniqueSerdeName());
+  }
+
+  @Test(expected = NoSuchObjectException.class)
+  public void setVersionStateNoSuchSchema() throws TException {
+    client.setSchemaVersionState(DEFAULT_DATABASE_NAME, "no.such.schema", 1, SchemaVersionState.INITIATED);
+  }
+
+  @Test(expected = NoSuchObjectException.class)
+  public void setVersionStateNoSuchVersion() throws TException {
+    String schemaName = uniqueSchemaName();
+    ISchema schema = new ISchemaBuilder()
+        .setSchemaType(SchemaType.AVRO)
+        .setName(schemaName)
+        .build();
+    client.createISchema(schema);
+    client.setSchemaVersionState(DEFAULT_DATABASE_NAME, schemaName, 1, SchemaVersionState.INITIATED);
+  }
+
+  @Test
+  public void setVersionState() throws TException {
+    String schemaName = uniqueSchemaName();
+    ISchema schema = new ISchemaBuilder()
+        .setSchemaType(SchemaType.AVRO)
+        .setName(schemaName)
+        .build();
+    client.createISchema(schema);
+
+    SchemaVersion schemaVersion = new SchemaVersionBuilder()
+        .versionOf(schema)
+        .setVersion(1)
+        .addCol("a", ColumnType.BINARY_TYPE_NAME)
+        .build();
+    client.addSchemaVersion(schemaVersion);
+
+    schemaVersion = client.getSchemaVersion(DEFAULT_DATABASE_NAME, schemaName, 1);
+    Assert.assertNull(schemaVersion.getState());
+
+    client.setSchemaVersionState(DEFAULT_DATABASE_NAME, schemaName, 1, SchemaVersionState.INITIATED);
+    Assert.assertEquals(1, (int)preEvents.get(PreEventContext.PreEventType.ALTER_SCHEMA_VERSION));
+    Assert.assertEquals(1, (int)events.get(EventMessage.EventType.ALTER_SCHEMA_VERSION));
+    Assert.assertEquals(1, (int)transactionalEvents.get(EventMessage.EventType.ALTER_SCHEMA_VERSION));
+    schemaVersion = client.getSchemaVersion(DEFAULT_DATABASE_NAME, schemaName, 1);
+    Assert.assertEquals(SchemaVersionState.INITIATED, schemaVersion.getState());
+
+    client.setSchemaVersionState(DEFAULT_DATABASE_NAME, schemaName, 1, SchemaVersionState.REVIEWED);
+    Assert.assertEquals(2, (int)preEvents.get(PreEventContext.PreEventType.ALTER_SCHEMA_VERSION));
+    Assert.assertEquals(2, (int)events.get(EventMessage.EventType.ALTER_SCHEMA_VERSION));
+    Assert.assertEquals(2, (int)transactionalEvents.get(EventMessage.EventType.ALTER_SCHEMA_VERSION));
+    schemaVersion = client.getSchemaVersion(DEFAULT_DATABASE_NAME, schemaName, 1);
+    Assert.assertEquals(SchemaVersionState.REVIEWED, schemaVersion.getState());
+  }
+
+  @Test
+  public void setVersionStateOtherDb() throws TException {
+    String dbName = "other_db_set_state";
+    Database db = new DatabaseBuilder()
+        .setName(dbName)
+        .build();
+    client.createDatabase(db);
+
+    String schemaName = uniqueSchemaName();
+    ISchema schema = new ISchemaBuilder()
+        .setSchemaType(SchemaType.AVRO)
+        .setName(schemaName)
+        .setDbName(dbName)
+        .build();
+    client.createISchema(schema);
+
+    SchemaVersion schemaVersion = new SchemaVersionBuilder()
+        .versionOf(schema)
+        .setVersion(1)
+        .addCol("a", ColumnType.BINARY_TYPE_NAME)
+        .build();
+    client.addSchemaVersion(schemaVersion);
+
+    schemaVersion = client.getSchemaVersion(dbName, schemaName, 1);
+    Assert.assertNull(schemaVersion.getState());
+
+    client.setSchemaVersionState(dbName, schemaName, 1, SchemaVersionState.INITIATED);
+    Assert.assertEquals(1, (int)preEvents.get(PreEventContext.PreEventType.ALTER_SCHEMA_VERSION));
+    Assert.assertEquals(1, (int)events.get(EventMessage.EventType.ALTER_SCHEMA_VERSION));
+    Assert.assertEquals(1, (int)transactionalEvents.get(EventMessage.EventType.ALTER_SCHEMA_VERSION));
+    schemaVersion = client.getSchemaVersion(dbName, schemaName, 1);
+    Assert.assertEquals(SchemaVersionState.INITIATED, schemaVersion.getState());
+
+    client.setSchemaVersionState(dbName, schemaName, 1, SchemaVersionState.REVIEWED);
+    Assert.assertEquals(2, (int)preEvents.get(PreEventContext.PreEventType.ALTER_SCHEMA_VERSION));
+    Assert.assertEquals(2, (int)events.get(EventMessage.EventType.ALTER_SCHEMA_VERSION));
+    Assert.assertEquals(2, (int)transactionalEvents.get(EventMessage.EventType.ALTER_SCHEMA_VERSION));
+    schemaVersion = client.getSchemaVersion(dbName, schemaName, 1);
+    Assert.assertEquals(SchemaVersionState.REVIEWED, schemaVersion.getState());
+  }
+
+  @Test(expected = NoSuchObjectException.class)
+  public void dropNonExistentSchemaVersion() throws TException {
+    client.dropSchemaVersion(DEFAULT_DATABASE_NAME, "ther is no schema named this", 23);
+  }
+
+  @Test
+  public void schemaQuery() throws TException {
+    String dbName = "schema_query_db";
+    Database db = new DatabaseBuilder()
+        .setName(dbName)
+        .build();
+    client.createDatabase(db);
+
+    String schemaName1 = uniqueSchemaName();
+    ISchema schema1 = new ISchemaBuilder()
+        .setSchemaType(SchemaType.AVRO)
+        .setDbName(dbName)
+        .setName(schemaName1)
+        .build();
+    client.createISchema(schema1);
+
+    String schemaName2 = uniqueSchemaName();
+    ISchema schema2 = new ISchemaBuilder()
+        .setSchemaType(SchemaType.AVRO)
+        .setDbName(dbName)
+        .setName(schemaName2)
+        .build();
+    client.createISchema(schema2);
+
+    SchemaVersion schemaVersion1_1 = new SchemaVersionBuilder()
+        .versionOf(schema1)
+        .setVersion(1)
+        .addCol("alpha", ColumnType.BIGINT_TYPE_NAME)
+        .addCol("beta", ColumnType.DATE_TYPE_NAME)
+        .build();
+    client.addSchemaVersion(schemaVersion1_1);
+
+    SchemaVersion schemaVersion1_2 = new SchemaVersionBuilder()
+        .versionOf(schema1)
+        .setVersion(2)
+        .addCol("alpha", ColumnType.BIGINT_TYPE_NAME)
+        .addCol("beta", ColumnType.DATE_TYPE_NAME)
+        .addCol("gamma", ColumnType.BIGINT_TYPE_NAME, "namespace=x")
+        .build();
+    client.addSchemaVersion(schemaVersion1_2);
+
+    SchemaVersion schemaVersion2_1 = new SchemaVersionBuilder()
+        .versionOf(schema2)
+        .setVersion(1)
+        .addCol("ALPHA", ColumnType.SMALLINT_TYPE_NAME)
+        .addCol("delta", ColumnType.DOUBLE_TYPE_NAME)
+        .build();
+    client.addSchemaVersion(schemaVersion2_1);
+
+    SchemaVersion schemaVersion2_2 = new SchemaVersionBuilder()
+        .versionOf(schema2)
+        .setVersion(2)
+        .addCol("ALPHA", ColumnType.SMALLINT_TYPE_NAME)
+        .addCol("delta", ColumnType.DOUBLE_TYPE_NAME)
+        .addCol("epsilon", ColumnType.STRING_TYPE_NAME, "namespace=x")
+        .build();
+    client.addSchemaVersion(schemaVersion2_2);
+
+    // Query that should return nothing
+    FindSchemasByColsRqst rqst = new FindSchemasByColsRqst();
+    rqst.setColName("x");
+    rqst.setColNamespace("y");
+    rqst.setType("z");
+    FindSchemasByColsResp rsp = client.getSchemaByCols(rqst);
+    Assert.assertEquals(0, rsp.getSchemaVersionsSize());
+
+    // Query that should fetch one column
+    rqst = new FindSchemasByColsRqst();
+    rqst.setColName("gamma");
+    rsp = client.getSchemaByCols(rqst);
+    Assert.assertEquals(1, rsp.getSchemaVersionsSize());
+    Assert.assertEquals(schemaName1, rsp.getSchemaVersions().get(0).getSchema().getSchemaName());
+    Assert.assertEquals(dbName, rsp.getSchemaVersions().get(0).getSchema().getDbName());
+    Assert.assertEquals(2, rsp.getSchemaVersions().get(0).getVersion());
+
+    // fetch 2 in same schema
+    rqst = new FindSchemasByColsRqst();
+    rqst.setColName("beta");
+    rsp = client.getSchemaByCols(rqst);
+    Assert.assertEquals(2, rsp.getSchemaVersionsSize());
+    List<SchemaVersionDescriptor> results = new ArrayList<>(rsp.getSchemaVersions());
+    Collections.sort(results);
+    Assert.assertEquals(schemaName1, results.get(0).getSchema().getSchemaName());
+    Assert.assertEquals(dbName, results.get(0).getSchema().getDbName());
+    Assert.assertEquals(1, results.get(0).getVersion());
+    Assert.assertEquals(schemaName1, results.get(1).getSchema().getSchemaName());
+    Assert.assertEquals(dbName, results.get(1).getSchema().getDbName());
+    Assert.assertEquals(2, results.get(1).getVersion());
+
+    // fetch across schemas
+    rqst = new FindSchemasByColsRqst();
+    rqst.setColName("alpha");
+    rsp = client.getSchemaByCols(rqst);
+    Assert.assertEquals(4, rsp.getSchemaVersionsSize());
+    results = new ArrayList<>(rsp.getSchemaVersions());
+    Collections.sort(results);
+    Assert.assertEquals(schemaName1, results.get(0).getSchema().getSchemaName());
+    Assert.assertEquals(dbName, results.get(0).getSchema().getDbName());
+    Assert.assertEquals(1, results.get(0).getVersion());
+    Assert.assertEquals(schemaName1, results.get(1).getSchema().getSchemaName());
+    Assert.assertEquals(dbName, results.get(1).getSchema().getDbName());
+    Assert.assertEquals(2, results.get(1).getVersion());
+    Assert.assertEquals(schemaName2, results.get(2).getSchema().getSchemaName());
+    Assert.assertEquals(dbName, results.get(2).getSchema().getDbName());
+    Assert.assertEquals(1, results.get(2).getVersion());
+    Assert.assertEquals(schemaName2, results.get(3).getSchema().getSchemaName());
+    Assert.assertEquals(dbName, results.get(3).getSchema().getDbName());
+    Assert.assertEquals(2, results.get(3).getVersion());
+
+    // fetch by namespace
+    rqst = new FindSchemasByColsRqst();
+    rqst.setColNamespace("namespace=x");
+    rsp = client.getSchemaByCols(rqst);
+    Assert.assertEquals(2, rsp.getSchemaVersionsSize());
+    results = new ArrayList<>(rsp.getSchemaVersions());
+    Collections.sort(results);
+    Assert.assertEquals(schemaName1, results.get(0).getSchema().getSchemaName());
+    Assert.assertEquals(dbName, results.get(0).getSchema().getDbName());
+    Assert.assertEquals(2, results.get(0).getVersion());
+    Assert.assertEquals(schemaName2, results.get(1).getSchema().getSchemaName());
+    Assert.assertEquals(dbName, results.get(1).getSchema().getDbName());
+    Assert.assertEquals(2, results.get(1).getVersion());
+
+    // fetch by name and type
+    rqst = new FindSchemasByColsRqst();
+    rqst.setColName("alpha");
+    rqst.setType(ColumnType.SMALLINT_TYPE_NAME);
+    rsp = client.getSchemaByCols(rqst);
+    Assert.assertEquals(2, rsp.getSchemaVersionsSize());
+    results = new ArrayList<>(rsp.getSchemaVersions());
+    Collections.sort(results);
+    Assert.assertEquals(schemaName2, results.get(0).getSchema().getSchemaName());
+    Assert.assertEquals(dbName, results.get(0).getSchema().getDbName());
+    Assert.assertEquals(1, results.get(0).getVersion());
+    Assert.assertEquals(schemaName2, results.get(1).getSchema().getSchemaName());
+    Assert.assertEquals(dbName, results.get(1).getSchema().getDbName());
+    Assert.assertEquals(2, results.get(1).getVersion());
+
+    // Make sure matching name but wrong type doesn't return
+    rqst = new FindSchemasByColsRqst();
+    rqst.setColName("alpha");
+    rqst.setType(ColumnType.STRING_TYPE_NAME);
+    rsp = client.getSchemaByCols(rqst);
+    Assert.assertEquals(0, rsp.getSchemaVersionsSize());
+  }
+
+  @Test(expected = MetaException.class)
+  public void schemaVersionQueryNoNameOrNamespace() throws TException {
+    FindSchemasByColsRqst rqst = new FindSchemasByColsRqst();
+    rqst.setType(ColumnType.STRING_TYPE_NAME);
+    client.getSchemaByCols(rqst);
+  }
+
+  private static int nextSchemaNum = 1;
+
+  private String uniqueSchemaName() {
+    return "uniqueschema" + nextSchemaNum++;
+
+  }
+
+  private String uniqueSerdeName() {
+    return "uniqueSerde" + nextSchemaNum++;
+  }
+
+  public static class SchemaEventListener extends MetaStoreEventListener {
+
+    public SchemaEventListener(Configuration config) {
+      super(config);
+    }
+
+    @Override
+    public void onCreateISchema(CreateISchemaEvent createISchemaEvent) throws MetaException {
+      Integer cnt = events.get(EventMessage.EventType.CREATE_ISCHEMA);
+      events.put(EventMessage.EventType.CREATE_ISCHEMA, cnt == null ? 1 : cnt + 1);
+    }
+
+    @Override
+    public void onAlterISchema(AlterISchemaEvent alterISchemaEvent) throws MetaException {
+      Integer cnt = events.get(EventMessage.EventType.ALTER_ISCHEMA);
+      events.put(EventMessage.EventType.ALTER_ISCHEMA, cnt == null ? 1 : cnt + 1);
+    }
+
+    @Override
+    public void onDropISchema(DropISchemaEvent dropISchemaEvent) throws MetaException {
+      Integer cnt = events.get(EventMessage.EventType.DROP_ISCHEMA);
+      events.put(EventMessage.EventType.DROP_ISCHEMA, cnt == null ? 1 : cnt + 1);
+    }
+
+    @Override
+    public void onAddSchemaVersion(AddSchemaVersionEvent addSchemaVersionEvent) throws
+        MetaException {
+      Integer cnt = events.get(EventMessage.EventType.ADD_SCHEMA_VERSION);
+      events.put(EventMessage.EventType.ADD_SCHEMA_VERSION, cnt == null ? 1 : cnt + 1);
+    }
+
+    @Override
+    public void onAlterSchemaVersion(AlterSchemaVersionEvent alterSchemaVersionEvent) throws
+        MetaException {
+      Integer cnt = events.get(EventMessage.EventType.ALTER_SCHEMA_VERSION);
+      events.put(EventMessage.EventType.ALTER_SCHEMA_VERSION, cnt == null ? 1 : cnt + 1);
+    }
+
+    @Override
+    public void onDropSchemaVersion(DropSchemaVersionEvent dropSchemaVersionEvent) throws
+        MetaException {
+      Integer cnt = events.get(EventMessage.EventType.DROP_SCHEMA_VERSION);
+      events.put(EventMessage.EventType.DROP_SCHEMA_VERSION, cnt == null ? 1 : cnt + 1);
+    }
+  }
+
+  public static class TransactionalSchemaEventListener extends MetaStoreEventListener {
+
+    public TransactionalSchemaEventListener(Configuration config) {
+      super(config);
+    }
+
+    @Override
+    public void onCreateISchema(CreateISchemaEvent createISchemaEvent) throws MetaException {
+      Integer cnt = transactionalEvents.get(EventMessage.EventType.CREATE_ISCHEMA);
+      transactionalEvents.put(EventMessage.EventType.CREATE_ISCHEMA, cnt == null ? 1 : cnt + 1);
+    }
+
+    @Override
+    public void onAlterISchema(AlterISchemaEvent alterISchemaEvent) throws MetaException {
+      Integer cnt = transactionalEvents.get(EventMessage.EventType.ALTER_ISCHEMA);
+      transactionalEvents.put(EventMessage.EventType.ALTER_ISCHEMA, cnt == null ? 1 : cnt + 1);
+    }
+
+    @Override
+    public void onDropISchema(DropISchemaEvent dropISchemaEvent) throws MetaException {
+      Integer cnt = transactionalEvents.get(EventMessage.EventType.DROP_ISCHEMA);
+      transactionalEvents.put(EventMessage.EventType.DROP_ISCHEMA, cnt == null ? 1 : cnt + 1);
+    }
+
+    @Override
+    public void onAddSchemaVersion(AddSchemaVersionEvent addSchemaVersionEvent) throws
+        MetaException {
+      Integer cnt = transactionalEvents.get(EventMessage.EventType.ADD_SCHEMA_VERSION);
+      transactionalEvents.put(EventMessage.EventType.ADD_SCHEMA_VERSION, cnt == null ? 1 : cnt + 1);
+    }
+
+    @Override
+    public void onAlterSchemaVersion(AlterSchemaVersionEvent alterSchemaVersionEvent) throws
+        MetaException {
+      Integer cnt = transactionalEvents.get(EventMessage.EventType.ALTER_SCHEMA_VERSION);
+      transactionalEvents.put(EventMessage.EventType.ALTER_SCHEMA_VERSION, cnt == null ? 1 : cnt + 1);
+    }
+
+    @Override
+    public void onDropSchemaVersion(DropSchemaVersionEvent dropSchemaVersionEvent) throws
+        MetaException {
+      Integer cnt = transactionalEvents.get(EventMessage.EventType.DROP_SCHEMA_VERSION);
+      transactionalEvents.put(EventMessage.EventType.DROP_SCHEMA_VERSION, cnt == null ? 1 : cnt + 1);
+    }
+  }
+
+  public static class SchemaPreEventListener extends MetaStorePreEventListener {
+
+    public SchemaPreEventListener(Configuration config) {
+      super(config);
+    }
+
+    @Override
+    public void onEvent(PreEventContext context) throws MetaException, NoSuchObjectException,
+        InvalidOperationException {
+      Integer cnt = preEvents.get(context.getEventType());
+      preEvents.put(context.getEventType(), cnt == null ? 1 : cnt + 1);
+
+    }
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStore.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStore.java
index b7f7be92..ca33b7d 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStore.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStore.java
@@ -27,7 +27,6 @@ import org.apache.hadoop.hive.metastore.annotation.MetastoreUnitTest;
 import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId;
 import org.apache.hadoop.hive.metastore.api.Database;
 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.InvalidInputException;
 import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
@@ -50,7 +49,6 @@ import org.apache.hadoop.hive.metastore.metrics.Metrics;
 import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
 import org.apache.hadoop.hive.metastore.model.MNotificationLog;
 import org.apache.hadoop.hive.metastore.model.MNotificationNextId;
-import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
 import org.junit.Assert;
 import org.junit.Assume;
 import org.junit.Before;
@@ -98,36 +96,6 @@ public class TestObjectStore {
     }
   }
 
-  public static class MockPartitionExpressionProxy implements PartitionExpressionProxy {
-    @Override
-    public String convertExprToFilter(byte[] expr) throws MetaException {
-      return null;
-    }
-
-    @Override
-    public boolean filterPartitionsByExpr(List<FieldSchema> partColumns,
-                                          byte[] expr, String defaultPartitionName,
-                                          List<String> partitionNames)
-        throws MetaException {
-      return false;
-    }
-
-    @Override
-    public FileMetadataExprType getMetadataType(String inputFormat) {
-      return null;
-    }
-
-    @Override
-    public SearchArgument createSarg(byte[] expr) {
-      return null;
-    }
-
-    @Override
-    public FileFormatProxy getFileFormatProxy(FileMetadataExprType type) {
-      return null;
-    }
-  }
-
   @Before
   public void setUp() throws Exception {
     Configuration conf = MetastoreConf.newMetastoreConf();


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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
index c99c3af..db09407 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@ -53,6 +53,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.locks.Condition;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
+import java.util.function.Consumer;
 import java.util.regex.Pattern;
 
 import javax.jdo.JDOException;
@@ -79,33 +80,47 @@ import org.apache.hadoop.hive.metastore.events.AddPartitionEvent;
 import org.apache.hadoop.hive.metastore.events.AddPrimaryKeyEvent;
 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.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.AddSchemaVersionEvent;
 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.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;
 import org.apache.hadoop.hive.metastore.events.PreAddPartitionEvent;
 import org.apache.hadoop.hive.metastore.events.PreAlterDatabaseEvent;
+import org.apache.hadoop.hive.metastore.events.PreAlterISchemaEvent;
 import org.apache.hadoop.hive.metastore.events.PreAlterPartitionEvent;
+import org.apache.hadoop.hive.metastore.events.PreAlterSchemaVersionEvent;
 import org.apache.hadoop.hive.metastore.events.PreAlterTableEvent;
 import org.apache.hadoop.hive.metastore.events.PreAuthorizationCallEvent;
 import org.apache.hadoop.hive.metastore.events.PreCreateDatabaseEvent;
+import org.apache.hadoop.hive.metastore.events.PreCreateISchemaEvent;
+import org.apache.hadoop.hive.metastore.events.PreAddSchemaVersionEvent;
 import org.apache.hadoop.hive.metastore.events.PreCreateTableEvent;
 import org.apache.hadoop.hive.metastore.events.PreDropDatabaseEvent;
+import org.apache.hadoop.hive.metastore.events.PreDropISchemaEvent;
 import org.apache.hadoop.hive.metastore.events.PreDropPartitionEvent;
+import org.apache.hadoop.hive.metastore.events.PreDropSchemaVersionEvent;
 import org.apache.hadoop.hive.metastore.events.PreDropTableEvent;
 import org.apache.hadoop.hive.metastore.events.PreEventContext;
 import org.apache.hadoop.hive.metastore.events.PreLoadPartitionDoneEvent;
 import org.apache.hadoop.hive.metastore.events.PreReadDatabaseEvent;
+import org.apache.hadoop.hive.metastore.events.PreReadISchemaEvent;
 import org.apache.hadoop.hive.metastore.events.PreReadTableEvent;
+import org.apache.hadoop.hive.metastore.events.PreReadhSchemaVersionEvent;
 import org.apache.hadoop.hive.metastore.messaging.EventMessage.EventType;
 import org.apache.hadoop.hive.metastore.metrics.JvmPauseMonitor;
 import org.apache.hadoop.hive.metastore.metrics.Metrics;
@@ -7106,7 +7121,6 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       }
     }
 
-
     @Override
     public WMCreateResourcePlanResponse create_resource_plan(WMCreateResourcePlanRequest request)
         throws AlreadyExistsException, InvalidObjectException, MetaException, TException {
@@ -7254,7 +7268,8 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         WMGetTriggersForResourePlanRequest request)
         throws NoSuchObjectException, MetaException, TException {
       try {
-        List<WMTrigger> triggers = getMS().getTriggersForResourcePlan(request.getResourcePlanName());
+        List<WMTrigger> triggers =
+            getMS().getTriggersForResourcePlan(request.getResourcePlanName());
         WMGetTriggersForResourePlanResponse response = new WMGetTriggersForResourePlanResponse();
         response.setTriggers(triggers);
         return response;
@@ -7264,33 +7279,31 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       }
     }
 
-    @Override
-    public WMCreatePoolResponse create_wm_pool(WMCreatePoolRequest request)
+    public WMAlterPoolResponse alter_wm_pool(WMAlterPoolRequest request)
         throws AlreadyExistsException, NoSuchObjectException, InvalidObjectException, MetaException,
         TException {
       try {
-        getMS().createPool(request.getPool());
-        return new WMCreatePoolResponse();
+        getMS().alterPool(request.getPool(), request.getPoolPath());
+        return new WMAlterPoolResponse();
       } catch (MetaException e) {
-        LOG.error("Exception while trying to create WMPool", e);
+        LOG.error("Exception while trying to alter WMPool", e);
         throw e;
       }
     }
 
     @Override
-    public WMAlterPoolResponse alter_wm_pool(WMAlterPoolRequest request)
+    public WMCreatePoolResponse create_wm_pool(WMCreatePoolRequest request)
         throws AlreadyExistsException, NoSuchObjectException, InvalidObjectException, MetaException,
         TException {
       try {
-        getMS().alterPool(request.getPool(), request.getPoolPath());
-        return new WMAlterPoolResponse();
+        getMS().createPool(request.getPool());
+        return new WMCreatePoolResponse();
       } catch (MetaException e) {
-        LOG.error("Exception while trying to alter WMPool", e);
+        LOG.error("Exception while trying to create WMPool", e);
         throw e;
       }
     }
 
-    @Override
     public WMDropPoolResponse drop_wm_pool(WMDropPoolRequest request)
         throws NoSuchObjectException, InvalidOperationException, MetaException, TException {
       try {
@@ -7302,7 +7315,6 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       }
     }
 
-    @Override
     public WMCreateOrUpdateMappingResponse create_or_update_wm_mapping(
         WMCreateOrUpdateMappingRequest request) throws AlreadyExistsException,
         NoSuchObjectException, InvalidObjectException, MetaException, TException {
@@ -7315,7 +7327,6 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       }
     }
 
-    @Override
     public WMDropMappingResponse drop_wm_mapping(WMDropMappingRequest request)
         throws NoSuchObjectException, InvalidOperationException, MetaException, TException {
       try {
@@ -7327,7 +7338,6 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       }
     }
 
-    @Override
     public WMCreateOrDropTriggerToPoolMappingResponse create_or_drop_wm_trigger_to_pool_mapping(
         WMCreateOrDropTriggerToPoolMappingRequest request) throws AlreadyExistsException,
         NoSuchObjectException, InvalidObjectException, MetaException, TException {
@@ -7345,6 +7355,438 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         throw e;
       }
     }
+
+    public void create_ischema(ISchema schema) throws TException {
+      startFunction("create_ischema", ": " + schema.getName());
+      boolean success = false;
+      Exception ex = null;
+      RawStore ms = getMS();
+      try {
+        firePreEvent(new PreCreateISchemaEvent(this, schema));
+        Map<String, String> transactionalListenersResponses = Collections.emptyMap();
+        ms.openTransaction();
+        try {
+          ms.createISchema(schema);
+
+          if (!transactionalListeners.isEmpty()) {
+            transactionalListenersResponses =
+                MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
+                    EventType.CREATE_ISCHEMA, new CreateISchemaEvent(true, this, schema));
+          }
+          success = ms.commitTransaction();
+        } finally {
+          if (!success) ms.rollbackTransaction();
+          if (!listeners.isEmpty()) {
+            MetaStoreListenerNotifier.notifyEvent(listeners, EventType.CREATE_ISCHEMA,
+                new CreateISchemaEvent(success, this, schema), null,
+                transactionalListenersResponses, ms);
+          }
+        }
+      } catch (MetaException|AlreadyExistsException e) {
+        LOG.error("Caught exception creating schema", e);
+        ex = e;
+        throw e;
+      } finally {
+        endFunction("create_ischema", success, ex);
+      }
+    }
+
+    @Override
+    public void alter_ischema(AlterISchemaRequest rqst) throws TException {
+      startFunction("alter_ischema", ": " + rqst);
+      boolean success = false;
+      Exception ex = null;
+      RawStore ms = getMS();
+      try {
+        ISchema oldSchema = ms.getISchema(rqst.getName());
+        if (oldSchema == null) {
+          throw new NoSuchObjectException("Could not find schema " + rqst.getName());
+        }
+        firePreEvent(new PreAlterISchemaEvent(this, oldSchema, rqst.getNewSchema()));
+        Map<String, String> transactionalListenersResponses = Collections.emptyMap();
+        ms.openTransaction();
+        try {
+          ms.alterISchema(rqst.getName(), rqst.getNewSchema());
+          if (!transactionalListeners.isEmpty()) {
+            transactionalListenersResponses =
+                MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
+                    EventType.ALTER_ISCHEMA, new AlterISchemaEvent(true, this, oldSchema, rqst.getNewSchema()));
+          }
+          success = ms.commitTransaction();
+        } finally {
+          if (!success) ms.rollbackTransaction();
+          if (!listeners.isEmpty()) {
+            MetaStoreListenerNotifier.notifyEvent(listeners, EventType.ALTER_ISCHEMA,
+                new AlterISchemaEvent(success, this, oldSchema, rqst.getNewSchema()), null,
+                transactionalListenersResponses, ms);
+          }
+        }
+      } catch (MetaException|NoSuchObjectException e) {
+        LOG.error("Caught exception altering schema", e);
+        ex = e;
+        throw e;
+      } finally {
+        endFunction("alter_ischema", success, ex);
+      }
+    }
+
+    @Override
+    public ISchema get_ischema(ISchemaName schemaName) throws TException {
+      startFunction("get_ischema", ": " + schemaName);
+      Exception ex = null;
+      ISchema schema = null;
+      try {
+        schema = getMS().getISchema(schemaName);
+        if (schema == null) {
+          throw new NoSuchObjectException("No schema named " + schemaName + " exists");
+        }
+        firePreEvent(new PreReadISchemaEvent(this, schema));
+        return schema;
+      } catch (MetaException e) {
+        LOG.error("Caught exception getting schema", e);
+        ex = e;
+        throw e;
+      } finally {
+        endFunction("get_ischema", schema != null, ex);
+      }
+    }
+
+    @Override
+    public void drop_ischema(ISchemaName schemaName) throws TException {
+      startFunction("drop_ischema", ": " + schemaName);
+      Exception ex = null;
+      boolean success = false;
+      RawStore ms = getMS();
+      try {
+        // look for any valid versions.  This will also throw NoSuchObjectException if the schema
+        // itself doesn't exist, which is what we want.
+        SchemaVersion latest = ms.getLatestSchemaVersion(schemaName);
+        if (latest != null) {
+          ex = new InvalidOperationException("Schema " + schemaName + " cannot be dropped, it has" +
+              " at least one valid version");
+          throw (InvalidObjectException)ex;
+        }
+        ISchema schema = ms.getISchema(schemaName);
+        firePreEvent(new PreDropISchemaEvent(this, schema));
+        Map<String, String> transactionalListenersResponses = Collections.emptyMap();
+        ms.openTransaction();
+        try {
+          ms.dropISchema(schemaName);
+          if (!transactionalListeners.isEmpty()) {
+            transactionalListenersResponses =
+                MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
+                    EventType.DROP_ISCHEMA, new DropISchemaEvent(true, this, schema));
+          }
+          success = ms.commitTransaction();
+        } finally {
+          if (!success) ms.rollbackTransaction();
+          if (!listeners.isEmpty()) {
+            MetaStoreListenerNotifier.notifyEvent(listeners, EventType.DROP_ISCHEMA,
+                new DropISchemaEvent(success, this, schema), null,
+                transactionalListenersResponses, ms);
+          }
+        }
+      } catch (MetaException|NoSuchObjectException e) {
+        LOG.error("Caught exception dropping schema", e);
+        ex = e;
+        throw e;
+      } finally {
+        endFunction("drop_ischema", success, ex);
+      }
+    }
+
+    @Override
+    public void add_schema_version(SchemaVersion schemaVersion) throws TException {
+      startFunction("add_schema_version", ": " + schemaVersion);
+      boolean success = false;
+      Exception ex = null;
+      RawStore ms = getMS();
+      try {
+        // Make sure the referenced schema exists
+        if (ms.getISchema(schemaVersion.getSchema()) == null) {
+          throw new NoSuchObjectException("No schema named " + schemaVersion.getSchema());
+        }
+        firePreEvent(new PreAddSchemaVersionEvent(this, schemaVersion));
+        Map<String, String> transactionalListenersResponses = Collections.emptyMap();
+        ms.openTransaction();
+        try {
+          ms.addSchemaVersion(schemaVersion);
+
+          if (!transactionalListeners.isEmpty()) {
+            transactionalListenersResponses =
+                MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
+                    EventType.ADD_SCHEMA_VERSION, new AddSchemaVersionEvent(true, this, schemaVersion));
+          }
+          success = ms.commitTransaction();
+        } finally {
+          if (!success) ms.rollbackTransaction();
+          if (!listeners.isEmpty()) {
+            MetaStoreListenerNotifier.notifyEvent(listeners, EventType.ADD_SCHEMA_VERSION,
+                new AddSchemaVersionEvent(success, this, schemaVersion), null,
+                transactionalListenersResponses, ms);
+          }
+        }
+      } catch (MetaException|AlreadyExistsException e) {
+        LOG.error("Caught exception adding schema version", e);
+        ex = e;
+        throw e;
+      } finally {
+        endFunction("add_schema_version", success, ex);
+      }
+    }
+
+    @Override
+    public SchemaVersion get_schema_version(SchemaVersionDescriptor version) throws TException {
+      startFunction("get_schema_version", ": " + version);
+      Exception ex = null;
+      SchemaVersion schemaVersion = null;
+      try {
+        schemaVersion = getMS().getSchemaVersion(version);
+        if (schemaVersion == null) {
+          throw new NoSuchObjectException("No schema version " + version + "exists");
+        }
+        firePreEvent(new PreReadhSchemaVersionEvent(this, Collections.singletonList(schemaVersion)));
+        return schemaVersion;
+      } catch (MetaException e) {
+        LOG.error("Caught exception getting schema version", e);
+        ex = e;
+        throw e;
+      } finally {
+        endFunction("get_schema_version", schemaVersion != null, ex);
+      }
+    }
+
+    @Override
+    public SchemaVersion get_schema_latest_version(ISchemaName schemaName) throws TException {
+      startFunction("get_latest_schema_version", ": " + schemaName);
+      Exception ex = null;
+      SchemaVersion schemaVersion = null;
+      try {
+        schemaVersion = getMS().getLatestSchemaVersion(schemaName);
+        if (schemaVersion == null) {
+          throw new NoSuchObjectException("No versions of schema " + schemaName + "exist");
+        }
+        firePreEvent(new PreReadhSchemaVersionEvent(this, Collections.singletonList(schemaVersion)));
+        return schemaVersion;
+      } catch (MetaException e) {
+        LOG.error("Caught exception getting latest schema version", e);
+        ex = e;
+        throw e;
+      } finally {
+        endFunction("get_latest_schema_version", schemaVersion != null, ex);
+      }
+    }
+
+    @Override
+    public List<SchemaVersion> get_schema_all_versions(ISchemaName schemaName) throws TException {
+      startFunction("get_all_schema_versions", ": " + schemaName);
+      Exception ex = null;
+      List<SchemaVersion> schemaVersions = null;
+      try {
+        schemaVersions = getMS().getAllSchemaVersion(schemaName);
+        if (schemaVersions == null) {
+          throw new NoSuchObjectException("No versions of schema " + schemaName + "exist");
+        }
+        firePreEvent(new PreReadhSchemaVersionEvent(this, schemaVersions));
+        return schemaVersions;
+      } catch (MetaException e) {
+        LOG.error("Caught exception getting all schema versions", e);
+        ex = e;
+        throw e;
+      } finally {
+        endFunction("get_all_schema_versions", schemaVersions != null, ex);
+      }
+    }
+
+    @Override
+    public void drop_schema_version(SchemaVersionDescriptor version) throws TException {
+      startFunction("drop_schema_version", ": " + version);
+      Exception ex = null;
+      boolean success = false;
+      RawStore ms = getMS();
+      try {
+        SchemaVersion schemaVersion = ms.getSchemaVersion(version);
+        if (schemaVersion == null) {
+          throw new NoSuchObjectException("No schema version " + version);
+        }
+        firePreEvent(new PreDropSchemaVersionEvent(this, schemaVersion));
+        Map<String, String> transactionalListenersResponses = Collections.emptyMap();
+        ms.openTransaction();
+        try {
+          ms.dropSchemaVersion(version);
+          if (!transactionalListeners.isEmpty()) {
+            transactionalListenersResponses =
+                MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
+                    EventType.DROP_SCHEMA_VERSION, new DropSchemaVersionEvent(true, this, schemaVersion));
+          }
+          success = ms.commitTransaction();
+        } finally {
+          if (!success) ms.rollbackTransaction();
+          if (!listeners.isEmpty()) {
+            MetaStoreListenerNotifier.notifyEvent(listeners, EventType.DROP_SCHEMA_VERSION,
+                new DropSchemaVersionEvent(success, this, schemaVersion), null,
+                transactionalListenersResponses, ms);
+          }
+        }
+      } catch (MetaException|NoSuchObjectException e) {
+        LOG.error("Caught exception dropping schema version", e);
+        ex = e;
+        throw e;
+      } finally {
+        endFunction("drop_schema_version", success, ex);
+      }
+    }
+
+    @Override
+    public FindSchemasByColsResp get_schemas_by_cols(FindSchemasByColsRqst rqst) throws TException {
+      startFunction("get_schemas_by_cols");
+      Exception ex = null;
+      List<SchemaVersion> schemaVersions = Collections.emptyList();
+      try {
+        schemaVersions = getMS().getSchemaVersionsByColumns(rqst.getColName(),
+            rqst.getColNamespace(), rqst.getType());
+        firePreEvent(new PreReadhSchemaVersionEvent(this, schemaVersions));
+        final List<SchemaVersionDescriptor> entries = new ArrayList<>(schemaVersions.size());
+        schemaVersions.forEach(schemaVersion -> entries.add(
+            new SchemaVersionDescriptor(schemaVersion.getSchema(), schemaVersion.getVersion())));
+        return new FindSchemasByColsResp(entries);
+      } catch (MetaException e) {
+        LOG.error("Caught exception doing schema version query", e);
+        ex = e;
+        throw e;
+      } finally {
+        endFunction("get_schemas_by_cols", !schemaVersions.isEmpty(), ex);
+      }
+    }
+
+    @Override
+    public void map_schema_version_to_serde(MapSchemaVersionToSerdeRequest rqst)
+        throws TException {
+      startFunction("map_schema_version_to_serde, :" + rqst);
+      boolean success = false;
+      Exception ex = null;
+      RawStore ms = getMS();
+      try {
+        SchemaVersion oldSchemaVersion = ms.getSchemaVersion(rqst.getSchemaVersion());
+        if (oldSchemaVersion == null) {
+          throw new NoSuchObjectException("No schema version " + rqst.getSchemaVersion());
+        }
+        SerDeInfo serde = ms.getSerDeInfo(rqst.getSerdeName());
+        if (serde == null) {
+          throw new NoSuchObjectException("No SerDe named " + rqst.getSerdeName());
+        }
+        SchemaVersion newSchemaVersion = new SchemaVersion(oldSchemaVersion);
+        newSchemaVersion.setSerDe(serde);
+        firePreEvent(new PreAlterSchemaVersionEvent(this, oldSchemaVersion, newSchemaVersion));
+        Map<String, String> transactionalListenersResponses = Collections.emptyMap();
+        ms.openTransaction();
+        try {
+          ms.alterSchemaVersion(rqst.getSchemaVersion(), newSchemaVersion);
+          if (!transactionalListeners.isEmpty()) {
+            transactionalListenersResponses =
+                MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
+                    EventType.ALTER_SCHEMA_VERSION, new AlterSchemaVersionEvent(true, this,
+                        oldSchemaVersion, newSchemaVersion));
+          }
+          success = ms.commitTransaction();
+        } finally {
+          if (!success) ms.rollbackTransaction();
+          if (!listeners.isEmpty()) {
+            MetaStoreListenerNotifier.notifyEvent(listeners, EventType.ALTER_SCHEMA_VERSION,
+                new AlterSchemaVersionEvent(success, this, oldSchemaVersion, newSchemaVersion), null,
+                transactionalListenersResponses, ms);
+          }
+        }
+      } catch (MetaException|NoSuchObjectException e) {
+        LOG.error("Caught exception mapping schema version to serde", e);
+        ex = e;
+        throw e;
+      } finally {
+        endFunction("map_schema_version_to_serde", success, ex);
+      }
+    }
+
+    @Override
+    public void set_schema_version_state(SetSchemaVersionStateRequest rqst) throws TException {
+      startFunction("set_schema_version_state, :" + rqst);
+      boolean success = false;
+      Exception ex = null;
+      RawStore ms = getMS();
+      try {
+        SchemaVersion oldSchemaVersion = ms.getSchemaVersion(rqst.getSchemaVersion());
+        if (oldSchemaVersion == null) {
+          throw new NoSuchObjectException("No schema version " + rqst.getSchemaVersion());
+        }
+        SchemaVersion newSchemaVersion = new SchemaVersion(oldSchemaVersion);
+        newSchemaVersion.setState(rqst.getState());
+        firePreEvent(new PreAlterSchemaVersionEvent(this, oldSchemaVersion, newSchemaVersion));
+        Map<String, String> transactionalListenersResponses = Collections.emptyMap();
+        ms.openTransaction();
+        try {
+          ms.alterSchemaVersion(rqst.getSchemaVersion(), newSchemaVersion);
+          if (!transactionalListeners.isEmpty()) {
+            transactionalListenersResponses =
+                MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
+                    EventType.ALTER_SCHEMA_VERSION, new AlterSchemaVersionEvent(true, this,
+                        oldSchemaVersion, newSchemaVersion));
+          }
+          success = ms.commitTransaction();
+        } finally {
+          if (!success) ms.rollbackTransaction();
+          if (!listeners.isEmpty()) {
+            MetaStoreListenerNotifier.notifyEvent(listeners, EventType.ALTER_SCHEMA_VERSION,
+                new AlterSchemaVersionEvent(success, this, oldSchemaVersion, newSchemaVersion), null,
+                transactionalListenersResponses, ms);
+          }
+        }
+      } catch (MetaException|NoSuchObjectException e) {
+        LOG.error("Caught exception changing schema version state", e);
+        ex = e;
+        throw e;
+      } finally {
+        endFunction("set_schema_version_state", success, ex);
+      }
+    }
+
+    @Override
+    public void add_serde(SerDeInfo serde) throws TException {
+      startFunction("create_serde", ": " + serde.getName());
+      Exception ex = null;
+      boolean success = false;
+      RawStore ms = getMS();
+      try {
+        ms.openTransaction();
+        ms.addSerde(serde);
+        success = ms.commitTransaction();
+      } catch (MetaException|AlreadyExistsException e) {
+        LOG.error("Caught exception creating serde", e);
+        ex = e;
+        throw e;
+      } finally {
+        if (!success) ms.rollbackTransaction();
+        endFunction("create_serde", success, ex);
+      }
+    }
+
+    @Override
+    public SerDeInfo get_serde(GetSerdeRequest rqst) throws TException {
+      startFunction("get_serde", ": " + rqst);
+      Exception ex = null;
+      SerDeInfo serde = null;
+      try {
+        serde = getMS().getSerDeInfo(rqst.getSerdeName());
+        if (serde == null) {
+          throw new NoSuchObjectException("No serde named " + rqst.getSerdeName() + " exists");
+        }
+        return serde;
+      } catch (MetaException e) {
+        LOG.error("Caught exception getting serde", e);
+        ex = e;
+        throw e;
+      } finally {
+        endFunction("get_serde", serde != null, ex);
+      }
+    }
   }
 
   private static IHMSHandler newRetryingHMSHandler(IHMSHandler baseHandler, Configuration conf)

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/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 3128089..1755700b 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
@@ -2794,4 +2794,76 @@ 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 dbName, String schemaName, ISchema newSchema) throws TException {
+    client.alter_ischema(new AlterISchemaRequest(new ISchemaName(dbName, schemaName), newSchema));
+  }
+
+  @Override
+  public ISchema getISchema(String dbName, String name) throws TException {
+    return client.get_ischema(new ISchemaName(dbName, name));
+  }
+
+  @Override
+  public void dropISchema(String dbName, String name) throws TException {
+    client.drop_ischema(new ISchemaName(dbName, name));
+  }
+
+  @Override
+  public void addSchemaVersion(SchemaVersion schemaVersion) throws TException {
+    client.add_schema_version(schemaVersion);
+  }
+
+  @Override
+  public SchemaVersion getSchemaVersion(String dbName, String schemaName, int version) throws TException {
+    return client.get_schema_version(new SchemaVersionDescriptor(new ISchemaName(dbName, schemaName), version));
+  }
+
+  @Override
+  public SchemaVersion getSchemaLatestVersion(String dbName, String schemaName) throws TException {
+    return client.get_schema_latest_version(new ISchemaName(dbName, schemaName));
+  }
+
+  @Override
+  public List<SchemaVersion> getSchemaAllVersions(String dbName, String schemaName) throws TException {
+    return client.get_schema_all_versions(new ISchemaName(dbName, schemaName));
+  }
+
+  @Override
+  public void dropSchemaVersion(String dbName, String schemaName, int version) throws TException {
+    client.drop_schema_version(new SchemaVersionDescriptor(new ISchemaName(dbName, schemaName), version));
+  }
+
+  @Override
+  public FindSchemasByColsResp getSchemaByCols(FindSchemasByColsRqst rqst) throws TException {
+    return client.get_schemas_by_cols(rqst);
+  }
+
+  @Override
+  public void mapSchemaVersionToSerde(String dbName, String schemaName, int version, String serdeName)
+      throws TException {
+    client.map_schema_version_to_serde(new MapSchemaVersionToSerdeRequest(
+        new SchemaVersionDescriptor(new ISchemaName(dbName, schemaName), version), serdeName));
+  }
+
+  @Override
+  public void setSchemaVersionState(String dbName, String schemaName, int version, SchemaVersionState state)
+      throws TException {
+    client.set_schema_version_state(new SetSchemaVersionStateRequest(new SchemaVersionDescriptor(
+        new ISchemaName(dbName, 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(new GetSerdeRequest(serDeName));
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/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 46984cb..f1d5066 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
@@ -49,6 +49,8 @@ import org.apache.hadoop.hive.metastore.api.DefaultConstraintsRequest;
 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;
@@ -62,6 +64,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.InvalidInputException;
 import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
 import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
@@ -94,6 +97,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;
@@ -1828,4 +1834,165 @@ 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 dbName database the schema is in
+   * @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 dbName, String schemaName, ISchema newSchema) throws TException;
+
+  /**
+   * Fetch a schema.
+   * @param dbName database the schema is in
+   * @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 dbName, String name) throws TException;
+
+  /**
+   * Drop an existing schema.  If there are schema versions of this, this call will fail.
+   * @param dbName database the schema is in
+   * @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 dbName, 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 dbName database the schema is in
+   * @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 dbName, String schemaName, int version) throws TException;
+
+  /**
+   * Get the latest version of a schema.
+   * @param dbName database the schema is in
+   * @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 dbName, String schemaName) throws TException;
+
+  /**
+   * Get all the extant versions of a schema.
+   * @param dbName database the schema is in
+   * @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 dbName, 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 dbName database the schema is in
+   * @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 dbName, 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 dbName database the schema is in
+   * @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 dbName, String schemaName, int version, String serdeName) throws TException;
+
+  /**
+   * Set the state of a schema version.
+   * @param dbName database the schema is in
+   * @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 dbName, 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/12041d39/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 88d22d6..67600e1 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
@@ -26,19 +26,25 @@ import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.events.AddForeignKeyEvent;
 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.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.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;
@@ -196,6 +202,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/12041d39/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 af01e8a..f5a91b4 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
@@ -28,16 +28,22 @@ import org.apache.hadoop.hive.metastore.events.AddForeignKeyEvent;
 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.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.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;
@@ -158,6 +164,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/12041d39/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 3633c03..1f75105 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
@@ -92,6 +92,8 @@ 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.ISchemaName;
 import org.apache.hadoop.hive.metastore.api.InvalidInputException;
 import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
 import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
@@ -121,7 +123,14 @@ 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.SchemaVersionDescriptor;
+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;
@@ -155,6 +164,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.MMasterKey;
 import org.apache.hadoop.hive.metastore.model.MMetastoreDBProperties;
 import org.apache.hadoop.hive.metastore.model.MNotificationLog;
@@ -168,6 +178,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;
@@ -1780,15 +1791,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());
   }
 
   /**
@@ -9569,6 +9587,408 @@ 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.getDbName(), schema.getName()) != null) {
+        throw new AlreadyExistsException("Schema with name " + schema.getDbName() + "." +
+            schema.getName() + " already exists");
+      }
+      pm.makePersistent(mSchema);
+      committed = commitTransaction();
+    } finally {
+      if (!committed) rollbackTransaction();
+    }
+  }
+
+  @Override
+  public void alterISchema(ISchemaName schemaName, ISchema newSchema)
+      throws NoSuchObjectException, MetaException {
+    boolean committed = false;
+    try {
+      openTransaction();
+      MISchema oldMSchema = getMISchema(schemaName.getDbName(), schemaName.getSchemaName());
+      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(ISchemaName schemaName) throws MetaException {
+    boolean committed = false;
+    try {
+      openTransaction();
+      ISchema schema = convertToISchema(getMISchema(schemaName.getDbName(), schemaName.getSchemaName()));
+      committed = commitTransaction();
+      return schema;
+    } finally {
+      if (!committed) rollbackTransaction();
+    }
+  }
+
+  private MISchema getMISchema(String dbName, String name) {
+    Query query = null;
+    try {
+      name = normalizeIdentifier(name);
+      dbName = normalizeIdentifier(dbName);
+      query = pm.newQuery(MISchema.class, "name == schemaName && db.name == dbname");
+      query.declareParameters("java.lang.String schemaName, java.lang.String dbname");
+      query.setUnique(true);
+      MISchema mSchema = (MISchema)query.execute(name, dbName);
+      pm.retrieve(mSchema);
+      return mSchema;
+    } finally {
+      if (query != null) query.closeAll();
+    }
+  }
+
+  @Override
+  public void dropISchema(ISchemaName schemaName) throws NoSuchObjectException, MetaException {
+    boolean committed = false;
+    try {
+      openTransaction();
+      MISchema mSchema = getMISchema(schemaName.getDbName(), schemaName.getSchemaName());
+      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.getSchema().getDbName(),
+          schemaVersion.getSchema().getSchemaName(), schemaVersion.getVersion()) != null) {
+        throw new AlreadyExistsException("Schema name " + schemaVersion.getSchema() +
+            " version " + schemaVersion.getVersion() + " already exists");
+      }
+      // Make sure the referenced Schema exists
+      if (getMISchema(schemaVersion.getSchema().getDbName(), schemaVersion.getSchema().getSchemaName()) == null) {
+        throw new NoSuchObjectException("Schema " + schemaVersion.getSchema() + " does not exist");
+      }
+      pm.makePersistent(mSchemaVersion);
+      committed = commitTransaction();
+    } finally {
+      if (!committed) rollbackTransaction();;
+    }
+  }
+
+  @Override
+  public void alterSchemaVersion(SchemaVersionDescriptor version, SchemaVersion newVersion)
+      throws NoSuchObjectException, MetaException {
+    boolean committed = false;
+    try {
+      openTransaction();
+      MSchemaVersion oldMSchemaVersion = getMSchemaVersion(version.getSchema().getDbName(),
+          version.getSchema().getSchemaName(), version.getVersion());
+      if (oldMSchemaVersion == null) {
+        throw new NoSuchObjectException("No schema 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(SchemaVersionDescriptor version) throws MetaException {
+    boolean committed = false;
+    try {
+      openTransaction();
+      SchemaVersion schemaVersion =
+          convertToSchemaVersion(getMSchemaVersion(version.getSchema().getDbName(),
+              version.getSchema().getSchemaName(), version.getVersion()));
+      committed = commitTransaction();
+      return schemaVersion;
+    } finally {
+      if (!committed) rollbackTransaction();;
+    }
+  }
+
+  private MSchemaVersion getMSchemaVersion(String dbName, String schemaName, int version) {
+    Query query = null;
+    try {
+      dbName = normalizeIdentifier(dbName);
+      schemaName = normalizeIdentifier(schemaName);
+      query = pm.newQuery(MSchemaVersion.class,
+          "iSchema.name == schemaName && iSchema.db.name == dbName && version == schemaVersion");
+      query.declareParameters(
+          "java.lang.String schemaName, java.lang.String dbName, java.lang.Integer schemaVersion");
+      query.setUnique(true);
+      MSchemaVersion mSchemaVersion = (MSchemaVersion)query.execute(schemaName, dbName, 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(ISchemaName schemaName) throws MetaException {
+    boolean committed = false;
+    Query query = null;
+    try {
+      openTransaction();
+      String name = normalizeIdentifier(schemaName.getSchemaName());
+      String dbName = normalizeIdentifier(schemaName.getDbName());
+      query = pm.newQuery(MSchemaVersion.class,
+          "iSchema.name == schemaName && iSchema.db.name == dbName");
+      query.declareParameters("java.lang.String schemaName, java.lang.String dbName");
+      query.setUnique(true);
+      query.setOrdering("version descending");
+      query.setRange(0, 1);
+      MSchemaVersion mSchemaVersion = (MSchemaVersion)query.execute(name, dbName);
+      pm.retrieve(mSchemaVersion);
+      if (mSchemaVersion != null) {
+        pm.retrieveAll(mSchemaVersion.getCols());
+        if (mSchemaVersion.getSerDe() != null) pm.retrieve(mSchemaVersion.getSerDe());
+      }
+      SchemaVersion version = mSchemaVersion == null ? null : convertToSchemaVersion(mSchemaVersion);
+      committed = commitTransaction();
+      return version;
+    } finally {
+      rollbackAndCleanup(committed, query);
+    }
+  }
+
+  @Override
+  public List<SchemaVersion> getAllSchemaVersion(ISchemaName schemaName) throws MetaException {
+    boolean committed = false;
+    Query query = null;
+    try {
+      openTransaction();
+      String name = normalizeIdentifier(schemaName.getSchemaName());
+      String dbName = normalizeIdentifier(schemaName.getDbName());
+      query = pm.newQuery(MSchemaVersion.class,
+          "iSchema.name == schemaName && iSchema.db.name == dbName");
+      query.declareParameters("java.lang.String schemaName, java.lang.String dbName");
+      query.setOrdering("version descending");
+      List<MSchemaVersion> mSchemaVersions = query.setParameters(name, dbName).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(SchemaVersionDescriptor version) throws NoSuchObjectException,
+      MetaException {
+    boolean committed = false;
+    try {
+      openTransaction();
+      MSchemaVersion mSchemaVersion = getMSchemaVersion(version.getSchema().getDbName(),
+          version.getSchema().getSchemaName(), version.getVersion());
+      if (mSchemaVersion != null) {
+        pm.deletePersistentAll(mSchemaVersion);
+      } else {
+        throw new NoSuchObjectException("Schema 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.getSchema().getDbName()),
+          normalizeIdentifier(schemaVersion.getSchema().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(
+        new ISchemaName(mSchemaVersion.getiSchema().getDb().getName(),
+            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/12041d39/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 40a7497..b079f8b 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
@@ -18,7 +18,10 @@
 
 package org.apache.hadoop.hive.metastore;
 
+import org.apache.hadoop.hive.metastore.api.AlterISchemaRequest;
 import org.apache.hadoop.hive.metastore.api.CreationMetadata;
+import org.apache.hadoop.hive.metastore.api.ISchemaName;
+import org.apache.hadoop.hive.metastore.api.SchemaVersionDescriptor;
 import org.apache.hadoop.hive.metastore.api.WMFullResourcePlan;
 
 import java.lang.annotation.ElementType;
@@ -40,6 +43,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.InvalidInputException;
 import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
 import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
@@ -69,6 +73,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;
@@ -805,4 +811,133 @@ 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(ISchemaName schemaName, ISchema newSchema) throws NoSuchObjectException, MetaException;
+
+  /**
+   * Get an ISchema by name.
+   * @param schemaName schema descriptor
+   * @return ISchema
+   * @throws MetaException general database exception
+   */
+  ISchema getISchema(ISchemaName 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 schema descriptor
+   * @throws NoSuchObjectException no schema of this name exists
+   * @throws MetaException general database exception
+   */
+  void dropISchema(ISchemaName 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 version version descriptor for the schema
+   * @param newVersion altered SchemaVersion
+   * @throws NoSuchObjectException no such version of the named schema exists
+   * @throws MetaException general database exception
+   */
+  void alterSchemaVersion(SchemaVersionDescriptor version, SchemaVersion newVersion)
+      throws NoSuchObjectException, MetaException;
+
+  /**
+   * Get a specific schema version.
+   * @param version version descriptor for the schema
+   * @return the SchemaVersion
+   * @throws MetaException general database exception
+   */
+  SchemaVersion getSchemaVersion(SchemaVersionDescriptor 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(ISchemaName 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(ISchemaName 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 version version descriptor for the schema
+   * @throws NoSuchObjectException no such version of the named schema exists
+   * @throws MetaException general database exception
+   */
+  void dropSchemaVersion(SchemaVersionDescriptor 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/12041d39/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 f58ba04..d28b196 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
@@ -18,6 +18,8 @@
 package org.apache.hadoop.hive.metastore.cache;
 
 import org.apache.hadoop.hive.metastore.api.CreationMetadata;
+import org.apache.hadoop.hive.metastore.api.ISchemaName;
+import org.apache.hadoop.hive.metastore.api.SchemaVersionDescriptor;
 import org.apache.hadoop.hive.metastore.api.WMFullResourcePlan;
 
 import java.nio.ByteBuffer;
@@ -59,6 +61,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.InvalidInputException;
 import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
 import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
@@ -91,6 +94,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;
@@ -2500,6 +2505,78 @@ public class CachedStore implements RawStore, Configurable {
     return rawStore.getPartitionColStatsForDatabase(dbName);
   }
 
+  // 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(ISchemaName schemaName, ISchema newSchema)
+      throws NoSuchObjectException, MetaException {
+    rawStore.alterISchema(schemaName, newSchema);
+  }
+
+  @Override
+  public ISchema getISchema(ISchemaName schemaName) throws MetaException {
+    return rawStore.getISchema(schemaName);
+  }
+
+  @Override
+  public void dropISchema(ISchemaName 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(SchemaVersionDescriptor version, SchemaVersion newVersion) throws
+      NoSuchObjectException, MetaException {
+    rawStore.alterSchemaVersion(version, newVersion);
+  }
+
+  @Override
+  public SchemaVersion getSchemaVersion(SchemaVersionDescriptor version) throws MetaException {
+    return rawStore.getSchemaVersion(version);
+  }
+
+  @Override
+  public SchemaVersion getLatestSchemaVersion(ISchemaName schemaName) throws MetaException {
+    return rawStore.getLatestSchemaVersion(schemaName);
+  }
+
+  @Override
+  public List<SchemaVersion> getAllSchemaVersion(ISchemaName 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(SchemaVersionDescriptor version) throws NoSuchObjectException,
+      MetaException {
+    rawStore.dropSchemaVersion(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/12041d39/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/12041d39/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..32a84ac
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/ISchemaBuilder.java
@@ -0,0 +1,94 @@
+/*
+ * 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.Warehouse;
+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 = Warehouse.DEFAULT_DATABASE_NAME;
+  }
+
+  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;
+  }
+}


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

Posted by ga...@apache.org.
HIVE-17990 Add Thrift and DB storage for Schema Registry objects (Alan Gates, reviewed by Thejas Nair)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/12041d39
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/12041d39
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/12041d39

Branch: refs/heads/master
Commit: 12041d39f052dc8e4858815da15c967cb378fae9
Parents: d717d38
Author: Alan Gates <ga...@hortonworks.com>
Authored: Mon Mar 12 16:54:46 2018 -0700
Committer: Alan Gates <ga...@hortonworks.com>
Committed: Mon Mar 12 16:54:46 2018 -0700

----------------------------------------------------------------------
 .../listener/DummyRawStoreFailEvent.java        |    75 +
 standalone-metastore/pom.xml                    |     3 +-
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.cpp  | 35798 ++++++++++-------
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.h    |  2378 +-
 .../ThriftHiveMetastore_server.skeleton.cpp     |    70 +
 .../gen/thrift/gen-cpp/hive_metastore_types.cpp |  6134 +--
 .../gen/thrift/gen-cpp/hive_metastore_types.h   |   729 +-
 .../hive/metastore/api/AlterISchemaRequest.java |   509 +
 .../metastore/api/FindSchemasByColsResp.java    |   449 +
 .../metastore/api/FindSchemasByColsRqst.java    |   605 +
 .../hive/metastore/api/GetSerdeRequest.java     |   395 +
 .../hadoop/hive/metastore/api/ISchema.java      |  1162 +
 .../hadoop/hive/metastore/api/ISchemaName.java  |   499 +
 .../api/MapSchemaVersionToSerdeRequest.java     |   504 +
 .../hive/metastore/api/SchemaCompatibility.java |    51 +
 .../hadoop/hive/metastore/api/SchemaType.java   |    45 +
 .../hive/metastore/api/SchemaValidation.java    |    45 +
 .../hive/metastore/api/SchemaVersion.java       |  1412 +
 .../metastore/api/SchemaVersionDescriptor.java  |   502 +
 .../hive/metastore/api/SchemaVersionState.java  |    63 +
 .../hadoop/hive/metastore/api/SerDeInfo.java    |   443 +-
 .../hadoop/hive/metastore/api/SerdeType.java    |    45 +
 .../api/SetSchemaVersionStateRequest.java       |   516 +
 .../hive/metastore/api/ThriftHiveMetastore.java | 17889 +++++++-
 .../gen-php/metastore/ThriftHiveMetastore.php   | 27954 +++++++------
 .../src/gen/thrift/gen-php/metastore/Types.php  |  1523 +
 .../hive_metastore/ThriftHiveMetastore-remote   |    98 +
 .../hive_metastore/ThriftHiveMetastore.py       |  5189 ++-
 .../gen/thrift/gen-py/hive_metastore/ttypes.py  |  1123 +-
 .../gen/thrift/gen-rb/hive_metastore_types.rb   |   277 +-
 .../gen/thrift/gen-rb/thrift_hive_metastore.rb  |   915 +
 .../hadoop/hive/metastore/HiveMetaStore.java    |   472 +-
 .../hive/metastore/HiveMetaStoreClient.java     |    72 +
 .../hadoop/hive/metastore/IMetaStoreClient.java |   167 +
 .../hive/metastore/MetaStoreEventListener.java  |    26 +
 .../metastore/MetaStoreListenerNotifier.java    |    42 +
 .../hadoop/hive/metastore/ObjectStore.java      |   426 +-
 .../apache/hadoop/hive/metastore/RawStore.java  |   135 +
 .../hive/metastore/cache/CachedStore.java       |    77 +
 .../client/builder/DatabaseBuilder.java         |     2 +-
 .../client/builder/ISchemaBuilder.java          |    94 +
 .../client/builder/SchemaVersionBuilder.java    |   108 +
 .../client/builder/SerdeAndColsBuilder.java     |   124 +
 .../builder/StorageDescriptorBuilder.java       |    57 +-
 .../metastore/events/AddSchemaVersionEvent.java |    40 +
 .../metastore/events/AlterISchemaEvent.java     |    45 +
 .../events/AlterSchemaVersionEvent.java         |    46 +
 .../metastore/events/CreateISchemaEvent.java    |    39 +
 .../hive/metastore/events/DropISchemaEvent.java |    39 +
 .../events/DropSchemaVersionEvent.java          |    40 +
 .../events/PreAddSchemaVersionEvent.java        |    39 +
 .../metastore/events/PreAlterISchemaEvent.java  |    44 +
 .../events/PreAlterSchemaVersionEvent.java      |    45 +
 .../metastore/events/PreCreateISchemaEvent.java |    39 +
 .../metastore/events/PreDropISchemaEvent.java   |    39 +
 .../events/PreDropSchemaVersionEvent.java       |    39 +
 .../hive/metastore/events/PreEventContext.java  |    10 +-
 .../metastore/events/PreReadISchemaEvent.java   |    39 +
 .../events/PreReadhSchemaVersionEvent.java      |    36 +
 .../hive/metastore/messaging/EventMessage.java  |     8 +-
 .../metastore/messaging/MessageFactory.java     |     7 +
 .../hadoop/hive/metastore/model/MISchema.java   |   107 +
 .../hive/metastore/model/MSchemaVersion.java    |   127 +
 .../hadoop/hive/metastore/model/MSerDeInfo.java |    48 +-
 .../hive/metastore/utils/MetaStoreUtils.java    |     1 +
 .../main/resources/datanucleus-log4j.properties |    17 +
 .../src/main/resources/package.jdo              |    77 +
 .../main/sql/derby/hive-schema-3.0.0.derby.sql  |    30 +-
 .../sql/derby/upgrade-2.3.0-to-3.0.0.derby.sql  |    34 +
 .../main/sql/mssql/hive-schema-3.0.0.mssql.sql  |    33 +-
 .../sql/mssql/upgrade-2.3.0-to-3.0.0.mssql.sql  |    33 +
 .../main/sql/mysql/hive-schema-3.0.0.mysql.sql  |    36 +
 .../sql/mysql/upgrade-2.3.0-to-3.0.0.mysql.sql  |    38 +
 .../sql/oracle/hive-schema-3.0.0.oracle.sql     |    33 +-
 .../oracle/upgrade-2.3.0-to-3.0.0.oracle.sql    |    34 +
 .../sql/postgres/hive-schema-3.0.0.postgres.sql |    33 +-
 .../upgrade-2.3.0-to-3.0.0.postgres.sql         |    34 +
 .../src/main/thrift/hive_metastore.thrift       |   136 +-
 .../DummyRawStoreControlledCommit.java          |    75 +
 .../DummyRawStoreForJdoConnection.java          |    74 +
 .../TestHiveMetaStoreSchemaMethods.java         |  1180 +
 .../hadoop/hive/metastore/TestObjectStore.java  |    32 -
 .../metastore/TestObjectStoreSchemaMethods.java |   573 +
 .../hive/metastore/cache/TestCachedStore.java   |     2 -
 84 files changed, 80370 insertions(+), 32213 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
----------------------------------------------------------------------
diff --git a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
index 4ecd89c..6144b61 100644
--- a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
+++ b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
@@ -18,6 +18,8 @@
 
 package org.apache.hive.hcatalog.listener;
 
+import org.apache.hadoop.hive.metastore.api.ISchemaName;
+import org.apache.hadoop.hive.metastore.api.SchemaVersionDescriptor;
 import org.apache.hadoop.hive.metastore.api.WMFullResourcePlan;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
@@ -40,6 +42,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.InvalidInputException;
 import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
 import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
@@ -57,6 +60,8 @@ import org.apache.hadoop.hive.metastore.api.PartitionValuesResponse;
 import org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet;
 import org.apache.hadoop.hive.metastore.api.PrincipalType;
 import org.apache.hadoop.hive.metastore.api.PrivilegeBag;
+import org.apache.hadoop.hive.metastore.api.SchemaVersion;
+import org.apache.hadoop.hive.metastore.api.SerDeInfo;
 import org.apache.hadoop.hive.metastore.api.WMResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMNullableResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMTrigger;
@@ -1070,4 +1075,74 @@ public class DummyRawStoreFailEvent implements RawStore, Configurable {
     // TODO Auto-generated method stub
     return null;
   }
+
+  public void createISchema(ISchema schema) throws AlreadyExistsException, MetaException,
+      NoSuchObjectException {
+    objectStore.createISchema(schema);
+  }
+
+  @Override
+  public void alterISchema(ISchemaName schemaName, ISchema newSchema) throws NoSuchObjectException,
+      MetaException {
+    objectStore.alterISchema(schemaName, newSchema);
+  }
+
+  @Override
+  public ISchema getISchema(ISchemaName schemaName) throws MetaException {
+    return objectStore.getISchema(schemaName);
+  }
+
+  @Override
+  public void dropISchema(ISchemaName schemaName) throws NoSuchObjectException, MetaException {
+    objectStore.dropISchema(schemaName);
+  }
+
+  @Override
+  public void addSchemaVersion(SchemaVersion schemaVersion) throws AlreadyExistsException,
+      InvalidObjectException, NoSuchObjectException, MetaException {
+    objectStore.addSchemaVersion(schemaVersion);
+  }
+
+  @Override
+  public void alterSchemaVersion(SchemaVersionDescriptor version, SchemaVersion newVersion) throws
+      NoSuchObjectException, MetaException {
+    objectStore.alterSchemaVersion(version, newVersion);
+  }
+
+  @Override
+  public SchemaVersion getSchemaVersion(SchemaVersionDescriptor version) throws MetaException {
+    return objectStore.getSchemaVersion(version);
+  }
+
+  @Override
+  public SchemaVersion getLatestSchemaVersion(ISchemaName schemaName) throws MetaException {
+    return objectStore.getLatestSchemaVersion(schemaName);
+  }
+
+  @Override
+  public List<SchemaVersion> getAllSchemaVersion(ISchemaName schemaName) throws MetaException {
+    return objectStore.getAllSchemaVersion(schemaName);
+  }
+
+  @Override
+  public List<SchemaVersion> getSchemaVersionsByColumns(String colName, String colNamespace,
+                                                        String type) throws MetaException {
+    return objectStore.getSchemaVersionsByColumns(colName, colNamespace, type);
+  }
+
+  @Override
+  public void dropSchemaVersion(SchemaVersionDescriptor version) throws NoSuchObjectException,
+      MetaException {
+    objectStore.dropSchemaVersion(version);
+  }
+
+  @Override
+  public SerDeInfo getSerDeInfo(String serDeName) throws NoSuchObjectException, MetaException {
+    return objectStore.getSerDeInfo(serDeName);
+  }
+
+  @Override
+  public void addSerde(SerDeInfo serde) throws AlreadyExistsException, MetaException {
+    objectStore.addSerde(serde);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/pom.xml
----------------------------------------------------------------------
diff --git a/standalone-metastore/pom.xml b/standalone-metastore/pom.xml
index fa76e63..e5b13be 100644
--- a/standalone-metastore/pom.xml
+++ b/standalone-metastore/pom.xml
@@ -755,7 +755,8 @@
         <version>4.0.5</version>
         <configuration>
           <api>JDO</api>
-          <verbose>true</verbose>
+          <verbose>false</verbose>
+          <log4jConfiguration>${basedir}/src/main/resources/datanucleus-log4j.properties</log4jConfiguration>
           <metadataIncludes>**/*.jdo</metadataIncludes>
           <fork>false</fork>
         </configuration>


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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ISchema.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ISchema.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ISchema.java
new file mode 100644
index 0000000..92d8b52
--- /dev/null
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ISchema.java
@@ -0,0 +1,1162 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hive.metastore.api;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+@org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public class ISchema implements org.apache.thrift.TBase<ISchema, ISchema._Fields>, java.io.Serializable, Cloneable, Comparable<ISchema> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ISchema");
+
+  private static final org.apache.thrift.protocol.TField SCHEMA_TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("schemaType", org.apache.thrift.protocol.TType.I32, (short)1);
+  private static final org.apache.thrift.protocol.TField NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("name", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField DB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("dbName", org.apache.thrift.protocol.TType.STRING, (short)3);
+  private static final org.apache.thrift.protocol.TField COMPATIBILITY_FIELD_DESC = new org.apache.thrift.protocol.TField("compatibility", org.apache.thrift.protocol.TType.I32, (short)4);
+  private static final org.apache.thrift.protocol.TField VALIDATION_LEVEL_FIELD_DESC = new org.apache.thrift.protocol.TField("validationLevel", org.apache.thrift.protocol.TType.I32, (short)5);
+  private static final org.apache.thrift.protocol.TField CAN_EVOLVE_FIELD_DESC = new org.apache.thrift.protocol.TField("canEvolve", org.apache.thrift.protocol.TType.BOOL, (short)6);
+  private static final org.apache.thrift.protocol.TField SCHEMA_GROUP_FIELD_DESC = new org.apache.thrift.protocol.TField("schemaGroup", org.apache.thrift.protocol.TType.STRING, (short)7);
+  private static final org.apache.thrift.protocol.TField DESCRIPTION_FIELD_DESC = new org.apache.thrift.protocol.TField("description", org.apache.thrift.protocol.TType.STRING, (short)8);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new ISchemaStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new ISchemaTupleSchemeFactory());
+  }
+
+  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
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    /**
+     * 
+     * @see SchemaType
+     */
+    SCHEMA_TYPE((short)1, "schemaType"),
+    NAME((short)2, "name"),
+    DB_NAME((short)3, "dbName"),
+    /**
+     * 
+     * @see SchemaCompatibility
+     */
+    COMPATIBILITY((short)4, "compatibility"),
+    /**
+     * 
+     * @see SchemaValidation
+     */
+    VALIDATION_LEVEL((short)5, "validationLevel"),
+    CAN_EVOLVE((short)6, "canEvolve"),
+    SCHEMA_GROUP((short)7, "schemaGroup"),
+    DESCRIPTION((short)8, "description");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // SCHEMA_TYPE
+          return SCHEMA_TYPE;
+        case 2: // NAME
+          return NAME;
+        case 3: // DB_NAME
+          return DB_NAME;
+        case 4: // COMPATIBILITY
+          return COMPATIBILITY;
+        case 5: // VALIDATION_LEVEL
+          return VALIDATION_LEVEL;
+        case 6: // CAN_EVOLVE
+          return CAN_EVOLVE;
+        case 7: // SCHEMA_GROUP
+          return SCHEMA_GROUP;
+        case 8: // DESCRIPTION
+          return DESCRIPTION;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __CANEVOLVE_ISSET_ID = 0;
+  private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.SCHEMA_GROUP,_Fields.DESCRIPTION};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.SCHEMA_TYPE, new org.apache.thrift.meta_data.FieldMetaData("schemaType", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, SchemaType.class)));
+    tmpMap.put(_Fields.NAME, new org.apache.thrift.meta_data.FieldMetaData("name", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.DB_NAME, new org.apache.thrift.meta_data.FieldMetaData("dbName", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.COMPATIBILITY, new org.apache.thrift.meta_data.FieldMetaData("compatibility", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, SchemaCompatibility.class)));
+    tmpMap.put(_Fields.VALIDATION_LEVEL, new org.apache.thrift.meta_data.FieldMetaData("validationLevel", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, SchemaValidation.class)));
+    tmpMap.put(_Fields.CAN_EVOLVE, new org.apache.thrift.meta_data.FieldMetaData("canEvolve", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+    tmpMap.put(_Fields.SCHEMA_GROUP, new org.apache.thrift.meta_data.FieldMetaData("schemaGroup", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.DESCRIPTION, new org.apache.thrift.meta_data.FieldMetaData("description", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ISchema.class, metaDataMap);
+  }
+
+  public ISchema() {
+  }
+
+  public ISchema(
+    SchemaType schemaType,
+    String name,
+    String dbName,
+    SchemaCompatibility compatibility,
+    SchemaValidation validationLevel,
+    boolean canEvolve)
+  {
+    this();
+    this.schemaType = schemaType;
+    this.name = name;
+    this.dbName = dbName;
+    this.compatibility = compatibility;
+    this.validationLevel = validationLevel;
+    this.canEvolve = canEvolve;
+    setCanEvolveIsSet(true);
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public ISchema(ISchema other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.isSetSchemaType()) {
+      this.schemaType = other.schemaType;
+    }
+    if (other.isSetName()) {
+      this.name = other.name;
+    }
+    if (other.isSetDbName()) {
+      this.dbName = other.dbName;
+    }
+    if (other.isSetCompatibility()) {
+      this.compatibility = other.compatibility;
+    }
+    if (other.isSetValidationLevel()) {
+      this.validationLevel = other.validationLevel;
+    }
+    this.canEvolve = other.canEvolve;
+    if (other.isSetSchemaGroup()) {
+      this.schemaGroup = other.schemaGroup;
+    }
+    if (other.isSetDescription()) {
+      this.description = other.description;
+    }
+  }
+
+  public ISchema deepCopy() {
+    return new ISchema(this);
+  }
+
+  @Override
+  public void clear() {
+    this.schemaType = null;
+    this.name = null;
+    this.dbName = null;
+    this.compatibility = null;
+    this.validationLevel = null;
+    setCanEvolveIsSet(false);
+    this.canEvolve = false;
+    this.schemaGroup = null;
+    this.description = null;
+  }
+
+  /**
+   * 
+   * @see SchemaType
+   */
+  public SchemaType getSchemaType() {
+    return this.schemaType;
+  }
+
+  /**
+   * 
+   * @see SchemaType
+   */
+  public void setSchemaType(SchemaType schemaType) {
+    this.schemaType = schemaType;
+  }
+
+  public void unsetSchemaType() {
+    this.schemaType = null;
+  }
+
+  /** Returns true if field schemaType is set (has been assigned a value) and false otherwise */
+  public boolean isSetSchemaType() {
+    return this.schemaType != null;
+  }
+
+  public void setSchemaTypeIsSet(boolean value) {
+    if (!value) {
+      this.schemaType = null;
+    }
+  }
+
+  public String getName() {
+    return this.name;
+  }
+
+  public void setName(String name) {
+    this.name = name;
+  }
+
+  public void unsetName() {
+    this.name = null;
+  }
+
+  /** Returns true if field name is set (has been assigned a value) and false otherwise */
+  public boolean isSetName() {
+    return this.name != null;
+  }
+
+  public void setNameIsSet(boolean value) {
+    if (!value) {
+      this.name = null;
+    }
+  }
+
+  public String getDbName() {
+    return this.dbName;
+  }
+
+  public void setDbName(String dbName) {
+    this.dbName = dbName;
+  }
+
+  public void unsetDbName() {
+    this.dbName = null;
+  }
+
+  /** Returns true if field dbName is set (has been assigned a value) and false otherwise */
+  public boolean isSetDbName() {
+    return this.dbName != null;
+  }
+
+  public void setDbNameIsSet(boolean value) {
+    if (!value) {
+      this.dbName = null;
+    }
+  }
+
+  /**
+   * 
+   * @see SchemaCompatibility
+   */
+  public SchemaCompatibility getCompatibility() {
+    return this.compatibility;
+  }
+
+  /**
+   * 
+   * @see SchemaCompatibility
+   */
+  public void setCompatibility(SchemaCompatibility compatibility) {
+    this.compatibility = compatibility;
+  }
+
+  public void unsetCompatibility() {
+    this.compatibility = null;
+  }
+
+  /** Returns true if field compatibility is set (has been assigned a value) and false otherwise */
+  public boolean isSetCompatibility() {
+    return this.compatibility != null;
+  }
+
+  public void setCompatibilityIsSet(boolean value) {
+    if (!value) {
+      this.compatibility = null;
+    }
+  }
+
+  /**
+   * 
+   * @see SchemaValidation
+   */
+  public SchemaValidation getValidationLevel() {
+    return this.validationLevel;
+  }
+
+  /**
+   * 
+   * @see SchemaValidation
+   */
+  public void setValidationLevel(SchemaValidation validationLevel) {
+    this.validationLevel = validationLevel;
+  }
+
+  public void unsetValidationLevel() {
+    this.validationLevel = null;
+  }
+
+  /** Returns true if field validationLevel is set (has been assigned a value) and false otherwise */
+  public boolean isSetValidationLevel() {
+    return this.validationLevel != null;
+  }
+
+  public void setValidationLevelIsSet(boolean value) {
+    if (!value) {
+      this.validationLevel = null;
+    }
+  }
+
+  public boolean isCanEvolve() {
+    return this.canEvolve;
+  }
+
+  public void setCanEvolve(boolean canEvolve) {
+    this.canEvolve = canEvolve;
+    setCanEvolveIsSet(true);
+  }
+
+  public void unsetCanEvolve() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __CANEVOLVE_ISSET_ID);
+  }
+
+  /** Returns true if field canEvolve is set (has been assigned a value) and false otherwise */
+  public boolean isSetCanEvolve() {
+    return EncodingUtils.testBit(__isset_bitfield, __CANEVOLVE_ISSET_ID);
+  }
+
+  public void setCanEvolveIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __CANEVOLVE_ISSET_ID, value);
+  }
+
+  public String getSchemaGroup() {
+    return this.schemaGroup;
+  }
+
+  public void setSchemaGroup(String schemaGroup) {
+    this.schemaGroup = schemaGroup;
+  }
+
+  public void unsetSchemaGroup() {
+    this.schemaGroup = null;
+  }
+
+  /** Returns true if field schemaGroup is set (has been assigned a value) and false otherwise */
+  public boolean isSetSchemaGroup() {
+    return this.schemaGroup != null;
+  }
+
+  public void setSchemaGroupIsSet(boolean value) {
+    if (!value) {
+      this.schemaGroup = null;
+    }
+  }
+
+  public String getDescription() {
+    return this.description;
+  }
+
+  public void setDescription(String description) {
+    this.description = description;
+  }
+
+  public void unsetDescription() {
+    this.description = null;
+  }
+
+  /** Returns true if field description is set (has been assigned a value) and false otherwise */
+  public boolean isSetDescription() {
+    return this.description != null;
+  }
+
+  public void setDescriptionIsSet(boolean value) {
+    if (!value) {
+      this.description = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case SCHEMA_TYPE:
+      if (value == null) {
+        unsetSchemaType();
+      } else {
+        setSchemaType((SchemaType)value);
+      }
+      break;
+
+    case NAME:
+      if (value == null) {
+        unsetName();
+      } else {
+        setName((String)value);
+      }
+      break;
+
+    case DB_NAME:
+      if (value == null) {
+        unsetDbName();
+      } else {
+        setDbName((String)value);
+      }
+      break;
+
+    case COMPATIBILITY:
+      if (value == null) {
+        unsetCompatibility();
+      } else {
+        setCompatibility((SchemaCompatibility)value);
+      }
+      break;
+
+    case VALIDATION_LEVEL:
+      if (value == null) {
+        unsetValidationLevel();
+      } else {
+        setValidationLevel((SchemaValidation)value);
+      }
+      break;
+
+    case CAN_EVOLVE:
+      if (value == null) {
+        unsetCanEvolve();
+      } else {
+        setCanEvolve((Boolean)value);
+      }
+      break;
+
+    case SCHEMA_GROUP:
+      if (value == null) {
+        unsetSchemaGroup();
+      } else {
+        setSchemaGroup((String)value);
+      }
+      break;
+
+    case DESCRIPTION:
+      if (value == null) {
+        unsetDescription();
+      } else {
+        setDescription((String)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case SCHEMA_TYPE:
+      return getSchemaType();
+
+    case NAME:
+      return getName();
+
+    case DB_NAME:
+      return getDbName();
+
+    case COMPATIBILITY:
+      return getCompatibility();
+
+    case VALIDATION_LEVEL:
+      return getValidationLevel();
+
+    case CAN_EVOLVE:
+      return isCanEvolve();
+
+    case SCHEMA_GROUP:
+      return getSchemaGroup();
+
+    case DESCRIPTION:
+      return getDescription();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case SCHEMA_TYPE:
+      return isSetSchemaType();
+    case NAME:
+      return isSetName();
+    case DB_NAME:
+      return isSetDbName();
+    case COMPATIBILITY:
+      return isSetCompatibility();
+    case VALIDATION_LEVEL:
+      return isSetValidationLevel();
+    case CAN_EVOLVE:
+      return isSetCanEvolve();
+    case SCHEMA_GROUP:
+      return isSetSchemaGroup();
+    case DESCRIPTION:
+      return isSetDescription();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof ISchema)
+      return this.equals((ISchema)that);
+    return false;
+  }
+
+  public boolean equals(ISchema that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_schemaType = true && this.isSetSchemaType();
+    boolean that_present_schemaType = true && that.isSetSchemaType();
+    if (this_present_schemaType || that_present_schemaType) {
+      if (!(this_present_schemaType && that_present_schemaType))
+        return false;
+      if (!this.schemaType.equals(that.schemaType))
+        return false;
+    }
+
+    boolean this_present_name = true && this.isSetName();
+    boolean that_present_name = true && that.isSetName();
+    if (this_present_name || that_present_name) {
+      if (!(this_present_name && that_present_name))
+        return false;
+      if (!this.name.equals(that.name))
+        return false;
+    }
+
+    boolean this_present_dbName = true && this.isSetDbName();
+    boolean that_present_dbName = true && that.isSetDbName();
+    if (this_present_dbName || that_present_dbName) {
+      if (!(this_present_dbName && that_present_dbName))
+        return false;
+      if (!this.dbName.equals(that.dbName))
+        return false;
+    }
+
+    boolean this_present_compatibility = true && this.isSetCompatibility();
+    boolean that_present_compatibility = true && that.isSetCompatibility();
+    if (this_present_compatibility || that_present_compatibility) {
+      if (!(this_present_compatibility && that_present_compatibility))
+        return false;
+      if (!this.compatibility.equals(that.compatibility))
+        return false;
+    }
+
+    boolean this_present_validationLevel = true && this.isSetValidationLevel();
+    boolean that_present_validationLevel = true && that.isSetValidationLevel();
+    if (this_present_validationLevel || that_present_validationLevel) {
+      if (!(this_present_validationLevel && that_present_validationLevel))
+        return false;
+      if (!this.validationLevel.equals(that.validationLevel))
+        return false;
+    }
+
+    boolean this_present_canEvolve = true;
+    boolean that_present_canEvolve = true;
+    if (this_present_canEvolve || that_present_canEvolve) {
+      if (!(this_present_canEvolve && that_present_canEvolve))
+        return false;
+      if (this.canEvolve != that.canEvolve)
+        return false;
+    }
+
+    boolean this_present_schemaGroup = true && this.isSetSchemaGroup();
+    boolean that_present_schemaGroup = true && that.isSetSchemaGroup();
+    if (this_present_schemaGroup || that_present_schemaGroup) {
+      if (!(this_present_schemaGroup && that_present_schemaGroup))
+        return false;
+      if (!this.schemaGroup.equals(that.schemaGroup))
+        return false;
+    }
+
+    boolean this_present_description = true && this.isSetDescription();
+    boolean that_present_description = true && that.isSetDescription();
+    if (this_present_description || that_present_description) {
+      if (!(this_present_description && that_present_description))
+        return false;
+      if (!this.description.equals(that.description))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_schemaType = true && (isSetSchemaType());
+    list.add(present_schemaType);
+    if (present_schemaType)
+      list.add(schemaType.getValue());
+
+    boolean present_name = true && (isSetName());
+    list.add(present_name);
+    if (present_name)
+      list.add(name);
+
+    boolean present_dbName = true && (isSetDbName());
+    list.add(present_dbName);
+    if (present_dbName)
+      list.add(dbName);
+
+    boolean present_compatibility = true && (isSetCompatibility());
+    list.add(present_compatibility);
+    if (present_compatibility)
+      list.add(compatibility.getValue());
+
+    boolean present_validationLevel = true && (isSetValidationLevel());
+    list.add(present_validationLevel);
+    if (present_validationLevel)
+      list.add(validationLevel.getValue());
+
+    boolean present_canEvolve = true;
+    list.add(present_canEvolve);
+    if (present_canEvolve)
+      list.add(canEvolve);
+
+    boolean present_schemaGroup = true && (isSetSchemaGroup());
+    list.add(present_schemaGroup);
+    if (present_schemaGroup)
+      list.add(schemaGroup);
+
+    boolean present_description = true && (isSetDescription());
+    list.add(present_description);
+    if (present_description)
+      list.add(description);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(ISchema other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetSchemaType()).compareTo(other.isSetSchemaType());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetSchemaType()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.schemaType, other.schemaType);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetName()).compareTo(other.isSetName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.name, other.name);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetDbName()).compareTo(other.isSetDbName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetDbName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.dbName, other.dbName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetCompatibility()).compareTo(other.isSetCompatibility());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCompatibility()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.compatibility, other.compatibility);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetValidationLevel()).compareTo(other.isSetValidationLevel());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetValidationLevel()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.validationLevel, other.validationLevel);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetCanEvolve()).compareTo(other.isSetCanEvolve());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCanEvolve()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.canEvolve, other.canEvolve);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetSchemaGroup()).compareTo(other.isSetSchemaGroup());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetSchemaGroup()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.schemaGroup, other.schemaGroup);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetDescription()).compareTo(other.isSetDescription());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetDescription()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.description, other.description);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("ISchema(");
+    boolean first = true;
+
+    sb.append("schemaType:");
+    if (this.schemaType == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.schemaType);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("name:");
+    if (this.name == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.name);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("dbName:");
+    if (this.dbName == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.dbName);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("compatibility:");
+    if (this.compatibility == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.compatibility);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("validationLevel:");
+    if (this.validationLevel == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.validationLevel);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("canEvolve:");
+    sb.append(this.canEvolve);
+    first = false;
+    if (isSetSchemaGroup()) {
+      if (!first) sb.append(", ");
+      sb.append("schemaGroup:");
+      if (this.schemaGroup == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.schemaGroup);
+      }
+      first = false;
+    }
+    if (isSetDescription()) {
+      if (!first) sb.append(", ");
+      sb.append("description:");
+      if (this.description == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.description);
+      }
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class ISchemaStandardSchemeFactory implements SchemeFactory {
+    public ISchemaStandardScheme getScheme() {
+      return new ISchemaStandardScheme();
+    }
+  }
+
+  private static class ISchemaStandardScheme extends StandardScheme<ISchema> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, ISchema struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // SCHEMA_TYPE
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.schemaType = org.apache.hadoop.hive.metastore.api.SchemaType.findByValue(iprot.readI32());
+              struct.setSchemaTypeIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.name = iprot.readString();
+              struct.setNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // DB_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.dbName = iprot.readString();
+              struct.setDbNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // COMPATIBILITY
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.compatibility = org.apache.hadoop.hive.metastore.api.SchemaCompatibility.findByValue(iprot.readI32());
+              struct.setCompatibilityIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 5: // VALIDATION_LEVEL
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.validationLevel = org.apache.hadoop.hive.metastore.api.SchemaValidation.findByValue(iprot.readI32());
+              struct.setValidationLevelIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 6: // CAN_EVOLVE
+            if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+              struct.canEvolve = iprot.readBool();
+              struct.setCanEvolveIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 7: // SCHEMA_GROUP
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.schemaGroup = iprot.readString();
+              struct.setSchemaGroupIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 8: // DESCRIPTION
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.description = iprot.readString();
+              struct.setDescriptionIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, ISchema struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.schemaType != null) {
+        oprot.writeFieldBegin(SCHEMA_TYPE_FIELD_DESC);
+        oprot.writeI32(struct.schemaType.getValue());
+        oprot.writeFieldEnd();
+      }
+      if (struct.name != null) {
+        oprot.writeFieldBegin(NAME_FIELD_DESC);
+        oprot.writeString(struct.name);
+        oprot.writeFieldEnd();
+      }
+      if (struct.dbName != null) {
+        oprot.writeFieldBegin(DB_NAME_FIELD_DESC);
+        oprot.writeString(struct.dbName);
+        oprot.writeFieldEnd();
+      }
+      if (struct.compatibility != null) {
+        oprot.writeFieldBegin(COMPATIBILITY_FIELD_DESC);
+        oprot.writeI32(struct.compatibility.getValue());
+        oprot.writeFieldEnd();
+      }
+      if (struct.validationLevel != null) {
+        oprot.writeFieldBegin(VALIDATION_LEVEL_FIELD_DESC);
+        oprot.writeI32(struct.validationLevel.getValue());
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldBegin(CAN_EVOLVE_FIELD_DESC);
+      oprot.writeBool(struct.canEvolve);
+      oprot.writeFieldEnd();
+      if (struct.schemaGroup != null) {
+        if (struct.isSetSchemaGroup()) {
+          oprot.writeFieldBegin(SCHEMA_GROUP_FIELD_DESC);
+          oprot.writeString(struct.schemaGroup);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.description != null) {
+        if (struct.isSetDescription()) {
+          oprot.writeFieldBegin(DESCRIPTION_FIELD_DESC);
+          oprot.writeString(struct.description);
+          oprot.writeFieldEnd();
+        }
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class ISchemaTupleSchemeFactory implements SchemeFactory {
+    public ISchemaTupleScheme getScheme() {
+      return new ISchemaTupleScheme();
+    }
+  }
+
+  private static class ISchemaTupleScheme extends TupleScheme<ISchema> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, ISchema struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.isSetSchemaType()) {
+        optionals.set(0);
+      }
+      if (struct.isSetName()) {
+        optionals.set(1);
+      }
+      if (struct.isSetDbName()) {
+        optionals.set(2);
+      }
+      if (struct.isSetCompatibility()) {
+        optionals.set(3);
+      }
+      if (struct.isSetValidationLevel()) {
+        optionals.set(4);
+      }
+      if (struct.isSetCanEvolve()) {
+        optionals.set(5);
+      }
+      if (struct.isSetSchemaGroup()) {
+        optionals.set(6);
+      }
+      if (struct.isSetDescription()) {
+        optionals.set(7);
+      }
+      oprot.writeBitSet(optionals, 8);
+      if (struct.isSetSchemaType()) {
+        oprot.writeI32(struct.schemaType.getValue());
+      }
+      if (struct.isSetName()) {
+        oprot.writeString(struct.name);
+      }
+      if (struct.isSetDbName()) {
+        oprot.writeString(struct.dbName);
+      }
+      if (struct.isSetCompatibility()) {
+        oprot.writeI32(struct.compatibility.getValue());
+      }
+      if (struct.isSetValidationLevel()) {
+        oprot.writeI32(struct.validationLevel.getValue());
+      }
+      if (struct.isSetCanEvolve()) {
+        oprot.writeBool(struct.canEvolve);
+      }
+      if (struct.isSetSchemaGroup()) {
+        oprot.writeString(struct.schemaGroup);
+      }
+      if (struct.isSetDescription()) {
+        oprot.writeString(struct.description);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, ISchema struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(8);
+      if (incoming.get(0)) {
+        struct.schemaType = org.apache.hadoop.hive.metastore.api.SchemaType.findByValue(iprot.readI32());
+        struct.setSchemaTypeIsSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.name = iprot.readString();
+        struct.setNameIsSet(true);
+      }
+      if (incoming.get(2)) {
+        struct.dbName = iprot.readString();
+        struct.setDbNameIsSet(true);
+      }
+      if (incoming.get(3)) {
+        struct.compatibility = org.apache.hadoop.hive.metastore.api.SchemaCompatibility.findByValue(iprot.readI32());
+        struct.setCompatibilityIsSet(true);
+      }
+      if (incoming.get(4)) {
+        struct.validationLevel = org.apache.hadoop.hive.metastore.api.SchemaValidation.findByValue(iprot.readI32());
+        struct.setValidationLevelIsSet(true);
+      }
+      if (incoming.get(5)) {
+        struct.canEvolve = iprot.readBool();
+        struct.setCanEvolveIsSet(true);
+      }
+      if (incoming.get(6)) {
+        struct.schemaGroup = iprot.readString();
+        struct.setSchemaGroupIsSet(true);
+      }
+      if (incoming.get(7)) {
+        struct.description = iprot.readString();
+        struct.setDescriptionIsSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ISchemaName.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ISchemaName.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ISchemaName.java
new file mode 100644
index 0000000..ad25051
--- /dev/null
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ISchemaName.java
@@ -0,0 +1,499 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hive.metastore.api;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+@org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public class ISchemaName implements org.apache.thrift.TBase<ISchemaName, ISchemaName._Fields>, java.io.Serializable, Cloneable, Comparable<ISchemaName> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ISchemaName");
+
+  private static final org.apache.thrift.protocol.TField DB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("dbName", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField SCHEMA_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("schemaName", org.apache.thrift.protocol.TType.STRING, (short)2);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new ISchemaNameStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new ISchemaNameTupleSchemeFactory());
+  }
+
+  private String dbName; // required
+  private String schemaName; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    DB_NAME((short)1, "dbName"),
+    SCHEMA_NAME((short)2, "schemaName");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // DB_NAME
+          return DB_NAME;
+        case 2: // SCHEMA_NAME
+          return SCHEMA_NAME;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.DB_NAME, new org.apache.thrift.meta_data.FieldMetaData("dbName", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.SCHEMA_NAME, new org.apache.thrift.meta_data.FieldMetaData("schemaName", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ISchemaName.class, metaDataMap);
+  }
+
+  public ISchemaName() {
+  }
+
+  public ISchemaName(
+    String dbName,
+    String schemaName)
+  {
+    this();
+    this.dbName = dbName;
+    this.schemaName = schemaName;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public ISchemaName(ISchemaName other) {
+    if (other.isSetDbName()) {
+      this.dbName = other.dbName;
+    }
+    if (other.isSetSchemaName()) {
+      this.schemaName = other.schemaName;
+    }
+  }
+
+  public ISchemaName deepCopy() {
+    return new ISchemaName(this);
+  }
+
+  @Override
+  public void clear() {
+    this.dbName = null;
+    this.schemaName = null;
+  }
+
+  public String getDbName() {
+    return this.dbName;
+  }
+
+  public void setDbName(String dbName) {
+    this.dbName = dbName;
+  }
+
+  public void unsetDbName() {
+    this.dbName = null;
+  }
+
+  /** Returns true if field dbName is set (has been assigned a value) and false otherwise */
+  public boolean isSetDbName() {
+    return this.dbName != null;
+  }
+
+  public void setDbNameIsSet(boolean value) {
+    if (!value) {
+      this.dbName = null;
+    }
+  }
+
+  public String getSchemaName() {
+    return this.schemaName;
+  }
+
+  public void setSchemaName(String schemaName) {
+    this.schemaName = schemaName;
+  }
+
+  public void unsetSchemaName() {
+    this.schemaName = null;
+  }
+
+  /** Returns true if field schemaName is set (has been assigned a value) and false otherwise */
+  public boolean isSetSchemaName() {
+    return this.schemaName != null;
+  }
+
+  public void setSchemaNameIsSet(boolean value) {
+    if (!value) {
+      this.schemaName = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case DB_NAME:
+      if (value == null) {
+        unsetDbName();
+      } else {
+        setDbName((String)value);
+      }
+      break;
+
+    case SCHEMA_NAME:
+      if (value == null) {
+        unsetSchemaName();
+      } else {
+        setSchemaName((String)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case DB_NAME:
+      return getDbName();
+
+    case SCHEMA_NAME:
+      return getSchemaName();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case DB_NAME:
+      return isSetDbName();
+    case SCHEMA_NAME:
+      return isSetSchemaName();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof ISchemaName)
+      return this.equals((ISchemaName)that);
+    return false;
+  }
+
+  public boolean equals(ISchemaName that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_dbName = true && this.isSetDbName();
+    boolean that_present_dbName = true && that.isSetDbName();
+    if (this_present_dbName || that_present_dbName) {
+      if (!(this_present_dbName && that_present_dbName))
+        return false;
+      if (!this.dbName.equals(that.dbName))
+        return false;
+    }
+
+    boolean this_present_schemaName = true && this.isSetSchemaName();
+    boolean that_present_schemaName = true && that.isSetSchemaName();
+    if (this_present_schemaName || that_present_schemaName) {
+      if (!(this_present_schemaName && that_present_schemaName))
+        return false;
+      if (!this.schemaName.equals(that.schemaName))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_dbName = true && (isSetDbName());
+    list.add(present_dbName);
+    if (present_dbName)
+      list.add(dbName);
+
+    boolean present_schemaName = true && (isSetSchemaName());
+    list.add(present_schemaName);
+    if (present_schemaName)
+      list.add(schemaName);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(ISchemaName other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetDbName()).compareTo(other.isSetDbName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetDbName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.dbName, other.dbName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetSchemaName()).compareTo(other.isSetSchemaName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetSchemaName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.schemaName, other.schemaName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("ISchemaName(");
+    boolean first = true;
+
+    sb.append("dbName:");
+    if (this.dbName == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.dbName);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("schemaName:");
+    if (this.schemaName == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.schemaName);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class ISchemaNameStandardSchemeFactory implements SchemeFactory {
+    public ISchemaNameStandardScheme getScheme() {
+      return new ISchemaNameStandardScheme();
+    }
+  }
+
+  private static class ISchemaNameStandardScheme extends StandardScheme<ISchemaName> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, ISchemaName struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // DB_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.dbName = iprot.readString();
+              struct.setDbNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // SCHEMA_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.schemaName = iprot.readString();
+              struct.setSchemaNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, ISchemaName struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.dbName != null) {
+        oprot.writeFieldBegin(DB_NAME_FIELD_DESC);
+        oprot.writeString(struct.dbName);
+        oprot.writeFieldEnd();
+      }
+      if (struct.schemaName != null) {
+        oprot.writeFieldBegin(SCHEMA_NAME_FIELD_DESC);
+        oprot.writeString(struct.schemaName);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class ISchemaNameTupleSchemeFactory implements SchemeFactory {
+    public ISchemaNameTupleScheme getScheme() {
+      return new ISchemaNameTupleScheme();
+    }
+  }
+
+  private static class ISchemaNameTupleScheme extends TupleScheme<ISchemaName> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, ISchemaName struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.isSetDbName()) {
+        optionals.set(0);
+      }
+      if (struct.isSetSchemaName()) {
+        optionals.set(1);
+      }
+      oprot.writeBitSet(optionals, 2);
+      if (struct.isSetDbName()) {
+        oprot.writeString(struct.dbName);
+      }
+      if (struct.isSetSchemaName()) {
+        oprot.writeString(struct.schemaName);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, ISchemaName struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(2);
+      if (incoming.get(0)) {
+        struct.dbName = iprot.readString();
+        struct.setDbNameIsSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.schemaName = iprot.readString();
+        struct.setSchemaNameIsSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/MapSchemaVersionToSerdeRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/MapSchemaVersionToSerdeRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/MapSchemaVersionToSerdeRequest.java
new file mode 100644
index 0000000..81eb732
--- /dev/null
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/MapSchemaVersionToSerdeRequest.java
@@ -0,0 +1,504 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hive.metastore.api;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+@org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public class MapSchemaVersionToSerdeRequest implements org.apache.thrift.TBase<MapSchemaVersionToSerdeRequest, MapSchemaVersionToSerdeRequest._Fields>, java.io.Serializable, Cloneable, Comparable<MapSchemaVersionToSerdeRequest> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("MapSchemaVersionToSerdeRequest");
+
+  private static final org.apache.thrift.protocol.TField SCHEMA_VERSION_FIELD_DESC = new org.apache.thrift.protocol.TField("schemaVersion", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+  private static final org.apache.thrift.protocol.TField SERDE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("serdeName", org.apache.thrift.protocol.TType.STRING, (short)2);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new MapSchemaVersionToSerdeRequestStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new MapSchemaVersionToSerdeRequestTupleSchemeFactory());
+  }
+
+  private SchemaVersionDescriptor schemaVersion; // required
+  private String serdeName; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    SCHEMA_VERSION((short)1, "schemaVersion"),
+    SERDE_NAME((short)2, "serdeName");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // SCHEMA_VERSION
+          return SCHEMA_VERSION;
+        case 2: // SERDE_NAME
+          return SERDE_NAME;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.SCHEMA_VERSION, new org.apache.thrift.meta_data.FieldMetaData("schemaVersion", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, SchemaVersionDescriptor.class)));
+    tmpMap.put(_Fields.SERDE_NAME, new org.apache.thrift.meta_data.FieldMetaData("serdeName", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(MapSchemaVersionToSerdeRequest.class, metaDataMap);
+  }
+
+  public MapSchemaVersionToSerdeRequest() {
+  }
+
+  public MapSchemaVersionToSerdeRequest(
+    SchemaVersionDescriptor schemaVersion,
+    String serdeName)
+  {
+    this();
+    this.schemaVersion = schemaVersion;
+    this.serdeName = serdeName;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public MapSchemaVersionToSerdeRequest(MapSchemaVersionToSerdeRequest other) {
+    if (other.isSetSchemaVersion()) {
+      this.schemaVersion = new SchemaVersionDescriptor(other.schemaVersion);
+    }
+    if (other.isSetSerdeName()) {
+      this.serdeName = other.serdeName;
+    }
+  }
+
+  public MapSchemaVersionToSerdeRequest deepCopy() {
+    return new MapSchemaVersionToSerdeRequest(this);
+  }
+
+  @Override
+  public void clear() {
+    this.schemaVersion = null;
+    this.serdeName = null;
+  }
+
+  public SchemaVersionDescriptor getSchemaVersion() {
+    return this.schemaVersion;
+  }
+
+  public void setSchemaVersion(SchemaVersionDescriptor schemaVersion) {
+    this.schemaVersion = schemaVersion;
+  }
+
+  public void unsetSchemaVersion() {
+    this.schemaVersion = null;
+  }
+
+  /** Returns true if field schemaVersion is set (has been assigned a value) and false otherwise */
+  public boolean isSetSchemaVersion() {
+    return this.schemaVersion != null;
+  }
+
+  public void setSchemaVersionIsSet(boolean value) {
+    if (!value) {
+      this.schemaVersion = null;
+    }
+  }
+
+  public String getSerdeName() {
+    return this.serdeName;
+  }
+
+  public void setSerdeName(String serdeName) {
+    this.serdeName = serdeName;
+  }
+
+  public void unsetSerdeName() {
+    this.serdeName = null;
+  }
+
+  /** Returns true if field serdeName is set (has been assigned a value) and false otherwise */
+  public boolean isSetSerdeName() {
+    return this.serdeName != null;
+  }
+
+  public void setSerdeNameIsSet(boolean value) {
+    if (!value) {
+      this.serdeName = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case SCHEMA_VERSION:
+      if (value == null) {
+        unsetSchemaVersion();
+      } else {
+        setSchemaVersion((SchemaVersionDescriptor)value);
+      }
+      break;
+
+    case SERDE_NAME:
+      if (value == null) {
+        unsetSerdeName();
+      } else {
+        setSerdeName((String)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case SCHEMA_VERSION:
+      return getSchemaVersion();
+
+    case SERDE_NAME:
+      return getSerdeName();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case SCHEMA_VERSION:
+      return isSetSchemaVersion();
+    case SERDE_NAME:
+      return isSetSerdeName();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof MapSchemaVersionToSerdeRequest)
+      return this.equals((MapSchemaVersionToSerdeRequest)that);
+    return false;
+  }
+
+  public boolean equals(MapSchemaVersionToSerdeRequest that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_schemaVersion = true && this.isSetSchemaVersion();
+    boolean that_present_schemaVersion = true && that.isSetSchemaVersion();
+    if (this_present_schemaVersion || that_present_schemaVersion) {
+      if (!(this_present_schemaVersion && that_present_schemaVersion))
+        return false;
+      if (!this.schemaVersion.equals(that.schemaVersion))
+        return false;
+    }
+
+    boolean this_present_serdeName = true && this.isSetSerdeName();
+    boolean that_present_serdeName = true && that.isSetSerdeName();
+    if (this_present_serdeName || that_present_serdeName) {
+      if (!(this_present_serdeName && that_present_serdeName))
+        return false;
+      if (!this.serdeName.equals(that.serdeName))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_schemaVersion = true && (isSetSchemaVersion());
+    list.add(present_schemaVersion);
+    if (present_schemaVersion)
+      list.add(schemaVersion);
+
+    boolean present_serdeName = true && (isSetSerdeName());
+    list.add(present_serdeName);
+    if (present_serdeName)
+      list.add(serdeName);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(MapSchemaVersionToSerdeRequest other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetSchemaVersion()).compareTo(other.isSetSchemaVersion());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetSchemaVersion()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.schemaVersion, other.schemaVersion);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetSerdeName()).compareTo(other.isSetSerdeName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetSerdeName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.serdeName, other.serdeName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("MapSchemaVersionToSerdeRequest(");
+    boolean first = true;
+
+    sb.append("schemaVersion:");
+    if (this.schemaVersion == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.schemaVersion);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("serdeName:");
+    if (this.serdeName == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.serdeName);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+    if (schemaVersion != null) {
+      schemaVersion.validate();
+    }
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class MapSchemaVersionToSerdeRequestStandardSchemeFactory implements SchemeFactory {
+    public MapSchemaVersionToSerdeRequestStandardScheme getScheme() {
+      return new MapSchemaVersionToSerdeRequestStandardScheme();
+    }
+  }
+
+  private static class MapSchemaVersionToSerdeRequestStandardScheme extends StandardScheme<MapSchemaVersionToSerdeRequest> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, MapSchemaVersionToSerdeRequest struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // SCHEMA_VERSION
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.schemaVersion = new SchemaVersionDescriptor();
+              struct.schemaVersion.read(iprot);
+              struct.setSchemaVersionIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // SERDE_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.serdeName = iprot.readString();
+              struct.setSerdeNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, MapSchemaVersionToSerdeRequest struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.schemaVersion != null) {
+        oprot.writeFieldBegin(SCHEMA_VERSION_FIELD_DESC);
+        struct.schemaVersion.write(oprot);
+        oprot.writeFieldEnd();
+      }
+      if (struct.serdeName != null) {
+        oprot.writeFieldBegin(SERDE_NAME_FIELD_DESC);
+        oprot.writeString(struct.serdeName);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class MapSchemaVersionToSerdeRequestTupleSchemeFactory implements SchemeFactory {
+    public MapSchemaVersionToSerdeRequestTupleScheme getScheme() {
+      return new MapSchemaVersionToSerdeRequestTupleScheme();
+    }
+  }
+
+  private static class MapSchemaVersionToSerdeRequestTupleScheme extends TupleScheme<MapSchemaVersionToSerdeRequest> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, MapSchemaVersionToSerdeRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.isSetSchemaVersion()) {
+        optionals.set(0);
+      }
+      if (struct.isSetSerdeName()) {
+        optionals.set(1);
+      }
+      oprot.writeBitSet(optionals, 2);
+      if (struct.isSetSchemaVersion()) {
+        struct.schemaVersion.write(oprot);
+      }
+      if (struct.isSetSerdeName()) {
+        oprot.writeString(struct.serdeName);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, MapSchemaVersionToSerdeRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(2);
+      if (incoming.get(0)) {
+        struct.schemaVersion = new SchemaVersionDescriptor();
+        struct.schemaVersion.read(iprot);
+        struct.setSchemaVersionIsSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.serdeName = iprot.readString();
+        struct.setSerdeNameIsSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaCompatibility.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaCompatibility.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaCompatibility.java
new file mode 100644
index 0000000..4c0bb5a
--- /dev/null
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaCompatibility.java
@@ -0,0 +1,51 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hive.metastore.api;
+
+
+import java.util.Map;
+import java.util.HashMap;
+import org.apache.thrift.TEnum;
+
+public enum SchemaCompatibility implements org.apache.thrift.TEnum {
+  NONE(1),
+  BACKWARD(2),
+  FORWARD(3),
+  BOTH(4);
+
+  private final int value;
+
+  private SchemaCompatibility(int value) {
+    this.value = value;
+  }
+
+  /**
+   * Get the integer value of this enum value, as defined in the Thrift IDL.
+   */
+  public int getValue() {
+    return value;
+  }
+
+  /**
+   * Find a the enum type by its integer value, as defined in the Thrift IDL.
+   * @return null if the value is not found.
+   */
+  public static SchemaCompatibility findByValue(int value) { 
+    switch (value) {
+      case 1:
+        return NONE;
+      case 2:
+        return BACKWARD;
+      case 3:
+        return FORWARD;
+      case 4:
+        return BOTH;
+      default:
+        return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaType.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaType.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaType.java
new file mode 100644
index 0000000..f4adeb9
--- /dev/null
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaType.java
@@ -0,0 +1,45 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hive.metastore.api;
+
+
+import java.util.Map;
+import java.util.HashMap;
+import org.apache.thrift.TEnum;
+
+public enum SchemaType implements org.apache.thrift.TEnum {
+  HIVE(1),
+  AVRO(2);
+
+  private final int value;
+
+  private SchemaType(int value) {
+    this.value = value;
+  }
+
+  /**
+   * Get the integer value of this enum value, as defined in the Thrift IDL.
+   */
+  public int getValue() {
+    return value;
+  }
+
+  /**
+   * Find a the enum type by its integer value, as defined in the Thrift IDL.
+   * @return null if the value is not found.
+   */
+  public static SchemaType findByValue(int value) { 
+    switch (value) {
+      case 1:
+        return HIVE;
+      case 2:
+        return AVRO;
+      default:
+        return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaValidation.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaValidation.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaValidation.java
new file mode 100644
index 0000000..1983f56
--- /dev/null
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaValidation.java
@@ -0,0 +1,45 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hive.metastore.api;
+
+
+import java.util.Map;
+import java.util.HashMap;
+import org.apache.thrift.TEnum;
+
+public enum SchemaValidation implements org.apache.thrift.TEnum {
+  LATEST(1),
+  ALL(2);
+
+  private final int value;
+
+  private SchemaValidation(int value) {
+    this.value = value;
+  }
+
+  /**
+   * Get the integer value of this enum value, as defined in the Thrift IDL.
+   */
+  public int getValue() {
+    return value;
+  }
+
+  /**
+   * Find a the enum type by its integer value, as defined in the Thrift IDL.
+   * @return null if the value is not found.
+   */
+  public static SchemaValidation findByValue(int value) { 
+    switch (value) {
+      case 1:
+        return LATEST;
+      case 2:
+        return ALL;
+      default:
+        return null;
+    }
+  }
+}


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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php b/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php
index 0b11532..3be16de 100644
--- a/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php
+++ b/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php
@@ -142,6 +142,67 @@ final class EventRequestType {
   );
 }
 
+final class SerdeType {
+  const HIVE = 1;
+  const SCHEMA_REGISTRY = 2;
+  static public $__names = array(
+    1 => 'HIVE',
+    2 => 'SCHEMA_REGISTRY',
+  );
+}
+
+final class SchemaType {
+  const HIVE = 1;
+  const AVRO = 2;
+  static public $__names = array(
+    1 => 'HIVE',
+    2 => 'AVRO',
+  );
+}
+
+final class SchemaCompatibility {
+  const NONE = 1;
+  const BACKWARD = 2;
+  const FORWARD = 3;
+  const BOTH = 4;
+  static public $__names = array(
+    1 => 'NONE',
+    2 => 'BACKWARD',
+    3 => 'FORWARD',
+    4 => 'BOTH',
+  );
+}
+
+final class SchemaValidation {
+  const LATEST = 1;
+  const ALL = 2;
+  static public $__names = array(
+    1 => 'LATEST',
+    2 => 'ALL',
+  );
+}
+
+final class SchemaVersionState {
+  const INITIATED = 1;
+  const START_REVIEW = 2;
+  const CHANGES_REQUIRED = 3;
+  const REVIEWED = 4;
+  const ENABLED = 5;
+  const DISABLED = 6;
+  const ARCHIVED = 7;
+  const DELETED = 8;
+  static public $__names = array(
+    1 => 'INITIATED',
+    2 => 'START_REVIEW',
+    3 => 'CHANGES_REQUIRED',
+    4 => 'REVIEWED',
+    5 => 'ENABLED',
+    6 => 'DISABLED',
+    7 => 'ARCHIVED',
+    8 => 'DELETED',
+  );
+}
+
 final class FunctionType {
   const JAVA = 1;
   static public $__names = array(
@@ -4266,6 +4327,22 @@ class SerDeInfo {
    * @var array
    */
   public $parameters = null;
+  /**
+   * @var string
+   */
+  public $description = null;
+  /**
+   * @var string
+   */
+  public $serializerClass = null;
+  /**
+   * @var string
+   */
+  public $deserializerClass = null;
+  /**
+   * @var int
+   */
+  public $serdeType = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
@@ -4290,6 +4367,22 @@ class SerDeInfo {
             'type' => TType::STRING,
             ),
           ),
+        4 => array(
+          'var' => 'description',
+          'type' => TType::STRING,
+          ),
+        5 => array(
+          'var' => 'serializerClass',
+          'type' => TType::STRING,
+          ),
+        6 => array(
+          'var' => 'deserializerClass',
+          'type' => TType::STRING,
+          ),
+        7 => array(
+          'var' => 'serdeType',
+          'type' => TType::I32,
+          ),
         );
     }
     if (is_array($vals)) {
@@ -4302,6 +4395,18 @@ class SerDeInfo {
       if (isset($vals['parameters'])) {
         $this->parameters = $vals['parameters'];
       }
+      if (isset($vals['description'])) {
+        $this->description = $vals['description'];
+      }
+      if (isset($vals['serializerClass'])) {
+        $this->serializerClass = $vals['serializerClass'];
+      }
+      if (isset($vals['deserializerClass'])) {
+        $this->deserializerClass = $vals['deserializerClass'];
+      }
+      if (isset($vals['serdeType'])) {
+        $this->serdeType = $vals['serdeType'];
+      }
     }
   }
 
@@ -4358,6 +4463,34 @@ class SerDeInfo {
             $xfer += $input->skip($ftype);
           }
           break;
+        case 4:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->description);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 5:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->serializerClass);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 6:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->deserializerClass);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 7:
+          if ($ftype == TType::I32) {
+            $xfer += $input->readI32($this->serdeType);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
         default:
           $xfer += $input->skip($ftype);
           break;
@@ -4399,6 +4532,26 @@ class SerDeInfo {
       }
       $xfer += $output->writeFieldEnd();
     }
+    if ($this->description !== null) {
+      $xfer += $output->writeFieldBegin('description', TType::STRING, 4);
+      $xfer += $output->writeString($this->description);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->serializerClass !== null) {
+      $xfer += $output->writeFieldBegin('serializerClass', TType::STRING, 5);
+      $xfer += $output->writeString($this->serializerClass);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->deserializerClass !== null) {
+      $xfer += $output->writeFieldBegin('deserializerClass', TType::STRING, 6);
+      $xfer += $output->writeString($this->deserializerClass);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->serdeType !== null) {
+      $xfer += $output->writeFieldBegin('serdeType', TType::I32, 7);
+      $xfer += $output->writeI32($this->serdeType);
+      $xfer += $output->writeFieldEnd();
+    }
     $xfer += $output->writeFieldStop();
     $xfer += $output->writeStructEnd();
     return $xfer;
@@ -26107,6 +26260,1376 @@ class WMCreateOrDropTriggerToPoolMappingResponse {
 
 }
 
+class ISchema {
+  static $_TSPEC;
+
+  /**
+   * @var int
+   */
+  public $schemaType = null;
+  /**
+   * @var string
+   */
+  public $name = null;
+  /**
+   * @var string
+   */
+  public $dbName = null;
+  /**
+   * @var int
+   */
+  public $compatibility = null;
+  /**
+   * @var int
+   */
+  public $validationLevel = null;
+  /**
+   * @var bool
+   */
+  public $canEvolve = null;
+  /**
+   * @var string
+   */
+  public $schemaGroup = null;
+  /**
+   * @var string
+   */
+  public $description = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'schemaType',
+          'type' => TType::I32,
+          ),
+        2 => array(
+          'var' => 'name',
+          'type' => TType::STRING,
+          ),
+        3 => array(
+          'var' => 'dbName',
+          'type' => TType::STRING,
+          ),
+        4 => array(
+          'var' => 'compatibility',
+          'type' => TType::I32,
+          ),
+        5 => array(
+          'var' => 'validationLevel',
+          'type' => TType::I32,
+          ),
+        6 => array(
+          'var' => 'canEvolve',
+          'type' => TType::BOOL,
+          ),
+        7 => array(
+          'var' => 'schemaGroup',
+          'type' => TType::STRING,
+          ),
+        8 => array(
+          'var' => 'description',
+          'type' => TType::STRING,
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['schemaType'])) {
+        $this->schemaType = $vals['schemaType'];
+      }
+      if (isset($vals['name'])) {
+        $this->name = $vals['name'];
+      }
+      if (isset($vals['dbName'])) {
+        $this->dbName = $vals['dbName'];
+      }
+      if (isset($vals['compatibility'])) {
+        $this->compatibility = $vals['compatibility'];
+      }
+      if (isset($vals['validationLevel'])) {
+        $this->validationLevel = $vals['validationLevel'];
+      }
+      if (isset($vals['canEvolve'])) {
+        $this->canEvolve = $vals['canEvolve'];
+      }
+      if (isset($vals['schemaGroup'])) {
+        $this->schemaGroup = $vals['schemaGroup'];
+      }
+      if (isset($vals['description'])) {
+        $this->description = $vals['description'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ISchema';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::I32) {
+            $xfer += $input->readI32($this->schemaType);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->name);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 3:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->dbName);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 4:
+          if ($ftype == TType::I32) {
+            $xfer += $input->readI32($this->compatibility);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 5:
+          if ($ftype == TType::I32) {
+            $xfer += $input->readI32($this->validationLevel);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 6:
+          if ($ftype == TType::BOOL) {
+            $xfer += $input->readBool($this->canEvolve);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 7:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->schemaGroup);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 8:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->description);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('ISchema');
+    if ($this->schemaType !== null) {
+      $xfer += $output->writeFieldBegin('schemaType', TType::I32, 1);
+      $xfer += $output->writeI32($this->schemaType);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->name !== null) {
+      $xfer += $output->writeFieldBegin('name', TType::STRING, 2);
+      $xfer += $output->writeString($this->name);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->dbName !== null) {
+      $xfer += $output->writeFieldBegin('dbName', TType::STRING, 3);
+      $xfer += $output->writeString($this->dbName);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->compatibility !== null) {
+      $xfer += $output->writeFieldBegin('compatibility', TType::I32, 4);
+      $xfer += $output->writeI32($this->compatibility);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->validationLevel !== null) {
+      $xfer += $output->writeFieldBegin('validationLevel', TType::I32, 5);
+      $xfer += $output->writeI32($this->validationLevel);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->canEvolve !== null) {
+      $xfer += $output->writeFieldBegin('canEvolve', TType::BOOL, 6);
+      $xfer += $output->writeBool($this->canEvolve);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->schemaGroup !== null) {
+      $xfer += $output->writeFieldBegin('schemaGroup', TType::STRING, 7);
+      $xfer += $output->writeString($this->schemaGroup);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->description !== null) {
+      $xfer += $output->writeFieldBegin('description', TType::STRING, 8);
+      $xfer += $output->writeString($this->description);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class ISchemaName {
+  static $_TSPEC;
+
+  /**
+   * @var string
+   */
+  public $dbName = null;
+  /**
+   * @var string
+   */
+  public $schemaName = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'dbName',
+          'type' => TType::STRING,
+          ),
+        2 => array(
+          'var' => 'schemaName',
+          'type' => TType::STRING,
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['dbName'])) {
+        $this->dbName = $vals['dbName'];
+      }
+      if (isset($vals['schemaName'])) {
+        $this->schemaName = $vals['schemaName'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ISchemaName';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->dbName);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->schemaName);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('ISchemaName');
+    if ($this->dbName !== null) {
+      $xfer += $output->writeFieldBegin('dbName', TType::STRING, 1);
+      $xfer += $output->writeString($this->dbName);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->schemaName !== null) {
+      $xfer += $output->writeFieldBegin('schemaName', TType::STRING, 2);
+      $xfer += $output->writeString($this->schemaName);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class AlterISchemaRequest {
+  static $_TSPEC;
+
+  /**
+   * @var \metastore\ISchemaName
+   */
+  public $name = null;
+  /**
+   * @var \metastore\ISchema
+   */
+  public $newSchema = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'name',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\ISchemaName',
+          ),
+        3 => array(
+          'var' => 'newSchema',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\ISchema',
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['name'])) {
+        $this->name = $vals['name'];
+      }
+      if (isset($vals['newSchema'])) {
+        $this->newSchema = $vals['newSchema'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'AlterISchemaRequest';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRUCT) {
+            $this->name = new \metastore\ISchemaName();
+            $xfer += $this->name->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 3:
+          if ($ftype == TType::STRUCT) {
+            $this->newSchema = new \metastore\ISchema();
+            $xfer += $this->newSchema->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('AlterISchemaRequest');
+    if ($this->name !== null) {
+      if (!is_object($this->name)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('name', TType::STRUCT, 1);
+      $xfer += $this->name->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->newSchema !== null) {
+      if (!is_object($this->newSchema)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('newSchema', TType::STRUCT, 3);
+      $xfer += $this->newSchema->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class SchemaVersion {
+  static $_TSPEC;
+
+  /**
+   * @var \metastore\ISchemaName
+   */
+  public $schema = null;
+  /**
+   * @var int
+   */
+  public $version = null;
+  /**
+   * @var int
+   */
+  public $createdAt = null;
+  /**
+   * @var \metastore\FieldSchema[]
+   */
+  public $cols = null;
+  /**
+   * @var int
+   */
+  public $state = null;
+  /**
+   * @var string
+   */
+  public $description = null;
+  /**
+   * @var string
+   */
+  public $schemaText = null;
+  /**
+   * @var string
+   */
+  public $fingerprint = null;
+  /**
+   * @var string
+   */
+  public $name = null;
+  /**
+   * @var \metastore\SerDeInfo
+   */
+  public $serDe = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'schema',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\ISchemaName',
+          ),
+        2 => array(
+          'var' => 'version',
+          'type' => TType::I32,
+          ),
+        3 => array(
+          'var' => 'createdAt',
+          'type' => TType::I64,
+          ),
+        4 => array(
+          'var' => 'cols',
+          'type' => TType::LST,
+          'etype' => TType::STRUCT,
+          'elem' => array(
+            'type' => TType::STRUCT,
+            'class' => '\metastore\FieldSchema',
+            ),
+          ),
+        5 => array(
+          'var' => 'state',
+          'type' => TType::I32,
+          ),
+        6 => array(
+          'var' => 'description',
+          'type' => TType::STRING,
+          ),
+        7 => array(
+          'var' => 'schemaText',
+          'type' => TType::STRING,
+          ),
+        8 => array(
+          'var' => 'fingerprint',
+          'type' => TType::STRING,
+          ),
+        9 => array(
+          'var' => 'name',
+          'type' => TType::STRING,
+          ),
+        10 => array(
+          'var' => 'serDe',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\SerDeInfo',
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['schema'])) {
+        $this->schema = $vals['schema'];
+      }
+      if (isset($vals['version'])) {
+        $this->version = $vals['version'];
+      }
+      if (isset($vals['createdAt'])) {
+        $this->createdAt = $vals['createdAt'];
+      }
+      if (isset($vals['cols'])) {
+        $this->cols = $vals['cols'];
+      }
+      if (isset($vals['state'])) {
+        $this->state = $vals['state'];
+      }
+      if (isset($vals['description'])) {
+        $this->description = $vals['description'];
+      }
+      if (isset($vals['schemaText'])) {
+        $this->schemaText = $vals['schemaText'];
+      }
+      if (isset($vals['fingerprint'])) {
+        $this->fingerprint = $vals['fingerprint'];
+      }
+      if (isset($vals['name'])) {
+        $this->name = $vals['name'];
+      }
+      if (isset($vals['serDe'])) {
+        $this->serDe = $vals['serDe'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'SchemaVersion';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRUCT) {
+            $this->schema = new \metastore\ISchemaName();
+            $xfer += $this->schema->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::I32) {
+            $xfer += $input->readI32($this->version);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 3:
+          if ($ftype == TType::I64) {
+            $xfer += $input->readI64($this->createdAt);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 4:
+          if ($ftype == TType::LST) {
+            $this->cols = array();
+            $_size764 = 0;
+            $_etype767 = 0;
+            $xfer += $input->readListBegin($_etype767, $_size764);
+            for ($_i768 = 0; $_i768 < $_size764; ++$_i768)
+            {
+              $elem769 = null;
+              $elem769 = new \metastore\FieldSchema();
+              $xfer += $elem769->read($input);
+              $this->cols []= $elem769;
+            }
+            $xfer += $input->readListEnd();
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 5:
+          if ($ftype == TType::I32) {
+            $xfer += $input->readI32($this->state);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 6:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->description);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 7:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->schemaText);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 8:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->fingerprint);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 9:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->name);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 10:
+          if ($ftype == TType::STRUCT) {
+            $this->serDe = new \metastore\SerDeInfo();
+            $xfer += $this->serDe->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('SchemaVersion');
+    if ($this->schema !== null) {
+      if (!is_object($this->schema)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('schema', TType::STRUCT, 1);
+      $xfer += $this->schema->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->version !== null) {
+      $xfer += $output->writeFieldBegin('version', TType::I32, 2);
+      $xfer += $output->writeI32($this->version);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->createdAt !== null) {
+      $xfer += $output->writeFieldBegin('createdAt', TType::I64, 3);
+      $xfer += $output->writeI64($this->createdAt);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->cols !== null) {
+      if (!is_array($this->cols)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('cols', TType::LST, 4);
+      {
+        $output->writeListBegin(TType::STRUCT, count($this->cols));
+        {
+          foreach ($this->cols as $iter770)
+          {
+            $xfer += $iter770->write($output);
+          }
+        }
+        $output->writeListEnd();
+      }
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->state !== null) {
+      $xfer += $output->writeFieldBegin('state', TType::I32, 5);
+      $xfer += $output->writeI32($this->state);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->description !== null) {
+      $xfer += $output->writeFieldBegin('description', TType::STRING, 6);
+      $xfer += $output->writeString($this->description);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->schemaText !== null) {
+      $xfer += $output->writeFieldBegin('schemaText', TType::STRING, 7);
+      $xfer += $output->writeString($this->schemaText);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->fingerprint !== null) {
+      $xfer += $output->writeFieldBegin('fingerprint', TType::STRING, 8);
+      $xfer += $output->writeString($this->fingerprint);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->name !== null) {
+      $xfer += $output->writeFieldBegin('name', TType::STRING, 9);
+      $xfer += $output->writeString($this->name);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->serDe !== null) {
+      if (!is_object($this->serDe)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('serDe', TType::STRUCT, 10);
+      $xfer += $this->serDe->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class SchemaVersionDescriptor {
+  static $_TSPEC;
+
+  /**
+   * @var \metastore\ISchemaName
+   */
+  public $schema = null;
+  /**
+   * @var int
+   */
+  public $version = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'schema',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\ISchemaName',
+          ),
+        2 => array(
+          'var' => 'version',
+          'type' => TType::I32,
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['schema'])) {
+        $this->schema = $vals['schema'];
+      }
+      if (isset($vals['version'])) {
+        $this->version = $vals['version'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'SchemaVersionDescriptor';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRUCT) {
+            $this->schema = new \metastore\ISchemaName();
+            $xfer += $this->schema->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::I32) {
+            $xfer += $input->readI32($this->version);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('SchemaVersionDescriptor');
+    if ($this->schema !== null) {
+      if (!is_object($this->schema)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('schema', TType::STRUCT, 1);
+      $xfer += $this->schema->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->version !== null) {
+      $xfer += $output->writeFieldBegin('version', TType::I32, 2);
+      $xfer += $output->writeI32($this->version);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class FindSchemasByColsRqst {
+  static $_TSPEC;
+
+  /**
+   * @var string
+   */
+  public $colName = null;
+  /**
+   * @var string
+   */
+  public $colNamespace = null;
+  /**
+   * @var string
+   */
+  public $type = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'colName',
+          'type' => TType::STRING,
+          ),
+        2 => array(
+          'var' => 'colNamespace',
+          'type' => TType::STRING,
+          ),
+        3 => array(
+          'var' => 'type',
+          'type' => TType::STRING,
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['colName'])) {
+        $this->colName = $vals['colName'];
+      }
+      if (isset($vals['colNamespace'])) {
+        $this->colNamespace = $vals['colNamespace'];
+      }
+      if (isset($vals['type'])) {
+        $this->type = $vals['type'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'FindSchemasByColsRqst';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->colName);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->colNamespace);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 3:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->type);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('FindSchemasByColsRqst');
+    if ($this->colName !== null) {
+      $xfer += $output->writeFieldBegin('colName', TType::STRING, 1);
+      $xfer += $output->writeString($this->colName);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->colNamespace !== null) {
+      $xfer += $output->writeFieldBegin('colNamespace', TType::STRING, 2);
+      $xfer += $output->writeString($this->colNamespace);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->type !== null) {
+      $xfer += $output->writeFieldBegin('type', TType::STRING, 3);
+      $xfer += $output->writeString($this->type);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class FindSchemasByColsResp {
+  static $_TSPEC;
+
+  /**
+   * @var \metastore\SchemaVersionDescriptor[]
+   */
+  public $schemaVersions = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'schemaVersions',
+          'type' => TType::LST,
+          'etype' => TType::STRUCT,
+          'elem' => array(
+            'type' => TType::STRUCT,
+            'class' => '\metastore\SchemaVersionDescriptor',
+            ),
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['schemaVersions'])) {
+        $this->schemaVersions = $vals['schemaVersions'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'FindSchemasByColsResp';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::LST) {
+            $this->schemaVersions = array();
+            $_size771 = 0;
+            $_etype774 = 0;
+            $xfer += $input->readListBegin($_etype774, $_size771);
+            for ($_i775 = 0; $_i775 < $_size771; ++$_i775)
+            {
+              $elem776 = null;
+              $elem776 = new \metastore\SchemaVersionDescriptor();
+              $xfer += $elem776->read($input);
+              $this->schemaVersions []= $elem776;
+            }
+            $xfer += $input->readListEnd();
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('FindSchemasByColsResp');
+    if ($this->schemaVersions !== null) {
+      if (!is_array($this->schemaVersions)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('schemaVersions', TType::LST, 1);
+      {
+        $output->writeListBegin(TType::STRUCT, count($this->schemaVersions));
+        {
+          foreach ($this->schemaVersions as $iter777)
+          {
+            $xfer += $iter777->write($output);
+          }
+        }
+        $output->writeListEnd();
+      }
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class MapSchemaVersionToSerdeRequest {
+  static $_TSPEC;
+
+  /**
+   * @var \metastore\SchemaVersionDescriptor
+   */
+  public $schemaVersion = null;
+  /**
+   * @var string
+   */
+  public $serdeName = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'schemaVersion',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\SchemaVersionDescriptor',
+          ),
+        2 => array(
+          'var' => 'serdeName',
+          'type' => TType::STRING,
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['schemaVersion'])) {
+        $this->schemaVersion = $vals['schemaVersion'];
+      }
+      if (isset($vals['serdeName'])) {
+        $this->serdeName = $vals['serdeName'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'MapSchemaVersionToSerdeRequest';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRUCT) {
+            $this->schemaVersion = new \metastore\SchemaVersionDescriptor();
+            $xfer += $this->schemaVersion->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->serdeName);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('MapSchemaVersionToSerdeRequest');
+    if ($this->schemaVersion !== null) {
+      if (!is_object($this->schemaVersion)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('schemaVersion', TType::STRUCT, 1);
+      $xfer += $this->schemaVersion->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->serdeName !== null) {
+      $xfer += $output->writeFieldBegin('serdeName', TType::STRING, 2);
+      $xfer += $output->writeString($this->serdeName);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class SetSchemaVersionStateRequest {
+  static $_TSPEC;
+
+  /**
+   * @var \metastore\SchemaVersionDescriptor
+   */
+  public $schemaVersion = null;
+  /**
+   * @var int
+   */
+  public $state = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'schemaVersion',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\SchemaVersionDescriptor',
+          ),
+        2 => array(
+          'var' => 'state',
+          'type' => TType::I32,
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['schemaVersion'])) {
+        $this->schemaVersion = $vals['schemaVersion'];
+      }
+      if (isset($vals['state'])) {
+        $this->state = $vals['state'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'SetSchemaVersionStateRequest';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRUCT) {
+            $this->schemaVersion = new \metastore\SchemaVersionDescriptor();
+            $xfer += $this->schemaVersion->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::I32) {
+            $xfer += $input->readI32($this->state);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('SetSchemaVersionStateRequest');
+    if ($this->schemaVersion !== null) {
+      if (!is_object($this->schemaVersion)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('schemaVersion', TType::STRUCT, 1);
+      $xfer += $this->schemaVersion->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->state !== null) {
+      $xfer += $output->writeFieldBegin('state', TType::I32, 2);
+      $xfer += $output->writeI32($this->state);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class GetSerdeRequest {
+  static $_TSPEC;
+
+  /**
+   * @var string
+   */
+  public $serdeName = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'serdeName',
+          'type' => TType::STRING,
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['serdeName'])) {
+        $this->serdeName = $vals['serdeName'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'GetSerdeRequest';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->serdeName);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('GetSerdeRequest');
+    if ($this->serdeName !== null) {
+      $xfer += $output->writeFieldBegin('serdeName', TType::STRING, 1);
+      $xfer += $output->writeString($this->serdeName);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
 class MetaException extends TException {
   static $_TSPEC;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote b/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
index 76c7164..a8e8386 100755
--- a/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
+++ b/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
@@ -204,6 +204,20 @@ if len(sys.argv) <= 1 or sys.argv[1] == '--help':
   print('  WMCreateOrUpdateMappingResponse create_or_update_wm_mapping(WMCreateOrUpdateMappingRequest request)')
   print('  WMDropMappingResponse drop_wm_mapping(WMDropMappingRequest request)')
   print('  WMCreateOrDropTriggerToPoolMappingResponse create_or_drop_wm_trigger_to_pool_mapping(WMCreateOrDropTriggerToPoolMappingRequest request)')
+  print('  void create_ischema(ISchema schema)')
+  print('  void alter_ischema(AlterISchemaRequest rqst)')
+  print('  ISchema get_ischema(ISchemaName name)')
+  print('  void drop_ischema(ISchemaName name)')
+  print('  void add_schema_version(SchemaVersion schemaVersion)')
+  print('  SchemaVersion get_schema_version(SchemaVersionDescriptor schemaVersion)')
+  print('  SchemaVersion get_schema_latest_version(ISchemaName schemaName)')
+  print('   get_schema_all_versions(ISchemaName schemaName)')
+  print('  void drop_schema_version(SchemaVersionDescriptor schemaVersion)')
+  print('  FindSchemasByColsResp get_schemas_by_cols(FindSchemasByColsRqst rqst)')
+  print('  void map_schema_version_to_serde(MapSchemaVersionToSerdeRequest rqst)')
+  print('  void set_schema_version_state(SetSchemaVersionStateRequest rqst)')
+  print('  void add_serde(SerDeInfo serde)')
+  print('  SerDeInfo get_serde(GetSerdeRequest rqst)')
   print('  string getName()')
   print('  string getVersion()')
   print('  fb_status getStatus()')
@@ -1353,6 +1367,90 @@ elif cmd == 'create_or_drop_wm_trigger_to_pool_mapping':
     sys.exit(1)
   pp.pprint(client.create_or_drop_wm_trigger_to_pool_mapping(eval(args[0]),))
 
+elif cmd == 'create_ischema':
+  if len(args) != 1:
+    print('create_ischema requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.create_ischema(eval(args[0]),))
+
+elif cmd == 'alter_ischema':
+  if len(args) != 1:
+    print('alter_ischema requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.alter_ischema(eval(args[0]),))
+
+elif cmd == 'get_ischema':
+  if len(args) != 1:
+    print('get_ischema requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.get_ischema(eval(args[0]),))
+
+elif cmd == 'drop_ischema':
+  if len(args) != 1:
+    print('drop_ischema requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.drop_ischema(eval(args[0]),))
+
+elif cmd == 'add_schema_version':
+  if len(args) != 1:
+    print('add_schema_version requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.add_schema_version(eval(args[0]),))
+
+elif cmd == 'get_schema_version':
+  if len(args) != 1:
+    print('get_schema_version requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.get_schema_version(eval(args[0]),))
+
+elif cmd == 'get_schema_latest_version':
+  if len(args) != 1:
+    print('get_schema_latest_version requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.get_schema_latest_version(eval(args[0]),))
+
+elif cmd == 'get_schema_all_versions':
+  if len(args) != 1:
+    print('get_schema_all_versions requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.get_schema_all_versions(eval(args[0]),))
+
+elif cmd == 'drop_schema_version':
+  if len(args) != 1:
+    print('drop_schema_version requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.drop_schema_version(eval(args[0]),))
+
+elif cmd == 'get_schemas_by_cols':
+  if len(args) != 1:
+    print('get_schemas_by_cols requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.get_schemas_by_cols(eval(args[0]),))
+
+elif cmd == 'map_schema_version_to_serde':
+  if len(args) != 1:
+    print('map_schema_version_to_serde requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.map_schema_version_to_serde(eval(args[0]),))
+
+elif cmd == 'set_schema_version_state':
+  if len(args) != 1:
+    print('set_schema_version_state requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.set_schema_version_state(eval(args[0]),))
+
+elif cmd == 'add_serde':
+  if len(args) != 1:
+    print('add_serde requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.add_serde(eval(args[0]),))
+
+elif cmd == 'get_serde':
+  if len(args) != 1:
+    print('get_serde requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.get_serde(eval(args[0]),))
+
 elif cmd == 'getName':
   if len(args) != 0:
     print('getName requires 0 args')


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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php b/standalone-metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
index 02242a7..efe693a 100644
--- a/standalone-metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
+++ b/standalone-metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
@@ -1380,6 +1380,99 @@ interface ThriftHiveMetastoreIf extends \FacebookServiceIf {
    * @throws \metastore\MetaException
    */
   public function create_or_drop_wm_trigger_to_pool_mapping(\metastore\WMCreateOrDropTriggerToPoolMappingRequest $request);
+  /**
+   * @param \metastore\ISchema $schema
+   * @throws \metastore\AlreadyExistsException
+   * @throws \metastore\NoSuchObjectException
+   * @throws \metastore\MetaException
+   */
+  public function create_ischema(\metastore\ISchema $schema);
+  /**
+   * @param \metastore\AlterISchemaRequest $rqst
+   * @throws \metastore\NoSuchObjectException
+   * @throws \metastore\MetaException
+   */
+  public function alter_ischema(\metastore\AlterISchemaRequest $rqst);
+  /**
+   * @param \metastore\ISchemaName $name
+   * @return \metastore\ISchema
+   * @throws \metastore\NoSuchObjectException
+   * @throws \metastore\MetaException
+   */
+  public function get_ischema(\metastore\ISchemaName $name);
+  /**
+   * @param \metastore\ISchemaName $name
+   * @throws \metastore\NoSuchObjectException
+   * @throws \metastore\InvalidOperationException
+   * @throws \metastore\MetaException
+   */
+  public function drop_ischema(\metastore\ISchemaName $name);
+  /**
+   * @param \metastore\SchemaVersion $schemaVersion
+   * @throws \metastore\AlreadyExistsException
+   * @throws \metastore\NoSuchObjectException
+   * @throws \metastore\MetaException
+   */
+  public function add_schema_version(\metastore\SchemaVersion $schemaVersion);
+  /**
+   * @param \metastore\SchemaVersionDescriptor $schemaVersion
+   * @return \metastore\SchemaVersion
+   * @throws \metastore\NoSuchObjectException
+   * @throws \metastore\MetaException
+   */
+  public function get_schema_version(\metastore\SchemaVersionDescriptor $schemaVersion);
+  /**
+   * @param \metastore\ISchemaName $schemaName
+   * @return \metastore\SchemaVersion
+   * @throws \metastore\NoSuchObjectException
+   * @throws \metastore\MetaException
+   */
+  public function get_schema_latest_version(\metastore\ISchemaName $schemaName);
+  /**
+   * @param \metastore\ISchemaName $schemaName
+   * @return \metastore\SchemaVersion[]
+   * @throws \metastore\NoSuchObjectException
+   * @throws \metastore\MetaException
+   */
+  public function get_schema_all_versions(\metastore\ISchemaName $schemaName);
+  /**
+   * @param \metastore\SchemaVersionDescriptor $schemaVersion
+   * @throws \metastore\NoSuchObjectException
+   * @throws \metastore\MetaException
+   */
+  public function drop_schema_version(\metastore\SchemaVersionDescriptor $schemaVersion);
+  /**
+   * @param \metastore\FindSchemasByColsRqst $rqst
+   * @return \metastore\FindSchemasByColsResp
+   * @throws \metastore\MetaException
+   */
+  public function get_schemas_by_cols(\metastore\FindSchemasByColsRqst $rqst);
+  /**
+   * @param \metastore\MapSchemaVersionToSerdeRequest $rqst
+   * @throws \metastore\NoSuchObjectException
+   * @throws \metastore\MetaException
+   */
+  public function map_schema_version_to_serde(\metastore\MapSchemaVersionToSerdeRequest $rqst);
+  /**
+   * @param \metastore\SetSchemaVersionStateRequest $rqst
+   * @throws \metastore\NoSuchObjectException
+   * @throws \metastore\InvalidOperationException
+   * @throws \metastore\MetaException
+   */
+  public function set_schema_version_state(\metastore\SetSchemaVersionStateRequest $rqst);
+  /**
+   * @param \metastore\SerDeInfo $serde
+   * @throws \metastore\AlreadyExistsException
+   * @throws \metastore\MetaException
+   */
+  public function add_serde(\metastore\SerDeInfo $serde);
+  /**
+   * @param \metastore\GetSerdeRequest $rqst
+   * @return \metastore\SerDeInfo
+   * @throws \metastore\NoSuchObjectException
+   * @throws \metastore\MetaException
+   */
+  public function get_serde(\metastore\GetSerdeRequest $rqst);
 }
 
 class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metastore\ThriftHiveMetastoreIf {
@@ -11633,322 +11726,4430 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas
     throw new \Exception("create_or_drop_wm_trigger_to_pool_mapping failed: unknown result");
   }
 
-}
-
-// HELPER FUNCTIONS AND STRUCTURES
+  public function create_ischema(\metastore\ISchema $schema)
+  {
+    $this->send_create_ischema($schema);
+    $this->recv_create_ischema();
+  }
 
-class ThriftHiveMetastore_getMetaConf_args {
-  static $_TSPEC;
+  public function send_create_ischema(\metastore\ISchema $schema)
+  {
+    $args = new \metastore\ThriftHiveMetastore_create_ischema_args();
+    $args->schema = $schema;
+    $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
+    if ($bin_accel)
+    {
+      thrift_protocol_write_binary($this->output_, 'create_ischema', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
+    }
+    else
+    {
+      $this->output_->writeMessageBegin('create_ischema', TMessageType::CALL, $this->seqid_);
+      $args->write($this->output_);
+      $this->output_->writeMessageEnd();
+      $this->output_->getTransport()->flush();
+    }
+  }
 
-  /**
-   * @var string
-   */
-  public $key = null;
+  public function recv_create_ischema()
+  {
+    $bin_accel = ($this->input_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_read_binary');
+    if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_create_ischema_result', $this->input_->isStrictRead());
+    else
+    {
+      $rseqid = 0;
+      $fname = null;
+      $mtype = 0;
 
-  public function __construct($vals=null) {
-    if (!isset(self::$_TSPEC)) {
-      self::$_TSPEC = array(
-        1 => array(
-          'var' => 'key',
-          'type' => TType::STRING,
-          ),
-        );
-    }
-    if (is_array($vals)) {
-      if (isset($vals['key'])) {
-        $this->key = $vals['key'];
+      $this->input_->readMessageBegin($fname, $mtype, $rseqid);
+      if ($mtype == TMessageType::EXCEPTION) {
+        $x = new TApplicationException();
+        $x->read($this->input_);
+        $this->input_->readMessageEnd();
+        throw $x;
       }
+      $result = new \metastore\ThriftHiveMetastore_create_ischema_result();
+      $result->read($this->input_);
+      $this->input_->readMessageEnd();
+    }
+    if ($result->o1 !== null) {
+      throw $result->o1;
     }
+    if ($result->o2 !== null) {
+      throw $result->o2;
+    }
+    if ($result->o3 !== null) {
+      throw $result->o3;
+    }
+    return;
   }
 
-  public function getName() {
-    return 'ThriftHiveMetastore_getMetaConf_args';
+  public function alter_ischema(\metastore\AlterISchemaRequest $rqst)
+  {
+    $this->send_alter_ischema($rqst);
+    $this->recv_alter_ischema();
   }
 
-  public function read($input)
+  public function send_alter_ischema(\metastore\AlterISchemaRequest $rqst)
   {
-    $xfer = 0;
-    $fname = null;
-    $ftype = 0;
-    $fid = 0;
-    $xfer += $input->readStructBegin($fname);
-    while (true)
+    $args = new \metastore\ThriftHiveMetastore_alter_ischema_args();
+    $args->rqst = $rqst;
+    $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
+    if ($bin_accel)
     {
-      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
-      if ($ftype == TType::STOP) {
-        break;
-      }
-      switch ($fid)
-      {
-        case 1:
-          if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->key);
-          } else {
-            $xfer += $input->skip($ftype);
-          }
-          break;
-        default:
-          $xfer += $input->skip($ftype);
-          break;
-      }
-      $xfer += $input->readFieldEnd();
+      thrift_protocol_write_binary($this->output_, 'alter_ischema', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
     }
-    $xfer += $input->readStructEnd();
-    return $xfer;
-  }
-
-  public function write($output) {
-    $xfer = 0;
-    $xfer += $output->writeStructBegin('ThriftHiveMetastore_getMetaConf_args');
-    if ($this->key !== null) {
-      $xfer += $output->writeFieldBegin('key', TType::STRING, 1);
-      $xfer += $output->writeString($this->key);
-      $xfer += $output->writeFieldEnd();
+    else
+    {
+      $this->output_->writeMessageBegin('alter_ischema', TMessageType::CALL, $this->seqid_);
+      $args->write($this->output_);
+      $this->output_->writeMessageEnd();
+      $this->output_->getTransport()->flush();
     }
-    $xfer += $output->writeFieldStop();
-    $xfer += $output->writeStructEnd();
-    return $xfer;
   }
 
-}
-
-class ThriftHiveMetastore_getMetaConf_result {
-  static $_TSPEC;
-
-  /**
-   * @var string
-   */
-  public $success = null;
-  /**
-   * @var \metastore\MetaException
-   */
-  public $o1 = null;
+  public function recv_alter_ischema()
+  {
+    $bin_accel = ($this->input_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_read_binary');
+    if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_alter_ischema_result', $this->input_->isStrictRead());
+    else
+    {
+      $rseqid = 0;
+      $fname = null;
+      $mtype = 0;
 
-  public function __construct($vals=null) {
-    if (!isset(self::$_TSPEC)) {
-      self::$_TSPEC = array(
-        0 => array(
-          'var' => 'success',
-          'type' => TType::STRING,
-          ),
-        1 => array(
-          'var' => 'o1',
-          'type' => TType::STRUCT,
-          'class' => '\metastore\MetaException',
-          ),
-        );
-    }
-    if (is_array($vals)) {
-      if (isset($vals['success'])) {
-        $this->success = $vals['success'];
-      }
-      if (isset($vals['o1'])) {
-        $this->o1 = $vals['o1'];
+      $this->input_->readMessageBegin($fname, $mtype, $rseqid);
+      if ($mtype == TMessageType::EXCEPTION) {
+        $x = new TApplicationException();
+        $x->read($this->input_);
+        $this->input_->readMessageEnd();
+        throw $x;
       }
+      $result = new \metastore\ThriftHiveMetastore_alter_ischema_result();
+      $result->read($this->input_);
+      $this->input_->readMessageEnd();
+    }
+    if ($result->o1 !== null) {
+      throw $result->o1;
     }
+    if ($result->o2 !== null) {
+      throw $result->o2;
+    }
+    return;
   }
 
-  public function getName() {
-    return 'ThriftHiveMetastore_getMetaConf_result';
+  public function get_ischema(\metastore\ISchemaName $name)
+  {
+    $this->send_get_ischema($name);
+    return $this->recv_get_ischema();
   }
 
-  public function read($input)
+  public function send_get_ischema(\metastore\ISchemaName $name)
   {
-    $xfer = 0;
-    $fname = null;
-    $ftype = 0;
-    $fid = 0;
-    $xfer += $input->readStructBegin($fname);
-    while (true)
+    $args = new \metastore\ThriftHiveMetastore_get_ischema_args();
+    $args->name = $name;
+    $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
+    if ($bin_accel)
     {
-      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
-      if ($ftype == TType::STOP) {
-        break;
-      }
-      switch ($fid)
-      {
-        case 0:
-          if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->success);
-          } else {
-            $xfer += $input->skip($ftype);
-          }
-          break;
-        case 1:
-          if ($ftype == TType::STRUCT) {
-            $this->o1 = new \metastore\MetaException();
-            $xfer += $this->o1->read($input);
-          } else {
-            $xfer += $input->skip($ftype);
-          }
-          break;
-        default:
-          $xfer += $input->skip($ftype);
-          break;
-      }
-      $xfer += $input->readFieldEnd();
+      thrift_protocol_write_binary($this->output_, 'get_ischema', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
+    }
+    else
+    {
+      $this->output_->writeMessageBegin('get_ischema', TMessageType::CALL, $this->seqid_);
+      $args->write($this->output_);
+      $this->output_->writeMessageEnd();
+      $this->output_->getTransport()->flush();
     }
-    $xfer += $input->readStructEnd();
-    return $xfer;
   }
 
-  public function write($output) {
-    $xfer = 0;
-    $xfer += $output->writeStructBegin('ThriftHiveMetastore_getMetaConf_result');
-    if ($this->success !== null) {
-      $xfer += $output->writeFieldBegin('success', TType::STRING, 0);
-      $xfer += $output->writeString($this->success);
-      $xfer += $output->writeFieldEnd();
+  public function recv_get_ischema()
+  {
+    $bin_accel = ($this->input_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_read_binary');
+    if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_get_ischema_result', $this->input_->isStrictRead());
+    else
+    {
+      $rseqid = 0;
+      $fname = null;
+      $mtype = 0;
+
+      $this->input_->readMessageBegin($fname, $mtype, $rseqid);
+      if ($mtype == TMessageType::EXCEPTION) {
+        $x = new TApplicationException();
+        $x->read($this->input_);
+        $this->input_->readMessageEnd();
+        throw $x;
+      }
+      $result = new \metastore\ThriftHiveMetastore_get_ischema_result();
+      $result->read($this->input_);
+      $this->input_->readMessageEnd();
     }
-    if ($this->o1 !== null) {
-      $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1);
-      $xfer += $this->o1->write($output);
-      $xfer += $output->writeFieldEnd();
+    if ($result->success !== null) {
+      return $result->success;
     }
-    $xfer += $output->writeFieldStop();
-    $xfer += $output->writeStructEnd();
-    return $xfer;
+    if ($result->o1 !== null) {
+      throw $result->o1;
+    }
+    if ($result->o2 !== null) {
+      throw $result->o2;
+    }
+    throw new \Exception("get_ischema failed: unknown result");
   }
 
-}
+  public function drop_ischema(\metastore\ISchemaName $name)
+  {
+    $this->send_drop_ischema($name);
+    $this->recv_drop_ischema();
+  }
 
-class ThriftHiveMetastore_setMetaConf_args {
-  static $_TSPEC;
+  public function send_drop_ischema(\metastore\ISchemaName $name)
+  {
+    $args = new \metastore\ThriftHiveMetastore_drop_ischema_args();
+    $args->name = $name;
+    $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
+    if ($bin_accel)
+    {
+      thrift_protocol_write_binary($this->output_, 'drop_ischema', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
+    }
+    else
+    {
+      $this->output_->writeMessageBegin('drop_ischema', TMessageType::CALL, $this->seqid_);
+      $args->write($this->output_);
+      $this->output_->writeMessageEnd();
+      $this->output_->getTransport()->flush();
+    }
+  }
 
-  /**
-   * @var string
-   */
-  public $key = null;
-  /**
-   * @var string
-   */
-  public $value = null;
+  public function recv_drop_ischema()
+  {
+    $bin_accel = ($this->input_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_read_binary');
+    if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_drop_ischema_result', $this->input_->isStrictRead());
+    else
+    {
+      $rseqid = 0;
+      $fname = null;
+      $mtype = 0;
 
-  public function __construct($vals=null) {
-    if (!isset(self::$_TSPEC)) {
-      self::$_TSPEC = array(
-        1 => array(
-          'var' => 'key',
-          'type' => TType::STRING,
-          ),
-        2 => array(
-          'var' => 'value',
-          'type' => TType::STRING,
-          ),
-        );
-    }
-    if (is_array($vals)) {
-      if (isset($vals['key'])) {
-        $this->key = $vals['key'];
-      }
-      if (isset($vals['value'])) {
-        $this->value = $vals['value'];
+      $this->input_->readMessageBegin($fname, $mtype, $rseqid);
+      if ($mtype == TMessageType::EXCEPTION) {
+        $x = new TApplicationException();
+        $x->read($this->input_);
+        $this->input_->readMessageEnd();
+        throw $x;
       }
+      $result = new \metastore\ThriftHiveMetastore_drop_ischema_result();
+      $result->read($this->input_);
+      $this->input_->readMessageEnd();
+    }
+    if ($result->o1 !== null) {
+      throw $result->o1;
+    }
+    if ($result->o2 !== null) {
+      throw $result->o2;
     }
+    if ($result->o3 !== null) {
+      throw $result->o3;
+    }
+    return;
   }
 
-  public function getName() {
-    return 'ThriftHiveMetastore_setMetaConf_args';
+  public function add_schema_version(\metastore\SchemaVersion $schemaVersion)
+  {
+    $this->send_add_schema_version($schemaVersion);
+    $this->recv_add_schema_version();
   }
 
-  public function read($input)
+  public function send_add_schema_version(\metastore\SchemaVersion $schemaVersion)
   {
-    $xfer = 0;
-    $fname = null;
-    $ftype = 0;
-    $fid = 0;
-    $xfer += $input->readStructBegin($fname);
-    while (true)
+    $args = new \metastore\ThriftHiveMetastore_add_schema_version_args();
+    $args->schemaVersion = $schemaVersion;
+    $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
+    if ($bin_accel)
     {
-      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
-      if ($ftype == TType::STOP) {
-        break;
-      }
-      switch ($fid)
-      {
-        case 1:
-          if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->key);
-          } else {
-            $xfer += $input->skip($ftype);
-          }
-          break;
-        case 2:
-          if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->value);
-          } else {
-            $xfer += $input->skip($ftype);
-          }
-          break;
-        default:
-          $xfer += $input->skip($ftype);
-          break;
-      }
-      $xfer += $input->readFieldEnd();
+      thrift_protocol_write_binary($this->output_, 'add_schema_version', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
+    }
+    else
+    {
+      $this->output_->writeMessageBegin('add_schema_version', TMessageType::CALL, $this->seqid_);
+      $args->write($this->output_);
+      $this->output_->writeMessageEnd();
+      $this->output_->getTransport()->flush();
     }
-    $xfer += $input->readStructEnd();
-    return $xfer;
   }
 
-  public function write($output) {
-    $xfer = 0;
-    $xfer += $output->writeStructBegin('ThriftHiveMetastore_setMetaConf_args');
-    if ($this->key !== null) {
-      $xfer += $output->writeFieldBegin('key', TType::STRING, 1);
-      $xfer += $output->writeString($this->key);
-      $xfer += $output->writeFieldEnd();
+  public function recv_add_schema_version()
+  {
+    $bin_accel = ($this->input_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_read_binary');
+    if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_add_schema_version_result', $this->input_->isStrictRead());
+    else
+    {
+      $rseqid = 0;
+      $fname = null;
+      $mtype = 0;
+
+      $this->input_->readMessageBegin($fname, $mtype, $rseqid);
+      if ($mtype == TMessageType::EXCEPTION) {
+        $x = new TApplicationException();
+        $x->read($this->input_);
+        $this->input_->readMessageEnd();
+        throw $x;
+      }
+      $result = new \metastore\ThriftHiveMetastore_add_schema_version_result();
+      $result->read($this->input_);
+      $this->input_->readMessageEnd();
     }
-    if ($this->value !== null) {
-      $xfer += $output->writeFieldBegin('value', TType::STRING, 2);
-      $xfer += $output->writeString($this->value);
-      $xfer += $output->writeFieldEnd();
+    if ($result->o1 !== null) {
+      throw $result->o1;
     }
-    $xfer += $output->writeFieldStop();
-    $xfer += $output->writeStructEnd();
-    return $xfer;
+    if ($result->o2 !== null) {
+      throw $result->o2;
+    }
+    if ($result->o3 !== null) {
+      throw $result->o3;
+    }
+    return;
   }
 
-}
-
-class ThriftHiveMetastore_setMetaConf_result {
-  static $_TSPEC;
-
-  /**
-   * @var \metastore\MetaException
-   */
-  public $o1 = null;
+  public function get_schema_version(\metastore\SchemaVersionDescriptor $schemaVersion)
+  {
+    $this->send_get_schema_version($schemaVersion);
+    return $this->recv_get_schema_version();
+  }
 
-  public function __construct($vals=null) {
-    if (!isset(self::$_TSPEC)) {
-      self::$_TSPEC = array(
-        1 => array(
-          'var' => 'o1',
-          'type' => TType::STRUCT,
-          'class' => '\metastore\MetaException',
-          ),
-        );
+  public function send_get_schema_version(\metastore\SchemaVersionDescriptor $schemaVersion)
+  {
+    $args = new \metastore\ThriftHiveMetastore_get_schema_version_args();
+    $args->schemaVersion = $schemaVersion;
+    $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
+    if ($bin_accel)
+    {
+      thrift_protocol_write_binary($this->output_, 'get_schema_version', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
     }
-    if (is_array($vals)) {
-      if (isset($vals['o1'])) {
-        $this->o1 = $vals['o1'];
-      }
+    else
+    {
+      $this->output_->writeMessageBegin('get_schema_version', TMessageType::CALL, $this->seqid_);
+      $args->write($this->output_);
+      $this->output_->writeMessageEnd();
+      $this->output_->getTransport()->flush();
     }
   }
 
-  public function getName() {
-    return 'ThriftHiveMetastore_setMetaConf_result';
-  }
-
-  public function read($input)
+  public function recv_get_schema_version()
   {
-    $xfer = 0;
-    $fname = null;
-    $ftype = 0;
-    $fid = 0;
-    $xfer += $input->readStructBegin($fname);
-    while (true)
+    $bin_accel = ($this->input_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_read_binary');
+    if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_get_schema_version_result', $this->input_->isStrictRead());
+    else
     {
-      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      $rseqid = 0;
+      $fname = null;
+      $mtype = 0;
+
+      $this->input_->readMessageBegin($fname, $mtype, $rseqid);
+      if ($mtype == TMessageType::EXCEPTION) {
+        $x = new TApplicationException();
+        $x->read($this->input_);
+        $this->input_->readMessageEnd();
+        throw $x;
+      }
+      $result = new \metastore\ThriftHiveMetastore_get_schema_version_result();
+      $result->read($this->input_);
+      $this->input_->readMessageEnd();
+    }
+    if ($result->success !== null) {
+      return $result->success;
+    }
+    if ($result->o1 !== null) {
+      throw $result->o1;
+    }
+    if ($result->o2 !== null) {
+      throw $result->o2;
+    }
+    throw new \Exception("get_schema_version failed: unknown result");
+  }
+
+  public function get_schema_latest_version(\metastore\ISchemaName $schemaName)
+  {
+    $this->send_get_schema_latest_version($schemaName);
+    return $this->recv_get_schema_latest_version();
+  }
+
+  public function send_get_schema_latest_version(\metastore\ISchemaName $schemaName)
+  {
+    $args = new \metastore\ThriftHiveMetastore_get_schema_latest_version_args();
+    $args->schemaName = $schemaName;
+    $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
+    if ($bin_accel)
+    {
+      thrift_protocol_write_binary($this->output_, 'get_schema_latest_version', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
+    }
+    else
+    {
+      $this->output_->writeMessageBegin('get_schema_latest_version', TMessageType::CALL, $this->seqid_);
+      $args->write($this->output_);
+      $this->output_->writeMessageEnd();
+      $this->output_->getTransport()->flush();
+    }
+  }
+
+  public function recv_get_schema_latest_version()
+  {
+    $bin_accel = ($this->input_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_read_binary');
+    if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_get_schema_latest_version_result', $this->input_->isStrictRead());
+    else
+    {
+      $rseqid = 0;
+      $fname = null;
+      $mtype = 0;
+
+      $this->input_->readMessageBegin($fname, $mtype, $rseqid);
+      if ($mtype == TMessageType::EXCEPTION) {
+        $x = new TApplicationException();
+        $x->read($this->input_);
+        $this->input_->readMessageEnd();
+        throw $x;
+      }
+      $result = new \metastore\ThriftHiveMetastore_get_schema_latest_version_result();
+      $result->read($this->input_);
+      $this->input_->readMessageEnd();
+    }
+    if ($result->success !== null) {
+      return $result->success;
+    }
+    if ($result->o1 !== null) {
+      throw $result->o1;
+    }
+    if ($result->o2 !== null) {
+      throw $result->o2;
+    }
+    throw new \Exception("get_schema_latest_version failed: unknown result");
+  }
+
+  public function get_schema_all_versions(\metastore\ISchemaName $schemaName)
+  {
+    $this->send_get_schema_all_versions($schemaName);
+    return $this->recv_get_schema_all_versions();
+  }
+
+  public function send_get_schema_all_versions(\metastore\ISchemaName $schemaName)
+  {
+    $args = new \metastore\ThriftHiveMetastore_get_schema_all_versions_args();
+    $args->schemaName = $schemaName;
+    $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
+    if ($bin_accel)
+    {
+      thrift_protocol_write_binary($this->output_, 'get_schema_all_versions', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
+    }
+    else
+    {
+      $this->output_->writeMessageBegin('get_schema_all_versions', TMessageType::CALL, $this->seqid_);
+      $args->write($this->output_);
+      $this->output_->writeMessageEnd();
+      $this->output_->getTransport()->flush();
+    }
+  }
+
+  public function recv_get_schema_all_versions()
+  {
+    $bin_accel = ($this->input_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_read_binary');
+    if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_get_schema_all_versions_result', $this->input_->isStrictRead());
+    else
+    {
+      $rseqid = 0;
+      $fname = null;
+      $mtype = 0;
+
+      $this->input_->readMessageBegin($fname, $mtype, $rseqid);
+      if ($mtype == TMessageType::EXCEPTION) {
+        $x = new TApplicationException();
+        $x->read($this->input_);
+        $this->input_->readMessageEnd();
+        throw $x;
+      }
+      $result = new \metastore\ThriftHiveMetastore_get_schema_all_versions_result();
+      $result->read($this->input_);
+      $this->input_->readMessageEnd();
+    }
+    if ($result->success !== null) {
+      return $result->success;
+    }
+    if ($result->o1 !== null) {
+      throw $result->o1;
+    }
+    if ($result->o2 !== null) {
+      throw $result->o2;
+    }
+    throw new \Exception("get_schema_all_versions failed: unknown result");
+  }
+
+  public function drop_schema_version(\metastore\SchemaVersionDescriptor $schemaVersion)
+  {
+    $this->send_drop_schema_version($schemaVersion);
+    $this->recv_drop_schema_version();
+  }
+
+  public function send_drop_schema_version(\metastore\SchemaVersionDescriptor $schemaVersion)
+  {
+    $args = new \metastore\ThriftHiveMetastore_drop_schema_version_args();
+    $args->schemaVersion = $schemaVersion;
+    $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
+    if ($bin_accel)
+    {
+      thrift_protocol_write_binary($this->output_, 'drop_schema_version', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
+    }
+    else
+    {
+      $this->output_->writeMessageBegin('drop_schema_version', TMessageType::CALL, $this->seqid_);
+      $args->write($this->output_);
+      $this->output_->writeMessageEnd();
+      $this->output_->getTransport()->flush();
+    }
+  }
+
+  public function recv_drop_schema_version()
+  {
+    $bin_accel = ($this->input_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_read_binary');
+    if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_drop_schema_version_result', $this->input_->isStrictRead());
+    else
+    {
+      $rseqid = 0;
+      $fname = null;
+      $mtype = 0;
+
+      $this->input_->readMessageBegin($fname, $mtype, $rseqid);
+      if ($mtype == TMessageType::EXCEPTION) {
+        $x = new TApplicationException();
+        $x->read($this->input_);
+        $this->input_->readMessageEnd();
+        throw $x;
+      }
+      $result = new \metastore\ThriftHiveMetastore_drop_schema_version_result();
+      $result->read($this->input_);
+      $this->input_->readMessageEnd();
+    }
+    if ($result->o1 !== null) {
+      throw $result->o1;
+    }
+    if ($result->o2 !== null) {
+      throw $result->o2;
+    }
+    return;
+  }
+
+  public function get_schemas_by_cols(\metastore\FindSchemasByColsRqst $rqst)
+  {
+    $this->send_get_schemas_by_cols($rqst);
+    return $this->recv_get_schemas_by_cols();
+  }
+
+  public function send_get_schemas_by_cols(\metastore\FindSchemasByColsRqst $rqst)
+  {
+    $args = new \metastore\ThriftHiveMetastore_get_schemas_by_cols_args();
+    $args->rqst = $rqst;
+    $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
+    if ($bin_accel)
+    {
+      thrift_protocol_write_binary($this->output_, 'get_schemas_by_cols', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
+    }
+    else
+    {
+      $this->output_->writeMessageBegin('get_schemas_by_cols', TMessageType::CALL, $this->seqid_);
+      $args->write($this->output_);
+      $this->output_->writeMessageEnd();
+      $this->output_->getTransport()->flush();
+    }
+  }
+
+  public function recv_get_schemas_by_cols()
+  {
+    $bin_accel = ($this->input_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_read_binary');
+    if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_get_schemas_by_cols_result', $this->input_->isStrictRead());
+    else
+    {
+      $rseqid = 0;
+      $fname = null;
+      $mtype = 0;
+
+      $this->input_->readMessageBegin($fname, $mtype, $rseqid);
+      if ($mtype == TMessageType::EXCEPTION) {
+        $x = new TApplicationException();
+        $x->read($this->input_);
+        $this->input_->readMessageEnd();
+        throw $x;
+      }
+      $result = new \metastore\ThriftHiveMetastore_get_schemas_by_cols_result();
+      $result->read($this->input_);
+      $this->input_->readMessageEnd();
+    }
+    if ($result->success !== null) {
+      return $result->success;
+    }
+    if ($result->o1 !== null) {
+      throw $result->o1;
+    }
+    throw new \Exception("get_schemas_by_cols failed: unknown result");
+  }
+
+  public function map_schema_version_to_serde(\metastore\MapSchemaVersionToSerdeRequest $rqst)
+  {
+    $this->send_map_schema_version_to_serde($rqst);
+    $this->recv_map_schema_version_to_serde();
+  }
+
+  public function send_map_schema_version_to_serde(\metastore\MapSchemaVersionToSerdeRequest $rqst)
+  {
+    $args = new \metastore\ThriftHiveMetastore_map_schema_version_to_serde_args();
+    $args->rqst = $rqst;
+    $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
+    if ($bin_accel)
+    {
+      thrift_protocol_write_binary($this->output_, 'map_schema_version_to_serde', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
+    }
+    else
+    {
+      $this->output_->writeMessageBegin('map_schema_version_to_serde', TMessageType::CALL, $this->seqid_);
+      $args->write($this->output_);
+      $this->output_->writeMessageEnd();
+      $this->output_->getTransport()->flush();
+    }
+  }
+
+  public function recv_map_schema_version_to_serde()
+  {
+    $bin_accel = ($this->input_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_read_binary');
+    if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_map_schema_version_to_serde_result', $this->input_->isStrictRead());
+    else
+    {
+      $rseqid = 0;
+      $fname = null;
+      $mtype = 0;
+
+      $this->input_->readMessageBegin($fname, $mtype, $rseqid);
+      if ($mtype == TMessageType::EXCEPTION) {
+        $x = new TApplicationException();
+        $x->read($this->input_);
+        $this->input_->readMessageEnd();
+        throw $x;
+      }
+      $result = new \metastore\ThriftHiveMetastore_map_schema_version_to_serde_result();
+      $result->read($this->input_);
+      $this->input_->readMessageEnd();
+    }
+    if ($result->o1 !== null) {
+      throw $result->o1;
+    }
+    if ($result->o2 !== null) {
+      throw $result->o2;
+    }
+    return;
+  }
+
+  public function set_schema_version_state(\metastore\SetSchemaVersionStateRequest $rqst)
+  {
+    $this->send_set_schema_version_state($rqst);
+    $this->recv_set_schema_version_state();
+  }
+
+  public function send_set_schema_version_state(\metastore\SetSchemaVersionStateRequest $rqst)
+  {
+    $args = new \metastore\ThriftHiveMetastore_set_schema_version_state_args();
+    $args->rqst = $rqst;
+    $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
+    if ($bin_accel)
+    {
+      thrift_protocol_write_binary($this->output_, 'set_schema_version_state', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
+    }
+    else
+    {
+      $this->output_->writeMessageBegin('set_schema_version_state', TMessageType::CALL, $this->seqid_);
+      $args->write($this->output_);
+      $this->output_->writeMessageEnd();
+      $this->output_->getTransport()->flush();
+    }
+  }
+
+  public function recv_set_schema_version_state()
+  {
+    $bin_accel = ($this->input_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_read_binary');
+    if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_set_schema_version_state_result', $this->input_->isStrictRead());
+    else
+    {
+      $rseqid = 0;
+      $fname = null;
+      $mtype = 0;
+
+      $this->input_->readMessageBegin($fname, $mtype, $rseqid);
+      if ($mtype == TMessageType::EXCEPTION) {
+        $x = new TApplicationException();
+        $x->read($this->input_);
+        $this->input_->readMessageEnd();
+        throw $x;
+      }
+      $result = new \metastore\ThriftHiveMetastore_set_schema_version_state_result();
+      $result->read($this->input_);
+      $this->input_->readMessageEnd();
+    }
+    if ($result->o1 !== null) {
+      throw $result->o1;
+    }
+    if ($result->o2 !== null) {
+      throw $result->o2;
+    }
+    if ($result->o3 !== null) {
+      throw $result->o3;
+    }
+    return;
+  }
+
+  public function add_serde(\metastore\SerDeInfo $serde)
+  {
+    $this->send_add_serde($serde);
+    $this->recv_add_serde();
+  }
+
+  public function send_add_serde(\metastore\SerDeInfo $serde)
+  {
+    $args = new \metastore\ThriftHiveMetastore_add_serde_args();
+    $args->serde = $serde;
+    $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
+    if ($bin_accel)
+    {
+      thrift_protocol_write_binary($this->output_, 'add_serde', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
+    }
+    else
+    {
+      $this->output_->writeMessageBegin('add_serde', TMessageType::CALL, $this->seqid_);
+      $args->write($this->output_);
+      $this->output_->writeMessageEnd();
+      $this->output_->getTransport()->flush();
+    }
+  }
+
+  public function recv_add_serde()
+  {
+    $bin_accel = ($this->input_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_read_binary');
+    if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_add_serde_result', $this->input_->isStrictRead());
+    else
+    {
+      $rseqid = 0;
+      $fname = null;
+      $mtype = 0;
+
+      $this->input_->readMessageBegin($fname, $mtype, $rseqid);
+      if ($mtype == TMessageType::EXCEPTION) {
+        $x = new TApplicationException();
+        $x->read($this->input_);
+        $this->input_->readMessageEnd();
+        throw $x;
+      }
+      $result = new \metastore\ThriftHiveMetastore_add_serde_result();
+      $result->read($this->input_);
+      $this->input_->readMessageEnd();
+    }
+    if ($result->o1 !== null) {
+      throw $result->o1;
+    }
+    if ($result->o2 !== null) {
+      throw $result->o2;
+    }
+    return;
+  }
+
+  public function get_serde(\metastore\GetSerdeRequest $rqst)
+  {
+    $this->send_get_serde($rqst);
+    return $this->recv_get_serde();
+  }
+
+  public function send_get_serde(\metastore\GetSerdeRequest $rqst)
+  {
+    $args = new \metastore\ThriftHiveMetastore_get_serde_args();
+    $args->rqst = $rqst;
+    $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
+    if ($bin_accel)
+    {
+      thrift_protocol_write_binary($this->output_, 'get_serde', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
+    }
+    else
+    {
+      $this->output_->writeMessageBegin('get_serde', TMessageType::CALL, $this->seqid_);
+      $args->write($this->output_);
+      $this->output_->writeMessageEnd();
+      $this->output_->getTransport()->flush();
+    }
+  }
+
+  public function recv_get_serde()
+  {
+    $bin_accel = ($this->input_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_read_binary');
+    if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_get_serde_result', $this->input_->isStrictRead());
+    else
+    {
+      $rseqid = 0;
+      $fname = null;
+      $mtype = 0;
+
+      $this->input_->readMessageBegin($fname, $mtype, $rseqid);
+      if ($mtype == TMessageType::EXCEPTION) {
+        $x = new TApplicationException();
+        $x->read($this->input_);
+        $this->input_->readMessageEnd();
+        throw $x;
+      }
+      $result = new \metastore\ThriftHiveMetastore_get_serde_result();
+      $result->read($this->input_);
+      $this->input_->readMessageEnd();
+    }
+    if ($result->success !== null) {
+      return $result->success;
+    }
+    if ($result->o1 !== null) {
+      throw $result->o1;
+    }
+    if ($result->o2 !== null) {
+      throw $result->o2;
+    }
+    throw new \Exception("get_serde failed: unknown result");
+  }
+
+}
+
+// HELPER FUNCTIONS AND STRUCTURES
+
+class ThriftHiveMetastore_getMetaConf_args {
+  static $_TSPEC;
+
+  /**
+   * @var string
+   */
+  public $key = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'key',
+          'type' => TType::STRING,
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['key'])) {
+        $this->key = $vals['key'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_getMetaConf_args';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->key);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_getMetaConf_args');
+    if ($this->key !== null) {
+      $xfer += $output->writeFieldBegin('key', TType::STRING, 1);
+      $xfer += $output->writeString($this->key);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class ThriftHiveMetastore_getMetaConf_result {
+  static $_TSPEC;
+
+  /**
+   * @var string
+   */
+  public $success = null;
+  /**
+   * @var \metastore\MetaException
+   */
+  public $o1 = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        0 => array(
+          'var' => 'success',
+          'type' => TType::STRING,
+          ),
+        1 => array(
+          'var' => 'o1',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\MetaException',
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['success'])) {
+        $this->success = $vals['success'];
+      }
+      if (isset($vals['o1'])) {
+        $this->o1 = $vals['o1'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_getMetaConf_result';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 0:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->success);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 1:
+          if ($ftype == TType::STRUCT) {
+            $this->o1 = new \metastore\MetaException();
+            $xfer += $this->o1->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_getMetaConf_result');
+    if ($this->success !== null) {
+      $xfer += $output->writeFieldBegin('success', TType::STRING, 0);
+      $xfer += $output->writeString($this->success);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->o1 !== null) {
+      $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1);
+      $xfer += $this->o1->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class ThriftHiveMetastore_setMetaConf_args {
+  static $_TSPEC;
+
+  /**
+   * @var string
+   */
+  public $key = null;
+  /**
+   * @var string
+   */
+  public $value = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'key',
+          'type' => TType::STRING,
+          ),
+        2 => array(
+          'var' => 'value',
+          'type' => TType::STRING,
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['key'])) {
+        $this->key = $vals['key'];
+      }
+      if (isset($vals['value'])) {
+        $this->value = $vals['value'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_setMetaConf_args';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->key);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->value);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_setMetaConf_args');
+    if ($this->key !== null) {
+      $xfer += $output->writeFieldBegin('key', TType::STRING, 1);
+      $xfer += $output->writeString($this->key);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->value !== null) {
+      $xfer += $output->writeFieldBegin('value', TType::STRING, 2);
+      $xfer += $output->writeString($this->value);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class ThriftHiveMetastore_setMetaConf_result {
+  static $_TSPEC;
+
+  /**
+   * @var \metastore\MetaException
+   */
+  public $o1 = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'o1',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\MetaException',
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['o1'])) {
+        $this->o1 = $vals['o1'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_setMetaConf_result';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRUCT) {
+            $this->o1 = new \metastore\MetaException();
+            $xfer += $this->o1->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_setMetaConf_result');
+    if ($this->o1 !== null) {
+      $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1);
+      $xfer += $this->o1->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class ThriftHiveMetastore_create_database_args {
+  static $_TSPEC;
+
+  /**
+   * @var \metastore\Database
+   */
+  public $database = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'database',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\Database',
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['database'])) {
+        $this->database = $vals['database'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_create_database_args';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRUCT) {
+            $this->database = new \metastore\Database();
+            $xfer += $this->database->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_create_database_args');
+    if ($this->database !== null) {
+      if (!is_object($this->database)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('database', TType::STRUCT, 1);
+      $xfer += $this->database->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class ThriftHiveMetastore_create_database_result {
+  static $_TSPEC;
+
+  /**
+   * @var \metastore\AlreadyExistsException
+   */
+  public $o1 = null;
+  /**
+   * @var \metastore\InvalidObjectException
+   */
+  public $o2 = null;
+  /**
+   * @var \metastore\MetaException
+   */
+  public $o3 = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'o1',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\AlreadyExistsException',
+          ),
+        2 => array(
+          'var' => 'o2',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\InvalidObjectException',
+          ),
+        3 => array(
+          'var' => 'o3',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\MetaException',
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['o1'])) {
+        $this->o1 = $vals['o1'];
+      }
+      if (isset($vals['o2'])) {
+        $this->o2 = $vals['o2'];
+      }
+      if (isset($vals['o3'])) {
+        $this->o3 = $vals['o3'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_create_database_result';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRUCT) {
+            $this->o1 = new \metastore\AlreadyExistsException();
+            $xfer += $this->o1->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::STRUCT) {
+            $this->o2 = new \metastore\InvalidObjectException();
+            $xfer += $this->o2->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 3:
+          if ($ftype == TType::STRUCT) {
+            $this->o3 = new \metastore\MetaException();
+            $xfer += $this->o3->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_create_database_result');
+    if ($this->o1 !== null) {
+      $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1);
+      $xfer += $this->o1->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->o2 !== null) {
+      $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2);
+      $xfer += $this->o2->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->o3 !== null) {
+      $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3);
+      $xfer += $this->o3->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class ThriftHiveMetastore_get_database_args {
+  static $_TSPEC;
+
+  /**
+   * @var string
+   */
+  public $name = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'name',
+          'type' => TType::STRING,
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['name'])) {
+        $this->name = $vals['name'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_get_database_args';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->name);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_database_args');
+    if ($this->name !== null) {
+      $xfer += $output->writeFieldBegin('name', TType::STRING, 1);
+      $xfer += $output->writeString($this->name);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class ThriftHiveMetastore_get_database_result {
+  static $_TSPEC;
+
+  /**
+   * @var \metastore\Database
+   */
+  public $success = null;
+  /**
+   * @var \metastore\NoSuchObjectException
+   */
+  public $o1 = null;
+  /**
+   * @var \metastore\MetaException
+   */
+  public $o2 = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        0 => array(
+          'var' => 'success',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\Database',
+          ),
+        1 => array(
+          'var' => 'o1',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\NoSuchObjectException',
+          ),
+        2 => array(
+          'var' => 'o2',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\MetaException',
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['success'])) {
+        $this->success = $vals['success'];
+      }
+      if (isset($vals['o1'])) {
+        $this->o1 = $vals['o1'];
+      }
+      if (isset($vals['o2'])) {
+        $this->o2 = $vals['o2'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_get_database_result';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 0:
+          if ($ftype == TType::STRUCT) {
+            $this->success = new \metastore\Database();
+            $xfer += $this->success->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 1:
+          if ($ftype == TType::STRUCT) {
+            $this->o1 = new \metastore\NoSuchObjectException();
+            $xfer += $this->o1->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::STRUCT) {
+            $this->o2 = new \metastore\MetaException();
+            $xfer += $this->o2->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_database_result');
+    if ($this->success !== null) {
+      if (!is_object($this->success)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0);
+      $xfer += $this->success->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->o1 !== null) {
+      $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1);
+      $xfer += $this->o1->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->o2 !== null) {
+      $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2);
+      $xfer += $this->o2->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class ThriftHiveMetastore_drop_database_args {
+  static $_TSPEC;
+
+  /**
+   * @var string
+   */
+  public $name = null;
+  /**
+   * @var bool
+   */
+  public $deleteData = null;
+  /**
+   * @var bool
+   */
+  public $cascade = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'name',
+          'type' => TType::STRING,
+          ),
+        2 => array(
+          'var' => 'deleteData',
+          'type' => TType::BOOL,
+          ),
+        3 => array(
+          'var' => 'cascade',
+          'type' => TType::BOOL,
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['name'])) {
+        $this->name = $vals['name'];
+      }
+      if (isset($vals['deleteData'])) {
+        $this->deleteData = $vals['deleteData'];
+      }
+      if (isset($vals['cascade'])) {
+        $this->cascade = $vals['cascade'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_drop_database_args';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->name);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::BOOL) {
+            $xfer += $input->readBool($this->deleteData);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 3:
+          if ($ftype == TType::BOOL) {
+            $xfer += $input->readBool($this->cascade);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_database_args');
+    if ($this->name !== null) {
+      $xfer += $output->writeFieldBegin('name', TType::STRING, 1);
+      $xfer += $output->writeString($this->name);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->deleteData !== null) {
+      $xfer += $output->writeFieldBegin('deleteData', TType::BOOL, 2);
+      $xfer += $output->writeBool($this->deleteData);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->cascade !== null) {
+      $xfer += $output->writeFieldBegin('cascade', TType::BOOL, 3);
+      $xfer += $output->writeBool($this->cascade);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class ThriftHiveMetastore_drop_database_result {
+  static $_TSPEC;
+
+  /**
+   * @var \metastore\NoSuchObjectException
+   */
+  public $o1 = null;
+  /**
+   * @var \metastore\InvalidOperationException
+   */
+  public $o2 = null;
+  /**
+   * @var \metastore\MetaException
+   */
+  public $o3 = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'o1',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\NoSuchObjectException',
+          ),
+        2 => array(
+          'var' => 'o2',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\InvalidOperationException',
+          ),
+        3 => array(
+          'var' => 'o3',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\MetaException',
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['o1'])) {
+        $this->o1 = $vals['o1'];
+      }
+      if (isset($vals['o2'])) {
+        $this->o2 = $vals['o2'];
+      }
+      if (isset($vals['o3'])) {
+        $this->o3 = $vals['o3'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_drop_database_result';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRUCT) {
+            $this->o1 = new \metastore\NoSuchObjectException();
+            $xfer += $this->o1->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::STRUCT) {
+            $this->o2 = new \metastore\InvalidOperationException();
+            $xfer += $this->o2->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 3:
+          if ($ftype == TType::STRUCT) {
+            $this->o3 = new \metastore\MetaException();
+            $xfer += $this->o3->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_drop_database_result');
+    if ($this->o1 !== null) {
+      $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1);
+      $xfer += $this->o1->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->o2 !== null) {
+      $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2);
+      $xfer += $this->o2->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->o3 !== null) {
+      $xfer += $output->writeFieldBegin('o3', TType::STRUCT, 3);
+      $xfer += $this->o3->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class ThriftHiveMetastore_get_databases_args {
+  static $_TSPEC;
+
+  /**
+   * @var string
+   */
+  public $pattern = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'pattern',
+          'type' => TType::STRING,
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['pattern'])) {
+        $this->pattern = $vals['pattern'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_get_databases_args';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->pattern);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_databases_args');
+    if ($this->pattern !== null) {
+      $xfer += $output->writeFieldBegin('pattern', TType::STRING, 1);
+      $xfer += $output->writeString($this->pattern);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class ThriftHiveMetastore_get_databases_result {
+  static $_TSPEC;
+
+  /**
+   * @var string[]
+   */
+  public $success = null;
+  /**
+   * @var \metastore\MetaException
+   */
+  public $o1 = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        0 => array(
+          'var' => 'success',
+          'type' => TType::LST,
+          'etype' => TType::STRING,
+          'elem' => array(
+            'type' => TType::STRING,
+            ),
+          ),
+        1 => array(
+          'var' => 'o1',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\MetaException',
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['success'])) {
+        $this->success = $vals['success'];
+      }
+      if (isset($vals['o1'])) {
+        $this->o1 = $vals['o1'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_get_databases_result';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 0:
+          if ($ftype == TType::LST) {
+            $this->success = array();
+            $_size778 = 0;
+            $_etype781 = 0;
+            $xfer += $input->readListBegin($_etype781, $_size778);
+            for ($_i782 = 0; $_i782 < $_size778; ++$_i782)
+            {
+              $elem783 = null;
+              $xfer += $input->readString($elem783);
+              $this->success []= $elem783;
+            }
+            $xfer += $input->readListEnd();
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 1:
+          if ($ftype == TType::STRUCT) {
+            $this->o1 = new \metastore\MetaException();
+            $xfer += $this->o1->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_databases_result');
+    if ($this->success !== null) {
+      if (!is_array($this->success)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('success', TType::LST, 0);
+      {
+        $output->writeListBegin(TType::STRING, count($this->success));
+        {
+          foreach ($this->success as $iter784)
+          {
+            $xfer += $output->writeString($iter784);
+          }
+        }
+        $output->writeListEnd();
+      }
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->o1 !== null) {
+      $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1);
+      $xfer += $this->o1->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class ThriftHiveMetastore_get_all_databases_args {
+  static $_TSPEC;
+
+
+  public function __construct() {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        );
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_get_all_databases_args';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_all_databases_args');
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class ThriftHiveMetastore_get_all_databases_result {
+  static $_TSPEC;
+
+  /**
+   * @var string[]
+   */
+  public $success = null;
+  /**
+   * @var \metastore\MetaException
+   */
+  public $o1 = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        0 => array(
+          'var' => 'success',
+          'type' => TType::LST,
+          'etype' => TType::STRING,
+          'elem' => array(
+            'type' => TType::STRING,
+            ),
+          ),
+        1 => array(
+          'var' => 'o1',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\MetaException',
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['success'])) {
+        $this->success = $vals['success'];
+      }
+      if (isset($vals['o1'])) {
+        $this->o1 = $vals['o1'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_get_all_databases_result';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 0:
+          if ($ftype == TType::LST) {
+            $this->success = array();
+            $_size785 = 0;
+            $_etype788 = 0;
+            $xfer += $input->readListBegin($_etype788, $_size785);
+            for ($_i789 = 0; $_i789 < $_size785; ++$_i789)
+            {
+              $elem790 = null;
+              $xfer += $input->readString($elem790);
+              $this->success []= $elem790;
+            }
+            $xfer += $input->readListEnd();
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 1:
+          if ($ftype == TType::STRUCT) {
+            $this->o1 = new \metastore\MetaException();
+            $xfer += $this->o1->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_all_databases_result');
+    if ($this->success !== null) {
+      if (!is_array($this->success)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('success', TType::LST, 0);
+      {
+        $output->writeListBegin(TType::STRING, count($this->success));
+        {
+          foreach ($this->success as $iter791)
+          {
+            $xfer += $output->writeString($iter791);
+          }
+        }
+        $output->writeListEnd();
+      }
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->o1 !== null) {
+      $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1);
+      $xfer += $this->o1->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class ThriftHiveMetastore_alter_database_args {
+  static $_TSPEC;
+
+  /**
+   * @var string
+   */
+  public $dbname = null;
+  /**
+   * @var \metastore\Database
+   */
+  public $db = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'dbname',
+          'type' => TType::STRING,
+          ),
+        2 => array(
+          'var' => 'db',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\Database',
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['dbname'])) {
+        $this->dbname = $vals['dbname'];
+      }
+      if (isset($vals['db'])) {
+        $this->db = $vals['db'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_alter_database_args';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->dbname);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::STRUCT) {
+            $this->db = new \metastore\Database();
+            $xfer += $this->db->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_database_args');
+    if ($this->dbname !== null) {
+      $xfer += $output->writeFieldBegin('dbname', TType::STRING, 1);
+      $xfer += $output->writeString($this->dbname);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->db !== null) {
+      if (!is_object($this->db)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('db', TType::STRUCT, 2);
+      $xfer += $this->db->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class ThriftHiveMetastore_alter_database_result {
+  static $_TSPEC;
+
+  /**
+   * @var \metastore\MetaException
+   */
+  public $o1 = null;
+  /**
+   * @var \metastore\NoSuchObjectException
+   */
+  public $o2 = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'o1',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\MetaException',
+          ),
+        2 => array(
+          'var' => 'o2',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\NoSuchObjectException',
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['o1'])) {
+        $this->o1 = $vals['o1'];
+      }
+      if (isset($vals['o2'])) {
+        $this->o2 = $vals['o2'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_alter_database_result';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRUCT) {
+            $this->o1 = new \metastore\MetaException();
+            $xfer += $this->o1->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::STRUCT) {
+            $this->o2 = new \metastore\NoSuchObjectException();
+            $xfer += $this->o2->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_alter_database_result');
+    if ($this->o1 !== null) {
+      $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1);
+      $xfer += $this->o1->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->o2 !== null) {
+      $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2);
+      $xfer += $this->o2->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class ThriftHiveMetastore_get_type_args {
+  static $_TSPEC;
+
+  /**
+   * @var string
+   */
+  public $name = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'name',
+          'type' => TType::STRING,
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['name'])) {
+        $this->name = $vals['name'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_get_type_args';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->name);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_type_args');
+    if ($this->name !== null) {
+      $xfer += $output->writeFieldBegin('name', TType::STRING, 1);
+      $xfer += $output->writeString($this->name);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class ThriftHiveMetastore_get_type_result {
+  static $_TSPEC;
+
+  /**
+   * @var \metastore\Type
+   */
+  public $success = null;
+  /**
+   * @var \metastore\MetaException
+   */
+  public $o1 = null;
+  /**
+   * @var \metastore\NoSuchObjectException
+   */
+  public $o2 = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        0 => array(
+          'var' => 'success',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\Type',
+          ),
+        1 => array(
+          'var' => 'o1',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\MetaException',
+          ),
+        2 => array(
+          'var' => 'o2',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\NoSuchObjectException',
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['success'])) {
+        $this->success = $vals['success'];
+      }
+      if (isset($vals['o1'])) {
+        $this->o1 = $vals['o1'];
+      }
+      if (isset($vals['o2'])) {
+        $this->o2 = $vals['o2'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_get_type_result';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 0:
+          if ($ftype == TType::STRUCT) {
+            $this->success = new \metastore\Type();
+            $xfer += $this->success->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 1:
+          if ($ftype == TType::STRUCT) {
+            $this->o1 = new \metastore\MetaException();
+            $xfer += $this->o1->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::STRUCT) {
+            $this->o2 = new \metastore\NoSuchObjectException();
+            $xfer += $this->o2->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_type_result');
+    if ($this->success !== null) {
+      if (!is_object($this->success)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0);
+      $xfer += $this->success->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->o1 !== null) {
+      $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1);
+      $xfer += $this->o1->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->o2 !== null) {
+      $xfer += $output->writeFieldBegin('o2', TType::STRUCT, 2);
+      $xfer += $this->o2->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class ThriftHiveMetastore_create_type_args {
+  static $_TSPEC;
+
+  /**
+   * @var \metastore\Type
+   */
+  public $type = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'type',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\Type',
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['type'])) {
+        $this->type = $vals['type'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_create_type_args';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRUCT) {
+            $this->type = new \metastore\Type();
+            $xfer += $this->type->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_create_type_args');
+    if ($this->type !== null) {
+      if (!is_object($this->type)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('type', TType::STRUCT, 1);
+      $xfer += $this->type->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class ThriftHiveMetastore_create_type_result {
+  static $_TSPEC;
+
+  /**
+   * @var bool
+   */
+  public $success = null;
+  /**
+   * @var \metastore\AlreadyExistsException
+   */
+  public $o1 = null;
+  /**
+   * @var \metastore\InvalidObjectException
+   */
+  public $o2 = null;
+  /**
+   * @var \metastore\MetaException
+   */
+  public $o3 = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        0 => array(
+          'var' => 'success',
+          'type' => TType::BOOL,
+          ),
+        1 => array(
+          'var' => 'o1',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\AlreadyExistsException',
+          ),
+        2 => array(
+          'var' => 'o2',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\InvalidObjectException',
+          ),
+        3 => array(
+          'var' => 'o3',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\MetaException',
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['success'])) {
+        $this->success = $vals['success'];
+      }
+      if (isset($vals['o1'])) {
+        $this->o1 = $vals['o1'];
+      }
+      if (isset($vals['o2'])) {
+        $this->o2 = $vals['o2'];
+      }
+      if (isset($vals['o3'])) {
+        $this->o3 = $vals['o3'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_create

<TRUNCATED>

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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
index 0290b7d..8d9ad25 100644
--- a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
+++ b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
@@ -922,6 +922,76 @@ class ThriftHiveMetastoreHandler : virtual public ThriftHiveMetastoreIf {
     printf("create_or_drop_wm_trigger_to_pool_mapping\n");
   }
 
+  void create_ischema(const ISchema& schema) {
+    // Your implementation goes here
+    printf("create_ischema\n");
+  }
+
+  void alter_ischema(const AlterISchemaRequest& rqst) {
+    // Your implementation goes here
+    printf("alter_ischema\n");
+  }
+
+  void get_ischema(ISchema& _return, const ISchemaName& name) {
+    // Your implementation goes here
+    printf("get_ischema\n");
+  }
+
+  void drop_ischema(const ISchemaName& name) {
+    // Your implementation goes here
+    printf("drop_ischema\n");
+  }
+
+  void add_schema_version(const SchemaVersion& schemaVersion) {
+    // Your implementation goes here
+    printf("add_schema_version\n");
+  }
+
+  void get_schema_version(SchemaVersion& _return, const SchemaVersionDescriptor& schemaVersion) {
+    // Your implementation goes here
+    printf("get_schema_version\n");
+  }
+
+  void get_schema_latest_version(SchemaVersion& _return, const ISchemaName& schemaName) {
+    // Your implementation goes here
+    printf("get_schema_latest_version\n");
+  }
+
+  void get_schema_all_versions(std::vector<SchemaVersion> & _return, const ISchemaName& schemaName) {
+    // Your implementation goes here
+    printf("get_schema_all_versions\n");
+  }
+
+  void drop_schema_version(const SchemaVersionDescriptor& schemaVersion) {
+    // Your implementation goes here
+    printf("drop_schema_version\n");
+  }
+
+  void get_schemas_by_cols(FindSchemasByColsResp& _return, const FindSchemasByColsRqst& rqst) {
+    // Your implementation goes here
+    printf("get_schemas_by_cols\n");
+  }
+
+  void map_schema_version_to_serde(const MapSchemaVersionToSerdeRequest& rqst) {
+    // Your implementation goes here
+    printf("map_schema_version_to_serde\n");
+  }
+
+  void set_schema_version_state(const SetSchemaVersionStateRequest& rqst) {
+    // Your implementation goes here
+    printf("set_schema_version_state\n");
+  }
+
+  void add_serde(const SerDeInfo& serde) {
+    // Your implementation goes here
+    printf("add_serde\n");
+  }
+
+  void get_serde(SerDeInfo& _return, const GetSerdeRequest& rqst) {
+    // Your implementation goes here
+    printf("get_serde\n");
+  }
+
 };
 
 int main(int argc, char **argv) {


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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
index 32a7086..b7a3b92 100644
--- a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
+++ b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
@@ -1240,14 +1240,14 @@ uint32_t ThriftHiveMetastore_get_databases_result::read(::apache::thrift::protoc
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1099;
-            ::apache::thrift::protocol::TType _etype1102;
-            xfer += iprot->readListBegin(_etype1102, _size1099);
-            this->success.resize(_size1099);
-            uint32_t _i1103;
-            for (_i1103 = 0; _i1103 < _size1099; ++_i1103)
+            uint32_t _size1137;
+            ::apache::thrift::protocol::TType _etype1140;
+            xfer += iprot->readListBegin(_etype1140, _size1137);
+            this->success.resize(_size1137);
+            uint32_t _i1141;
+            for (_i1141 = 0; _i1141 < _size1137; ++_i1141)
             {
-              xfer += iprot->readString(this->success[_i1103]);
+              xfer += iprot->readString(this->success[_i1141]);
             }
             xfer += iprot->readListEnd();
           }
@@ -1286,10 +1286,10 @@ uint32_t ThriftHiveMetastore_get_databases_result::write(::apache::thrift::proto
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter1104;
-      for (_iter1104 = this->success.begin(); _iter1104 != this->success.end(); ++_iter1104)
+      std::vector<std::string> ::const_iterator _iter1142;
+      for (_iter1142 = this->success.begin(); _iter1142 != this->success.end(); ++_iter1142)
       {
-        xfer += oprot->writeString((*_iter1104));
+        xfer += oprot->writeString((*_iter1142));
       }
       xfer += oprot->writeListEnd();
     }
@@ -1334,14 +1334,14 @@ uint32_t ThriftHiveMetastore_get_databases_presult::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1105;
-            ::apache::thrift::protocol::TType _etype1108;
-            xfer += iprot->readListBegin(_etype1108, _size1105);
-            (*(this->success)).resize(_size1105);
-            uint32_t _i1109;
-            for (_i1109 = 0; _i1109 < _size1105; ++_i1109)
+            uint32_t _size1143;
+            ::apache::thrift::protocol::TType _etype1146;
+            xfer += iprot->readListBegin(_etype1146, _size1143);
+            (*(this->success)).resize(_size1143);
+            uint32_t _i1147;
+            for (_i1147 = 0; _i1147 < _size1143; ++_i1147)
             {
-              xfer += iprot->readString((*(this->success))[_i1109]);
+              xfer += iprot->readString((*(this->success))[_i1147]);
             }
             xfer += iprot->readListEnd();
           }
@@ -1458,14 +1458,14 @@ uint32_t ThriftHiveMetastore_get_all_databases_result::read(::apache::thrift::pr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1110;
-            ::apache::thrift::protocol::TType _etype1113;
-            xfer += iprot->readListBegin(_etype1113, _size1110);
-            this->success.resize(_size1110);
-            uint32_t _i1114;
-            for (_i1114 = 0; _i1114 < _size1110; ++_i1114)
+            uint32_t _size1148;
+            ::apache::thrift::protocol::TType _etype1151;
+            xfer += iprot->readListBegin(_etype1151, _size1148);
+            this->success.resize(_size1148);
+            uint32_t _i1152;
+            for (_i1152 = 0; _i1152 < _size1148; ++_i1152)
             {
-              xfer += iprot->readString(this->success[_i1114]);
+              xfer += iprot->readString(this->success[_i1152]);
             }
             xfer += iprot->readListEnd();
           }
@@ -1504,10 +1504,10 @@ uint32_t ThriftHiveMetastore_get_all_databases_result::write(::apache::thrift::p
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter1115;
-      for (_iter1115 = this->success.begin(); _iter1115 != this->success.end(); ++_iter1115)
+      std::vector<std::string> ::const_iterator _iter1153;
+      for (_iter1153 = this->success.begin(); _iter1153 != this->success.end(); ++_iter1153)
       {
-        xfer += oprot->writeString((*_iter1115));
+        xfer += oprot->writeString((*_iter1153));
       }
       xfer += oprot->writeListEnd();
     }
@@ -1552,14 +1552,14 @@ uint32_t ThriftHiveMetastore_get_all_databases_presult::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1116;
-            ::apache::thrift::protocol::TType _etype1119;
-            xfer += iprot->readListBegin(_etype1119, _size1116);
-            (*(this->success)).resize(_size1116);
-            uint32_t _i1120;
-            for (_i1120 = 0; _i1120 < _size1116; ++_i1120)
+            uint32_t _size1154;
+            ::apache::thrift::protocol::TType _etype1157;
+            xfer += iprot->readListBegin(_etype1157, _size1154);
+            (*(this->success)).resize(_size1154);
+            uint32_t _i1158;
+            for (_i1158 = 0; _i1158 < _size1154; ++_i1158)
             {
-              xfer += iprot->readString((*(this->success))[_i1120]);
+              xfer += iprot->readString((*(this->success))[_i1158]);
             }
             xfer += iprot->readListEnd();
           }
@@ -2621,17 +2621,17 @@ uint32_t ThriftHiveMetastore_get_type_all_result::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->success.clear();
-            uint32_t _size1121;
-            ::apache::thrift::protocol::TType _ktype1122;
-            ::apache::thrift::protocol::TType _vtype1123;
-            xfer += iprot->readMapBegin(_ktype1122, _vtype1123, _size1121);
-            uint32_t _i1125;
-            for (_i1125 = 0; _i1125 < _size1121; ++_i1125)
+            uint32_t _size1159;
+            ::apache::thrift::protocol::TType _ktype1160;
+            ::apache::thrift::protocol::TType _vtype1161;
+            xfer += iprot->readMapBegin(_ktype1160, _vtype1161, _size1159);
+            uint32_t _i1163;
+            for (_i1163 = 0; _i1163 < _size1159; ++_i1163)
             {
-              std::string _key1126;
-              xfer += iprot->readString(_key1126);
-              Type& _val1127 = this->success[_key1126];
-              xfer += _val1127.read(iprot);
+              std::string _key1164;
+              xfer += iprot->readString(_key1164);
+              Type& _val1165 = this->success[_key1164];
+              xfer += _val1165.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -2670,11 +2670,11 @@ uint32_t ThriftHiveMetastore_get_type_all_result::write(::apache::thrift::protoc
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_MAP, 0);
     {
       xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::map<std::string, Type> ::const_iterator _iter1128;
-      for (_iter1128 = this->success.begin(); _iter1128 != this->success.end(); ++_iter1128)
+      std::map<std::string, Type> ::const_iterator _iter1166;
+      for (_iter1166 = this->success.begin(); _iter1166 != this->success.end(); ++_iter1166)
       {
-        xfer += oprot->writeString(_iter1128->first);
-        xfer += _iter1128->second.write(oprot);
+        xfer += oprot->writeString(_iter1166->first);
+        xfer += _iter1166->second.write(oprot);
       }
       xfer += oprot->writeMapEnd();
     }
@@ -2719,17 +2719,17 @@ uint32_t ThriftHiveMetastore_get_type_all_presult::read(::apache::thrift::protoc
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             (*(this->success)).clear();
-            uint32_t _size1129;
-            ::apache::thrift::protocol::TType _ktype1130;
-            ::apache::thrift::protocol::TType _vtype1131;
-            xfer += iprot->readMapBegin(_ktype1130, _vtype1131, _size1129);
-            uint32_t _i1133;
-            for (_i1133 = 0; _i1133 < _size1129; ++_i1133)
+            uint32_t _size1167;
+            ::apache::thrift::protocol::TType _ktype1168;
+            ::apache::thrift::protocol::TType _vtype1169;
+            xfer += iprot->readMapBegin(_ktype1168, _vtype1169, _size1167);
+            uint32_t _i1171;
+            for (_i1171 = 0; _i1171 < _size1167; ++_i1171)
             {
-              std::string _key1134;
-              xfer += iprot->readString(_key1134);
-              Type& _val1135 = (*(this->success))[_key1134];
-              xfer += _val1135.read(iprot);
+              std::string _key1172;
+              xfer += iprot->readString(_key1172);
+              Type& _val1173 = (*(this->success))[_key1172];
+              xfer += _val1173.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -2883,14 +2883,14 @@ uint32_t ThriftHiveMetastore_get_fields_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1136;
-            ::apache::thrift::protocol::TType _etype1139;
-            xfer += iprot->readListBegin(_etype1139, _size1136);
-            this->success.resize(_size1136);
-            uint32_t _i1140;
-            for (_i1140 = 0; _i1140 < _size1136; ++_i1140)
+            uint32_t _size1174;
+            ::apache::thrift::protocol::TType _etype1177;
+            xfer += iprot->readListBegin(_etype1177, _size1174);
+            this->success.resize(_size1174);
+            uint32_t _i1178;
+            for (_i1178 = 0; _i1178 < _size1174; ++_i1178)
             {
-              xfer += this->success[_i1140].read(iprot);
+              xfer += this->success[_i1178].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -2945,10 +2945,10 @@ uint32_t ThriftHiveMetastore_get_fields_result::write(::apache::thrift::protocol
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter1141;
-      for (_iter1141 = this->success.begin(); _iter1141 != this->success.end(); ++_iter1141)
+      std::vector<FieldSchema> ::const_iterator _iter1179;
+      for (_iter1179 = this->success.begin(); _iter1179 != this->success.end(); ++_iter1179)
       {
-        xfer += (*_iter1141).write(oprot);
+        xfer += (*_iter1179).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -3001,14 +3001,14 @@ uint32_t ThriftHiveMetastore_get_fields_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1142;
-            ::apache::thrift::protocol::TType _etype1145;
-            xfer += iprot->readListBegin(_etype1145, _size1142);
-            (*(this->success)).resize(_size1142);
-            uint32_t _i1146;
-            for (_i1146 = 0; _i1146 < _size1142; ++_i1146)
+            uint32_t _size1180;
+            ::apache::thrift::protocol::TType _etype1183;
+            xfer += iprot->readListBegin(_etype1183, _size1180);
+            (*(this->success)).resize(_size1180);
+            uint32_t _i1184;
+            for (_i1184 = 0; _i1184 < _size1180; ++_i1184)
             {
-              xfer += (*(this->success))[_i1146].read(iprot);
+              xfer += (*(this->success))[_i1184].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3194,14 +3194,14 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_result::read(::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1147;
-            ::apache::thrift::protocol::TType _etype1150;
-            xfer += iprot->readListBegin(_etype1150, _size1147);
-            this->success.resize(_size1147);
-            uint32_t _i1151;
-            for (_i1151 = 0; _i1151 < _size1147; ++_i1151)
+            uint32_t _size1185;
+            ::apache::thrift::protocol::TType _etype1188;
+            xfer += iprot->readListBegin(_etype1188, _size1185);
+            this->success.resize(_size1185);
+            uint32_t _i1189;
+            for (_i1189 = 0; _i1189 < _size1185; ++_i1189)
             {
-              xfer += this->success[_i1151].read(iprot);
+              xfer += this->success[_i1189].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3256,10 +3256,10 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_result::write(:
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter1152;
-      for (_iter1152 = this->success.begin(); _iter1152 != this->success.end(); ++_iter1152)
+      std::vector<FieldSchema> ::const_iterator _iter1190;
+      for (_iter1190 = this->success.begin(); _iter1190 != this->success.end(); ++_iter1190)
       {
-        xfer += (*_iter1152).write(oprot);
+        xfer += (*_iter1190).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -3312,14 +3312,14 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_presult::read(:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1153;
-            ::apache::thrift::protocol::TType _etype1156;
-            xfer += iprot->readListBegin(_etype1156, _size1153);
-            (*(this->success)).resize(_size1153);
-            uint32_t _i1157;
-            for (_i1157 = 0; _i1157 < _size1153; ++_i1157)
+            uint32_t _size1191;
+            ::apache::thrift::protocol::TType _etype1194;
+            xfer += iprot->readListBegin(_etype1194, _size1191);
+            (*(this->success)).resize(_size1191);
+            uint32_t _i1195;
+            for (_i1195 = 0; _i1195 < _size1191; ++_i1195)
             {
-              xfer += (*(this->success))[_i1157].read(iprot);
+              xfer += (*(this->success))[_i1195].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3489,14 +3489,14 @@ uint32_t ThriftHiveMetastore_get_schema_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1158;
-            ::apache::thrift::protocol::TType _etype1161;
-            xfer += iprot->readListBegin(_etype1161, _size1158);
-            this->success.resize(_size1158);
-            uint32_t _i1162;
-            for (_i1162 = 0; _i1162 < _size1158; ++_i1162)
+            uint32_t _size1196;
+            ::apache::thrift::protocol::TType _etype1199;
+            xfer += iprot->readListBegin(_etype1199, _size1196);
+            this->success.resize(_size1196);
+            uint32_t _i1200;
+            for (_i1200 = 0; _i1200 < _size1196; ++_i1200)
             {
-              xfer += this->success[_i1162].read(iprot);
+              xfer += this->success[_i1200].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3551,10 +3551,10 @@ uint32_t ThriftHiveMetastore_get_schema_result::write(::apache::thrift::protocol
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter1163;
-      for (_iter1163 = this->success.begin(); _iter1163 != this->success.end(); ++_iter1163)
+      std::vector<FieldSchema> ::const_iterator _iter1201;
+      for (_iter1201 = this->success.begin(); _iter1201 != this->success.end(); ++_iter1201)
       {
-        xfer += (*_iter1163).write(oprot);
+        xfer += (*_iter1201).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -3607,14 +3607,14 @@ uint32_t ThriftHiveMetastore_get_schema_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1164;
-            ::apache::thrift::protocol::TType _etype1167;
-            xfer += iprot->readListBegin(_etype1167, _size1164);
-            (*(this->success)).resize(_size1164);
-            uint32_t _i1168;
-            for (_i1168 = 0; _i1168 < _size1164; ++_i1168)
+            uint32_t _size1202;
+            ::apache::thrift::protocol::TType _etype1205;
+            xfer += iprot->readListBegin(_etype1205, _size1202);
+            (*(this->success)).resize(_size1202);
+            uint32_t _i1206;
+            for (_i1206 = 0; _i1206 < _size1202; ++_i1206)
             {
-              xfer += (*(this->success))[_i1168].read(iprot);
+              xfer += (*(this->success))[_i1206].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3800,14 +3800,14 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_result::read(::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1169;
-            ::apache::thrift::protocol::TType _etype1172;
-            xfer += iprot->readListBegin(_etype1172, _size1169);
-            this->success.resize(_size1169);
-            uint32_t _i1173;
-            for (_i1173 = 0; _i1173 < _size1169; ++_i1173)
+            uint32_t _size1207;
+            ::apache::thrift::protocol::TType _etype1210;
+            xfer += iprot->readListBegin(_etype1210, _size1207);
+            this->success.resize(_size1207);
+            uint32_t _i1211;
+            for (_i1211 = 0; _i1211 < _size1207; ++_i1211)
             {
-              xfer += this->success[_i1173].read(iprot);
+              xfer += this->success[_i1211].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3862,10 +3862,10 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_result::write(:
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter1174;
-      for (_iter1174 = this->success.begin(); _iter1174 != this->success.end(); ++_iter1174)
+      std::vector<FieldSchema> ::const_iterator _iter1212;
+      for (_iter1212 = this->success.begin(); _iter1212 != this->success.end(); ++_iter1212)
       {
-        xfer += (*_iter1174).write(oprot);
+        xfer += (*_iter1212).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -3918,14 +3918,14 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_presult::read(:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1175;
-            ::apache::thrift::protocol::TType _etype1178;
-            xfer += iprot->readListBegin(_etype1178, _size1175);
-            (*(this->success)).resize(_size1175);
-            uint32_t _i1179;
-            for (_i1179 = 0; _i1179 < _size1175; ++_i1179)
+            uint32_t _size1213;
+            ::apache::thrift::protocol::TType _etype1216;
+            xfer += iprot->readListBegin(_etype1216, _size1213);
+            (*(this->success)).resize(_size1213);
+            uint32_t _i1217;
+            for (_i1217 = 0; _i1217 < _size1213; ++_i1217)
             {
-              xfer += (*(this->success))[_i1179].read(iprot);
+              xfer += (*(this->success))[_i1217].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4518,14 +4518,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->primaryKeys.clear();
-            uint32_t _size1180;
-            ::apache::thrift::protocol::TType _etype1183;
-            xfer += iprot->readListBegin(_etype1183, _size1180);
-            this->primaryKeys.resize(_size1180);
-            uint32_t _i1184;
-            for (_i1184 = 0; _i1184 < _size1180; ++_i1184)
+            uint32_t _size1218;
+            ::apache::thrift::protocol::TType _etype1221;
+            xfer += iprot->readListBegin(_etype1221, _size1218);
+            this->primaryKeys.resize(_size1218);
+            uint32_t _i1222;
+            for (_i1222 = 0; _i1222 < _size1218; ++_i1222)
             {
-              xfer += this->primaryKeys[_i1184].read(iprot);
+              xfer += this->primaryKeys[_i1222].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4538,14 +4538,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->foreignKeys.clear();
-            uint32_t _size1185;
-            ::apache::thrift::protocol::TType _etype1188;
-            xfer += iprot->readListBegin(_etype1188, _size1185);
-            this->foreignKeys.resize(_size1185);
-            uint32_t _i1189;
-            for (_i1189 = 0; _i1189 < _size1185; ++_i1189)
+            uint32_t _size1223;
+            ::apache::thrift::protocol::TType _etype1226;
+            xfer += iprot->readListBegin(_etype1226, _size1223);
+            this->foreignKeys.resize(_size1223);
+            uint32_t _i1227;
+            for (_i1227 = 0; _i1227 < _size1223; ++_i1227)
             {
-              xfer += this->foreignKeys[_i1189].read(iprot);
+              xfer += this->foreignKeys[_i1227].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4558,14 +4558,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->uniqueConstraints.clear();
-            uint32_t _size1190;
-            ::apache::thrift::protocol::TType _etype1193;
-            xfer += iprot->readListBegin(_etype1193, _size1190);
-            this->uniqueConstraints.resize(_size1190);
-            uint32_t _i1194;
-            for (_i1194 = 0; _i1194 < _size1190; ++_i1194)
+            uint32_t _size1228;
+            ::apache::thrift::protocol::TType _etype1231;
+            xfer += iprot->readListBegin(_etype1231, _size1228);
+            this->uniqueConstraints.resize(_size1228);
+            uint32_t _i1232;
+            for (_i1232 = 0; _i1232 < _size1228; ++_i1232)
             {
-              xfer += this->uniqueConstraints[_i1194].read(iprot);
+              xfer += this->uniqueConstraints[_i1232].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4578,14 +4578,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->notNullConstraints.clear();
-            uint32_t _size1195;
-            ::apache::thrift::protocol::TType _etype1198;
-            xfer += iprot->readListBegin(_etype1198, _size1195);
-            this->notNullConstraints.resize(_size1195);
-            uint32_t _i1199;
-            for (_i1199 = 0; _i1199 < _size1195; ++_i1199)
+            uint32_t _size1233;
+            ::apache::thrift::protocol::TType _etype1236;
+            xfer += iprot->readListBegin(_etype1236, _size1233);
+            this->notNullConstraints.resize(_size1233);
+            uint32_t _i1237;
+            for (_i1237 = 0; _i1237 < _size1233; ++_i1237)
             {
-              xfer += this->notNullConstraints[_i1199].read(iprot);
+              xfer += this->notNullConstraints[_i1237].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4598,14 +4598,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->defaultConstraints.clear();
-            uint32_t _size1200;
-            ::apache::thrift::protocol::TType _etype1203;
-            xfer += iprot->readListBegin(_etype1203, _size1200);
-            this->defaultConstraints.resize(_size1200);
-            uint32_t _i1204;
-            for (_i1204 = 0; _i1204 < _size1200; ++_i1204)
+            uint32_t _size1238;
+            ::apache::thrift::protocol::TType _etype1241;
+            xfer += iprot->readListBegin(_etype1241, _size1238);
+            this->defaultConstraints.resize(_size1238);
+            uint32_t _i1242;
+            for (_i1242 = 0; _i1242 < _size1238; ++_i1242)
             {
-              xfer += this->defaultConstraints[_i1204].read(iprot);
+              xfer += this->defaultConstraints[_i1242].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4638,10 +4638,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache:
   xfer += oprot->writeFieldBegin("primaryKeys", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->primaryKeys.size()));
-    std::vector<SQLPrimaryKey> ::const_iterator _iter1205;
-    for (_iter1205 = this->primaryKeys.begin(); _iter1205 != this->primaryKeys.end(); ++_iter1205)
+    std::vector<SQLPrimaryKey> ::const_iterator _iter1243;
+    for (_iter1243 = this->primaryKeys.begin(); _iter1243 != this->primaryKeys.end(); ++_iter1243)
     {
-      xfer += (*_iter1205).write(oprot);
+      xfer += (*_iter1243).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -4650,10 +4650,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache:
   xfer += oprot->writeFieldBegin("foreignKeys", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->foreignKeys.size()));
-    std::vector<SQLForeignKey> ::const_iterator _iter1206;
-    for (_iter1206 = this->foreignKeys.begin(); _iter1206 != this->foreignKeys.end(); ++_iter1206)
+    std::vector<SQLForeignKey> ::const_iterator _iter1244;
+    for (_iter1244 = this->foreignKeys.begin(); _iter1244 != this->foreignKeys.end(); ++_iter1244)
     {
-      xfer += (*_iter1206).write(oprot);
+      xfer += (*_iter1244).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -4662,10 +4662,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache:
   xfer += oprot->writeFieldBegin("uniqueConstraints", ::apache::thrift::protocol::T_LIST, 4);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->uniqueConstraints.size()));
-    std::vector<SQLUniqueConstraint> ::const_iterator _iter1207;
-    for (_iter1207 = this->uniqueConstraints.begin(); _iter1207 != this->uniqueConstraints.end(); ++_iter1207)
+    std::vector<SQLUniqueConstraint> ::const_iterator _iter1245;
+    for (_iter1245 = this->uniqueConstraints.begin(); _iter1245 != this->uniqueConstraints.end(); ++_iter1245)
     {
-      xfer += (*_iter1207).write(oprot);
+      xfer += (*_iter1245).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -4674,10 +4674,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache:
   xfer += oprot->writeFieldBegin("notNullConstraints", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->notNullConstraints.size()));
-    std::vector<SQLNotNullConstraint> ::const_iterator _iter1208;
-    for (_iter1208 = this->notNullConstraints.begin(); _iter1208 != this->notNullConstraints.end(); ++_iter1208)
+    std::vector<SQLNotNullConstraint> ::const_iterator _iter1246;
+    for (_iter1246 = this->notNullConstraints.begin(); _iter1246 != this->notNullConstraints.end(); ++_iter1246)
     {
-      xfer += (*_iter1208).write(oprot);
+      xfer += (*_iter1246).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -4686,10 +4686,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache:
   xfer += oprot->writeFieldBegin("defaultConstraints", ::apache::thrift::protocol::T_LIST, 6);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->defaultConstraints.size()));
-    std::vector<SQLDefaultConstraint> ::const_iterator _iter1209;
-    for (_iter1209 = this->defaultConstraints.begin(); _iter1209 != this->defaultConstraints.end(); ++_iter1209)
+    std::vector<SQLDefaultConstraint> ::const_iterator _iter1247;
+    for (_iter1247 = this->defaultConstraints.begin(); _iter1247 != this->defaultConstraints.end(); ++_iter1247)
     {
-      xfer += (*_iter1209).write(oprot);
+      xfer += (*_iter1247).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -4717,10 +4717,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache
   xfer += oprot->writeFieldBegin("primaryKeys", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->primaryKeys)).size()));
-    std::vector<SQLPrimaryKey> ::const_iterator _iter1210;
-    for (_iter1210 = (*(this->primaryKeys)).begin(); _iter1210 != (*(this->primaryKeys)).end(); ++_iter1210)
+    std::vector<SQLPrimaryKey> ::const_iterator _iter1248;
+    for (_iter1248 = (*(this->primaryKeys)).begin(); _iter1248 != (*(this->primaryKeys)).end(); ++_iter1248)
     {
-      xfer += (*_iter1210).write(oprot);
+      xfer += (*_iter1248).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -4729,10 +4729,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache
   xfer += oprot->writeFieldBegin("foreignKeys", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->foreignKeys)).size()));
-    std::vector<SQLForeignKey> ::const_iterator _iter1211;
-    for (_iter1211 = (*(this->foreignKeys)).begin(); _iter1211 != (*(this->foreignKeys)).end(); ++_iter1211)
+    std::vector<SQLForeignKey> ::const_iterator _iter1249;
+    for (_iter1249 = (*(this->foreignKeys)).begin(); _iter1249 != (*(this->foreignKeys)).end(); ++_iter1249)
     {
-      xfer += (*_iter1211).write(oprot);
+      xfer += (*_iter1249).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -4741,10 +4741,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache
   xfer += oprot->writeFieldBegin("uniqueConstraints", ::apache::thrift::protocol::T_LIST, 4);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->uniqueConstraints)).size()));
-    std::vector<SQLUniqueConstraint> ::const_iterator _iter1212;
-    for (_iter1212 = (*(this->uniqueConstraints)).begin(); _iter1212 != (*(this->uniqueConstraints)).end(); ++_iter1212)
+    std::vector<SQLUniqueConstraint> ::const_iterator _iter1250;
+    for (_iter1250 = (*(this->uniqueConstraints)).begin(); _iter1250 != (*(this->uniqueConstraints)).end(); ++_iter1250)
     {
-      xfer += (*_iter1212).write(oprot);
+      xfer += (*_iter1250).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -4753,10 +4753,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache
   xfer += oprot->writeFieldBegin("notNullConstraints", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->notNullConstraints)).size()));
-    std::vector<SQLNotNullConstraint> ::const_iterator _iter1213;
-    for (_iter1213 = (*(this->notNullConstraints)).begin(); _iter1213 != (*(this->notNullConstraints)).end(); ++_iter1213)
+    std::vector<SQLNotNullConstraint> ::const_iterator _iter1251;
+    for (_iter1251 = (*(this->notNullConstraints)).begin(); _iter1251 != (*(this->notNullConstraints)).end(); ++_iter1251)
     {
-      xfer += (*_iter1213).write(oprot);
+      xfer += (*_iter1251).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -4765,10 +4765,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache
   xfer += oprot->writeFieldBegin("defaultConstraints", ::apache::thrift::protocol::T_LIST, 6);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->defaultConstraints)).size()));
-    std::vector<SQLDefaultConstraint> ::const_iterator _iter1214;
-    for (_iter1214 = (*(this->defaultConstraints)).begin(); _iter1214 != (*(this->defaultConstraints)).end(); ++_iter1214)
+    std::vector<SQLDefaultConstraint> ::const_iterator _iter1252;
+    for (_iter1252 = (*(this->defaultConstraints)).begin(); _iter1252 != (*(this->defaultConstraints)).end(); ++_iter1252)
     {
-      xfer += (*_iter1214).write(oprot);
+      xfer += (*_iter1252).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -6729,14 +6729,14 @@ uint32_t ThriftHiveMetastore_truncate_table_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partNames.clear();
-            uint32_t _size1215;
-            ::apache::thrift::protocol::TType _etype1218;
-            xfer += iprot->readListBegin(_etype1218, _size1215);
-            this->partNames.resize(_size1215);
-            uint32_t _i1219;
-            for (_i1219 = 0; _i1219 < _size1215; ++_i1219)
+            uint32_t _size1253;
+            ::apache::thrift::protocol::TType _etype1256;
+            xfer += iprot->readListBegin(_etype1256, _size1253);
+            this->partNames.resize(_size1253);
+            uint32_t _i1257;
+            for (_i1257 = 0; _i1257 < _size1253; ++_i1257)
             {
-              xfer += iprot->readString(this->partNames[_i1219]);
+              xfer += iprot->readString(this->partNames[_i1257]);
             }
             xfer += iprot->readListEnd();
           }
@@ -6773,10 +6773,10 @@ uint32_t ThriftHiveMetastore_truncate_table_args::write(::apache::thrift::protoc
   xfer += oprot->writeFieldBegin("partNames", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partNames.size()));
-    std::vector<std::string> ::const_iterator _iter1220;
-    for (_iter1220 = this->partNames.begin(); _iter1220 != this->partNames.end(); ++_iter1220)
+    std::vector<std::string> ::const_iterator _iter1258;
+    for (_iter1258 = this->partNames.begin(); _iter1258 != this->partNames.end(); ++_iter1258)
     {
-      xfer += oprot->writeString((*_iter1220));
+      xfer += oprot->writeString((*_iter1258));
     }
     xfer += oprot->writeListEnd();
   }
@@ -6808,10 +6808,10 @@ uint32_t ThriftHiveMetastore_truncate_table_pargs::write(::apache::thrift::proto
   xfer += oprot->writeFieldBegin("partNames", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->partNames)).size()));
-    std::vector<std::string> ::const_iterator _iter1221;
-    for (_iter1221 = (*(this->partNames)).begin(); _iter1221 != (*(this->partNames)).end(); ++_iter1221)
+    std::vector<std::string> ::const_iterator _iter1259;
+    for (_iter1259 = (*(this->partNames)).begin(); _iter1259 != (*(this->partNames)).end(); ++_iter1259)
     {
-      xfer += oprot->writeString((*_iter1221));
+      xfer += oprot->writeString((*_iter1259));
     }
     xfer += oprot->writeListEnd();
   }
@@ -7055,14 +7055,14 @@ uint32_t ThriftHiveMetastore_get_tables_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1222;
-            ::apache::thrift::protocol::TType _etype1225;
-            xfer += iprot->readListBegin(_etype1225, _size1222);
-            this->success.resize(_size1222);
-            uint32_t _i1226;
-            for (_i1226 = 0; _i1226 < _size1222; ++_i1226)
+            uint32_t _size1260;
+            ::apache::thrift::protocol::TType _etype1263;
+            xfer += iprot->readListBegin(_etype1263, _size1260);
+            this->success.resize(_size1260);
+            uint32_t _i1264;
+            for (_i1264 = 0; _i1264 < _size1260; ++_i1264)
             {
-              xfer += iprot->readString(this->success[_i1226]);
+              xfer += iprot->readString(this->success[_i1264]);
             }
             xfer += iprot->readListEnd();
           }
@@ -7101,10 +7101,10 @@ uint32_t ThriftHiveMetastore_get_tables_result::write(::apache::thrift::protocol
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter1227;
-      for (_iter1227 = this->success.begin(); _iter1227 != this->success.end(); ++_iter1227)
+      std::vector<std::string> ::const_iterator _iter1265;
+      for (_iter1265 = this->success.begin(); _iter1265 != this->success.end(); ++_iter1265)
       {
-        xfer += oprot->writeString((*_iter1227));
+        xfer += oprot->writeString((*_iter1265));
       }
       xfer += oprot->writeListEnd();
     }
@@ -7149,14 +7149,14 @@ uint32_t ThriftHiveMetastore_get_tables_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1228;
-            ::apache::thrift::protocol::TType _etype1231;
-            xfer += iprot->readListBegin(_etype1231, _size1228);
-            (*(this->success)).resize(_size1228);
-            uint32_t _i1232;
-            for (_i1232 = 0; _i1232 < _size1228; ++_i1232)
+            uint32_t _size1266;
+            ::apache::thrift::protocol::TType _etype1269;
+            xfer += iprot->readListBegin(_etype1269, _size1266);
+            (*(this->success)).resize(_size1266);
+            uint32_t _i1270;
+            for (_i1270 = 0; _i1270 < _size1266; ++_i1270)
             {
-              xfer += iprot->readString((*(this->success))[_i1232]);
+              xfer += iprot->readString((*(this->success))[_i1270]);
             }
             xfer += iprot->readListEnd();
           }
@@ -7326,253 +7326,253 @@ uint32_t ThriftHiveMetastore_get_tables_by_type_result::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1233;
-            ::apache::thrift::protocol::TType _etype1236;
-            xfer += iprot->readListBegin(_etype1236, _size1233);
-            this->success.resize(_size1233);
-            uint32_t _i1237;
-            for (_i1237 = 0; _i1237 < _size1233; ++_i1237)
-            {
-              xfer += iprot->readString(this->success[_i1237]);
-            }
-            xfer += iprot->readListEnd();
-          }
-          this->__isset.success = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 1:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->o1.read(iprot);
-          this->__isset.o1 = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      default:
-        xfer += iprot->skip(ftype);
-        break;
-    }
-    xfer += iprot->readFieldEnd();
-  }
-
-  xfer += iprot->readStructEnd();
-
-  return xfer;
-}
-
-uint32_t ThriftHiveMetastore_get_tables_by_type_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
-
-  uint32_t xfer = 0;
-
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_tables_by_type_result");
-
-  if (this->__isset.success) {
-    xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
-    {
-      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter1238;
-      for (_iter1238 = this->success.begin(); _iter1238 != this->success.end(); ++_iter1238)
-      {
-        xfer += oprot->writeString((*_iter1238));
-      }
-      xfer += oprot->writeListEnd();
-    }
-    xfer += oprot->writeFieldEnd();
-  } else if (this->__isset.o1) {
-    xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1);
-    xfer += this->o1.write(oprot);
-    xfer += oprot->writeFieldEnd();
-  }
-  xfer += oprot->writeFieldStop();
-  xfer += oprot->writeStructEnd();
-  return xfer;
-}
-
-
-ThriftHiveMetastore_get_tables_by_type_presult::~ThriftHiveMetastore_get_tables_by_type_presult() throw() {
-}
-
-
-uint32_t ThriftHiveMetastore_get_tables_by_type_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
-
-  apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
-  uint32_t xfer = 0;
-  std::string fname;
-  ::apache::thrift::protocol::TType ftype;
-  int16_t fid;
-
-  xfer += iprot->readStructBegin(fname);
-
-  using ::apache::thrift::protocol::TProtocolException;
-
-
-  while (true)
-  {
-    xfer += iprot->readFieldBegin(fname, ftype, fid);
-    if (ftype == ::apache::thrift::protocol::T_STOP) {
-      break;
-    }
-    switch (fid)
-    {
-      case 0:
-        if (ftype == ::apache::thrift::protocol::T_LIST) {
-          {
-            (*(this->success)).clear();
-            uint32_t _size1239;
-            ::apache::thrift::protocol::TType _etype1242;
-            xfer += iprot->readListBegin(_etype1242, _size1239);
-            (*(this->success)).resize(_size1239);
-            uint32_t _i1243;
-            for (_i1243 = 0; _i1243 < _size1239; ++_i1243)
-            {
-              xfer += iprot->readString((*(this->success))[_i1243]);
-            }
-            xfer += iprot->readListEnd();
-          }
-          this->__isset.success = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 1:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->o1.read(iprot);
-          this->__isset.o1 = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      default:
-        xfer += iprot->skip(ftype);
-        break;
-    }
-    xfer += iprot->readFieldEnd();
-  }
-
-  xfer += iprot->readStructEnd();
-
-  return xfer;
-}
-
-
-ThriftHiveMetastore_get_materialized_views_for_rewriting_args::~ThriftHiveMetastore_get_materialized_views_for_rewriting_args() throw() {
-}
-
-
-uint32_t ThriftHiveMetastore_get_materialized_views_for_rewriting_args::read(::apache::thrift::protocol::TProtocol* iprot) {
-
-  apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
-  uint32_t xfer = 0;
-  std::string fname;
-  ::apache::thrift::protocol::TType ftype;
-  int16_t fid;
-
-  xfer += iprot->readStructBegin(fname);
-
-  using ::apache::thrift::protocol::TProtocolException;
-
-
-  while (true)
-  {
-    xfer += iprot->readFieldBegin(fname, ftype, fid);
-    if (ftype == ::apache::thrift::protocol::T_STOP) {
-      break;
-    }
-    switch (fid)
-    {
-      case 1:
-        if (ftype == ::apache::thrift::protocol::T_STRING) {
-          xfer += iprot->readString(this->db_name);
-          this->__isset.db_name = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      default:
-        xfer += iprot->skip(ftype);
-        break;
-    }
-    xfer += iprot->readFieldEnd();
-  }
-
-  xfer += iprot->readStructEnd();
-
-  return xfer;
-}
-
-uint32_t ThriftHiveMetastore_get_materialized_views_for_rewriting_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
-  uint32_t xfer = 0;
-  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_materialized_views_for_rewriting_args");
-
-  xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1);
-  xfer += oprot->writeString(this->db_name);
-  xfer += oprot->writeFieldEnd();
-
-  xfer += oprot->writeFieldStop();
-  xfer += oprot->writeStructEnd();
-  return xfer;
-}
-
-
-ThriftHiveMetastore_get_materialized_views_for_rewriting_pargs::~ThriftHiveMetastore_get_materialized_views_for_rewriting_pargs() throw() {
-}
-
-
-uint32_t ThriftHiveMetastore_get_materialized_views_for_rewriting_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
-  uint32_t xfer = 0;
-  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_materialized_views_for_rewriting_pargs");
-
-  xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1);
-  xfer += oprot->writeString((*(this->db_name)));
-  xfer += oprot->writeFieldEnd();
-
-  xfer += oprot->writeFieldStop();
-  xfer += oprot->writeStructEnd();
-  return xfer;
-}
-
-
-ThriftHiveMetastore_get_materialized_views_for_rewriting_result::~ThriftHiveMetastore_get_materialized_views_for_rewriting_result() throw() {
-}
-
-
-uint32_t ThriftHiveMetastore_get_materialized_views_for_rewriting_result::read(::apache::thrift::protocol::TProtocol* iprot) {
-
-  apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
-  uint32_t xfer = 0;
-  std::string fname;
-  ::apache::thrift::protocol::TType ftype;
-  int16_t fid;
-
-  xfer += iprot->readStructBegin(fname);
-
-  using ::apache::thrift::protocol::TProtocolException;
-
-
-  while (true)
-  {
-    xfer += iprot->readFieldBegin(fname, ftype, fid);
-    if (ftype == ::apache::thrift::protocol::T_STOP) {
-      break;
-    }
-    switch (fid)
-    {
-      case 0:
-        if (ftype == ::apache::thrift::protocol::T_LIST) {
-          {
-            this->success.clear();
-            uint32_t _size1244;
-            ::apache::thrift::protocol::TType _etype1247;
-            xfer += iprot->readListBegin(_etype1247, _size1244);
-            this->success.resize(_size1244);
-            uint32_t _i1248;
-            for (_i1248 = 0; _i1248 < _size1244; ++_i1248)
+            uint32_t _size1271;
+            ::apache::thrift::protocol::TType _etype1274;
+            xfer += iprot->readListBegin(_etype1274, _size1271);
+            this->success.resize(_size1271);
+            uint32_t _i1275;
+            for (_i1275 = 0; _i1275 < _size1271; ++_i1275)
+            {
+              xfer += iprot->readString(this->success[_i1275]);
+            }
+            xfer += iprot->readListEnd();
+          }
+          this->__isset.success = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->o1.read(iprot);
+          this->__isset.o1 = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  return xfer;
+}
+
+uint32_t ThriftHiveMetastore_get_tables_by_type_result::write(::apache::thrift::protocol::TProtocol* oprot) const {
+
+  uint32_t xfer = 0;
+
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_tables_by_type_result");
+
+  if (this->__isset.success) {
+    xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
+    {
+      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
+      std::vector<std::string> ::const_iterator _iter1276;
+      for (_iter1276 = this->success.begin(); _iter1276 != this->success.end(); ++_iter1276)
+      {
+        xfer += oprot->writeString((*_iter1276));
+      }
+      xfer += oprot->writeListEnd();
+    }
+    xfer += oprot->writeFieldEnd();
+  } else if (this->__isset.o1) {
+    xfer += oprot->writeFieldBegin("o1", ::apache::thrift::protocol::T_STRUCT, 1);
+    xfer += this->o1.write(oprot);
+    xfer += oprot->writeFieldEnd();
+  }
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+
+ThriftHiveMetastore_get_tables_by_type_presult::~ThriftHiveMetastore_get_tables_by_type_presult() throw() {
+}
+
+
+uint32_t ThriftHiveMetastore_get_tables_by_type_presult::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
+  uint32_t xfer = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TType ftype;
+  int16_t fid;
+
+  xfer += iprot->readStructBegin(fname);
+
+  using ::apache::thrift::protocol::TProtocolException;
+
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    switch (fid)
+    {
+      case 0:
+        if (ftype == ::apache::thrift::protocol::T_LIST) {
+          {
+            (*(this->success)).clear();
+            uint32_t _size1277;
+            ::apache::thrift::protocol::TType _etype1280;
+            xfer += iprot->readListBegin(_etype1280, _size1277);
+            (*(this->success)).resize(_size1277);
+            uint32_t _i1281;
+            for (_i1281 = 0; _i1281 < _size1277; ++_i1281)
+            {
+              xfer += iprot->readString((*(this->success))[_i1281]);
+            }
+            xfer += iprot->readListEnd();
+          }
+          this->__isset.success = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->o1.read(iprot);
+          this->__isset.o1 = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  return xfer;
+}
+
+
+ThriftHiveMetastore_get_materialized_views_for_rewriting_args::~ThriftHiveMetastore_get_materialized_views_for_rewriting_args() throw() {
+}
+
+
+uint32_t ThriftHiveMetastore_get_materialized_views_for_rewriting_args::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
+  uint32_t xfer = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TType ftype;
+  int16_t fid;
+
+  xfer += iprot->readStructBegin(fname);
+
+  using ::apache::thrift::protocol::TProtocolException;
+
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    switch (fid)
+    {
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->db_name);
+          this->__isset.db_name = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  return xfer;
+}
+
+uint32_t ThriftHiveMetastore_get_materialized_views_for_rewriting_args::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_materialized_views_for_rewriting_args");
+
+  xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeString(this->db_name);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+
+ThriftHiveMetastore_get_materialized_views_for_rewriting_pargs::~ThriftHiveMetastore_get_materialized_views_for_rewriting_pargs() throw() {
+}
+
+
+uint32_t ThriftHiveMetastore_get_materialized_views_for_rewriting_pargs::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+  xfer += oprot->writeStructBegin("ThriftHiveMetastore_get_materialized_views_for_rewriting_pargs");
+
+  xfer += oprot->writeFieldBegin("db_name", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeString((*(this->db_name)));
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+
+ThriftHiveMetastore_get_materialized_views_for_rewriting_result::~ThriftHiveMetastore_get_materialized_views_for_rewriting_result() throw() {
+}
+
+
+uint32_t ThriftHiveMetastore_get_materialized_views_for_rewriting_result::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
+  uint32_t xfer = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TType ftype;
+  int16_t fid;
+
+  xfer += iprot->readStructBegin(fname);
+
+  using ::apache::thrift::protocol::TProtocolException;
+
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    switch (fid)
+    {
+      case 0:
+        if (ftype == ::apache::thrift::protocol::T_LIST) {
+          {
+            this->success.clear();
+            uint32_t _size1282;
+            ::apache::thrift::protocol::TType _etype1285;
+            xfer += iprot->readListBegin(_etype1285, _size1282);
+            this->success.resize(_size1282);
+            uint32_t _i1286;
+            for (_i1286 = 0; _i1286 < _size1282; ++_i1286)
             {
-              xfer += iprot->readString(this->success[_i1248]);
+              xfer += iprot->readString(this->success[_i1286]);
             }
             xfer += iprot->readListEnd();
           }
@@ -7611,10 +7611,10 @@ uint32_t ThriftHiveMetastore_get_materialized_views_for_rewriting_result::write(
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter1249;
-      for (_iter1249 = this->success.begin(); _iter1249 != this->success.end(); ++_iter1249)
+      std::vector<std::string> ::const_iterator _iter1287;
+      for (_iter1287 = this->success.begin(); _iter1287 != this->success.end(); ++_iter1287)
       {
-        xfer += oprot->writeString((*_iter1249));
+        xfer += oprot->writeString((*_iter1287));
       }
       xfer += oprot->writeListEnd();
     }
@@ -7659,14 +7659,14 @@ uint32_t ThriftHiveMetastore_get_materialized_views_for_rewriting_presult::read(
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1250;
-            ::apache::thrift::protocol::TType _etype1253;
-            xfer += iprot->readListBegin(_etype1253, _size1250);
-            (*(this->success)).resize(_size1250);
-            uint32_t _i1254;
-            for (_i1254 = 0; _i1254 < _size1250; ++_i1254)
+            uint32_t _size1288;
+            ::apache::thrift::protocol::TType _etype1291;
+            xfer += iprot->readListBegin(_etype1291, _size1288);
+            (*(this->success)).resize(_size1288);
+            uint32_t _i1292;
+            for (_i1292 = 0; _i1292 < _size1288; ++_i1292)
             {
-              xfer += iprot->readString((*(this->success))[_i1254]);
+              xfer += iprot->readString((*(this->success))[_i1292]);
             }
             xfer += iprot->readListEnd();
           }
@@ -7741,14 +7741,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->tbl_types.clear();
-            uint32_t _size1255;
-            ::apache::thrift::protocol::TType _etype1258;
-            xfer += iprot->readListBegin(_etype1258, _size1255);
-            this->tbl_types.resize(_size1255);
-            uint32_t _i1259;
-            for (_i1259 = 0; _i1259 < _size1255; ++_i1259)
+            uint32_t _size1293;
+            ::apache::thrift::protocol::TType _etype1296;
+            xfer += iprot->readListBegin(_etype1296, _size1293);
+            this->tbl_types.resize(_size1293);
+            uint32_t _i1297;
+            for (_i1297 = 0; _i1297 < _size1293; ++_i1297)
             {
-              xfer += iprot->readString(this->tbl_types[_i1259]);
+              xfer += iprot->readString(this->tbl_types[_i1297]);
             }
             xfer += iprot->readListEnd();
           }
@@ -7785,10 +7785,10 @@ uint32_t ThriftHiveMetastore_get_table_meta_args::write(::apache::thrift::protoc
   xfer += oprot->writeFieldBegin("tbl_types", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->tbl_types.size()));
-    std::vector<std::string> ::const_iterator _iter1260;
-    for (_iter1260 = this->tbl_types.begin(); _iter1260 != this->tbl_types.end(); ++_iter1260)
+    std::vector<std::string> ::const_iterator _iter1298;
+    for (_iter1298 = this->tbl_types.begin(); _iter1298 != this->tbl_types.end(); ++_iter1298)
     {
-      xfer += oprot->writeString((*_iter1260));
+      xfer += oprot->writeString((*_iter1298));
     }
     xfer += oprot->writeListEnd();
   }
@@ -7820,10 +7820,10 @@ uint32_t ThriftHiveMetastore_get_table_meta_pargs::write(::apache::thrift::proto
   xfer += oprot->writeFieldBegin("tbl_types", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->tbl_types)).size()));
-    std::vector<std::string> ::const_iterator _iter1261;
-    for (_iter1261 = (*(this->tbl_types)).begin(); _iter1261 != (*(this->tbl_types)).end(); ++_iter1261)
+    std::vector<std::string> ::const_iterator _iter1299;
+    for (_iter1299 = (*(this->tbl_types)).begin(); _iter1299 != (*(this->tbl_types)).end(); ++_iter1299)
     {
-      xfer += oprot->writeString((*_iter1261));
+      xfer += oprot->writeString((*_iter1299));
     }
     xfer += oprot->writeListEnd();
   }
@@ -7864,14 +7864,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1262;
-            ::apache::thrift::protocol::TType _etype1265;
-            xfer += iprot->readListBegin(_etype1265, _size1262);
-            this->success.resize(_size1262);
-            uint32_t _i1266;
-            for (_i1266 = 0; _i1266 < _size1262; ++_i1266)
+            uint32_t _size1300;
+            ::apache::thrift::protocol::TType _etype1303;
+            xfer += iprot->readListBegin(_etype1303, _size1300);
+            this->success.resize(_size1300);
+            uint32_t _i1304;
+            for (_i1304 = 0; _i1304 < _size1300; ++_i1304)
             {
-              xfer += this->success[_i1266].read(iprot);
+              xfer += this->success[_i1304].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -7910,10 +7910,10 @@ uint32_t ThriftHiveMetastore_get_table_meta_result::write(::apache::thrift::prot
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<TableMeta> ::const_iterator _iter1267;
-      for (_iter1267 = this->success.begin(); _iter1267 != this->success.end(); ++_iter1267)
+      std::vector<TableMeta> ::const_iterator _iter1305;
+      for (_iter1305 = this->success.begin(); _iter1305 != this->success.end(); ++_iter1305)
       {
-        xfer += (*_iter1267).write(oprot);
+        xfer += (*_iter1305).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -7958,14 +7958,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1268;
-            ::apache::thrift::protocol::TType _etype1271;
-            xfer += iprot->readListBegin(_etype1271, _size1268);
-            (*(this->success)).resize(_size1268);
-            uint32_t _i1272;
-            for (_i1272 = 0; _i1272 < _size1268; ++_i1272)
+            uint32_t _size1306;
+            ::apache::thrift::protocol::TType _etype1309;
+            xfer += iprot->readListBegin(_etype1309, _size1306);
+            (*(this->success)).resize(_size1306);
+            uint32_t _i1310;
+            for (_i1310 = 0; _i1310 < _size1306; ++_i1310)
             {
-              xfer += (*(this->success))[_i1272].read(iprot);
+              xfer += (*(this->success))[_i1310].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -8103,14 +8103,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1273;
-            ::apache::thrift::protocol::TType _etype1276;
-            xfer += iprot->readListBegin(_etype1276, _size1273);
-            this->success.resize(_size1273);
-            uint32_t _i1277;
-            for (_i1277 = 0; _i1277 < _size1273; ++_i1277)
+            uint32_t _size1311;
+            ::apache::thrift::protocol::TType _etype1314;
+            xfer += iprot->readListBegin(_etype1314, _size1311);
+            this->success.resize(_size1311);
+            uint32_t _i1315;
+            for (_i1315 = 0; _i1315 < _size1311; ++_i1315)
             {
-              xfer += iprot->readString(this->success[_i1277]);
+              xfer += iprot->readString(this->success[_i1315]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8149,10 +8149,10 @@ uint32_t ThriftHiveMetastore_get_all_tables_result::write(::apache::thrift::prot
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter1278;
-      for (_iter1278 = this->success.begin(); _iter1278 != this->success.end(); ++_iter1278)
+      std::vector<std::string> ::const_iterator _iter1316;
+      for (_iter1316 = this->success.begin(); _iter1316 != this->success.end(); ++_iter1316)
       {
-        xfer += oprot->writeString((*_iter1278));
+        xfer += oprot->writeString((*_iter1316));
       }
       xfer += oprot->writeListEnd();
     }
@@ -8197,14 +8197,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1279;
-            ::apache::thrift::protocol::TType _etype1282;
-            xfer += iprot->readListBegin(_etype1282, _size1279);
-            (*(this->success)).resize(_size1279);
-            uint32_t _i1283;
-            for (_i1283 = 0; _i1283 < _size1279; ++_i1283)
+            uint32_t _size1317;
+            ::apache::thrift::protocol::TType _etype1320;
+            xfer += iprot->readListBegin(_etype1320, _size1317);
+            (*(this->success)).resize(_size1317);
+            uint32_t _i1321;
+            for (_i1321 = 0; _i1321 < _size1317; ++_i1321)
             {
-              xfer += iprot->readString((*(this->success))[_i1283]);
+              xfer += iprot->readString((*(this->success))[_i1321]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8514,14 +8514,14 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_args::read(::apache::thri
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->tbl_names.clear();
-            uint32_t _size1284;
-            ::apache::thrift::protocol::TType _etype1287;
-            xfer += iprot->readListBegin(_etype1287, _size1284);
-            this->tbl_names.resize(_size1284);
-            uint32_t _i1288;
-            for (_i1288 = 0; _i1288 < _size1284; ++_i1288)
+            uint32_t _size1322;
+            ::apache::thrift::protocol::TType _etype1325;
+            xfer += iprot->readListBegin(_etype1325, _size1322);
+            this->tbl_names.resize(_size1322);
+            uint32_t _i1326;
+            for (_i1326 = 0; _i1326 < _size1322; ++_i1326)
             {
-              xfer += iprot->readString(this->tbl_names[_i1288]);
+              xfer += iprot->readString(this->tbl_names[_i1326]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8554,10 +8554,10 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_args::write(::apache::thr
   xfer += oprot->writeFieldBegin("tbl_names", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->tbl_names.size()));
-    std::vector<std::string> ::const_iterator _iter1289;
-    for (_iter1289 = this->tbl_names.begin(); _iter1289 != this->tbl_names.end(); ++_iter1289)
+    std::vector<std::string> ::const_iterator _iter1327;
+    for (_iter1327 = this->tbl_names.begin(); _iter1327 != this->tbl_names.end(); ++_iter1327)
     {
-      xfer += oprot->writeString((*_iter1289));
+      xfer += oprot->writeString((*_iter1327));
     }
     xfer += oprot->writeListEnd();
   }
@@ -8585,10 +8585,10 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_pargs::write(::apache::th
   xfer += oprot->writeFieldBegin("tbl_names", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->tbl_names)).size()));
-    std::vector<std::string> ::const_iterator _iter1290;
-    for (_iter1290 = (*(this->tbl_names)).begin(); _iter1290 != (*(this->tbl_names)).end(); ++_iter1290)
+    std::vector<std::string> ::const_iterator _iter1328;
+    for (_iter1328 = (*(this->tbl_names)).begin(); _iter1328 != (*(this->tbl_names)).end(); ++_iter1328)
     {
-      xfer += oprot->writeString((*_iter1290));
+      xfer += oprot->writeString((*_iter1328));
     }
     xfer += oprot->writeListEnd();
   }
@@ -8629,14 +8629,14 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_result::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1291;
-            ::apache::thrift::protocol::TType _etype1294;
-            xfer += iprot->readListBegin(_etype1294, _size1291);
-            this->success.resize(_size1291);
-            uint32_t _i1295;
-            for (_i1295 = 0; _i1295 < _size1291; ++_i1295)
+            uint32_t _size1329;
+            ::apache::thrift::protocol::TType _etype1332;
+            xfer += iprot->readListBegin(_etype1332, _size1329);
+            this->success.resize(_size1329);
+            uint32_t _i1333;
+            for (_i1333 = 0; _i1333 < _size1329; ++_i1333)
             {
-              xfer += this->success[_i1295].read(iprot);
+              xfer += this->success[_i1333].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -8667,10 +8667,10 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_result::write(::apache::t
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Table> ::const_iterator _iter1296;
-      for (_iter1296 = this->success.begin(); _iter1296 != this->success.end(); ++_iter1296)
+      std::vector<Table> ::const_iterator _iter1334;
+      for (_iter1334 = this->success.begin(); _iter1334 != this->success.end(); ++_iter1334)
       {
-        xfer += (*_iter1296).write(oprot);
+        xfer += (*_iter1334).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -8711,14 +8711,14 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_presult::read(::apache::t
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1297;
-            ::apache::thrift::protocol::TType _etype1300;
-            xfer += iprot->readListBegin(_etype1300, _size1297);
-            (*(this->success)).resize(_size1297);
-            uint32_t _i1301;
-            for (_i1301 = 0; _i1301 < _size1297; ++_i1301)
+            uint32_t _size1335;
+            ::apache::thrift::protocol::TType _etype1338;
+            xfer += iprot->readListBegin(_etype1338, _size1335);
+            (*(this->success)).resize(_size1335);
+            uint32_t _i1339;
+            for (_i1339 = 0; _i1339 < _size1335; ++_i1339)
             {
-              xfer += (*(this->success))[_i1301].read(iprot);
+              xfer += (*(this->success))[_i1339].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -9251,14 +9251,14 @@ uint32_t ThriftHiveMetastore_get_materialization_invalidation_info_args::read(::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->tbl_names.clear();
-            uint32_t _size1302;
-            ::apache::thrift::protocol::TType _etype1305;
-            xfer += iprot->readListBegin(_etype1305, _size1302);
-            this->tbl_names.resize(_size1302);
-            uint32_t _i1306;
-            for (_i1306 = 0; _i1306 < _size1302; ++_i1306)
+            uint32_t _size1340;
+            ::apache::thrift::protocol::TType _etype1343;
+            xfer += iprot->readListBegin(_etype1343, _size1340);
+            this->tbl_names.resize(_size1340);
+            uint32_t _i1344;
+            for (_i1344 = 0; _i1344 < _size1340; ++_i1344)
             {
-              xfer += iprot->readString(this->tbl_names[_i1306]);
+              xfer += iprot->readString(this->tbl_names[_i1344]);
             }
             xfer += iprot->readListEnd();
           }
@@ -9291,10 +9291,10 @@ uint32_t ThriftHiveMetastore_get_materialization_invalidation_info_args::write(:
   xfer += oprot->writeFieldBegin("tbl_names", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->tbl_names.size()));
-    std::vector<std::string> ::const_iterator _iter1307;
-    for (_iter1307 = this->tbl_names.begin(); _iter1307 != this->tbl_names.end(); ++_iter1307)
+    std::vector<std::string> ::const_iterator _iter1345;
+    for (_iter1345 = this->tbl_names.begin(); _iter1345 != this->tbl_names.end(); ++_iter1345)
     {
-      xfer += oprot->writeString((*_iter1307));
+      xfer += oprot->writeString((*_iter1345));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9322,10 +9322,10 @@ uint32_t ThriftHiveMetastore_get_materialization_invalidation_info_pargs::write(
   xfer += oprot->writeFieldBegin("tbl_names", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->tbl_names)).size()));
-    std::vector<std::string> ::const_iterator _iter1308;
-    for (_iter1308 = (*(this->tbl_names)).begin(); _iter1308 != (*(this->tbl_names)).end(); ++_iter1308)
+    std::vector<std::string> ::const_iterator _iter1346;
+    for (_iter1346 = (*(this->tbl_names)).begin(); _iter1346 != (*(this->tbl_names)).end(); ++_iter1346)
     {
-      xfer += oprot->writeString((*_iter1308));
+      xfer += oprot->writeString((*_iter1346));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9366,17 +9366,17 @@ uint32_t ThriftHiveMetastore_get_materialization_invalidation_info_result::read(
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->success.clear();
-            uint32_t _size1309;
-            ::apache::thrift::protocol::TType _ktype1310;
-            ::apache::thrift::protocol::TType _vtype1311;
-            xfer += iprot->readMapBegin(_ktype1310, _vtype1311, _size1309);
-            uint32_t _i1313;
-            for (_i1313 = 0; _i1313 < _size1309; ++_i1313)
+            uint32_t _size1347;
+            ::apache::thrift::protocol::TType _ktype1348;
+            ::apache::thrift::protocol::TType _vtype1349;
+            xfer += iprot->readMapBegin(_ktype1348, _vtype1349, _size1347);
+            uint32_t _i1351;
+            for (_i1351 = 0; _i1351 < _size1347; ++_i1351)
             {
-              std::string _key1314;
-              xfer += iprot->readString(_key1314);
-              Materialization& _val1315 = this->success[_key1314];
-              xfer += _val1315.read(iprot);
+              std::string _key1352;
+              xfer += iprot->readString(_key1352);
+              Materialization& _val1353 = this->success[_key1352];
+              xfer += _val1353.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -9431,11 +9431,11 @@ uint32_t ThriftHiveMetastore_get_materialization_invalidation_info_result::write
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_MAP, 0);
     {
       xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::map<std::string, Materialization> ::const_iterator _iter1316;
-      for (_iter1316 = this->success.begin(); _iter1316 != this->success.end(); ++_iter1316)
+      std::map<std::string, Materialization> ::const_iterator _iter1354;
+      for (_iter1354 = this->success.begin(); _iter1354 != this->success.end(); ++_iter1354)
       {
-        xfer += oprot->writeString(_iter1316->first);
-        xfer += _iter1316->second.write(oprot);
+        xfer += oprot->writeString(_iter1354->first);
+        xfer += _iter1354->second.write(oprot);
       }
       xfer += oprot->writeMapEnd();
     }
@@ -9488,17 +9488,17 @@ uint32_t ThriftHiveMetastore_get_materialization_invalidation_info_presult::read
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             (*(this->success)).clear();
-            uint32_t _size1317;
-            ::apache::thrift::protocol::TType _ktype1318;
-            ::apache::thrift::protocol::TType _vtype1319;
-            xfer += iprot->readMapBegin(_ktype1318, _vtype1319, _size1317);
-            uint32_t _i1321;
-            for (_i1321 = 0; _i1321 < _size1317; ++_i1321)
+            uint32_t _size1355;
+            ::apache::thrift::protocol::TType _ktype1356;
+            ::apache::thrift::protocol::TType _vtype1357;
+            xfer += iprot->readMapBegin(_ktype1356, _vtype1357, _size1355);
+            uint32_t _i1359;
+            for (_i1359 = 0; _i1359 < _size1355; ++_i1359)
             {
-              std::string _key1322;
-              xfer += iprot->readString(_key1322);
-              Materialization& _val1323 = (*(this->success))[_key1322];
-              xfer += _val1323.read(iprot);
+              std::string _key1360;
+              xfer += iprot->readString(_key1360);
+              Materialization& _val1361 = (*(this->success))[_key1360];
+              xfer += _val1361.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -9943,14 +9943,14 @@ uint32_t ThriftHiveMetastore_get_table_names_by_filter_result::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1324;
-            ::apache::thrift::protocol::TType _etype1327;
-            xfer += iprot->readListBegin(_etype1327, _size1324);
-            this->success.resize(_size1324);
-            uint32_t _i1328;
-            for (_i1328 = 0; _i1328 < _size1324; ++_i1328)
+            uint32_t _size1362;
+            ::apache::thrift::protocol::TType _etype1365;
+            xfer += iprot->readListBegin(_etype1365, _size1362);
+            this->success.resize(_size1362);
+            uint32_t _i1366;
+            for (_i1366 = 0; _i1366 < _size1362; ++_i1366)
             {
-              xfer += iprot->readString(this->success[_i1328]);
+              xfer += iprot->readString(this->success[_i1366]);
             }
             xfer += iprot->readListEnd();
           }
@@ -10005,10 +10005,10 @@ uint32_t ThriftHiveMetastore_get_table_names_by_filter_result::write(::apache::t
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter1329;
-      for (_iter1329 = this->success.begin(); _iter1329 != this->success.end(); ++_iter1329)
+      std::vector<std::string> ::const_iterator _iter1367;
+      for (_iter1367 = this->success.begin(); _iter1367 != this->success.end(); ++_iter1367)
       {
-        xfer += oprot->writeString((*_iter1329));
+        xfer += oprot->writeString((*_iter1367));
       }
       xfer += oprot->writeListEnd();
     }
@@ -10061,14 +10061,14 @@ uint32_t ThriftHiveMetastore_get_table_names_by_filter_presult::read(::apache::t
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1330;
-            ::apache::thrift::protocol::TType _etype1333;
-            xfer += iprot->readListBegin(_etype1333, _size1330);
-            (*(this->success)).resize(_size1330);
-            uint32_t _i1334;
-            for (_i1334 = 0; _i1334 < _size1330; ++_i1334)
+            uint32_t _size1368;
+            ::apache::thrift::protocol::TType _etype1371;
+            xfer += iprot->readListBegin(_etype1371, _size1368);
+            (*(this->success)).resize(_size1368);
+            uint32_t _i1372;
+            for (_i1372 = 0; _i1372 < _size1368; ++_i1372)
             {
-              xfer += iprot->readString((*(this->success))[_i1334]);
+              xfer += iprot->readString((*(this->success))[_i1372]);
             }
             xfer += iprot->readListEnd();
           }
@@ -11402,14 +11402,14 @@ uint32_t ThriftHiveMetastore_add_partitions_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->new_parts.clear();
-            uint32_t _size1335;
-            ::apache::thrift::protocol::TType _etype1338;
-            xfer += iprot->readListBegin(_etype1338, _size1335);
-            this->new_parts.resize(_size1335);
-            uint32_t _i1339;
-            for (_i1339 = 0; _i1339 < _size1335; ++_i1339)
+            uint32_t _size1373;
+            ::apache::thrift::protocol::TType _etype1376;
+            xfer += iprot->readListBegin(_etype1376, _size1373);
+            this->new_parts.resize(_size1373);
+            uint32_t _i1377;
+            for (_i1377 = 0; _i1377 < _size1373; ++_i1377)
             {
-              xfer += this->new_parts[_i1339].read(iprot);
+              xfer += this->new_parts[_i1377].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -11438,10 +11438,10 @@ uint32_t ThriftHiveMetastore_add_partitions_args::write(::apache::thrift::protoc
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->new_parts.size()));
-    std::vector<Partition> ::const_iterator _iter1340;
-    for (_iter1340 = this->new_parts.begin(); _iter1340 != this->new_parts.end(); ++_iter1340)
+    std::vector<Partition> ::const_iterator _iter1378;
+    for (_iter1378 = this->new_parts.begin(); _iter1378 != this->new_parts.end(); ++_iter1378)
     {
-      xfer += (*_iter1340).write(oprot);
+      xfer += (*_iter1378).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -11465,10 +11465,10 @@ uint32_t ThriftHiveMetastore_add_partitions_pargs::write(::apache::thrift::proto
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->new_parts)).size()));
-    std::vector<Partition> ::const_iterator _iter1341;
-    for (_iter1341 = (*(this->new_parts)).begin(); _iter1341 != (*(this->new_parts)).end(); ++_iter1341)
+    std::vector<Partition> ::const_iterator _iter1379;
+    for (_iter1379 = (*(this->new_parts)).begin(); _iter1379 != (*(this->new_parts)).end(); ++_iter1379)
     {
-      xfer += (*_iter1341).write(oprot);
+      xfer += (*_iter1379).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -11677,14 +11677,14 @@ uint32_t ThriftHiveMetastore_add_partitions_pspec_args::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->new_parts.clear();
-            uint32_t _size1342;
-            ::apache::thrift::protocol::TType _etype1345;
-            xfer += iprot->readListBegin(_etype1345, _size1342);
-            this->new_parts.resize(_size1342);
-            uint32_t _i1346;
-            for (_i1346 = 0; _i1346 < _size1342; ++_i1346)
+            uint32_t _size1380;
+            ::apache::thrift::protocol::TType _etype1383;
+            xfer += iprot->readListBegin(_etype1383, _size1380);
+            this->new_parts.resize(_size1380);
+            uint32_t _i1384;
+            for (_i1384 = 0; _i1384 < _size1380; ++_i1384)
             {
-              xfer += this->new_parts[_i1346].read(iprot);
+              xfer += this->new_parts[_i1384].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -11713,10 +11713,10 @@ uint32_t ThriftHiveMetastore_add_partitions_pspec_args::write(::apache::thrift::
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->new_parts.size()));
-    std::vector<PartitionSpec> ::const_iterator _iter1347;
-    for (_iter1347 = this->new_parts.begin(); _iter1347 != this->new_parts.end(); ++_iter1347)
+    std::vector<PartitionSpec> ::const_iterator _iter1385;
+    for (_iter1385 = this->new_parts.begin(); _iter1385 != this->new_parts.end(); ++_iter1385)
     {
-      xfer += (*_iter1347).write(oprot);
+      xfer += (*_iter1385).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -11740,10 +11740,10 @@ uint32_t ThriftHiveMetastore_add_partitions_pspec_pargs::write(::apache::thrift:
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->new_parts)).size()));
-    std::vector<PartitionSpec> ::const_iterator _iter1348;
-    for (_iter1348 = (*(this->new_parts)).begin(); _iter1348 != (*(this->new_parts)).end(); ++_iter1348)
+    std::vector<PartitionSpec> ::const_iterator _iter1386;
+    for (_iter1386 = (*(this->new_parts)).begin(); _iter1386 != (*(this->new_parts)).end(); ++_iter1386)
     {
-      xfer += (*_iter1348).write(oprot);
+      xfer += (*_iter1386).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -11968,14 +11968,14 @@ uint32_t ThriftHiveMetastore_append_partition_args::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1349;
-            ::apache::thrift::protocol::TType _etype1352;
-            xfer += iprot->readListBegin(_etype1352, _size1349);
-            this->part_vals.resize(_size1349);
-            uint32_t _i1353;
-            for (_i1353 = 0; _i1353 < _size1349; ++_i1353)
+            uint32_t _size1387;
+            ::apache::thrift::protocol::TType _etype1390;
+            xfer += iprot->readListBegin(_etype1390, _size1387);
+            this->part_vals.resize(_size1387);
+            uint32_t _i1391;
+            for (_i1391 = 0; _i1391 < _size1387; ++_i1391)
             {
-              xfer += iprot->readString(this->part_vals[_i1353]);
+              xfer += iprot->readString(this->part_vals[_i1391]);
             }
             xfer += iprot->readListEnd();
           }
@@ -12012,10 +12012,10 @@ uint32_t ThriftHiveMetastore_append_partition_args::write(::apache::thrift::prot
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter1354;
-    for (_iter1354 = this->part_vals.begin(); _iter1354 != this->part_vals.end(); ++_iter1354)
+    std::vector<std::string> ::const_iterator _iter1392;
+    for (_iter1392 = this->part_vals.begin(); _iter1392 != this->part_vals.end(); ++_iter1392)
     {
-      xfer += oprot->writeString((*_iter1354));
+      xfer += oprot->writeString((*_iter1392));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12047,10 +12047,10 @@ uint32_t ThriftHiveMetastore_append_partition_pargs::write(::apache::thrift::pro
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter1355;
-    for (_iter1355 = (*(this->part_vals)).begin(); _iter1355 != (*(this->part_vals)).end(); ++_iter1355)
+    std::vector<std::string> ::const_iterator _iter1393;
+    for (_iter1393 = (*(this->part_vals)).begin(); _iter1393 != (*(this->part_vals)).end(); ++_iter1393)
     {
-      xfer += oprot->writeString((*_iter1355));
+      xfer += oprot->writeString((*_iter1393));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12522,14 +12522,14 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_args::rea
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1356;
-            ::apache::thrift::protocol::TType _etype1359;
-            xfer += iprot->readListBegin(_etype1359, _size1356);
-            this->part_vals.resize(_size1356);
-            uint32_t _i1360;
-            for (_i1360 = 0; _i1360 < _size1356; ++_i1360)
+            uint32_t _size1394;
+            ::apache::thrift::protocol::TType _etype1397;
+            xfer += iprot->readListBegin(_etype1397, _size1394);
+            this->part_vals.resize(_size1394);
+            uint32_t _i1398;
+            for (_i1398 = 0; _i1398 < _size1394; ++_i1398)
             {
-              xfer += iprot->readString(this->part_vals[_i1360]);
+              xfer += iprot->readString(this->part_vals[_i1398]);
             }
             xfer += iprot->readListEnd();
           }
@@ -12574,10 +12574,10 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_args::wri
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter1361;
-    for (_iter1361 = this->part_vals.begin(); _iter1361 != this->part_vals.end(); ++_iter1361)
+    std::vector<std::string> ::const_iterator _iter1399;
+    for (_i

<TRUNCATED>

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

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp b/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
index 244b7ab..4a7c6bf 100644
--- a/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
+++ b/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
@@ -149,6 +149,72 @@ const char* _kEventRequestTypeNames[] = {
 };
 const std::map<int, const char*> _EventRequestType_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(3, _kEventRequestTypeValues, _kEventRequestTypeNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL));
 
+int _kSerdeTypeValues[] = {
+  SerdeType::HIVE,
+  SerdeType::SCHEMA_REGISTRY
+};
+const char* _kSerdeTypeNames[] = {
+  "HIVE",
+  "SCHEMA_REGISTRY"
+};
+const std::map<int, const char*> _SerdeType_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(2, _kSerdeTypeValues, _kSerdeTypeNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL));
+
+int _kSchemaTypeValues[] = {
+  SchemaType::HIVE,
+  SchemaType::AVRO
+};
+const char* _kSchemaTypeNames[] = {
+  "HIVE",
+  "AVRO"
+};
+const std::map<int, const char*> _SchemaType_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(2, _kSchemaTypeValues, _kSchemaTypeNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL));
+
+int _kSchemaCompatibilityValues[] = {
+  SchemaCompatibility::NONE,
+  SchemaCompatibility::BACKWARD,
+  SchemaCompatibility::FORWARD,
+  SchemaCompatibility::BOTH
+};
+const char* _kSchemaCompatibilityNames[] = {
+  "NONE",
+  "BACKWARD",
+  "FORWARD",
+  "BOTH"
+};
+const std::map<int, const char*> _SchemaCompatibility_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(4, _kSchemaCompatibilityValues, _kSchemaCompatibilityNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL));
+
+int _kSchemaValidationValues[] = {
+  SchemaValidation::LATEST,
+  SchemaValidation::ALL
+};
+const char* _kSchemaValidationNames[] = {
+  "LATEST",
+  "ALL"
+};
+const std::map<int, const char*> _SchemaValidation_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(2, _kSchemaValidationValues, _kSchemaValidationNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL));
+
+int _kSchemaVersionStateValues[] = {
+  SchemaVersionState::INITIATED,
+  SchemaVersionState::START_REVIEW,
+  SchemaVersionState::CHANGES_REQUIRED,
+  SchemaVersionState::REVIEWED,
+  SchemaVersionState::ENABLED,
+  SchemaVersionState::DISABLED,
+  SchemaVersionState::ARCHIVED,
+  SchemaVersionState::DELETED
+};
+const char* _kSchemaVersionStateNames[] = {
+  "INITIATED",
+  "START_REVIEW",
+  "CHANGES_REQUIRED",
+  "REVIEWED",
+  "ENABLED",
+  "DISABLED",
+  "ARCHIVED",
+  "DELETED"
+};
+const std::map<int, const char*> _SchemaVersionState_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(8, _kSchemaVersionStateValues, _kSchemaVersionStateNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL));
+
 int _kFunctionTypeValues[] = {
   FunctionType::JAVA
 };
@@ -4235,6 +4301,26 @@ void SerDeInfo::__set_parameters(const std::map<std::string, std::string> & val)
   this->parameters = val;
 }
 
+void SerDeInfo::__set_description(const std::string& val) {
+  this->description = val;
+__isset.description = true;
+}
+
+void SerDeInfo::__set_serializerClass(const std::string& val) {
+  this->serializerClass = val;
+__isset.serializerClass = true;
+}
+
+void SerDeInfo::__set_deserializerClass(const std::string& val) {
+  this->deserializerClass = val;
+__isset.deserializerClass = true;
+}
+
+void SerDeInfo::__set_serdeType(const SerdeType::type val) {
+  this->serdeType = val;
+__isset.serdeType = true;
+}
+
 uint32_t SerDeInfo::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
@@ -4295,6 +4381,40 @@ uint32_t SerDeInfo::read(::apache::thrift::protocol::TProtocol* iprot) {
           xfer += iprot->skip(ftype);
         }
         break;
+      case 4:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->description);
+          this->__isset.description = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 5:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->serializerClass);
+          this->__isset.serializerClass = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 6:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->deserializerClass);
+          this->__isset.deserializerClass = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 7:
+        if (ftype == ::apache::thrift::protocol::T_I32) {
+          int32_t ecast146;
+          xfer += iprot->readI32(ecast146);
+          this->serdeType = (SerdeType::type)ecast146;
+          this->__isset.serdeType = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -4323,16 +4443,36 @@ uint32_t SerDeInfo::write(::apache::thrift::protocol::TProtocol* oprot) const {
   xfer += oprot->writeFieldBegin("parameters", ::apache::thrift::protocol::T_MAP, 3);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->parameters.size()));
-    std::map<std::string, std::string> ::const_iterator _iter146;
-    for (_iter146 = this->parameters.begin(); _iter146 != this->parameters.end(); ++_iter146)
+    std::map<std::string, std::string> ::const_iterator _iter147;
+    for (_iter147 = this->parameters.begin(); _iter147 != this->parameters.end(); ++_iter147)
     {
-      xfer += oprot->writeString(_iter146->first);
-      xfer += oprot->writeString(_iter146->second);
+      xfer += oprot->writeString(_iter147->first);
+      xfer += oprot->writeString(_iter147->second);
     }
     xfer += oprot->writeMapEnd();
   }
   xfer += oprot->writeFieldEnd();
 
+  if (this->__isset.description) {
+    xfer += oprot->writeFieldBegin("description", ::apache::thrift::protocol::T_STRING, 4);
+    xfer += oprot->writeString(this->description);
+    xfer += oprot->writeFieldEnd();
+  }
+  if (this->__isset.serializerClass) {
+    xfer += oprot->writeFieldBegin("serializerClass", ::apache::thrift::protocol::T_STRING, 5);
+    xfer += oprot->writeString(this->serializerClass);
+    xfer += oprot->writeFieldEnd();
+  }
+  if (this->__isset.deserializerClass) {
+    xfer += oprot->writeFieldBegin("deserializerClass", ::apache::thrift::protocol::T_STRING, 6);
+    xfer += oprot->writeString(this->deserializerClass);
+    xfer += oprot->writeFieldEnd();
+  }
+  if (this->__isset.serdeType) {
+    xfer += oprot->writeFieldBegin("serdeType", ::apache::thrift::protocol::T_I32, 7);
+    xfer += oprot->writeI32((int32_t)this->serdeType);
+    xfer += oprot->writeFieldEnd();
+  }
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
   return xfer;
@@ -4343,20 +4483,32 @@ void swap(SerDeInfo &a, SerDeInfo &b) {
   swap(a.name, b.name);
   swap(a.serializationLib, b.serializationLib);
   swap(a.parameters, b.parameters);
+  swap(a.description, b.description);
+  swap(a.serializerClass, b.serializerClass);
+  swap(a.deserializerClass, b.deserializerClass);
+  swap(a.serdeType, b.serdeType);
   swap(a.__isset, b.__isset);
 }
 
-SerDeInfo::SerDeInfo(const SerDeInfo& other147) {
-  name = other147.name;
-  serializationLib = other147.serializationLib;
-  parameters = other147.parameters;
-  __isset = other147.__isset;
-}
-SerDeInfo& SerDeInfo::operator=(const SerDeInfo& other148) {
+SerDeInfo::SerDeInfo(const SerDeInfo& other148) {
   name = other148.name;
   serializationLib = other148.serializationLib;
   parameters = other148.parameters;
+  description = other148.description;
+  serializerClass = other148.serializerClass;
+  deserializerClass = other148.deserializerClass;
+  serdeType = other148.serdeType;
   __isset = other148.__isset;
+}
+SerDeInfo& SerDeInfo::operator=(const SerDeInfo& other149) {
+  name = other149.name;
+  serializationLib = other149.serializationLib;
+  parameters = other149.parameters;
+  description = other149.description;
+  serializerClass = other149.serializerClass;
+  deserializerClass = other149.deserializerClass;
+  serdeType = other149.serdeType;
+  __isset = other149.__isset;
   return *this;
 }
 void SerDeInfo::printTo(std::ostream& out) const {
@@ -4365,6 +4517,10 @@ void SerDeInfo::printTo(std::ostream& out) const {
   out << "name=" << to_string(name);
   out << ", " << "serializationLib=" << to_string(serializationLib);
   out << ", " << "parameters=" << to_string(parameters);
+  out << ", " << "description="; (__isset.description ? (out << to_string(description)) : (out << "<null>"));
+  out << ", " << "serializerClass="; (__isset.serializerClass ? (out << to_string(serializerClass)) : (out << "<null>"));
+  out << ", " << "deserializerClass="; (__isset.deserializerClass ? (out << to_string(deserializerClass)) : (out << "<null>"));
+  out << ", " << "serdeType="; (__isset.serdeType ? (out << to_string(serdeType)) : (out << "<null>"));
   out << ")";
 }
 
@@ -4455,15 +4611,15 @@ void swap(Order &a, Order &b) {
   swap(a.__isset, b.__isset);
 }
 
-Order::Order(const Order& other149) {
-  col = other149.col;
-  order = other149.order;
-  __isset = other149.__isset;
-}
-Order& Order::operator=(const Order& other150) {
+Order::Order(const Order& other150) {
   col = other150.col;
   order = other150.order;
   __isset = other150.__isset;
+}
+Order& Order::operator=(const Order& other151) {
+  col = other151.col;
+  order = other151.order;
+  __isset = other151.__isset;
   return *this;
 }
 void Order::printTo(std::ostream& out) const {
@@ -4516,14 +4672,14 @@ uint32_t SkewedInfo::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->skewedColNames.clear();
-            uint32_t _size151;
-            ::apache::thrift::protocol::TType _etype154;
-            xfer += iprot->readListBegin(_etype154, _size151);
-            this->skewedColNames.resize(_size151);
-            uint32_t _i155;
-            for (_i155 = 0; _i155 < _size151; ++_i155)
+            uint32_t _size152;
+            ::apache::thrift::protocol::TType _etype155;
+            xfer += iprot->readListBegin(_etype155, _size152);
+            this->skewedColNames.resize(_size152);
+            uint32_t _i156;
+            for (_i156 = 0; _i156 < _size152; ++_i156)
             {
-              xfer += iprot->readString(this->skewedColNames[_i155]);
+              xfer += iprot->readString(this->skewedColNames[_i156]);
             }
             xfer += iprot->readListEnd();
           }
@@ -4536,23 +4692,23 @@ uint32_t SkewedInfo::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->skewedColValues.clear();
-            uint32_t _size156;
-            ::apache::thrift::protocol::TType _etype159;
-            xfer += iprot->readListBegin(_etype159, _size156);
-            this->skewedColValues.resize(_size156);
-            uint32_t _i160;
-            for (_i160 = 0; _i160 < _size156; ++_i160)
+            uint32_t _size157;
+            ::apache::thrift::protocol::TType _etype160;
+            xfer += iprot->readListBegin(_etype160, _size157);
+            this->skewedColValues.resize(_size157);
+            uint32_t _i161;
+            for (_i161 = 0; _i161 < _size157; ++_i161)
             {
               {
-                this->skewedColValues[_i160].clear();
-                uint32_t _size161;
-                ::apache::thrift::protocol::TType _etype164;
-                xfer += iprot->readListBegin(_etype164, _size161);
-                this->skewedColValues[_i160].resize(_size161);
-                uint32_t _i165;
-                for (_i165 = 0; _i165 < _size161; ++_i165)
+                this->skewedColValues[_i161].clear();
+                uint32_t _size162;
+                ::apache::thrift::protocol::TType _etype165;
+                xfer += iprot->readListBegin(_etype165, _size162);
+                this->skewedColValues[_i161].resize(_size162);
+                uint32_t _i166;
+                for (_i166 = 0; _i166 < _size162; ++_i166)
                 {
-                  xfer += iprot->readString(this->skewedColValues[_i160][_i165]);
+                  xfer += iprot->readString(this->skewedColValues[_i161][_i166]);
                 }
                 xfer += iprot->readListEnd();
               }
@@ -4568,29 +4724,29 @@ uint32_t SkewedInfo::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->skewedColValueLocationMaps.clear();
-            uint32_t _size166;
-            ::apache::thrift::protocol::TType _ktype167;
-            ::apache::thrift::protocol::TType _vtype168;
-            xfer += iprot->readMapBegin(_ktype167, _vtype168, _size166);
-            uint32_t _i170;
-            for (_i170 = 0; _i170 < _size166; ++_i170)
+            uint32_t _size167;
+            ::apache::thrift::protocol::TType _ktype168;
+            ::apache::thrift::protocol::TType _vtype169;
+            xfer += iprot->readMapBegin(_ktype168, _vtype169, _size167);
+            uint32_t _i171;
+            for (_i171 = 0; _i171 < _size167; ++_i171)
             {
-              std::vector<std::string>  _key171;
+              std::vector<std::string>  _key172;
               {
-                _key171.clear();
-                uint32_t _size173;
-                ::apache::thrift::protocol::TType _etype176;
-                xfer += iprot->readListBegin(_etype176, _size173);
-                _key171.resize(_size173);
-                uint32_t _i177;
-                for (_i177 = 0; _i177 < _size173; ++_i177)
+                _key172.clear();
+                uint32_t _size174;
+                ::apache::thrift::protocol::TType _etype177;
+                xfer += iprot->readListBegin(_etype177, _size174);
+                _key172.resize(_size174);
+                uint32_t _i178;
+                for (_i178 = 0; _i178 < _size174; ++_i178)
                 {
-                  xfer += iprot->readString(_key171[_i177]);
+                  xfer += iprot->readString(_key172[_i178]);
                 }
                 xfer += iprot->readListEnd();
               }
-              std::string& _val172 = this->skewedColValueLocationMaps[_key171];
-              xfer += iprot->readString(_val172);
+              std::string& _val173 = this->skewedColValueLocationMaps[_key172];
+              xfer += iprot->readString(_val173);
             }
             xfer += iprot->readMapEnd();
           }
@@ -4619,10 +4775,10 @@ uint32_t SkewedInfo::write(::apache::thrift::protocol::TProtocol* oprot) const {
   xfer += oprot->writeFieldBegin("skewedColNames", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->skewedColNames.size()));
-    std::vector<std::string> ::const_iterator _iter178;
-    for (_iter178 = this->skewedColNames.begin(); _iter178 != this->skewedColNames.end(); ++_iter178)
+    std::vector<std::string> ::const_iterator _iter179;
+    for (_iter179 = this->skewedColNames.begin(); _iter179 != this->skewedColNames.end(); ++_iter179)
     {
-      xfer += oprot->writeString((*_iter178));
+      xfer += oprot->writeString((*_iter179));
     }
     xfer += oprot->writeListEnd();
   }
@@ -4631,15 +4787,15 @@ uint32_t SkewedInfo::write(::apache::thrift::protocol::TProtocol* oprot) const {
   xfer += oprot->writeFieldBegin("skewedColValues", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_LIST, static_cast<uint32_t>(this->skewedColValues.size()));
-    std::vector<std::vector<std::string> > ::const_iterator _iter179;
-    for (_iter179 = this->skewedColValues.begin(); _iter179 != this->skewedColValues.end(); ++_iter179)
+    std::vector<std::vector<std::string> > ::const_iterator _iter180;
+    for (_iter180 = this->skewedColValues.begin(); _iter180 != this->skewedColValues.end(); ++_iter180)
     {
       {
-        xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*_iter179).size()));
-        std::vector<std::string> ::const_iterator _iter180;
-        for (_iter180 = (*_iter179).begin(); _iter180 != (*_iter179).end(); ++_iter180)
+        xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*_iter180).size()));
+        std::vector<std::string> ::const_iterator _iter181;
+        for (_iter181 = (*_iter180).begin(); _iter181 != (*_iter180).end(); ++_iter181)
         {
-          xfer += oprot->writeString((*_iter180));
+          xfer += oprot->writeString((*_iter181));
         }
         xfer += oprot->writeListEnd();
       }
@@ -4651,19 +4807,19 @@ uint32_t SkewedInfo::write(::apache::thrift::protocol::TProtocol* oprot) const {
   xfer += oprot->writeFieldBegin("skewedColValueLocationMaps", ::apache::thrift::protocol::T_MAP, 3);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_LIST, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->skewedColValueLocationMaps.size()));
-    std::map<std::vector<std::string> , std::string> ::const_iterator _iter181;
-    for (_iter181 = this->skewedColValueLocationMaps.begin(); _iter181 != this->skewedColValueLocationMaps.end(); ++_iter181)
+    std::map<std::vector<std::string> , std::string> ::const_iterator _iter182;
+    for (_iter182 = this->skewedColValueLocationMaps.begin(); _iter182 != this->skewedColValueLocationMaps.end(); ++_iter182)
     {
       {
-        xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(_iter181->first.size()));
-        std::vector<std::string> ::const_iterator _iter182;
-        for (_iter182 = _iter181->first.begin(); _iter182 != _iter181->first.end(); ++_iter182)
+        xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(_iter182->first.size()));
+        std::vector<std::string> ::const_iterator _iter183;
+        for (_iter183 = _iter182->first.begin(); _iter183 != _iter182->first.end(); ++_iter183)
         {
-          xfer += oprot->writeString((*_iter182));
+          xfer += oprot->writeString((*_iter183));
         }
         xfer += oprot->writeListEnd();
       }
-      xfer += oprot->writeString(_iter181->second);
+      xfer += oprot->writeString(_iter182->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -4682,17 +4838,17 @@ void swap(SkewedInfo &a, SkewedInfo &b) {
   swap(a.__isset, b.__isset);
 }
 
-SkewedInfo::SkewedInfo(const SkewedInfo& other183) {
-  skewedColNames = other183.skewedColNames;
-  skewedColValues = other183.skewedColValues;
-  skewedColValueLocationMaps = other183.skewedColValueLocationMaps;
-  __isset = other183.__isset;
-}
-SkewedInfo& SkewedInfo::operator=(const SkewedInfo& other184) {
+SkewedInfo::SkewedInfo(const SkewedInfo& other184) {
   skewedColNames = other184.skewedColNames;
   skewedColValues = other184.skewedColValues;
   skewedColValueLocationMaps = other184.skewedColValueLocationMaps;
   __isset = other184.__isset;
+}
+SkewedInfo& SkewedInfo::operator=(const SkewedInfo& other185) {
+  skewedColNames = other185.skewedColNames;
+  skewedColValues = other185.skewedColValues;
+  skewedColValueLocationMaps = other185.skewedColValueLocationMaps;
+  __isset = other185.__isset;
   return *this;
 }
 void SkewedInfo::printTo(std::ostream& out) const {
@@ -4784,14 +4940,14 @@ uint32_t StorageDescriptor::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->cols.clear();
-            uint32_t _size185;
-            ::apache::thrift::protocol::TType _etype188;
-            xfer += iprot->readListBegin(_etype188, _size185);
-            this->cols.resize(_size185);
-            uint32_t _i189;
-            for (_i189 = 0; _i189 < _size185; ++_i189)
+            uint32_t _size186;
+            ::apache::thrift::protocol::TType _etype189;
+            xfer += iprot->readListBegin(_etype189, _size186);
+            this->cols.resize(_size186);
+            uint32_t _i190;
+            for (_i190 = 0; _i190 < _size186; ++_i190)
             {
-              xfer += this->cols[_i189].read(iprot);
+              xfer += this->cols[_i190].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4852,14 +5008,14 @@ uint32_t StorageDescriptor::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->bucketCols.clear();
-            uint32_t _size190;
-            ::apache::thrift::protocol::TType _etype193;
-            xfer += iprot->readListBegin(_etype193, _size190);
-            this->bucketCols.resize(_size190);
-            uint32_t _i194;
-            for (_i194 = 0; _i194 < _size190; ++_i194)
+            uint32_t _size191;
+            ::apache::thrift::protocol::TType _etype194;
+            xfer += iprot->readListBegin(_etype194, _size191);
+            this->bucketCols.resize(_size191);
+            uint32_t _i195;
+            for (_i195 = 0; _i195 < _size191; ++_i195)
             {
-              xfer += iprot->readString(this->bucketCols[_i194]);
+              xfer += iprot->readString(this->bucketCols[_i195]);
             }
             xfer += iprot->readListEnd();
           }
@@ -4872,14 +5028,14 @@ uint32_t StorageDescriptor::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->sortCols.clear();
-            uint32_t _size195;
-            ::apache::thrift::protocol::TType _etype198;
-            xfer += iprot->readListBegin(_etype198, _size195);
-            this->sortCols.resize(_size195);
-            uint32_t _i199;
-            for (_i199 = 0; _i199 < _size195; ++_i199)
+            uint32_t _size196;
+            ::apache::thrift::protocol::TType _etype199;
+            xfer += iprot->readListBegin(_etype199, _size196);
+            this->sortCols.resize(_size196);
+            uint32_t _i200;
+            for (_i200 = 0; _i200 < _size196; ++_i200)
             {
-              xfer += this->sortCols[_i199].read(iprot);
+              xfer += this->sortCols[_i200].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4892,17 +5048,17 @@ uint32_t StorageDescriptor::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->parameters.clear();
-            uint32_t _size200;
-            ::apache::thrift::protocol::TType _ktype201;
-            ::apache::thrift::protocol::TType _vtype202;
-            xfer += iprot->readMapBegin(_ktype201, _vtype202, _size200);
-            uint32_t _i204;
-            for (_i204 = 0; _i204 < _size200; ++_i204)
+            uint32_t _size201;
+            ::apache::thrift::protocol::TType _ktype202;
+            ::apache::thrift::protocol::TType _vtype203;
+            xfer += iprot->readMapBegin(_ktype202, _vtype203, _size201);
+            uint32_t _i205;
+            for (_i205 = 0; _i205 < _size201; ++_i205)
             {
-              std::string _key205;
-              xfer += iprot->readString(_key205);
-              std::string& _val206 = this->parameters[_key205];
-              xfer += iprot->readString(_val206);
+              std::string _key206;
+              xfer += iprot->readString(_key206);
+              std::string& _val207 = this->parameters[_key206];
+              xfer += iprot->readString(_val207);
             }
             xfer += iprot->readMapEnd();
           }
@@ -4947,10 +5103,10 @@ uint32_t StorageDescriptor::write(::apache::thrift::protocol::TProtocol* oprot)
   xfer += oprot->writeFieldBegin("cols", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->cols.size()));
-    std::vector<FieldSchema> ::const_iterator _iter207;
-    for (_iter207 = this->cols.begin(); _iter207 != this->cols.end(); ++_iter207)
+    std::vector<FieldSchema> ::const_iterator _iter208;
+    for (_iter208 = this->cols.begin(); _iter208 != this->cols.end(); ++_iter208)
     {
-      xfer += (*_iter207).write(oprot);
+      xfer += (*_iter208).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -4983,10 +5139,10 @@ uint32_t StorageDescriptor::write(::apache::thrift::protocol::TProtocol* oprot)
   xfer += oprot->writeFieldBegin("bucketCols", ::apache::thrift::protocol::T_LIST, 8);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->bucketCols.size()));
-    std::vector<std::string> ::const_iterator _iter208;
-    for (_iter208 = this->bucketCols.begin(); _iter208 != this->bucketCols.end(); ++_iter208)
+    std::vector<std::string> ::const_iterator _iter209;
+    for (_iter209 = this->bucketCols.begin(); _iter209 != this->bucketCols.end(); ++_iter209)
     {
-      xfer += oprot->writeString((*_iter208));
+      xfer += oprot->writeString((*_iter209));
     }
     xfer += oprot->writeListEnd();
   }
@@ -4995,10 +5151,10 @@ uint32_t StorageDescriptor::write(::apache::thrift::protocol::TProtocol* oprot)
   xfer += oprot->writeFieldBegin("sortCols", ::apache::thrift::protocol::T_LIST, 9);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->sortCols.size()));
-    std::vector<Order> ::const_iterator _iter209;
-    for (_iter209 = this->sortCols.begin(); _iter209 != this->sortCols.end(); ++_iter209)
+    std::vector<Order> ::const_iterator _iter210;
+    for (_iter210 = this->sortCols.begin(); _iter210 != this->sortCols.end(); ++_iter210)
     {
-      xfer += (*_iter209).write(oprot);
+      xfer += (*_iter210).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5007,11 +5163,11 @@ uint32_t StorageDescriptor::write(::apache::thrift::protocol::TProtocol* oprot)
   xfer += oprot->writeFieldBegin("parameters", ::apache::thrift::protocol::T_MAP, 10);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->parameters.size()));
-    std::map<std::string, std::string> ::const_iterator _iter210;
-    for (_iter210 = this->parameters.begin(); _iter210 != this->parameters.end(); ++_iter210)
+    std::map<std::string, std::string> ::const_iterator _iter211;
+    for (_iter211 = this->parameters.begin(); _iter211 != this->parameters.end(); ++_iter211)
     {
-      xfer += oprot->writeString(_iter210->first);
-      xfer += oprot->writeString(_iter210->second);
+      xfer += oprot->writeString(_iter211->first);
+      xfer += oprot->writeString(_iter211->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -5049,22 +5205,7 @@ void swap(StorageDescriptor &a, StorageDescriptor &b) {
   swap(a.__isset, b.__isset);
 }
 
-StorageDescriptor::StorageDescriptor(const StorageDescriptor& other211) {
-  cols = other211.cols;
-  location = other211.location;
-  inputFormat = other211.inputFormat;
-  outputFormat = other211.outputFormat;
-  compressed = other211.compressed;
-  numBuckets = other211.numBuckets;
-  serdeInfo = other211.serdeInfo;
-  bucketCols = other211.bucketCols;
-  sortCols = other211.sortCols;
-  parameters = other211.parameters;
-  skewedInfo = other211.skewedInfo;
-  storedAsSubDirectories = other211.storedAsSubDirectories;
-  __isset = other211.__isset;
-}
-StorageDescriptor& StorageDescriptor::operator=(const StorageDescriptor& other212) {
+StorageDescriptor::StorageDescriptor(const StorageDescriptor& other212) {
   cols = other212.cols;
   location = other212.location;
   inputFormat = other212.inputFormat;
@@ -5078,6 +5219,21 @@ StorageDescriptor& StorageDescriptor::operator=(const StorageDescriptor& other21
   skewedInfo = other212.skewedInfo;
   storedAsSubDirectories = other212.storedAsSubDirectories;
   __isset = other212.__isset;
+}
+StorageDescriptor& StorageDescriptor::operator=(const StorageDescriptor& other213) {
+  cols = other213.cols;
+  location = other213.location;
+  inputFormat = other213.inputFormat;
+  outputFormat = other213.outputFormat;
+  compressed = other213.compressed;
+  numBuckets = other213.numBuckets;
+  serdeInfo = other213.serdeInfo;
+  bucketCols = other213.bucketCols;
+  sortCols = other213.sortCols;
+  parameters = other213.parameters;
+  skewedInfo = other213.skewedInfo;
+  storedAsSubDirectories = other213.storedAsSubDirectories;
+  __isset = other213.__isset;
   return *this;
 }
 void StorageDescriptor::printTo(std::ostream& out) const {
@@ -5252,14 +5408,14 @@ uint32_t Table::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partitionKeys.clear();
-            uint32_t _size213;
-            ::apache::thrift::protocol::TType _etype216;
-            xfer += iprot->readListBegin(_etype216, _size213);
-            this->partitionKeys.resize(_size213);
-            uint32_t _i217;
-            for (_i217 = 0; _i217 < _size213; ++_i217)
+            uint32_t _size214;
+            ::apache::thrift::protocol::TType _etype217;
+            xfer += iprot->readListBegin(_etype217, _size214);
+            this->partitionKeys.resize(_size214);
+            uint32_t _i218;
+            for (_i218 = 0; _i218 < _size214; ++_i218)
             {
-              xfer += this->partitionKeys[_i217].read(iprot);
+              xfer += this->partitionKeys[_i218].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5272,17 +5428,17 @@ uint32_t Table::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->parameters.clear();
-            uint32_t _size218;
-            ::apache::thrift::protocol::TType _ktype219;
-            ::apache::thrift::protocol::TType _vtype220;
-            xfer += iprot->readMapBegin(_ktype219, _vtype220, _size218);
-            uint32_t _i222;
-            for (_i222 = 0; _i222 < _size218; ++_i222)
+            uint32_t _size219;
+            ::apache::thrift::protocol::TType _ktype220;
+            ::apache::thrift::protocol::TType _vtype221;
+            xfer += iprot->readMapBegin(_ktype220, _vtype221, _size219);
+            uint32_t _i223;
+            for (_i223 = 0; _i223 < _size219; ++_i223)
             {
-              std::string _key223;
-              xfer += iprot->readString(_key223);
-              std::string& _val224 = this->parameters[_key223];
-              xfer += iprot->readString(_val224);
+              std::string _key224;
+              xfer += iprot->readString(_key224);
+              std::string& _val225 = this->parameters[_key224];
+              xfer += iprot->readString(_val225);
             }
             xfer += iprot->readMapEnd();
           }
@@ -5395,10 +5551,10 @@ uint32_t Table::write(::apache::thrift::protocol::TProtocol* oprot) const {
   xfer += oprot->writeFieldBegin("partitionKeys", ::apache::thrift::protocol::T_LIST, 8);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->partitionKeys.size()));
-    std::vector<FieldSchema> ::const_iterator _iter225;
-    for (_iter225 = this->partitionKeys.begin(); _iter225 != this->partitionKeys.end(); ++_iter225)
+    std::vector<FieldSchema> ::const_iterator _iter226;
+    for (_iter226 = this->partitionKeys.begin(); _iter226 != this->partitionKeys.end(); ++_iter226)
     {
-      xfer += (*_iter225).write(oprot);
+      xfer += (*_iter226).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5407,11 +5563,11 @@ uint32_t Table::write(::apache::thrift::protocol::TProtocol* oprot) const {
   xfer += oprot->writeFieldBegin("parameters", ::apache::thrift::protocol::T_MAP, 9);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->parameters.size()));
-    std::map<std::string, std::string> ::const_iterator _iter226;
-    for (_iter226 = this->parameters.begin(); _iter226 != this->parameters.end(); ++_iter226)
+    std::map<std::string, std::string> ::const_iterator _iter227;
+    for (_iter227 = this->parameters.begin(); _iter227 != this->parameters.end(); ++_iter227)
     {
-      xfer += oprot->writeString(_iter226->first);
-      xfer += oprot->writeString(_iter226->second);
+      xfer += oprot->writeString(_iter227->first);
+      xfer += oprot->writeString(_iter227->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -5475,26 +5631,7 @@ void swap(Table &a, Table &b) {
   swap(a.__isset, b.__isset);
 }
 
-Table::Table(const Table& other227) {
-  tableName = other227.tableName;
-  dbName = other227.dbName;
-  owner = other227.owner;
-  createTime = other227.createTime;
-  lastAccessTime = other227.lastAccessTime;
-  retention = other227.retention;
-  sd = other227.sd;
-  partitionKeys = other227.partitionKeys;
-  parameters = other227.parameters;
-  viewOriginalText = other227.viewOriginalText;
-  viewExpandedText = other227.viewExpandedText;
-  tableType = other227.tableType;
-  privileges = other227.privileges;
-  temporary = other227.temporary;
-  rewriteEnabled = other227.rewriteEnabled;
-  creationMetadata = other227.creationMetadata;
-  __isset = other227.__isset;
-}
-Table& Table::operator=(const Table& other228) {
+Table::Table(const Table& other228) {
   tableName = other228.tableName;
   dbName = other228.dbName;
   owner = other228.owner;
@@ -5512,6 +5649,25 @@ Table& Table::operator=(const Table& other228) {
   rewriteEnabled = other228.rewriteEnabled;
   creationMetadata = other228.creationMetadata;
   __isset = other228.__isset;
+}
+Table& Table::operator=(const Table& other229) {
+  tableName = other229.tableName;
+  dbName = other229.dbName;
+  owner = other229.owner;
+  createTime = other229.createTime;
+  lastAccessTime = other229.lastAccessTime;
+  retention = other229.retention;
+  sd = other229.sd;
+  partitionKeys = other229.partitionKeys;
+  parameters = other229.parameters;
+  viewOriginalText = other229.viewOriginalText;
+  viewExpandedText = other229.viewExpandedText;
+  tableType = other229.tableType;
+  privileges = other229.privileges;
+  temporary = other229.temporary;
+  rewriteEnabled = other229.rewriteEnabled;
+  creationMetadata = other229.creationMetadata;
+  __isset = other229.__isset;
   return *this;
 }
 void Table::printTo(std::ostream& out) const {
@@ -5599,14 +5755,14 @@ uint32_t Partition::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->values.clear();
-            uint32_t _size229;
-            ::apache::thrift::protocol::TType _etype232;
-            xfer += iprot->readListBegin(_etype232, _size229);
-            this->values.resize(_size229);
-            uint32_t _i233;
-            for (_i233 = 0; _i233 < _size229; ++_i233)
+            uint32_t _size230;
+            ::apache::thrift::protocol::TType _etype233;
+            xfer += iprot->readListBegin(_etype233, _size230);
+            this->values.resize(_size230);
+            uint32_t _i234;
+            for (_i234 = 0; _i234 < _size230; ++_i234)
             {
-              xfer += iprot->readString(this->values[_i233]);
+              xfer += iprot->readString(this->values[_i234]);
             }
             xfer += iprot->readListEnd();
           }
@@ -5659,17 +5815,17 @@ uint32_t Partition::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->parameters.clear();
-            uint32_t _size234;
-            ::apache::thrift::protocol::TType _ktype235;
-            ::apache::thrift::protocol::TType _vtype236;
-            xfer += iprot->readMapBegin(_ktype235, _vtype236, _size234);
-            uint32_t _i238;
-            for (_i238 = 0; _i238 < _size234; ++_i238)
+            uint32_t _size235;
+            ::apache::thrift::protocol::TType _ktype236;
+            ::apache::thrift::protocol::TType _vtype237;
+            xfer += iprot->readMapBegin(_ktype236, _vtype237, _size235);
+            uint32_t _i239;
+            for (_i239 = 0; _i239 < _size235; ++_i239)
             {
-              std::string _key239;
-              xfer += iprot->readString(_key239);
-              std::string& _val240 = this->parameters[_key239];
-              xfer += iprot->readString(_val240);
+              std::string _key240;
+              xfer += iprot->readString(_key240);
+              std::string& _val241 = this->parameters[_key240];
+              xfer += iprot->readString(_val241);
             }
             xfer += iprot->readMapEnd();
           }
@@ -5706,10 +5862,10 @@ uint32_t Partition::write(::apache::thrift::protocol::TProtocol* oprot) const {
   xfer += oprot->writeFieldBegin("values", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->values.size()));
-    std::vector<std::string> ::const_iterator _iter241;
-    for (_iter241 = this->values.begin(); _iter241 != this->values.end(); ++_iter241)
+    std::vector<std::string> ::const_iterator _iter242;
+    for (_iter242 = this->values.begin(); _iter242 != this->values.end(); ++_iter242)
     {
-      xfer += oprot->writeString((*_iter241));
+      xfer += oprot->writeString((*_iter242));
     }
     xfer += oprot->writeListEnd();
   }
@@ -5738,11 +5894,11 @@ uint32_t Partition::write(::apache::thrift::protocol::TProtocol* oprot) const {
   xfer += oprot->writeFieldBegin("parameters", ::apache::thrift::protocol::T_MAP, 7);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->parameters.size()));
-    std::map<std::string, std::string> ::const_iterator _iter242;
-    for (_iter242 = this->parameters.begin(); _iter242 != this->parameters.end(); ++_iter242)
+    std::map<std::string, std::string> ::const_iterator _iter243;
+    for (_iter243 = this->parameters.begin(); _iter243 != this->parameters.end(); ++_iter243)
     {
-      xfer += oprot->writeString(_iter242->first);
-      xfer += oprot->writeString(_iter242->second);
+      xfer += oprot->writeString(_iter243->first);
+      xfer += oprot->writeString(_iter243->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -5771,18 +5927,7 @@ void swap(Partition &a, Partition &b) {
   swap(a.__isset, b.__isset);
 }
 
-Partition::Partition(const Partition& other243) {
-  values = other243.values;
-  dbName = other243.dbName;
-  tableName = other243.tableName;
-  createTime = other243.createTime;
-  lastAccessTime = other243.lastAccessTime;
-  sd = other243.sd;
-  parameters = other243.parameters;
-  privileges = other243.privileges;
-  __isset = other243.__isset;
-}
-Partition& Partition::operator=(const Partition& other244) {
+Partition::Partition(const Partition& other244) {
   values = other244.values;
   dbName = other244.dbName;
   tableName = other244.tableName;
@@ -5792,6 +5937,17 @@ Partition& Partition::operator=(const Partition& other244) {
   parameters = other244.parameters;
   privileges = other244.privileges;
   __isset = other244.__isset;
+}
+Partition& Partition::operator=(const Partition& other245) {
+  values = other245.values;
+  dbName = other245.dbName;
+  tableName = other245.tableName;
+  createTime = other245.createTime;
+  lastAccessTime = other245.lastAccessTime;
+  sd = other245.sd;
+  parameters = other245.parameters;
+  privileges = other245.privileges;
+  __isset = other245.__isset;
   return *this;
 }
 void Partition::printTo(std::ostream& out) const {
@@ -5863,14 +6019,14 @@ uint32_t PartitionWithoutSD::read(::apache::thrift::protocol::TProtocol* iprot)
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->values.clear();
-            uint32_t _size245;
-            ::apache::thrift::protocol::TType _etype248;
-            xfer += iprot->readListBegin(_etype248, _size245);
-            this->values.resize(_size245);
-            uint32_t _i249;
-            for (_i249 = 0; _i249 < _size245; ++_i249)
+            uint32_t _size246;
+            ::apache::thrift::protocol::TType _etype249;
+            xfer += iprot->readListBegin(_etype249, _size246);
+            this->values.resize(_size246);
+            uint32_t _i250;
+            for (_i250 = 0; _i250 < _size246; ++_i250)
             {
-              xfer += iprot->readString(this->values[_i249]);
+              xfer += iprot->readString(this->values[_i250]);
             }
             xfer += iprot->readListEnd();
           }
@@ -5907,17 +6063,17 @@ uint32_t PartitionWithoutSD::read(::apache::thrift::protocol::TProtocol* iprot)
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->parameters.clear();
-            uint32_t _size250;
-            ::apache::thrift::protocol::TType _ktype251;
-            ::apache::thrift::protocol::TType _vtype252;
-            xfer += iprot->readMapBegin(_ktype251, _vtype252, _size250);
-            uint32_t _i254;
-            for (_i254 = 0; _i254 < _size250; ++_i254)
+            uint32_t _size251;
+            ::apache::thrift::protocol::TType _ktype252;
+            ::apache::thrift::protocol::TType _vtype253;
+            xfer += iprot->readMapBegin(_ktype252, _vtype253, _size251);
+            uint32_t _i255;
+            for (_i255 = 0; _i255 < _size251; ++_i255)
             {
-              std::string _key255;
-              xfer += iprot->readString(_key255);
-              std::string& _val256 = this->parameters[_key255];
-              xfer += iprot->readString(_val256);
+              std::string _key256;
+              xfer += iprot->readString(_key256);
+              std::string& _val257 = this->parameters[_key256];
+              xfer += iprot->readString(_val257);
             }
             xfer += iprot->readMapEnd();
           }
@@ -5954,10 +6110,10 @@ uint32_t PartitionWithoutSD::write(::apache::thrift::protocol::TProtocol* oprot)
   xfer += oprot->writeFieldBegin("values", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->values.size()));
-    std::vector<std::string> ::const_iterator _iter257;
-    for (_iter257 = this->values.begin(); _iter257 != this->values.end(); ++_iter257)
+    std::vector<std::string> ::const_iterator _iter258;
+    for (_iter258 = this->values.begin(); _iter258 != this->values.end(); ++_iter258)
     {
-      xfer += oprot->writeString((*_iter257));
+      xfer += oprot->writeString((*_iter258));
     }
     xfer += oprot->writeListEnd();
   }
@@ -5978,11 +6134,11 @@ uint32_t PartitionWithoutSD::write(::apache::thrift::protocol::TProtocol* oprot)
   xfer += oprot->writeFieldBegin("parameters", ::apache::thrift::protocol::T_MAP, 5);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->parameters.size()));
-    std::map<std::string, std::string> ::const_iterator _iter258;
-    for (_iter258 = this->parameters.begin(); _iter258 != this->parameters.end(); ++_iter258)
+    std::map<std::string, std::string> ::const_iterator _iter259;
+    for (_iter259 = this->parameters.begin(); _iter259 != this->parameters.end(); ++_iter259)
     {
-      xfer += oprot->writeString(_iter258->first);
-      xfer += oprot->writeString(_iter258->second);
+      xfer += oprot->writeString(_iter259->first);
+      xfer += oprot->writeString(_iter259->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -6009,16 +6165,7 @@ void swap(PartitionWithoutSD &a, PartitionWithoutSD &b) {
   swap(a.__isset, b.__isset);
 }
 
-PartitionWithoutSD::PartitionWithoutSD(const PartitionWithoutSD& other259) {
-  values = other259.values;
-  createTime = other259.createTime;
-  lastAccessTime = other259.lastAccessTime;
-  relativePath = other259.relativePath;
-  parameters = other259.parameters;
-  privileges = other259.privileges;
-  __isset = other259.__isset;
-}
-PartitionWithoutSD& PartitionWithoutSD::operator=(const PartitionWithoutSD& other260) {
+PartitionWithoutSD::PartitionWithoutSD(const PartitionWithoutSD& other260) {
   values = other260.values;
   createTime = other260.createTime;
   lastAccessTime = other260.lastAccessTime;
@@ -6026,6 +6173,15 @@ PartitionWithoutSD& PartitionWithoutSD::operator=(const PartitionWithoutSD& othe
   parameters = other260.parameters;
   privileges = other260.privileges;
   __isset = other260.__isset;
+}
+PartitionWithoutSD& PartitionWithoutSD::operator=(const PartitionWithoutSD& other261) {
+  values = other261.values;
+  createTime = other261.createTime;
+  lastAccessTime = other261.lastAccessTime;
+  relativePath = other261.relativePath;
+  parameters = other261.parameters;
+  privileges = other261.privileges;
+  __isset = other261.__isset;
   return *this;
 }
 void PartitionWithoutSD::printTo(std::ostream& out) const {
@@ -6078,14 +6234,14 @@ uint32_t PartitionSpecWithSharedSD::read(::apache::thrift::protocol::TProtocol*
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partitions.clear();
-            uint32_t _size261;
-            ::apache::thrift::protocol::TType _etype264;
-            xfer += iprot->readListBegin(_etype264, _size261);
-            this->partitions.resize(_size261);
-            uint32_t _i265;
-            for (_i265 = 0; _i265 < _size261; ++_i265)
+            uint32_t _size262;
+            ::apache::thrift::protocol::TType _etype265;
+            xfer += iprot->readListBegin(_etype265, _size262);
+            this->partitions.resize(_size262);
+            uint32_t _i266;
+            for (_i266 = 0; _i266 < _size262; ++_i266)
             {
-              xfer += this->partitions[_i265].read(iprot);
+              xfer += this->partitions[_i266].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -6122,10 +6278,10 @@ uint32_t PartitionSpecWithSharedSD::write(::apache::thrift::protocol::TProtocol*
   xfer += oprot->writeFieldBegin("partitions", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->partitions.size()));
-    std::vector<PartitionWithoutSD> ::const_iterator _iter266;
-    for (_iter266 = this->partitions.begin(); _iter266 != this->partitions.end(); ++_iter266)
+    std::vector<PartitionWithoutSD> ::const_iterator _iter267;
+    for (_iter267 = this->partitions.begin(); _iter267 != this->partitions.end(); ++_iter267)
     {
-      xfer += (*_iter266).write(oprot);
+      xfer += (*_iter267).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -6147,15 +6303,15 @@ void swap(PartitionSpecWithSharedSD &a, PartitionSpecWithSharedSD &b) {
   swap(a.__isset, b.__isset);
 }
 
-PartitionSpecWithSharedSD::PartitionSpecWithSharedSD(const PartitionSpecWithSharedSD& other267) {
-  partitions = other267.partitions;
-  sd = other267.sd;
-  __isset = other267.__isset;
-}
-PartitionSpecWithSharedSD& PartitionSpecWithSharedSD::operator=(const PartitionSpecWithSharedSD& other268) {
+PartitionSpecWithSharedSD::PartitionSpecWithSharedSD(const PartitionSpecWithSharedSD& other268) {
   partitions = other268.partitions;
   sd = other268.sd;
   __isset = other268.__isset;
+}
+PartitionSpecWithSharedSD& PartitionSpecWithSharedSD::operator=(const PartitionSpecWithSharedSD& other269) {
+  partitions = other269.partitions;
+  sd = other269.sd;
+  __isset = other269.__isset;
   return *this;
 }
 void PartitionSpecWithSharedSD::printTo(std::ostream& out) const {
@@ -6200,14 +6356,14 @@ uint32_t PartitionListComposingSpec::read(::apache::thrift::protocol::TProtocol*
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partitions.clear();
-            uint32_t _size269;
-            ::apache::thrift::protocol::TType _etype272;
-            xfer += iprot->readListBegin(_etype272, _size269);
-            this->partitions.resize(_size269);
-            uint32_t _i273;
-            for (_i273 = 0; _i273 < _size269; ++_i273)
+            uint32_t _size270;
+            ::apache::thrift::protocol::TType _etype273;
+            xfer += iprot->readListBegin(_etype273, _size270);
+            this->partitions.resize(_size270);
+            uint32_t _i274;
+            for (_i274 = 0; _i274 < _size270; ++_i274)
             {
-              xfer += this->partitions[_i273].read(iprot);
+              xfer += this->partitions[_i274].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -6236,10 +6392,10 @@ uint32_t PartitionListComposingSpec::write(::apache::thrift::protocol::TProtocol
   xfer += oprot->writeFieldBegin("partitions", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->partitions.size()));
-    std::vector<Partition> ::const_iterator _iter274;
-    for (_iter274 = this->partitions.begin(); _iter274 != this->partitions.end(); ++_iter274)
+    std::vector<Partition> ::const_iterator _iter275;
+    for (_iter275 = this->partitions.begin(); _iter275 != this->partitions.end(); ++_iter275)
     {
-      xfer += (*_iter274).write(oprot);
+      xfer += (*_iter275).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -6256,13 +6412,13 @@ void swap(PartitionListComposingSpec &a, PartitionListComposingSpec &b) {
   swap(a.__isset, b.__isset);
 }
 
-PartitionListComposingSpec::PartitionListComposingSpec(const PartitionListComposingSpec& other275) {
-  partitions = other275.partitions;
-  __isset = other275.__isset;
-}
-PartitionListComposingSpec& PartitionListComposingSpec::operator=(const PartitionListComposingSpec& other276) {
+PartitionListComposingSpec::PartitionListComposingSpec(const PartitionListComposingSpec& other276) {
   partitions = other276.partitions;
   __isset = other276.__isset;
+}
+PartitionListComposingSpec& PartitionListComposingSpec::operator=(const PartitionListComposingSpec& other277) {
+  partitions = other277.partitions;
+  __isset = other277.__isset;
   return *this;
 }
 void PartitionListComposingSpec::printTo(std::ostream& out) const {
@@ -6414,21 +6570,21 @@ void swap(PartitionSpec &a, PartitionSpec &b) {
   swap(a.__isset, b.__isset);
 }
 
-PartitionSpec::PartitionSpec(const PartitionSpec& other277) {
-  dbName = other277.dbName;
-  tableName = other277.tableName;
-  rootPath = other277.rootPath;
-  sharedSDPartitionSpec = other277.sharedSDPartitionSpec;
-  partitionList = other277.partitionList;
-  __isset = other277.__isset;
-}
-PartitionSpec& PartitionSpec::operator=(const PartitionSpec& other278) {
+PartitionSpec::PartitionSpec(const PartitionSpec& other278) {
   dbName = other278.dbName;
   tableName = other278.tableName;
   rootPath = other278.rootPath;
   sharedSDPartitionSpec = other278.sharedSDPartitionSpec;
   partitionList = other278.partitionList;
   __isset = other278.__isset;
+}
+PartitionSpec& PartitionSpec::operator=(const PartitionSpec& other279) {
+  dbName = other279.dbName;
+  tableName = other279.tableName;
+  rootPath = other279.rootPath;
+  sharedSDPartitionSpec = other279.sharedSDPartitionSpec;
+  partitionList = other279.partitionList;
+  __isset = other279.__isset;
   return *this;
 }
 void PartitionSpec::printTo(std::ostream& out) const {
@@ -6574,19 +6730,19 @@ void swap(BooleanColumnStatsData &a, BooleanColumnStatsData &b) {
   swap(a.__isset, b.__isset);
 }
 
-BooleanColumnStatsData::BooleanColumnStatsData(const BooleanColumnStatsData& other279) {
-  numTrues = other279.numTrues;
-  numFalses = other279.numFalses;
-  numNulls = other279.numNulls;
-  bitVectors = other279.bitVectors;
-  __isset = other279.__isset;
-}
-BooleanColumnStatsData& BooleanColumnStatsData::operator=(const BooleanColumnStatsData& other280) {
+BooleanColumnStatsData::BooleanColumnStatsData(const BooleanColumnStatsData& other280) {
   numTrues = other280.numTrues;
   numFalses = other280.numFalses;
   numNulls = other280.numNulls;
   bitVectors = other280.bitVectors;
   __isset = other280.__isset;
+}
+BooleanColumnStatsData& BooleanColumnStatsData::operator=(const BooleanColumnStatsData& other281) {
+  numTrues = other281.numTrues;
+  numFalses = other281.numFalses;
+  numNulls = other281.numNulls;
+  bitVectors = other281.bitVectors;
+  __isset = other281.__isset;
   return *this;
 }
 void BooleanColumnStatsData::printTo(std::ostream& out) const {
@@ -6749,21 +6905,21 @@ void swap(DoubleColumnStatsData &a, DoubleColumnStatsData &b) {
   swap(a.__isset, b.__isset);
 }
 
-DoubleColumnStatsData::DoubleColumnStatsData(const DoubleColumnStatsData& other281) {
-  lowValue = other281.lowValue;
-  highValue = other281.highValue;
-  numNulls = other281.numNulls;
-  numDVs = other281.numDVs;
-  bitVectors = other281.bitVectors;
-  __isset = other281.__isset;
-}
-DoubleColumnStatsData& DoubleColumnStatsData::operator=(const DoubleColumnStatsData& other282) {
+DoubleColumnStatsData::DoubleColumnStatsData(const DoubleColumnStatsData& other282) {
   lowValue = other282.lowValue;
   highValue = other282.highValue;
   numNulls = other282.numNulls;
   numDVs = other282.numDVs;
   bitVectors = other282.bitVectors;
   __isset = other282.__isset;
+}
+DoubleColumnStatsData& DoubleColumnStatsData::operator=(const DoubleColumnStatsData& other283) {
+  lowValue = other283.lowValue;
+  highValue = other283.highValue;
+  numNulls = other283.numNulls;
+  numDVs = other283.numDVs;
+  bitVectors = other283.bitVectors;
+  __isset = other283.__isset;
   return *this;
 }
 void DoubleColumnStatsData::printTo(std::ostream& out) const {
@@ -6927,21 +7083,21 @@ void swap(LongColumnStatsData &a, LongColumnStatsData &b) {
   swap(a.__isset, b.__isset);
 }
 
-LongColumnStatsData::LongColumnStatsData(const LongColumnStatsData& other283) {
-  lowValue = other283.lowValue;
-  highValue = other283.highValue;
-  numNulls = other283.numNulls;
-  numDVs = other283.numDVs;
-  bitVectors = other283.bitVectors;
-  __isset = other283.__isset;
-}
-LongColumnStatsData& LongColumnStatsData::operator=(const LongColumnStatsData& other284) {
+LongColumnStatsData::LongColumnStatsData(const LongColumnStatsData& other284) {
   lowValue = other284.lowValue;
   highValue = other284.highValue;
   numNulls = other284.numNulls;
   numDVs = other284.numDVs;
   bitVectors = other284.bitVectors;
   __isset = other284.__isset;
+}
+LongColumnStatsData& LongColumnStatsData::operator=(const LongColumnStatsData& other285) {
+  lowValue = other285.lowValue;
+  highValue = other285.highValue;
+  numNulls = other285.numNulls;
+  numDVs = other285.numDVs;
+  bitVectors = other285.bitVectors;
+  __isset = other285.__isset;
   return *this;
 }
 void LongColumnStatsData::printTo(std::ostream& out) const {
@@ -7107,21 +7263,21 @@ void swap(StringColumnStatsData &a, StringColumnStatsData &b) {
   swap(a.__isset, b.__isset);
 }
 
-StringColumnStatsData::StringColumnStatsData(const StringColumnStatsData& other285) {
-  maxColLen = other285.maxColLen;
-  avgColLen = other285.avgColLen;
-  numNulls = other285.numNulls;
-  numDVs = other285.numDVs;
-  bitVectors = other285.bitVectors;
-  __isset = other285.__isset;
-}
-StringColumnStatsData& StringColumnStatsData::operator=(const StringColumnStatsData& other286) {
+StringColumnStatsData::StringColumnStatsData(const StringColumnStatsData& other286) {
   maxColLen = other286.maxColLen;
   avgColLen = other286.avgColLen;
   numNulls = other286.numNulls;
   numDVs = other286.numDVs;
   bitVectors = other286.bitVectors;
   __isset = other286.__isset;
+}
+StringColumnStatsData& StringColumnStatsData::operator=(const StringColumnStatsData& other287) {
+  maxColLen = other287.maxColLen;
+  avgColLen = other287.avgColLen;
+  numNulls = other287.numNulls;
+  numDVs = other287.numDVs;
+  bitVectors = other287.bitVectors;
+  __isset = other287.__isset;
   return *this;
 }
 void StringColumnStatsData::printTo(std::ostream& out) const {
@@ -7267,19 +7423,19 @@ void swap(BinaryColumnStatsData &a, BinaryColumnStatsData &b) {
   swap(a.__isset, b.__isset);
 }
 
-BinaryColumnStatsData::BinaryColumnStatsData(const BinaryColumnStatsData& other287) {
-  maxColLen = other287.maxColLen;
-  avgColLen = other287.avgColLen;
-  numNulls = other287.numNulls;
-  bitVectors = other287.bitVectors;
-  __isset = other287.__isset;
-}
-BinaryColumnStatsData& BinaryColumnStatsData::operator=(const BinaryColumnStatsData& other288) {
+BinaryColumnStatsData::BinaryColumnStatsData(const BinaryColumnStatsData& other288) {
   maxColLen = other288.maxColLen;
   avgColLen = other288.avgColLen;
   numNulls = other288.numNulls;
   bitVectors = other288.bitVectors;
   __isset = other288.__isset;
+}
+BinaryColumnStatsData& BinaryColumnStatsData::operator=(const BinaryColumnStatsData& other289) {
+  maxColLen = other289.maxColLen;
+  avgColLen = other289.avgColLen;
+  numNulls = other289.numNulls;
+  bitVectors = other289.bitVectors;
+  __isset = other289.__isset;
   return *this;
 }
 void BinaryColumnStatsData::printTo(std::ostream& out) const {
@@ -7384,13 +7540,13 @@ void swap(Decimal &a, Decimal &b) {
   swap(a.scale, b.scale);
 }
 
-Decimal::Decimal(const Decimal& other289) {
-  unscaled = other289.unscaled;
-  scale = other289.scale;
-}
-Decimal& Decimal::operator=(const Decimal& other290) {
+Decimal::Decimal(const Decimal& other290) {
   unscaled = other290.unscaled;
   scale = other290.scale;
+}
+Decimal& Decimal::operator=(const Decimal& other291) {
+  unscaled = other291.unscaled;
+  scale = other291.scale;
   return *this;
 }
 void Decimal::printTo(std::ostream& out) const {
@@ -7551,21 +7707,21 @@ void swap(DecimalColumnStatsData &a, DecimalColumnStatsData &b) {
   swap(a.__isset, b.__isset);
 }
 
-DecimalColumnStatsData::DecimalColumnStatsData(const DecimalColumnStatsData& other291) {
-  lowValue = other291.lowValue;
-  highValue = other291.highValue;
-  numNulls = other291.numNulls;
-  numDVs = other291.numDVs;
-  bitVectors = other291.bitVectors;
-  __isset = other291.__isset;
-}
-DecimalColumnStatsData& DecimalColumnStatsData::operator=(const DecimalColumnStatsData& other292) {
+DecimalColumnStatsData::DecimalColumnStatsData(const DecimalColumnStatsData& other292) {
   lowValue = other292.lowValue;
   highValue = other292.highValue;
   numNulls = other292.numNulls;
   numDVs = other292.numDVs;
   bitVectors = other292.bitVectors;
   __isset = other292.__isset;
+}
+DecimalColumnStatsData& DecimalColumnStatsData::operator=(const DecimalColumnStatsData& other293) {
+  lowValue = other293.lowValue;
+  highValue = other293.highValue;
+  numNulls = other293.numNulls;
+  numDVs = other293.numDVs;
+  bitVectors = other293.bitVectors;
+  __isset = other293.__isset;
   return *this;
 }
 void DecimalColumnStatsData::printTo(std::ostream& out) const {
@@ -7651,11 +7807,11 @@ void swap(Date &a, Date &b) {
   swap(a.daysSinceEpoch, b.daysSinceEpoch);
 }
 
-Date::Date(const Date& other293) {
-  daysSinceEpoch = other293.daysSinceEpoch;
-}
-Date& Date::operator=(const Date& other294) {
+Date::Date(const Date& other294) {
   daysSinceEpoch = other294.daysSinceEpoch;
+}
+Date& Date::operator=(const Date& other295) {
+  daysSinceEpoch = other295.daysSinceEpoch;
   return *this;
 }
 void Date::printTo(std::ostream& out) const {
@@ -7815,21 +7971,21 @@ void swap(DateColumnStatsData &a, DateColumnStatsData &b) {
   swap(a.__isset, b.__isset);
 }
 
-DateColumnStatsData::DateColumnStatsData(const DateColumnStatsData& other295) {
-  lowValue = other295.lowValue;
-  highValue = other295.highValue;
-  numNulls = other295.numNulls;
-  numDVs = other295.numDVs;
-  bitVectors = other295.bitVectors;
-  __isset = other295.__isset;
-}
-DateColumnStatsData& DateColumnStatsData::operator=(const DateColumnStatsData& other296) {
+DateColumnStatsData::DateColumnStatsData(const DateColumnStatsData& other296) {
   lowValue = other296.lowValue;
   highValue = other296.highValue;
   numNulls = other296.numNulls;
   numDVs = other296.numDVs;
   bitVectors = other296.bitVectors;
   __isset = other296.__isset;
+}
+DateColumnStatsData& DateColumnStatsData::operator=(const DateColumnStatsData& other297) {
+  lowValue = other297.lowValue;
+  highValue = other297.highValue;
+  numNulls = other297.numNulls;
+  numDVs = other297.numDVs;
+  bitVectors = other297.bitVectors;
+  __isset = other297.__isset;
   return *this;
 }
 void DateColumnStatsData::printTo(std::ostream& out) const {
@@ -8015,17 +8171,7 @@ void swap(ColumnStatisticsData &a, ColumnStatisticsData &b) {
   swap(a.__isset, b.__isset);
 }
 
-ColumnStatisticsData::ColumnStatisticsData(const ColumnStatisticsData& other297) {
-  booleanStats = other297.booleanStats;
-  longStats = other297.longStats;
-  doubleStats = other297.doubleStats;
-  stringStats = other297.stringStats;
-  binaryStats = other297.binaryStats;
-  decimalStats = other297.decimalStats;
-  dateStats = other297.dateStats;
-  __isset = other297.__isset;
-}
-ColumnStatisticsData& ColumnStatisticsData::operator=(const ColumnStatisticsData& other298) {
+ColumnStatisticsData::ColumnStatisticsData(const ColumnStatisticsData& other298) {
   booleanStats = other298.booleanStats;
   longStats = other298.longStats;
   doubleStats = other298.doubleStats;
@@ -8034,6 +8180,16 @@ ColumnStatisticsData& ColumnStatisticsData::operator=(const ColumnStatisticsData
   decimalStats = other298.decimalStats;
   dateStats = other298.dateStats;
   __isset = other298.__isset;
+}
+ColumnStatisticsData& ColumnStatisticsData::operator=(const ColumnStatisticsData& other299) {
+  booleanStats = other299.booleanStats;
+  longStats = other299.longStats;
+  doubleStats = other299.doubleStats;
+  stringStats = other299.stringStats;
+  binaryStats = other299.binaryStats;
+  decimalStats = other299.decimalStats;
+  dateStats = other299.dateStats;
+  __isset = other299.__isset;
   return *this;
 }
 void ColumnStatisticsData::printTo(std::ostream& out) const {
@@ -8161,15 +8317,15 @@ void swap(ColumnStatisticsObj &a, ColumnStatisticsObj &b) {
   swap(a.statsData, b.statsData);
 }
 
-ColumnStatisticsObj::ColumnStatisticsObj(const ColumnStatisticsObj& other299) {
-  colName = other299.colName;
-  colType = other299.colType;
-  statsData = other299.statsData;
-}
-ColumnStatisticsObj& ColumnStatisticsObj::operator=(const ColumnStatisticsObj& other300) {
+ColumnStatisticsObj::ColumnStatisticsObj(const ColumnStatisticsObj& other300) {
   colName = other300.colName;
   colType = other300.colType;
   statsData = other300.statsData;
+}
+ColumnStatisticsObj& ColumnStatisticsObj::operator=(const ColumnStatisticsObj& other301) {
+  colName = other301.colName;
+  colType = other301.colType;
+  statsData = other301.statsData;
   return *this;
 }
 void ColumnStatisticsObj::printTo(std::ostream& out) const {
@@ -8332,21 +8488,21 @@ void swap(ColumnStatisticsDesc &a, ColumnStatisticsDesc &b) {
   swap(a.__isset, b.__isset);
 }
 
-ColumnStatisticsDesc::ColumnStatisticsDesc(const ColumnStatisticsDesc& other301) {
-  isTblLevel = other301.isTblLevel;
-  dbName = other301.dbName;
-  tableName = other301.tableName;
-  partName = other301.partName;
-  lastAnalyzed = other301.lastAnalyzed;
-  __isset = other301.__isset;
-}
-ColumnStatisticsDesc& ColumnStatisticsDesc::operator=(const ColumnStatisticsDesc& other302) {
+ColumnStatisticsDesc::ColumnStatisticsDesc(const ColumnStatisticsDesc& other302) {
   isTblLevel = other302.isTblLevel;
   dbName = other302.dbName;
   tableName = other302.tableName;
   partName = other302.partName;
   lastAnalyzed = other302.lastAnalyzed;
   __isset = other302.__isset;
+}
+ColumnStatisticsDesc& ColumnStatisticsDesc::operator=(const ColumnStatisticsDesc& other303) {
+  isTblLevel = other303.isTblLevel;
+  dbName = other303.dbName;
+  tableName = other303.tableName;
+  partName = other303.partName;
+  lastAnalyzed = other303.lastAnalyzed;
+  __isset = other303.__isset;
   return *this;
 }
 void ColumnStatisticsDesc::printTo(std::ostream& out) const {
@@ -8408,14 +8564,14 @@ uint32_t ColumnStatistics::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->statsObj.clear();
-            uint32_t _size303;
-            ::apache::thrift::protocol::TType _etype306;
-            xfer += iprot->readListBegin(_etype306, _size303);
-            this->statsObj.resize(_size303);
-            uint32_t _i307;
-            for (_i307 = 0; _i307 < _size303; ++_i307)
+            uint32_t _size304;
+            ::apache::thrift::protocol::TType _etype307;
+            xfer += iprot->readListBegin(_etype307, _size304);
+            this->statsObj.resize(_size304);
+            uint32_t _i308;
+            for (_i308 = 0; _i308 < _size304; ++_i308)
             {
-              xfer += this->statsObj[_i307].read(iprot);
+              xfer += this->statsObj[_i308].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -8452,10 +8608,10 @@ uint32_t ColumnStatistics::write(::apache::thrift::protocol::TProtocol* oprot) c
   xfer += oprot->writeFieldBegin("statsObj", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->statsObj.size()));
-    std::vector<ColumnStatisticsObj> ::const_iterator _iter308;
-    for (_iter308 = this->statsObj.begin(); _iter308 != this->statsObj.end(); ++_iter308)
+    std::vector<ColumnStatisticsObj> ::const_iterator _iter309;
+    for (_iter309 = this->statsObj.begin(); _iter309 != this->statsObj.end(); ++_iter309)
     {
-      xfer += (*_iter308).write(oprot);
+      xfer += (*_iter309).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -8472,13 +8628,13 @@ void swap(ColumnStatistics &a, ColumnStatistics &b) {
   swap(a.statsObj, b.statsObj);
 }
 
-ColumnStatistics::ColumnStatistics(const ColumnStatistics& other309) {
-  statsDesc = other309.statsDesc;
-  statsObj = other309.statsObj;
-}
-ColumnStatistics& ColumnStatistics::operator=(const ColumnStatistics& other310) {
+ColumnStatistics::ColumnStatistics(const ColumnStatistics& other310) {
   statsDesc = other310.statsDesc;
   statsObj = other310.statsObj;
+}
+ColumnStatistics& ColumnStatistics::operator=(const ColumnStatistics& other311) {
+  statsDesc = other311.statsDesc;
+  statsObj = other311.statsObj;
   return *this;
 }
 void ColumnStatistics::printTo(std::ostream& out) const {
@@ -8529,14 +8685,14 @@ uint32_t AggrStats::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->colStats.clear();
-            uint32_t _size311;
-            ::apache::thrift::protocol::TType _etype314;
-            xfer += iprot->readListBegin(_etype314, _size311);
-            this->colStats.resize(_size311);
-            uint32_t _i315;
-            for (_i315 = 0; _i315 < _size311; ++_i315)
+            uint32_t _size312;
+            ::apache::thrift::protocol::TType _etype315;
+            xfer += iprot->readListBegin(_etype315, _size312);
+            this->colStats.resize(_size312);
+            uint32_t _i316;
+            for (_i316 = 0; _i316 < _size312; ++_i316)
             {
-              xfer += this->colStats[_i315].read(iprot);
+              xfer += this->colStats[_i316].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -8577,10 +8733,10 @@ uint32_t AggrStats::write(::apache::thrift::protocol::TProtocol* oprot) const {
   xfer += oprot->writeFieldBegin("colStats", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->colStats.size()));
-    std::vector<ColumnStatisticsObj> ::const_iterator _iter316;
-    for (_iter316 = this->colStats.begin(); _iter316 != this->colStats.end(); ++_iter316)
+    std::vector<ColumnStatisticsObj> ::const_iterator _iter317;
+    for (_iter317 = this->colStats.begin(); _iter317 != this->colStats.end(); ++_iter317)
     {
-      xfer += (*_iter316).write(oprot);
+      xfer += (*_iter317).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -8601,13 +8757,13 @@ void swap(AggrStats &a, AggrStats &b) {
   swap(a.partsFound, b.partsFound);
 }
 
-AggrStats::AggrStats(const AggrStats& other317) {
-  colStats = other317.colStats;
-  partsFound = other317.partsFound;
-}
-AggrStats& AggrStats::operator=(const AggrStats& other318) {
+AggrStats::AggrStats(const AggrStats& other318) {
   colStats = other318.colStats;
   partsFound = other318.partsFound;
+}
+AggrStats& AggrStats::operator=(const AggrStats& other319) {
+  colStats = other319.colStats;
+  partsFound = other319.partsFound;
   return *this;
 }
 void AggrStats::printTo(std::ostream& out) const {
@@ -8658,14 +8814,14 @@ uint32_t SetPartitionsStatsRequest::read(::apache::thrift::protocol::TProtocol*
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->colStats.clear();
-            uint32_t _size319;
-            ::apache::thrift::protocol::TType _etype322;
-            xfer += iprot->readListBegin(_etype322, _size319);
-            this->colStats.resize(_size319);
-            uint32_t _i323;
-            for (_i323 = 0; _i323 < _size319; ++_i323)
+            uint32_t _size320;
+            ::apache::thrift::protocol::TType _etype323;
+            xfer += iprot->readListBegin(_etype323, _size320);
+            this->colStats.resize(_size320);
+            uint32_t _i324;
+            for (_i324 = 0; _i324 < _size320; ++_i324)
             {
-              xfer += this->colStats[_i323].read(iprot);
+              xfer += this->colStats[_i324].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -8704,10 +8860,10 @@ uint32_t SetPartitionsStatsRequest::write(::apache::thrift::protocol::TProtocol*
   xfer += oprot->writeFieldBegin("colStats", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->colStats.size()));
-    std::vector<ColumnStatistics> ::const_iterator _iter324;
-    for (_iter324 = this->colStats.begin(); _iter324 != this->colStats.end(); ++_iter324)
+    std::vector<ColumnStatistics> ::const_iterator _iter325;
+    for (_iter325 = this->colStats.begin(); _iter325 != this->colStats.end(); ++_iter325)
     {
-      xfer += (*_iter324).write(oprot);
+      xfer += (*_iter325).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -8730,15 +8886,15 @@ void swap(SetPartitionsStatsRequest &a, SetPartitionsStatsRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-SetPartitionsStatsRequest::SetPartitionsStatsRequest(const SetPartitionsStatsRequest& other325) {
-  colStats = other325.colStats;
-  needMerge = other325.needMerge;
-  __isset = other325.__isset;
-}
-SetPartitionsStatsRequest& SetPartitionsStatsRequest::operator=(const SetPartitionsStatsRequest& other326) {
+SetPartitionsStatsRequest::SetPartitionsStatsRequest(const SetPartitionsStatsRequest& other326) {
   colStats = other326.colStats;
   needMerge = other326.needMerge;
   __isset = other326.__isset;
+}
+SetPartitionsStatsRequest& SetPartitionsStatsRequest::operator=(const SetPartitionsStatsRequest& other327) {
+  colStats = other327.colStats;
+  needMerge = other327.needMerge;
+  __isset = other327.__isset;
   return *this;
 }
 void SetPartitionsStatsRequest::printTo(std::ostream& out) const {
@@ -8787,14 +8943,14 @@ uint32_t Schema::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->fieldSchemas.clear();
-            uint32_t _size327;
-            ::apache::thrift::protocol::TType _etype330;
-            xfer += iprot->readListBegin(_etype330, _size327);
-            this->fieldSchemas.resize(_size327);
-            uint32_t _i331;
-            for (_i331 = 0; _i331 < _size327; ++_i331)
+            uint32_t _size328;
+            ::apache::thrift::protocol::TType _etype331;
+            xfer += iprot->readListBegin(_etype331, _size328);
+            this->fieldSchemas.resize(_size328);
+            uint32_t _i332;
+            for (_i332 = 0; _i332 < _size328; ++_i332)
             {
-              xfer += this->fieldSchemas[_i331].read(iprot);
+              xfer += this->fieldSchemas[_i332].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -8807,17 +8963,17 @@ uint32_t Schema::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->properties.clear();
-            uint32_t _size332;
-            ::apache::thrift::protocol::TType _ktype333;
-            ::apache::thrift::protocol::TType _vtype334;
-            xfer += iprot->readMapBegin(_ktype333, _vtype334, _size332);
-            uint32_t _i336;
-            for (_i336 = 0; _i336 < _size332; ++_i336)
+            uint32_t _size333;
+            ::apache::thrift::protocol::TType _ktype334;
+            ::apache::thrift::protocol::TType _vtype335;
+            xfer += iprot->readMapBegin(_ktype334, _vtype335, _size333);
+            uint32_t _i337;
+            for (_i337 = 0; _i337 < _size333; ++_i337)
             {
-              std::string _key337;
-              xfer += iprot->readString(_key337);
-              std::string& _val338 = this->properties[_key337];
-              xfer += iprot->readString(_val338);
+              std::string _key338;
+              xfer += iprot->readString(_key338);
+              std::string& _val339 = this->properties[_key338];
+              xfer += iprot->readString(_val339);
             }
             xfer += iprot->readMapEnd();
           }
@@ -8846,10 +9002,10 @@ uint32_t Schema::write(::apache::thrift::protocol::TProtocol* oprot) const {
   xfer += oprot->writeFieldBegin("fieldSchemas", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->fieldSchemas.size()));
-    std::vector<FieldSchema> ::const_iterator _iter339;
-    for (_iter339 = this->fieldSchemas.begin(); _iter339 != this->fieldSchemas.end(); ++_iter339)
+    std::vector<FieldSchema> ::const_iterator _iter340;
+    for (_iter340 = this->fieldSchemas.begin(); _iter340 != this->fieldSchemas.end(); ++_iter340)
     {
-      xfer += (*_iter339).write(oprot);
+      xfer += (*_iter340).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -8858,11 +9014,11 @@ uint32_t Schema::write(::apache::thrift::protocol::TProtocol* oprot) const {
   xfer += oprot->writeFieldBegin("properties", ::apache::thrift::protocol::T_MAP, 2);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->properties.size()));
-    std::map<std::string, std::string> ::const_iterator _iter340;
-    for (_iter340 = this->properties.begin(); _iter340 != this->properties.end(); ++_iter340)
+    std::map<std::string, std::string> ::const_iterator _iter341;
+    for (_iter341 = this->properties.begin(); _iter341 != this->properties.end(); ++_iter341)
     {
-      xfer += oprot->writeString(_iter340->first);
-      xfer += oprot->writeString(_iter340->second);
+      xfer += oprot->writeString(_iter341->first);
+      xfer += oprot->writeString(_iter341->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -8880,15 +9036,15 @@ void swap(Schema &a, Schema &b) {
   swap(a.__isset, b.__isset);
 }
 
-Schema::Schema(const Schema& other341) {
-  fieldSchemas = other341.fieldSchemas;
-  properties = other341.properties;
-  __isset = other341.__isset;
-}
-Schema& Schema::operator=(const Schema& other342) {
+Schema::Schema(const Schema& other342) {
   fieldSchemas = other342.fieldSchemas;
   properties = other342.properties;
   __isset = other342.__isset;
+}
+Schema& Schema::operator=(const Schema& other343) {
+  fieldSchemas = other343.fieldSchemas;
+  properties = other343.properties;
+  __isset = other343.__isset;
   return *this;
 }
 void Schema::printTo(std::ostream& out) const {
@@ -8933,17 +9089,17 @@ uint32_t EnvironmentContext::read(::apache::thrift::protocol::TProtocol* iprot)
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->properties.clear();
-            uint32_t _size343;
-            ::apache::thrift::protocol::TType _ktype344;
-            ::apache::thrift::protocol::TType _vtype345;
-            xfer += iprot->readMapBegin(_ktype344, _vtype345, _size343);
-            uint32_t _i347;
-            for (_i347 = 0; _i347 < _size343; ++_i347)
+            uint32_t _size344;
+            ::apache::thrift::protocol::TType _ktype345;
+            ::apache::thrift::protocol::TType _vtype346;
+            xfer += iprot->readMapBegin(_ktype345, _vtype346, _size344);
+            uint32_t _i348;
+            for (_i348 = 0; _i348 < _size344; ++_i348)
             {
-              std::string _key348;
-              xfer += iprot->readString(_key348);
-              std::string& _val349 = this->properties[_key348];
-              xfer += iprot->readString(_val349);
+              std::string _key349;
+              xfer += iprot->readString(_key349);
+              std::string& _val350 = this->properties[_key349];
+              xfer += iprot->readString(_val350);
             }
             xfer += iprot->readMapEnd();
           }
@@ -8972,11 +9128,11 @@ uint32_t EnvironmentContext::write(::apache::thrift::protocol::TProtocol* oprot)
   xfer += oprot->writeFieldBegin("properties", ::apache::thrift::protocol::T_MAP, 1);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->properties.size()));
-    std::map<std::string, std::string> ::const_iterator _iter350;
-    for (_iter350 = this->properties.begin(); _iter350 != this->properties.end(); ++_iter350)
+    std::map<std::string, std::string> ::const_iterator _iter351;
+    for (_iter351 = this->properties.begin(); _iter351 != this->properties.end(); ++_iter351)
     {
-      xfer += oprot->writeString(_iter350->first);
-      xfer += oprot->writeString(_iter350->second);
+      xfer += oprot->writeString(_iter351->first);
+      xfer += oprot->writeString(_iter351->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -8993,13 +9149,13 @@ void swap(EnvironmentContext &a, EnvironmentContext &b) {
   swap(a.__isset, b.__isset);
 }
 
-EnvironmentContext::EnvironmentContext(const EnvironmentContext& other351) {
-  properties = other351.properties;
-  __isset = other351.__isset;
-}
-EnvironmentContext& EnvironmentContext::operator=(const EnvironmentContext& other352) {
+EnvironmentContext::EnvironmentContext(const EnvironmentContext& other352) {
   properties = other352.properties;
   __isset = other352.__isset;
+}
+EnvironmentContext& EnvironmentContext::operator=(const EnvironmentContext& other353) {
+  properties = other353.properties;
+  __isset = other353.__isset;
   return *this;
 }
 void EnvironmentContext::printTo(std::ostream& out) const {
@@ -9101,13 +9257,13 @@ void swap(PrimaryKeysRequest &a, PrimaryKeysRequest &b) {
   swap(a.tbl_name, b.tbl_name);
 }
 
-PrimaryKeysRequest::PrimaryKeysRequest(const PrimaryKeysRequest& other353) {
-  db_name = other353.db_name;
-  tbl_name = other353.tbl_name;
-}
-PrimaryKeysRequest& PrimaryKeysRequest::operator=(const PrimaryKeysRequest& other354) {
+PrimaryKeysRequest::PrimaryKeysRequest(const PrimaryKeysRequest& other354) {
   db_name = other354.db_name;
   tbl_name = other354.tbl_name;
+}
+PrimaryKeysRequest& PrimaryKeysRequest::operator=(const PrimaryKeysRequest& other355) {
+  db_name = other355.db_name;
+  tbl_name = other355.tbl_name;
   return *this;
 }
 void PrimaryKeysRequest::printTo(std::ostream& out) const {
@@ -9153,14 +9309,14 @@ uint32_t PrimaryKeysResponse::read(::apache::thrift::protocol::TProtocol* iprot)
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->primaryKeys.clear();
-            uint32_t _size355;
-            ::apache::thrift::protocol::TType _etype358;
-            xfer += iprot->readListBegin(_etype358, _size355);
-            this->primaryKeys.resize(_size355);
-            uint32_t _i359;
-            for (_i359 = 0; _i359 < _size355; ++_i359)
+            uint32_t _size356;
+            ::apache::thrift::protocol::TType _etype359;
+            xfer += iprot->readListBegin(_etype359, _size356);
+            this->primaryKeys.resize(_size356);
+            uint32_t _i360;
+            for (_i360 = 0; _i360 < _size356; ++_i360)
             {
-              xfer += this->primaryKeys[_i359].read(iprot);
+              xfer += this->primaryKeys[_i360].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -9191,10 +9347,10 @@ uint32_t PrimaryKeysResponse::write(::apache::thrift::protocol::TProtocol* oprot
   xfer += oprot->writeFieldBegin("primaryKeys", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->primaryKeys.size()));
-    std::vector<SQLPrimaryKey> ::const_iterator _iter360;
-    for (_iter360 = this->primaryKeys.begin(); _iter360 != this->primaryKeys.end(); ++_iter360)
+    std::vector<SQLPrimaryKey> ::const_iterator _iter361;
+    for (_iter361 = this->primaryKeys.begin(); _iter361 != this->primaryKeys.end(); ++_iter361)
     {
-      xfer += (*_iter360).write(oprot);
+      xfer += (*_iter361).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -9210,11 +9366,11 @@ void swap(PrimaryKeysResponse &a, PrimaryKeysResponse &b) {
   swap(a.primaryKeys, b.primaryKeys);
 }
 
-PrimaryKeysResponse::PrimaryKeysResponse(const PrimaryKeysResponse& other361) {
-  primaryKeys = other361.primaryKeys;
-}
-PrimaryKeysResponse& PrimaryKeysResponse::operator=(const PrimaryKeysResponse& other362) {
+PrimaryKeysResponse::PrimaryKeysResponse(const PrimaryKeysResponse& other362) {
   primaryKeys = other362.primaryKeys;
+}
+PrimaryKeysResponse& PrimaryKeysResponse::operator=(const PrimaryKeysResponse& other363) {
+  primaryKeys = other363.primaryKeys;
   return *this;
 }
 void PrimaryKeysResponse::printTo(std::ostream& out) const {
@@ -9345,19 +9501,19 @@ void swap(ForeignKeysRequest &a, ForeignKeysRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-ForeignKeysRequest::ForeignKeysRequest(const ForeignKeysRequest& other363) {
-  parent_db_name = other363.parent_db_name;
-  parent_tbl_name = other363.parent_tbl_name;
-  foreign_db_name = other363.foreign_db_name;
-  foreign_tbl_name = other363.foreign_tbl_name;
-  __isset = other363.__isset;
-}
-ForeignKeysRequest& ForeignKeysRequest::operator=(const ForeignKeysRequest& other364) {
+ForeignKeysRequest::ForeignKeysRequest(const ForeignKeysRequest& other364) {
   parent_db_name = other364.parent_db_name;
   parent_tbl_name = other364.parent_tbl_name;
   foreign_db_name = other364.foreign_db_name;
   foreign_tbl_name = other364.foreign_tbl_name;
   __isset = other364.__isset;
+}
+ForeignKeysRequest& ForeignKeysRequest::operator=(const ForeignKeysRequest& other365) {
+  parent_db_name = other365.parent_db_name;
+  parent_tbl_name = other365.parent_tbl_name;
+  foreign_db_name = other365.foreign_db_name;
+  foreign_tbl_name = other365.foreign_tbl_name;
+  __isset = other365.__isset;
   return *this;
 }
 void ForeignKeysRequest::printTo(std::ostream& out) const {
@@ -9405,14 +9561,14 @@ uint32_t ForeignKeysResponse::read(::apache::thrift::protocol::TProtocol* iprot)
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->foreignKeys.clear();
-            uint32_t _size365;
-            ::apache::thrift::protocol::TType _etype368;
-            xfer += iprot->readListBegin(_etype368, _size365);
-            this->foreignKeys.resize(_size365);
-            uint32_t _i369;
-            for (_i369 = 0; _i369 < _size365; ++_i369)
+            uint32_t _size366;
+            ::apache::thrift::protocol::TType _etype369;
+            xfer += iprot->readListBegin(_etype369, _size366);
+            this->foreignKeys.resize(_size366);
+            uint32_t _i370;
+            for (_i370 = 0; _i370 < _size366; ++_i370)
             {
-              xfer += this->foreignKeys[_i369].read(iprot);
+              xfer += this->foreignKeys[_i370].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -9443,10 +9599,10 @@ uint32_t ForeignKeysResponse::write(::apache::thrift::protocol::TProtocol* oprot
   xfer += oprot->writeFieldBegin("foreignKeys", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->foreignKeys.size()));
-    std::vector<SQLForeignKey> ::const_iterator _iter370;
-    for (_iter370 = this->foreignKeys.begin(); _iter370 != this->foreignKeys.end(); ++_iter370)
+    std::vector<SQLForeignKey> ::const_iterator _iter371;
+    for (_iter371 = this->foreignKeys.begin(); _iter371 != this->foreignKeys.end(); ++_iter371)
     {
-      xfer += (*_iter370).write(oprot);
+      xfer += (*_iter371).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -9462,11 +9618,11 @@ void swap(ForeignKeysResponse &a, ForeignKeysResponse &b) {
   swap(a.foreignKeys, b.foreignKeys);
 }
 
-ForeignKeysResponse::ForeignKeysResponse(const ForeignKeysResponse& other371) {
-  foreignKeys = other371.foreignKeys;
-}
-ForeignKeysResponse& ForeignKeysResponse::operator=(const ForeignKeysResponse& other372) {
+ForeignKeysResponse::ForeignKeysResponse(const ForeignKeysResponse& other372) {
   foreignKeys = other372.foreignKeys;
+}
+ForeignKeysResponse& ForeignKeysResponse::operator=(const ForeignKeysResponse& other373) {
+  foreignKeys = other373.foreignKeys;
   return *this;
 }
 void ForeignKeysResponse::printTo(std::ostream& out) const {
@@ -9568,13 +9724,13 @@ void swap(UniqueConstraintsRequest &a, UniqueConstraintsRequest &b) {
   swap(a.tbl_name, b.tbl_name);
 }
 
-UniqueConstraintsRequest::UniqueConstraintsRequest(const UniqueConstraintsRequest& other373) {
-  db_name = other373.db_name;
-  tbl_name = other373.tbl_name;
-}
-UniqueConstraintsRequest& UniqueConstraintsRequest::operator=(const UniqueConstraintsRequest& other374) {
+UniqueConstraintsRequest::UniqueConstraintsRequest(const UniqueConstraintsRequest& other374) {
   db_name = other374.db_name;
   tbl_name = other374.tbl_name;
+}
+UniqueConstraintsRequest& UniqueConstraintsRequest::operator=(const UniqueConstraintsRequest& other375) {
+  db_name = other375.db_name;
+  tbl_name = other375.tbl_name;
   return *this;
 }
 void UniqueConstraintsRequest::printTo(std::ostream& out) const {
@@ -9620,14 +9776,14 @@ uint32_t UniqueConstraintsResponse::read(::apache::thrift::protocol::TProtocol*
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->uniqueConstraints.clear();
-            uint32_t _size375;
-            ::apache::thrift::protocol::TType _etype378;
-            xfer += iprot->readListBegin(_etype378, _size375);
-            this->uniqueConstraints.resize(_size375);
-            uint32_t _i379;
-            for (_i379 = 0; _i379 < _size375; ++_i379)
+            uint32_t _size376;
+            ::apache::thrift::protocol::TType _etype379;
+            xfer += iprot->readListBegin(_etype379, _size376);
+            this->uniqueConstraints.resize(_size376);
+            uint32_t _i380;
+            for (_i380 = 0; _i380 < _size376; ++_i380)
             {
-              xfer += this->uniqueConstraints[_i379].read(iprot);
+              xfer += this->uniqueConstraints[_i380].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -9658,10 +9814,10 @@ uint32_t UniqueConstraintsResponse::write(::apache::thrift::protocol::TProtocol*
   xfer += oprot->writeFieldBegin("uniqueConstraints", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->uniqueConstraints.size()));
-    std::vector<SQLUniqueConstraint> ::const_iterator _iter380;
-    for (_iter380 = this->uniqueConstraints.begin(); _iter380 != this->uniqueConstraints.end(); ++_iter380)
+    std::vector<SQLUniqueConstraint> ::const_iterator _iter381;
+    for (_iter381 = this->uniqueConstraints.begi

<TRUNCATED>