You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by vi...@apache.org on 2018/10/07 22:47:37 UTC

[01/13] hive git commit: HIVE-20306 : Implement projection spec for fetching only requested fields from partitions (Vihang Karajgaonkar, reviewed by Aihua Xu, Andrew Sherman and Alexander Kolbasov)

Repository: hive
Updated Branches:
  refs/heads/master 827f4f9c0 -> 44ef91a67


http://git-wip-us.apache.org/repos/asf/hive/blob/44ef91a6/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestGetPartitionsUsingProjection.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestGetPartitionsUsingProjection.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestGetPartitionsUsingProjection.java
new file mode 100644
index 0000000..dcff606
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestGetPartitionsUsingProjection.java
@@ -0,0 +1,700 @@
+/*
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.hive.metastore;
+
+import org.apache.commons.beanutils.PropertyUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.annotation.MetastoreCheckinTest;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.GetPartitionsFilterSpec;
+import org.apache.hadoop.hive.metastore.api.GetPartitionsProjectionSpec;
+import org.apache.hadoop.hive.metastore.api.GetPartitionsRequest;
+import org.apache.hadoop.hive.metastore.api.GetPartitionsResponse;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.PartitionListComposingSpec;
+import org.apache.hadoop.hive.metastore.api.PartitionSpec;
+import org.apache.hadoop.hive.metastore.api.PartitionSpecWithSharedSD;
+import org.apache.hadoop.hive.metastore.api.PartitionWithoutSD;
+import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.client.builder.DatabaseBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.PartitionBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.TableBuilder;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
+import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge;
+import org.apache.thrift.TException;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.reflect.InvocationTargetException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import static org.apache.hadoop.hive.metastore.ColumnType.SERIALIZATION_FORMAT;
+
+/**
+ * Tests for getPartitionsWithSpecs metastore API. This test create some partitions and makes sure
+ * that getPartitionsWithSpecs returns results which are comparable with the get_partitions API when
+ * various combinations of projection spec are set. Also checks the JDO code path in addition to
+ * directSQL code path
+ */
+@Category(MetastoreCheckinTest.class)
+public class TestGetPartitionsUsingProjection {
+  private static final Logger LOG = LoggerFactory.getLogger(TestGetPartitionsUsingProjection.class);
+  protected static Configuration conf = MetastoreConf.newMetastoreConf();
+  private static int port;
+  private static final String dbName = "test_projection_db";
+  private static final String tblName = "test_projection_table";
+  private List<Partition> origPartitions;
+  private Table tbl;
+  private static final String EXCLUDE_KEY_PREFIX = "exclude";
+  private HiveMetaStoreClient client;
+
+  @BeforeClass
+  public static void startMetaStoreServer() throws Exception {
+    conf.set("hive.in.test", "true");
+    MetaStoreTestUtils.setConfForStandloneMode(conf);
+    MetastoreConf.setLongVar(conf, ConfVars.BATCH_RETRIEVE_MAX, 2);
+    MetastoreConf.setLongVar(conf, ConfVars.LIMIT_PARTITION_REQUEST, 100);
+    port = MetaStoreTestUtils.startMetaStoreWithRetry(HadoopThriftAuthBridge.getBridge(), conf);
+    LOG.info("Starting MetaStore Server on port " + port);
+
+    try (HiveMetaStoreClient client = createClient()) {
+      new DatabaseBuilder().setName(dbName).create(client, conf);
+    }
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    try (HiveMetaStoreClient client = createClient()) {
+      client.dropDatabase(dbName, true, true, true);
+    }
+  }
+
+  @Before
+  public void setup() throws TException {
+    // This is default case with setugi off for both client and server
+    client = createClient();
+    createTestTables();
+    origPartitions = client.listPartitions(dbName, tblName, (short) -1);
+    tbl = client.getTable(dbName, tblName);
+    // set directSQL to true explicitly
+    client.setMetaConf(ConfVars.TRY_DIRECT_SQL.getVarname(), "true");
+    client.setMetaConf(ConfVars.TRY_DIRECT_SQL_DDL.getVarname(), "true");
+  }
+
+  @After
+  public void cleanup() {
+    dropTestTables();
+    client.close();
+    client = null;
+  }
+
+  private void dropTestTables() {
+    try {
+      client.dropTable(dbName, tblName);
+    } catch (TException e) {
+      // ignored
+    }
+  }
+
+  private void createTestTables() throws TException {
+    if (client.tableExists(dbName, tblName)) {
+      LOG.info("Table is already existing. Dropping it and then recreating");
+      client.dropTable(dbName, tblName);
+    }
+    new TableBuilder().setTableName(tblName).setDbName(dbName).setCols(Arrays
+        .asList(new FieldSchema("col1", "string", "c1 comment"),
+            new FieldSchema("col2", "int", "c2 comment"))).setPartCols(Arrays
+        .asList(new FieldSchema("state", "string", "state comment"),
+            new FieldSchema("city", "string", "city comment")))
+        .setTableParams(new HashMap<String, String>(2) {{
+          put("tableparam1", "tableval1");
+          put("tableparam2", "tableval2");
+        }})
+        .setBucketCols(Collections.singletonList("col1"))
+        .addSortCol("col2", 1)
+        .addSerdeParam(SERIALIZATION_FORMAT, "1").setSerdeName(tblName)
+        .setSerdeLib("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe")
+        .setInputFormat("org.apache.hadoop.hive.ql.io.HiveInputFormat")
+        .setOutputFormat("org.apache.hadoop.hive.ql.io.HiveOutputFormat")
+        .create(client, conf);
+
+    Table table = client.getTable(dbName, tblName);
+    Assert.assertTrue("Table " + dbName + "." + tblName + " does not exist",
+        client.tableExists(dbName, tblName));
+
+    List<Partition> partitions = new ArrayList<>();
+    partitions.add(createPartition(Arrays.asList("CA", "SanFrancisco"), table));
+    partitions.add(createPartition(Arrays.asList("CA", "PaloAlto"), table));
+    partitions.add(createPartition(Arrays.asList("WA", "Seattle"), table));
+    partitions.add(createPartition(Arrays.asList("AZ", "Phoenix"), table));
+
+    client.add_partitions(partitions);
+  }
+
+  private Partition createPartition(List<String> vals, Table table) throws MetaException {
+    return new PartitionBuilder()
+        .inTable(table)
+        .setValues(vals)
+        .addPartParam("key1", "S1")
+        .addPartParam("key2", "S2")
+        .addPartParam(EXCLUDE_KEY_PREFIX + "key1", "e1")
+        .addPartParam(EXCLUDE_KEY_PREFIX + "key2", "e2")
+        .setBucketCols(table.getSd().getBucketCols())
+        .setSortCols(table.getSd().getSortCols())
+        .setSerdeName(table.getSd().getSerdeInfo().getName())
+        .setSerdeLib(table.getSd().getSerdeInfo().getSerializationLib())
+        .setSerdeParams(table.getSd().getSerdeInfo().getParameters())
+        .build(conf);
+  }
+
+  private static HiveMetaStoreClient createClient() throws MetaException {
+    MetastoreConf.setVar(conf, ConfVars.THRIFT_URIS, "thrift://localhost:" + port);
+    MetastoreConf.setBoolVar(conf, ConfVars.EXECUTE_SET_UGI, false);
+    return new HiveMetaStoreClient(conf);
+  }
+
+  @Test
+  public void testGetPartitions() throws TException {
+    GetPartitionsRequest request = getGetPartitionsRequest();
+    GetPartitionsResponse response = client.getPartitionsWithSpecs(request);
+    validateBasic(response);
+  }
+
+  @Test
+  public void testPartitionProjectionEmptySpec() throws Throwable {
+    GetPartitionsRequest request = getGetPartitionsRequest();
+    GetPartitionsProjectionSpec projectSpec = request.getProjectionSpec();
+
+    projectSpec.setFieldList(new ArrayList<>(0));
+    projectSpec.setExcludeParamKeyPattern("exclude%");
+
+    GetPartitionsResponse response;
+    response = client.getPartitionsWithSpecs(request);
+    Assert.assertEquals(1, response.getPartitionSpec().size());
+    PartitionSpec partitionSpec = response.getPartitionSpec().get(0);
+    PartitionSpecWithSharedSD partitionSpecWithSharedSD = partitionSpec.getSharedSDPartitionSpec();
+
+    StorageDescriptor sharedSD = partitionSpecWithSharedSD.getSd();
+    Assert.assertNotNull(sharedSD);
+    // everything except location in sharedSD should be same
+    StorageDescriptor origSd = origPartitions.get(0).getSd().deepCopy();
+    origSd.unsetLocation();
+    StorageDescriptor sharedSDCopy = sharedSD.deepCopy();
+    sharedSDCopy.unsetLocation();
+    Assert.assertEquals(origSd, sharedSDCopy);
+
+    List<PartitionWithoutSD> partitionWithoutSDS = partitionSpecWithSharedSD.getPartitions();
+    Assert.assertNotNull(partitionWithoutSDS);
+    Assert.assertEquals("Unexpected number of partitions returned",
+        origPartitions.size(), partitionWithoutSDS.size());
+    for (int i = 0; i < origPartitions.size(); i++) {
+      Partition origPartition = origPartitions.get(i);
+      PartitionWithoutSD retPartition = partitionWithoutSDS.get(i);
+      Assert.assertEquals(origPartition.getCreateTime(), retPartition.getCreateTime());
+      Assert.assertEquals(origPartition.getLastAccessTime(), retPartition.getLastAccessTime());
+      Assert.assertEquals(origPartition.getSd().getLocation(),
+          sharedSD.getLocation() + retPartition.getRelativePath());
+      validateMap(origPartition.getParameters(), retPartition.getParameters());
+      validateList(origPartition.getValues(), retPartition.getValues());
+    }
+  }
+
+  @Test
+  public void testPartitionProjectionAllSingleValuedFields() throws Throwable {
+    GetPartitionsRequest request = getGetPartitionsRequest();
+    GetPartitionsProjectionSpec projectSpec = request.getProjectionSpec();
+
+    List<String> projectedFields = Arrays
+        .asList("dbName", "tableName", "createTime", "lastAccessTime", "sd.location",
+            "sd.inputFormat", "sd.outputFormat", "sd.compressed", "sd.numBuckets",
+            "sd.serdeInfo.name", "sd.serdeInfo.serializationLib"/*, "sd.serdeInfo.serdeType"*/);
+    //TODO directSQL does not support serdeType, serializerClass and deserializerClass in serdeInfo
+    projectSpec.setFieldList(projectedFields);
+
+    GetPartitionsResponse response = client.getPartitionsWithSpecs(request);
+    Assert.assertEquals(1, response.getPartitionSpec().size());
+    PartitionSpec partitionSpec = response.getPartitionSpec().get(0);
+    Assert.assertTrue("DbName is not set", partitionSpec.isSetDbName());
+    Assert.assertTrue("tableName is not set", partitionSpec.isSetTableName());
+    PartitionSpecWithSharedSD partitionSpecWithSharedSD = partitionSpec.getSharedSDPartitionSpec();
+
+    StorageDescriptor sharedSD = partitionSpecWithSharedSD.getSd();
+    Assert.assertNotNull(sharedSD);
+    List<PartitionWithoutSD> partitionWithoutSDS = partitionSpecWithSharedSD.getPartitions();
+    Assert.assertNotNull(partitionWithoutSDS);
+    Assert.assertEquals(partitionWithoutSDS.size(), origPartitions.size());
+    comparePartitionForSingleValuedFields(projectedFields, sharedSD, partitionWithoutSDS, 0);
+  }
+
+  @Test
+  public void testProjectionUsingJDO() throws Throwable {
+    // disable direct SQL to make sure
+    client.setMetaConf(ConfVars.TRY_DIRECT_SQL.getVarname(), "false");
+    GetPartitionsRequest request = getGetPartitionsRequest();
+    GetPartitionsProjectionSpec projectSpec = request.getProjectionSpec();
+    List<String> projectedFields = Collections.singletonList("sd.location");
+    projectSpec.setFieldList(projectedFields);
+
+    GetPartitionsResponse response = client.getPartitionsWithSpecs(request);
+    Assert.assertEquals(1, response.getPartitionSpec().size());
+    PartitionSpec partitionSpec = response.getPartitionSpec().get(0);
+    Assert.assertTrue("DbName is not set", partitionSpec.isSetDbName());
+    Assert.assertTrue("tableName is not set", partitionSpec.isSetTableName());
+    PartitionSpecWithSharedSD partitionSpecWithSharedSD = partitionSpec.getSharedSDPartitionSpec();
+
+    StorageDescriptor sharedSD = partitionSpecWithSharedSD.getSd();
+    Assert.assertNotNull(sharedSD);
+    List<PartitionWithoutSD> partitionWithoutSDS = partitionSpecWithSharedSD.getPartitions();
+    Assert.assertNotNull(partitionWithoutSDS);
+    Assert.assertEquals(partitionWithoutSDS.size(), origPartitions.size());
+    comparePartitionForSingleValuedFields(projectedFields, sharedSD, partitionWithoutSDS, 0);
+
+    // set all the single-valued fields and try using JDO
+    request = getGetPartitionsRequest();
+    projectSpec = request.getProjectionSpec();
+    projectedFields = Arrays
+        .asList("dbName", "tableName", "createTime", "lastAccessTime", "sd.location",
+            "sd.inputFormat", "sd.outputFormat", "sd.compressed", "sd.numBuckets",
+            "sd.serdeInfo.name", "sd.serdeInfo.serializationLib", "sd.serdeInfo.serdeType",
+            "sd.serdeInfo.serializerClass", "sd.serdeInfo.deserializerClass");
+    projectSpec.setFieldList(projectedFields);
+
+    response = client.getPartitionsWithSpecs(request);
+    Assert.assertEquals(1, response.getPartitionSpec().size());
+    partitionSpec = response.getPartitionSpec().get(0);
+    Assert.assertTrue("DbName is not set", partitionSpec.isSetDbName());
+    Assert.assertTrue("tableName is not set", partitionSpec.isSetTableName());
+    partitionSpecWithSharedSD = partitionSpec.getSharedSDPartitionSpec();
+
+    sharedSD = partitionSpecWithSharedSD.getSd();
+    Assert.assertNotNull(sharedSD);
+    partitionWithoutSDS = partitionSpecWithSharedSD.getPartitions();
+    Assert.assertNotNull(partitionWithoutSDS);
+    Assert.assertEquals(partitionWithoutSDS.size(), origPartitions.size());
+    comparePartitionForSingleValuedFields(projectedFields, sharedSD, partitionWithoutSDS, 0);
+  }
+
+  /**
+   * Confirms if the partitionWithoutSD object at partitionWithoutSDSIndex index has all the
+   * projected fields set to values which are same as the ones set in origPartitions
+   * @param projectedFields
+   * @param sharedSD
+   * @param partitionWithoutSDS
+   * @param partitionWithoutSDSIndex
+   * @throws IllegalAccessException
+   * @throws InvocationTargetException
+   * @throws NoSuchMethodException
+   */
+  private void comparePartitionForSingleValuedFields(List<String> projectedFields,
+      StorageDescriptor sharedSD, List<PartitionWithoutSD> partitionWithoutSDS, int partitionWithoutSDSIndex)
+      throws IllegalAccessException, InvocationTargetException, NoSuchMethodException {
+    for (Partition origPart : origPartitions) {
+      for (String projectField : projectedFields) {
+        // dbname, tableName and catName is not stored in partition
+        if (projectField.equals("dbName") || projectField.equals("tableName") || projectField
+            .equals("catName"))
+          continue;
+        if (projectField.startsWith("sd")) {
+          String sdPropertyName = projectField.substring(projectField.indexOf("sd.") + 3);
+          if (sdPropertyName.equals("location")) {
+            // in case of location sharedSD has the base location and partition has relative location
+            Assert.assertEquals("Location does not match", origPart.getSd().getLocation(),
+                sharedSD.getLocation() + partitionWithoutSDS.get(partitionWithoutSDSIndex).getRelativePath());
+          } else {
+            Assert.assertEquals(PropertyUtils.getNestedProperty(origPart, projectField),
+                PropertyUtils.getNestedProperty(sharedSD, sdPropertyName));
+          }
+        } else {
+          Assert.assertEquals(PropertyUtils.getNestedProperty(origPart, projectField),
+              PropertyUtils.getNestedProperty(partitionWithoutSDS.get(partitionWithoutSDSIndex), projectField));
+        }
+      }
+      partitionWithoutSDSIndex++;
+    }
+  }
+
+  @Test
+  public void testPartitionProjectionAllMultiValuedFields() throws Throwable {
+    GetPartitionsRequest request = getGetPartitionsRequest();
+    GetPartitionsProjectionSpec projectSpec = request.getProjectionSpec();
+    List<String> projectedFields = Arrays
+        .asList("values", "parameters", "sd.cols", "sd.bucketCols", "sd.sortCols", "sd.parameters",
+            "sd.skewedInfo", "sd.serdeInfo.parameters");
+    projectSpec.setFieldList(projectedFields);
+
+    GetPartitionsResponse response = client.getPartitionsWithSpecs(request);
+
+    Assert.assertEquals(1, response.getPartitionSpec().size());
+    PartitionSpec partitionSpec = response.getPartitionSpec().get(0);
+    PartitionSpecWithSharedSD partitionSpecWithSharedSD = partitionSpec.getSharedSDPartitionSpec();
+    Assert.assertEquals(origPartitions.size(), partitionSpecWithSharedSD.getPartitions().size());
+    StorageDescriptor sharedSD = partitionSpecWithSharedSD.getSd();
+    for (int i = 0; i < origPartitions.size(); i++) {
+      Partition origPartition = origPartitions.get(i);
+      PartitionWithoutSD retPartition = partitionSpecWithSharedSD.getPartitions().get(i);
+      for (String projectedField : projectedFields) {
+        switch (projectedField) {
+        case "values":
+          validateList(origPartition.getValues(), retPartition.getValues());
+          break;
+        case "parameters":
+          validateMap(origPartition.getParameters(), retPartition.getParameters());
+          break;
+        case "sd.cols":
+          validateList(origPartition.getSd().getCols(), sharedSD.getCols());
+          break;
+        case "sd.bucketCols":
+          validateList(origPartition.getSd().getBucketCols(), sharedSD.getBucketCols());
+          break;
+        case "sd.sortCols":
+          validateList(origPartition.getSd().getSortCols(), sharedSD.getSortCols());
+          break;
+        case "sd.parameters":
+          validateMap(origPartition.getSd().getParameters(), sharedSD.getParameters());
+          break;
+        case "sd.skewedInfo":
+          if (!origPartition.getSd().getSkewedInfo().getSkewedColNames().isEmpty()) {
+            validateList(origPartition.getSd().getSkewedInfo().getSkewedColNames(),
+                sharedSD.getSkewedInfo().getSkewedColNames());
+          }
+          if (!origPartition.getSd().getSkewedInfo().getSkewedColValues().isEmpty()) {
+            for (int i1 = 0;
+                 i1 < origPartition.getSd().getSkewedInfo().getSkewedColValuesSize(); i1++) {
+              validateList(origPartition.getSd().getSkewedInfo().getSkewedColValues().get(i1),
+                  sharedSD.getSkewedInfo().getSkewedColValues().get(i1));
+            }
+          }
+          if (!origPartition.getSd().getSkewedInfo().getSkewedColValueLocationMaps().isEmpty()) {
+            validateMap(origPartition.getSd().getSkewedInfo().getSkewedColValueLocationMaps(),
+                sharedSD.getSkewedInfo().getSkewedColValueLocationMaps());
+          }
+          break;
+        case "sd.serdeInfo.parameters":
+          validateMap(origPartition.getSd().getSerdeInfo().getParameters(),
+              sharedSD.getSerdeInfo().getParameters());
+          break;
+        default:
+          throw new IllegalArgumentException("Invalid field " + projectedField);
+        }
+      }
+    }
+  }
+
+  @Test
+  public void testPartitionProjectionIncludeParameters() throws Throwable {
+    GetPartitionsRequest request = getGetPartitionsRequest();
+    GetPartitionsProjectionSpec projectSpec = request.getProjectionSpec();
+    projectSpec
+        .setFieldList(Arrays.asList("dbName", "tableName", "catName", "parameters", "values"));
+    projectSpec.setIncludeParamKeyPattern(EXCLUDE_KEY_PREFIX + "%");
+
+    GetPartitionsResponse response = client.getPartitionsWithSpecs(request);
+
+    PartitionSpecWithSharedSD partitionSpecWithSharedSD =
+        response.getPartitionSpec().get(0).getSharedSDPartitionSpec();
+    Assert.assertNotNull("All the partitions should be returned in sharedSD spec",
+        partitionSpecWithSharedSD);
+    PartitionListComposingSpec partitionListComposingSpec =
+        response.getPartitionSpec().get(0).getPartitionList();
+    Assert.assertNull("Partition list composing spec should be null since all the "
+        + "partitions are expected to be in sharedSD spec", partitionListComposingSpec);
+    for (PartitionWithoutSD retPartion : partitionSpecWithSharedSD.getPartitions()) {
+      Assert.assertTrue("included parameter key is not found in the response",
+          retPartion.getParameters().containsKey(EXCLUDE_KEY_PREFIX + "key1"));
+      Assert.assertTrue("included parameter key is not found in the response",
+          retPartion.getParameters().containsKey(EXCLUDE_KEY_PREFIX + "key2"));
+      Assert.assertEquals("Additional parameters returned other than inclusion keys",
+          2, retPartion.getParameters().size());
+    }
+  }
+
+  @Test
+  public void testPartitionProjectionIncludeExcludeParameters() throws Throwable {
+    GetPartitionsRequest request = getGetPartitionsRequest();
+    GetPartitionsProjectionSpec projectSpec = request.getProjectionSpec();
+    projectSpec
+        .setFieldList(Arrays.asList("dbName", "tableName", "catName", "parameters", "values"));
+    // test parameter key inclusion using setIncludeParamKeyPattern
+    projectSpec.setIncludeParamKeyPattern(EXCLUDE_KEY_PREFIX + "%");
+    projectSpec.setExcludeParamKeyPattern("%key1%");
+
+    GetPartitionsResponse response = client.getPartitionsWithSpecs(request);
+
+    PartitionSpecWithSharedSD partitionSpecWithSharedSD =
+        response.getPartitionSpec().get(0).getSharedSDPartitionSpec();
+    Assert.assertNotNull("All the partitions should be returned in sharedSD spec",
+        partitionSpecWithSharedSD);
+    PartitionListComposingSpec partitionListComposingSpec =
+        response.getPartitionSpec().get(0).getPartitionList();
+    Assert.assertNull("Partition list composing spec should be null since all the "
+        + "partitions are expected to be in sharedSD spec", partitionListComposingSpec);
+    for (PartitionWithoutSD retPartion : partitionSpecWithSharedSD.getPartitions()) {
+      Assert.assertFalse("excluded parameter key is found in the response",
+          retPartion.getParameters().containsKey(EXCLUDE_KEY_PREFIX + "key1"));
+      Assert.assertTrue("included parameter key is not found in the response",
+          retPartion.getParameters().containsKey(EXCLUDE_KEY_PREFIX + "key2"));
+      Assert.assertEquals("Additional parameters returned other than inclusion keys",
+          1, retPartion.getParameters().size());
+    }
+  }
+
+  @Test
+  public void testPartitionProjectionExcludeParameters() throws Throwable {
+    GetPartitionsRequest request = getGetPartitionsRequest();
+    GetPartitionsProjectionSpec projectSpec = request.getProjectionSpec();
+    projectSpec
+        .setFieldList(Arrays.asList("dbName", "tableName", "catName", "parameters", "values"));
+    projectSpec.setExcludeParamKeyPattern(EXCLUDE_KEY_PREFIX + "%");
+
+    GetPartitionsResponse response = client.getPartitionsWithSpecs(request);
+
+    PartitionSpecWithSharedSD partitionSpecWithSharedSD =
+        response.getPartitionSpec().get(0).getSharedSDPartitionSpec();
+    Assert.assertNotNull("All the partitions should be returned in sharedSD spec",
+        partitionSpecWithSharedSD);
+    PartitionListComposingSpec partitionListComposingSpec =
+        response.getPartitionSpec().get(0).getPartitionList();
+    Assert.assertNull("Partition list composing spec should be null", partitionListComposingSpec);
+    for (PartitionWithoutSD retPartion : partitionSpecWithSharedSD.getPartitions()) {
+      Assert.assertFalse("excluded parameter key is found in the response",
+          retPartion.getParameters().containsKey(EXCLUDE_KEY_PREFIX + "key1"));
+      Assert.assertFalse("excluded parameter key is found in the response",
+          retPartion.getParameters().containsKey(EXCLUDE_KEY_PREFIX + "key2"));
+    }
+  }
+
+  @Test
+  public void testNestedMultiValuedFieldProjection() throws TException {
+    GetPartitionsRequest request = getGetPartitionsRequest();
+    GetPartitionsProjectionSpec projectSpec = request.getProjectionSpec();
+    projectSpec.setFieldList(Arrays.asList("sd.cols.name", "sd.cols.type"));
+
+    GetPartitionsResponse response = client.getPartitionsWithSpecs(request);
+
+    PartitionSpecWithSharedSD partitionSpecWithSharedSD =
+        response.getPartitionSpec().get(0).getSharedSDPartitionSpec();
+    StorageDescriptor sharedSD = partitionSpecWithSharedSD.getSd();
+    Assert.assertNotNull("sd.cols were requested but was not returned", sharedSD.getCols());
+    for (FieldSchema col : sharedSD.getCols()) {
+      Assert.assertTrue("sd.cols.name was requested but was not returned", col.isSetName());
+      Assert.assertTrue("sd.cols.type was requested but was not returned", col.isSetType());
+      Assert.assertFalse("sd.cols.comment was not requested but was returned", col.isSetComment());
+    }
+  }
+
+  @Test
+  public void testParameterExpansion() throws TException {
+    GetPartitionsRequest request = getGetPartitionsRequest();
+    GetPartitionsProjectionSpec projectSpec = request.getProjectionSpec();
+    projectSpec.setFieldList(Arrays.asList("sd.cols", "sd.serdeInfo"));
+
+    GetPartitionsResponse response = client.getPartitionsWithSpecs(request);
+
+    PartitionSpecWithSharedSD partitionSpecWithSharedSD =
+        response.getPartitionSpec().get(0).getSharedSDPartitionSpec();
+    StorageDescriptor sharedSD = partitionSpecWithSharedSD.getSd();
+    Assert.assertNotNull("sd.cols were requested but was not returned", sharedSD.getCols());
+    Assert.assertEquals("Returned serdeInfo does not match with original serdeInfo",
+        origPartitions.get(0).getSd().getCols(), sharedSD.getCols());
+
+    Assert
+        .assertNotNull("sd.serdeInfo were requested but was not returned", sharedSD.getSerdeInfo());
+    Assert.assertEquals("Returned serdeInfo does not match with original serdeInfo",
+        origPartitions.get(0).getSd().getSerdeInfo(), sharedSD.getSerdeInfo());
+  }
+
+  @Test
+  public void testNonStandardPartitions() throws TException {
+    String testTblName = "test_non_standard";
+    new TableBuilder()
+        .setTableName(testTblName)
+        .setDbName(dbName)
+        .addCol("ns_c1", "string", "comment 1")
+        .addCol("ns_c2", "int", "comment 2")
+        .addPartCol("part", "string")
+        .addPartCol("city", "string")
+        .addBucketCol("ns_c1")
+        .addSortCol("ns_c2", 1)
+        .addTableParam("tblparamKey", "Partitions of this table are not located within table directory")
+        .create(client, conf);
+
+    Table table = client.getTable(dbName, testTblName);
+    Assert.assertNotNull("Unable to create a test table ", table);
+
+    List<Partition> partitions = new ArrayList<>();
+    partitions.add(createPartition(Arrays.asList("p1", "SanFrancisco"), table));
+    partitions.add(createPartition(Arrays.asList("p1", "PaloAlto"), table));
+    partitions.add(createPartition(Arrays.asList("p2", "Seattle"), table));
+    partitions.add(createPartition(Arrays.asList("p2", "Phoenix"), table));
+
+    client.add_partitions(partitions);
+    // change locations of two of the partitions outside table directory
+    List<Partition> testPartitions = client.listPartitions(dbName, testTblName, (short) -1);
+    Assert.assertEquals(4, testPartitions.size());
+    Partition p1 = testPartitions.get(2);
+    p1.getSd().setLocation("/tmp/some_other_location/part=p2/city=Seattle");
+    Partition p2 = testPartitions.get(3);
+    p2.getSd().setLocation("/tmp/some_other_location/part=p2/city=Phoenix");
+    client.alter_partitions(dbName, testTblName, Arrays.asList(p1, p2));
+
+    GetPartitionsRequest request = getGetPartitionsRequest();
+    request.getProjectionSpec().setFieldList(Arrays.asList("values", "sd"));
+    request.setDbName(dbName);
+    request.setTblName(testTblName);
+
+    GetPartitionsResponse response = client.getPartitionsWithSpecs(request);
+    Assert.assertNotNull("Response should have returned partition specs",
+        response.getPartitionSpec());
+    Assert
+        .assertEquals("We should have two partition specs", 2, response.getPartitionSpec().size());
+    Assert.assertNotNull("One SharedSD spec is expected",
+        response.getPartitionSpec().get(0).getSharedSDPartitionSpec());
+    Assert.assertNotNull("One composing spec is expected",
+        response.getPartitionSpec().get(1).getPartitionList());
+
+    PartitionSpecWithSharedSD partitionSpecWithSharedSD =
+        response.getPartitionSpec().get(0).getSharedSDPartitionSpec();
+    Assert.assertNotNull("sd was requested but not returned", partitionSpecWithSharedSD.getSd());
+    Assert.assertEquals("shared SD should have table location", table.getSd().getLocation(),
+        partitionSpecWithSharedSD.getSd().getLocation());
+    List<List<String>> expectedVals = new ArrayList<>(2);
+    expectedVals.add(Arrays.asList("p1", "PaloAlto"));
+    expectedVals.add(Arrays.asList("p1", "SanFrancisco"));
+
+    for (int i=0; i<partitionSpecWithSharedSD.getPartitions().size(); i++) {
+      PartitionWithoutSD retPartition = partitionSpecWithSharedSD.getPartitions().get(i);
+      Assert.assertEquals(2, retPartition.getValuesSize());
+      validateList(expectedVals.get(i), retPartition.getValues());
+      Assert.assertNull("parameters were not requested so should have been null",
+          retPartition.getParameters());
+    }
+
+    PartitionListComposingSpec composingSpec =
+        response.getPartitionSpec().get(1).getPartitionList();
+    Assert.assertNotNull("composing spec should have returned 2 partitions",
+        composingSpec.getPartitions());
+    Assert.assertEquals("composing spec should have returned 2 partitions", 2,
+        composingSpec.getPartitionsSize());
+
+    expectedVals.clear();
+    expectedVals.add(Arrays.asList("p2", "Phoenix"));
+    expectedVals.add(Arrays.asList("p2", "Seattle"));
+    for (int i=0; i<composingSpec.getPartitions().size(); i++) {
+      Partition partition = composingSpec.getPartitions().get(i);
+      Assert.assertEquals(2, partition.getValuesSize());
+      validateList(expectedVals.get(i), partition.getValues());
+      Assert.assertNull("parameters were not requested so should have been null",
+          partition.getParameters());
+    }
+  }
+
+  @Test(expected = TException.class)
+  public void testInvalidProjectFieldNames() throws TException {
+    GetPartitionsRequest request = getGetPartitionsRequest();
+    GetPartitionsProjectionSpec projectSpec = request.getProjectionSpec();
+    projectSpec.setFieldList(Arrays.asList("values", "invalid.field.name"));
+    client.getPartitionsWithSpecs(request);
+  }
+
+  @Test(expected = TException.class)
+  public void testInvalidProjectFieldNames2() throws TException {
+    GetPartitionsRequest request = getGetPartitionsRequest();
+    GetPartitionsProjectionSpec projectSpec = request.getProjectionSpec();
+    projectSpec.setFieldList(Arrays.asList(""));
+    client.getPartitionsWithSpecs(request);
+  }
+
+  private void validateBasic(GetPartitionsResponse response) throws TException {
+    Assert.assertNotNull("Response is null", response);
+    Assert.assertNotNull("Returned partition spec is null", response.getPartitionSpec());
+    Assert.assertEquals(1, response.getPartitionSpecSize());
+    PartitionSpecWithSharedSD partitionSpecWithSharedSD =
+        response.getPartitionSpec().get(0).getSharedSDPartitionSpec();
+    Assert.assertNotNull(partitionSpecWithSharedSD.getSd());
+    StorageDescriptor sharedSD = partitionSpecWithSharedSD.getSd();
+    Assert.assertEquals("Root location should be set to table location", tbl.getSd().getLocation(),
+        sharedSD.getLocation());
+
+    List<PartitionWithoutSD> partitionWithoutSDS = partitionSpecWithSharedSD.getPartitions();
+    Assert.assertEquals(origPartitions.size(), partitionWithoutSDS.size());
+    for (int i = 0; i < origPartitions.size(); i++) {
+      Partition origPartition = origPartitions.get(i);
+      PartitionWithoutSD returnedPartitionWithoutSD = partitionWithoutSDS.get(i);
+      Assert.assertEquals(String.format("Location returned for Partition %d is not correct", i),
+          origPartition.getSd().getLocation(),
+          sharedSD.getLocation() + returnedPartitionWithoutSD.getRelativePath());
+    }
+  }
+
+  private GetPartitionsRequest getGetPartitionsRequest() {
+    GetPartitionsRequest request = new GetPartitionsRequest();
+    request.setProjectionSpec(new GetPartitionsProjectionSpec());
+    request.setFilterSpec(new GetPartitionsFilterSpec());
+    request.setTblName(tblName);
+    request.setDbName(dbName);
+    return request;
+  }
+
+  private <K, V> void validateMap(Map<K, V> aMap, Map<K, V> bMap) {
+    if ((aMap == null || aMap.isEmpty()) && (bMap == null || bMap.isEmpty())) {
+      return;
+    }
+    // Equality is verified here because metastore updates stats automatically
+    // and adds them in the returned partition. So the returned partition will
+    // have parameters + some more parameters for the basic stats
+    Assert.assertTrue(bMap.size() >= aMap.size());
+    for (Entry<K, V> entries : aMap.entrySet()) {
+      Assert.assertTrue("Expected " + entries.getKey() + " is missing from the map",
+          bMap.containsKey(entries.getKey()));
+      Assert.assertEquals("Expected value to be " + aMap.get(entries.getKey()) + " found" + bMap
+          .get(entries.getKey()), aMap.get(entries.getKey()), bMap.get(entries.getKey()));
+    }
+  }
+
+  private <T> void validateList(List<T> aList, List<T> bList) {
+    if ((aList == null || aList.isEmpty()) && (bList == null || bList.isEmpty())) {
+      return;
+    }
+    Assert.assertEquals(aList.size(), bList.size());
+    Iterator<T> origValuesIt = aList.iterator();
+    Iterator<T> retValuesIt = bList.iterator();
+    while (origValuesIt.hasNext()) {
+      Assert.assertTrue(retValuesIt.hasNext());
+      Assert.assertEquals(origValuesIt.next(), retValuesIt.next());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/44ef91a6/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
index d6f0d8c..7429d18 100644
--- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
+++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hive.metastore;
 
 import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
 import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.PreparedStatement;
@@ -28,9 +29,11 @@ import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.Set;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
@@ -41,6 +44,13 @@ import java.lang.reflect.*;
 import static org.mockito.Mockito.mock;
 
 import com.google.common.collect.Sets;
+import org.apache.hadoop.hive.metastore.api.GetPartitionsFilterSpec;
+import org.apache.hadoop.hive.metastore.api.GetPartitionsProjectionSpec;
+import org.apache.hadoop.hive.metastore.api.GetPartitionsRequest;
+import org.apache.hadoop.hive.metastore.api.GetPartitionsResponse;
+import org.apache.hadoop.hive.metastore.api.PartitionSpec;
+import org.apache.hadoop.hive.metastore.api.PartitionSpecWithSharedSD;
+import org.apache.hadoop.hive.metastore.api.PartitionWithoutSD;
 import org.apache.hadoop.hive.metastore.client.builder.DatabaseBuilder;
 import org.apache.hadoop.hive.metastore.client.builder.TableBuilder;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
@@ -125,6 +135,7 @@ public abstract class TestHiveMetaStore {
     conf.set("hive.key3", "");
     conf.set("hive.key4", "0");
     conf.set("datanucleus.autoCreateTables", "false");
+    conf.set("hive.in.test", "true");
 
     MetaStoreTestUtils.setConfForStandloneMode(conf);
     MetastoreConf.setLongVar(conf, ConfVars.BATCH_RETRIEVE_MAX, 2);
@@ -471,7 +482,6 @@ public abstract class TestHiveMetaStore {
 
   private static List<String> makeVals(String ds, String id) {
     List <String> vals4 = new ArrayList<>(2);
-    vals4 = new ArrayList<>(2);
     vals4.add(ds);
     vals4.add(id);
     return vals4;
@@ -666,6 +676,126 @@ public abstract class TestHiveMetaStore {
 
   }
 
+  @Test
+  public void testGetPartitionsWithSpec() throws Throwable {
+    // create a table with multiple partitions
+    List<Partition> createdPartitions = setupProjectionTestTable();
+    Table tbl = client.getTable("compdb", "comptbl");
+    GetPartitionsRequest request = new GetPartitionsRequest();
+    GetPartitionsProjectionSpec projectSpec = new GetPartitionsProjectionSpec();
+    projectSpec.setFieldList(Arrays
+        .asList("dbName", "tableName", "catName", "parameters", "lastAccessTime", "sd.location",
+            "values", "createTime", "sd.serdeInfo.serializationLib", "sd.cols"));
+    projectSpec.setExcludeParamKeyPattern("exclude%");
+    GetPartitionsFilterSpec filter = new GetPartitionsFilterSpec();
+    request.setDbName("compdb");
+    request.setTblName("comptbl");
+    request.setFilterSpec(filter);
+    request.setProjectionSpec(projectSpec);
+    GetPartitionsResponse response;
+    try {
+      response = client.getPartitionsWithSpecs(request);
+    } catch (Exception ex) {
+      ex.printStackTrace();
+      LOG.error("Exception while retriveing partitions", ex);
+      throw ex;
+    }
+
+    Assert.assertEquals(1, response.getPartitionSpecSize());
+    PartitionSpecWithSharedSD partitionSpecWithSharedSD =
+        response.getPartitionSpec().get(0).getSharedSDPartitionSpec();
+    Assert.assertNotNull(partitionSpecWithSharedSD.getSd());
+    StorageDescriptor sharedSD = partitionSpecWithSharedSD.getSd();
+    Assert.assertEquals("Root location should be set to table location", tbl.getSd().getLocation(),
+        sharedSD.getLocation());
+    Assert.assertFalse("Fields which are not requested should not be set",
+        sharedSD.isSetParameters());
+    Assert.assertNotNull(
+        "serializationLib class was requested but was not found in the returned partition",
+        sharedSD.getSerdeInfo().getSerializationLib());
+    Assert.assertNotNull("db name was requested but was not found in the returned partition",
+        response.getPartitionSpec().get(0).getDbName());
+    Assert.assertNotNull("Table name was requested but was not found in the returned partition",
+        response.getPartitionSpec().get(0).getTableName());
+    Assert.assertTrue("sd.cols was requested but was not found in the returned response",
+        partitionSpecWithSharedSD.getSd().isSetCols());
+    List<FieldSchema> origSdCols = createdPartitions.get(0).getSd().getCols();
+    Assert.assertEquals("Size of the requested sd.cols should be same", origSdCols.size(),
+        partitionSpecWithSharedSD.getSd().getCols().size());
+    for (int i = 0; i < origSdCols.size(); i++) {
+      FieldSchema origFs = origSdCols.get(i);
+      FieldSchema returnedFs = partitionSpecWithSharedSD.getSd().getCols().get(i);
+      Assert.assertEquals("Field schemas returned different than expected", origFs, returnedFs);
+    }
+    /*Assert
+        .assertNotNull("Catalog name was requested but was not found in the returned partition",
+            response.getPartitionSpec().get(0).getCatName());*/
+
+    List<PartitionWithoutSD> partitionWithoutSDS = partitionSpecWithSharedSD.getPartitions();
+    Assert.assertEquals(createdPartitions.size(), partitionWithoutSDS.size());
+    for (int i = 0; i < createdPartitions.size(); i++) {
+      Partition origPartition = createdPartitions.get(i);
+      PartitionWithoutSD returnedPartitionWithoutSD = partitionWithoutSDS.get(i);
+      Assert.assertEquals(String.format("Location returned for Partition %d is not correct", i),
+          origPartition.getSd().getLocation(),
+          sharedSD.getLocation() + returnedPartitionWithoutSD.getRelativePath());
+      Assert.assertTrue("createTime was request but is not set",
+          returnedPartitionWithoutSD.isSetCreateTime());
+      Assert.assertTrue("Partition parameters were requested but are not set",
+          returnedPartitionWithoutSD.isSetParameters());
+      // first partition has parameters set
+      if (i == 0) {
+        Assert.assertTrue("partition parameters not set",
+            returnedPartitionWithoutSD.getParameters().containsKey("key1"));
+        Assert.assertEquals("partition parameters does not contain included keys", "val1",
+            returnedPartitionWithoutSD.getParameters().get("key1"));
+        // excluded parameter should not be returned
+        Assert.assertFalse("Excluded parameter key returned",
+            returnedPartitionWithoutSD.getParameters().containsKey("excludeKey1"));
+        Assert.assertFalse("Excluded parameter key returned",
+            returnedPartitionWithoutSD.getParameters().containsKey("excludeKey2"));
+      }
+      List<String> returnedVals = returnedPartitionWithoutSD.getValues();
+      List<String> actualVals = origPartition.getValues();
+      for (int j = 0; j < actualVals.size(); j++) {
+        Assert.assertEquals(actualVals.get(j), returnedVals.get(j));
+      }
+    }
+  }
+
+
+  protected List<Partition> setupProjectionTestTable() throws Throwable {
+    //String catName = "catName";
+    String dbName = "compdb";
+    String tblName = "comptbl";
+    String typeName = "Person";
+    //String catName = "catName";
+    Map<String, String> dummyparams = new HashMap<>();
+    dummyparams.put("key1", "val1");
+    dummyparams.put("excludeKey1", "excludeVal1");
+    dummyparams.put("excludeKey2", "excludeVal2");
+    cleanUp(dbName, tblName, typeName);
+
+    List<List<String>> values = new ArrayList<>();
+    values.add(makeVals("2008-07-01 14:13:12", "14"));
+    values.add(makeVals("2008-07-01 14:13:12", "15"));
+    values.add(makeVals("2008-07-02 14:13:12", "15"));
+    values.add(makeVals("2008-07-03 14:13:12", "151"));
+
+    List<Partition> createdPartitions =
+        createMultiPartitionTableSchema(dbName, tblName, typeName, values);
+    Table tbl = client.getTable(dbName, tblName);
+    // add some dummy parameters to one of the partitions to confirm the fetching logic is working
+    Partition newPartition = createdPartitions.remove(0);
+    //Map<String, String> sdParams = new HashMap<>();
+    //dummyparams.put("sdkey1", "sdval1");
+    newPartition.setParameters(dummyparams);
+    //newPartition.getSd().setParameters(sdParams);
+
+    client.alter_partition(dbName, tblName, newPartition);
+    createdPartitions.add(0, newPartition);
+    return createdPartitions;
+  }
 
   @Test
   public void testDropTable() throws Throwable {
@@ -2802,7 +2932,11 @@ public abstract class TestHiveMetaStore {
     return partitions;
   }
 
-  private void createMultiPartitionTableSchema(String dbName, String tblName,
+  private List<Partition> createMultiPartitionTableSchema(String dbName, String tblName,
+      String typeName, List<List<String>> values) throws Throwable {
+    return createMultiPartitionTableSchema(null, dbName, tblName, typeName, values);
+  }
+  private List<Partition> createMultiPartitionTableSchema(String catName, String dbName, String tblName,
       String typeName, List<List<String>> values) throws Throwable {
     createDb(dbName);
 
@@ -2813,6 +2947,7 @@ public abstract class TestHiveMetaStore {
     Table tbl = new TableBuilder()
         .setDbName(dbName)
         .setTableName(tblName)
+        .setCatName(catName)
         .addCol("name", ColumnType.STRING_TYPE_NAME)
         .addCol("income", ColumnType.INT_TYPE_NAME)
         .addPartCol("ds", ColumnType.STRING_TYPE_NAME)
@@ -2827,7 +2962,7 @@ public abstract class TestHiveMetaStore {
       tbl = client.getTable(dbName, tblName);
     }
 
-    createPartitions(dbName, tbl, values);
+    return createPartitions(dbName, tbl, values);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hive/blob/44ef91a6/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestPartitionProjectionEvaluator.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestPartitionProjectionEvaluator.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestPartitionProjectionEvaluator.java
new file mode 100644
index 0000000..663ae81
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestPartitionProjectionEvaluator.java
@@ -0,0 +1,250 @@
+/*
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.hive.metastore;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.hadoop.hive.metastore.PartitionProjectionEvaluator.PartitionFieldNode;
+import org.apache.hadoop.hive.metastore.annotation.MetastoreUnitTest;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
+
+import javax.jdo.PersistenceManager;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.hive.metastore.PartitionProjectionEvaluator.CD_PATTERN;
+import static org.apache.hadoop.hive.metastore.PartitionProjectionEvaluator.SD_PATTERN;
+import static org.apache.hadoop.hive.metastore.PartitionProjectionEvaluator.SERDE_PATTERN;
+
+@Category(MetastoreUnitTest.class)
+public class TestPartitionProjectionEvaluator {
+  private ImmutableMap<String, String> fieldNameToColumnName =
+      ImmutableMap.<String, String>builder()
+          .put("createTime", "\"PARTITIONS\"" + ".\"CREATE_TIME\"")
+          .put("lastAccessTime", "\"PARTITIONS\"" + ".\"LAST_ACCESS_TIME\"")
+          .put("sd.location", "\"SDS\"" + ".\"LOCATION\"")
+          .put("sd.inputFormat", "\"SDS\"" + ".\"INPUT_FORMAT\"")
+          .put("sd.outputFormat", "\"SDS\"" + ".\"OUTPUT_FORMAT\"")
+          .put("sd.storedAsSubDirectories", "\"SDS\"" + ".\"IS_STOREDASSUBDIRECTORIES\"")
+          .put("sd.compressed", "\"SDS\"" + ".\"IS_COMPRESSED\"")
+          .put("sd.numBuckets", "\"SDS\"" + ".\"NUM_BUCKETS\"")
+          .put("sd.serdeInfo.name", "\"SDS\"" + ".\"NAME\"")
+          .put("sd.serdeInfo.serializationLib", "\"SDS\"" + ".\"SLIB\"")
+          .put("PART_ID", "\"PARTITIONS\"" + ".\"PART_ID\"").put("SD_ID", "\"SDS\"" + ".\"SD_ID\"")
+          .put("SERDE_ID", "\"SERDES\"" + ".\"SERDE_ID\"").put("CD_ID", "\"SDS\"" + ".\"CD_ID\"")
+          .build();
+
+  private static void compareTreeUtil(PartitionFieldNode expected, PartitionFieldNode given) {
+    if (expected == null || given == null) {
+      Assert.assertTrue(expected == null && given == null);
+    }
+    Assert.assertEquals("Field names should match", expected.getFieldName(), given.getFieldName());
+    Assert.assertEquals(
+        "IsLeafNode: Expected " + expected + " " + expected.isLeafNode() + " Given " + given + " " + given
+            .isLeafNode(), expected.isLeafNode(), given.isLeafNode());
+    Assert.assertEquals(
+        "IsMultivalued: Expected " + expected + " " + expected.isMultiValued() + " Given " + given + " " + given
+            .isMultiValued(), expected.isMultiValued(), given.isMultiValued());
+    for (PartitionFieldNode child : expected.getChildren()) {
+      Assert.assertTrue("given node " + given + " does not have the child node " + child,
+          given.getChildren().contains(child));
+      int counter = 0;
+      for (PartitionFieldNode giveChild : given.getChildren()) {
+        if (child.equals(giveChild)) {
+          compareTreeUtil(child, giveChild);
+          counter++;
+        }
+      }
+      Assert.assertEquals("More than one copies of node " + child + " found", 1, counter);
+    }
+  }
+
+  private static void compare(Set<PartitionFieldNode> roots, Set<PartitionFieldNode> giveRoots) {
+    Assert.assertEquals("Given roots size does not match with the size of expected number of roots",
+        roots.size(), giveRoots.size());
+    for (PartitionFieldNode root : roots) {
+      Assert.assertTrue(giveRoots.contains(root));
+      int counter = 0;
+      for (PartitionFieldNode givenRoot : giveRoots) {
+        if (givenRoot.equals(root)) {
+          compareTreeUtil(root, givenRoot);
+          counter++;
+        }
+      }
+      Assert.assertEquals("More than one copies of node found for " + root, 1, counter);
+    }
+  }
+
+  @Test
+  public void testPartitionFieldTree() throws MetaException {
+    PersistenceManager mockPm = Mockito.mock(PersistenceManager.class);
+    List<String> projectionFields = new ArrayList<>(2);
+    projectionFields.add("sd.location");
+    projectionFields.add("sd.parameters");
+    projectionFields.add("createTime");
+    projectionFields.add("sd.serdeInfo.serializationLib");
+    projectionFields.add("sd.cols");
+    projectionFields.add("parameters");
+    PartitionProjectionEvaluator projectionEvaluator =
+        new PartitionProjectionEvaluator(mockPm, fieldNameToColumnName, projectionFields, false,
+            false, null, null);
+    Set<PartitionFieldNode> roots = projectionEvaluator.getRoots();
+
+    Set<PartitionFieldNode> expected = new HashSet<>();
+    PartitionFieldNode sdNode = new PartitionFieldNode("sd");
+    sdNode.addChild(new PartitionFieldNode("sd.location"));
+    sdNode.addChild(new PartitionFieldNode("sd.parameters", true));
+    PartitionFieldNode sdColsNodes = new PartitionFieldNode("sd.cols", true);
+    sdColsNodes.addChild(new PartitionFieldNode("sd.cols.name", true));
+    sdColsNodes.addChild(new PartitionFieldNode("sd.cols.type", true));
+    sdColsNodes.addChild(new PartitionFieldNode("sd.cols.comment", true));
+    sdNode.addChild(sdColsNodes);
+
+    PartitionFieldNode serdeNode = new PartitionFieldNode("sd.serdeInfo");
+    serdeNode.addChild(new PartitionFieldNode("sd.serdeInfo.serializationLib"));
+
+    sdNode.addChild(serdeNode);
+    expected.add(sdNode);
+    expected.add(new PartitionFieldNode("parameters", true));
+    expected.add(new PartitionFieldNode("createTime"));
+    expected.add(new PartitionFieldNode("PART_ID"));
+    expected.add(new PartitionFieldNode("SD_ID"));
+    expected.add(new PartitionFieldNode("CD_ID"));
+    expected.add(new PartitionFieldNode("SERDE_ID"));
+    compare(expected, roots);
+  }
+
+  @Test
+  public void testProjectionCompaction() throws MetaException {
+    PersistenceManager mockPm = Mockito.mock(PersistenceManager.class);
+    List<String> projectionFields = new ArrayList<>(2);
+    projectionFields.add("sd.location");
+    projectionFields.add("sd.parameters");
+    projectionFields.add("createTime");
+    projectionFields.add("sd");
+    PartitionProjectionEvaluator projectionEvaluator =
+        new PartitionProjectionEvaluator(mockPm, fieldNameToColumnName, projectionFields, false,
+            false, null, null);
+    Set<PartitionFieldNode> roots = projectionEvaluator.getRoots();
+    Assert.assertFalse("sd.location should not contained since it is already included in sd",
+        roots.contains(new PartitionFieldNode("sd.location")));
+    Assert.assertFalse("sd.parameters should not contained since it is already included in sd",
+        roots.contains(new PartitionFieldNode("sd.parameters")));
+  }
+
+  @Test(expected = MetaException.class)
+  public void testInvalidProjectFields() throws MetaException {
+    PersistenceManager mockPm = Mockito.mock(PersistenceManager.class);
+    List<String> projectionFields = new ArrayList<>(2);
+    projectionFields.add("sd.location");
+    projectionFields.add("sd.parameters");
+    projectionFields.add("createTime");
+    projectionFields.add("sd");
+    projectionFields.add("invalid");
+    new PartitionProjectionEvaluator(mockPm, fieldNameToColumnName, projectionFields, false, false,
+        null, null);
+  }
+
+  @Test
+  public void testFind() throws MetaException {
+    PersistenceManager mockPm = Mockito.mock(PersistenceManager.class);
+    List<String> projectionFields = Arrays.asList("sd", "createTime", "sd.location", "parameters");
+    PartitionProjectionEvaluator projectionEvaluator =
+        new PartitionProjectionEvaluator(mockPm, fieldNameToColumnName, projectionFields, false,
+            false, null, null);
+    Assert.assertTrue(projectionEvaluator.find(SD_PATTERN));
+
+    projectionFields = Arrays.asList("sd", "createTime", "parameters");
+    projectionEvaluator =
+        new PartitionProjectionEvaluator(mockPm, fieldNameToColumnName, projectionFields, false,
+            false, null, null);
+    Assert.assertTrue(projectionEvaluator.find(SD_PATTERN));
+
+    projectionFields = Arrays.asList("createTime", "parameters", "sd.serdeInfo.serializationLib");
+    projectionEvaluator =
+        new PartitionProjectionEvaluator(mockPm, fieldNameToColumnName, projectionFields, false,
+            false, null, null);
+    Assert.assertTrue(projectionEvaluator.find(SD_PATTERN));
+
+    projectionFields = Arrays.asList("createTime", "parameters", "sd.location");
+    projectionEvaluator =
+        new PartitionProjectionEvaluator(mockPm, fieldNameToColumnName, projectionFields, false,
+            false, null, null);
+    Assert.assertTrue(projectionEvaluator.find(SD_PATTERN));
+
+    projectionFields = Arrays.asList("createTime", "parameters", "sd.location");
+    projectionEvaluator =
+        new PartitionProjectionEvaluator(mockPm, fieldNameToColumnName, projectionFields, false,
+            false, null, null);
+    Assert.assertFalse(projectionEvaluator.find(SERDE_PATTERN));
+
+    projectionFields = Arrays.asList("createTime", "parameters", "sd.serdeInfo.serializationLib");
+    projectionEvaluator =
+        new PartitionProjectionEvaluator(mockPm, fieldNameToColumnName, projectionFields, false,
+            false, null, null);
+    Assert.assertTrue(projectionEvaluator.find(SERDE_PATTERN));
+
+    projectionFields = Arrays.asList("createTime", "parameters", "sd.serdeInfo");
+    projectionEvaluator =
+        new PartitionProjectionEvaluator(mockPm, fieldNameToColumnName, projectionFields, false,
+            false, null, null);
+    Assert.assertTrue(projectionEvaluator.find(SERDE_PATTERN));
+
+    projectionFields = Arrays.asList("createTime", "parameters");
+    projectionEvaluator =
+        new PartitionProjectionEvaluator(mockPm, fieldNameToColumnName, projectionFields, false,
+            false, null, null);
+    Assert.assertFalse(projectionEvaluator.find(SD_PATTERN));
+
+    projectionFields = Arrays.asList("createTime", "parameters", "sd.cols");
+    projectionEvaluator =
+        new PartitionProjectionEvaluator(mockPm, fieldNameToColumnName, projectionFields, false,
+            false, null, null);
+    Assert.assertTrue(projectionEvaluator.find(CD_PATTERN));
+
+    projectionFields = Arrays.asList("createTime", "parameters", "sd.cols.name");
+    projectionEvaluator =
+        new PartitionProjectionEvaluator(mockPm, fieldNameToColumnName, projectionFields, false,
+            false, null, null);
+    Assert.assertTrue(projectionEvaluator.find(CD_PATTERN));
+
+    projectionFields = Arrays.asList("createTime", "parameters", "sd", "sd.location");
+    projectionEvaluator =
+        new PartitionProjectionEvaluator(mockPm, fieldNameToColumnName, projectionFields, false,
+            false, null, null);
+    // CD_PATTERN should exist since sd gets expanded to all the child nodes
+    Assert.assertTrue(projectionEvaluator.find(CD_PATTERN));
+  }
+
+  @Test(expected = MetaException.class)
+  public void testFindNegative() throws MetaException {
+    PersistenceManager mockPm = Mockito.mock(PersistenceManager.class);
+    List<String> projectionFields = projectionFields = Arrays.asList("createTime", "parameters", "sdxcols");
+    PartitionProjectionEvaluator projectionEvaluator =
+        new PartitionProjectionEvaluator(mockPm, fieldNameToColumnName, projectionFields, false,
+            false, null, null);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/44ef91a6/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/utils/TestMetaStoreServerUtils.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/utils/TestMetaStoreServerUtils.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/utils/TestMetaStoreServerUtils.java
index b05cb54..f4bdd73 100644
--- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/utils/TestMetaStoreServerUtils.java
+++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/utils/TestMetaStoreServerUtils.java
@@ -29,12 +29,19 @@ import org.apache.hadoop.hive.metastore.annotation.MetastoreUnitTest;
 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.Table;
+import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.PartitionSpec;
+import org.apache.hadoop.hive.metastore.api.PartitionSpecWithSharedSD;
+import org.apache.hadoop.hive.metastore.api.PartitionWithoutSD;
+import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
+import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.client.builder.DatabaseBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.PartitionBuilder;
 import org.apache.hadoop.hive.metastore.client.builder.TableBuilder;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.thrift.TException;
+import org.hamcrest.core.IsNot;
 import org.junit.Assert;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -50,11 +57,12 @@ import java.util.stream.Collectors;
 import static java.util.regex.Pattern.compile;
 import static org.apache.hadoop.hive.common.StatsSetupConst.COLUMN_STATS_ACCURATE;
 import static org.apache.hadoop.hive.common.StatsSetupConst.FAST_STATS;
-import static org.apache.hadoop.hive.common.StatsSetupConst.NUM_FILES;
 import static org.apache.hadoop.hive.common.StatsSetupConst.NUM_ERASURE_CODED_FILES;
+import static org.apache.hadoop.hive.common.StatsSetupConst.NUM_FILES;
 import static org.apache.hadoop.hive.common.StatsSetupConst.STATS_GENERATED;
 import static org.apache.hadoop.hive.common.StatsSetupConst.TOTAL_SIZE;
 import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.filterMapkeys;
+import static org.hamcrest.CoreMatchers.equalTo;
 import static org.hamcrest.core.Is.is;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertThat;
@@ -460,5 +468,349 @@ public class TestMetaStoreServerUtils {
         put("numFilesErasureCoded", "0");
       }};
   }
+  /**
+   * Two empty StorageDescriptorKey should be equal.
+   */
+  @Test
+  public void testCompareNullSdKey() {
+    assertThat(MetaStoreServerUtils.StorageDescriptorKey.UNSET_KEY,
+        is(new MetaStoreServerUtils.StorageDescriptorKey()));
+  }
+
+  /**
+   * Two StorageDescriptorKey objects with null storage descriptors should be
+   * equal iff the base location is equal.
+   */
+  @Test
+  public void testCompareNullSd()
+  {
+    assertThat(new MetaStoreServerUtils.StorageDescriptorKey("a", null),
+        is(new MetaStoreServerUtils.StorageDescriptorKey("a", null)));
+    // Different locations produce different objects
+    assertThat(new MetaStoreServerUtils.StorageDescriptorKey("a", null),
+        IsNot.not(equalTo(new MetaStoreServerUtils.StorageDescriptorKey("b", null))));
+  }
+
+  /**
+   * Two StorageDescriptorKey objects with the same base location but different
+   * SD location should be equal
+   */
+  @Test
+  public void testCompareWithSdSamePrefixDifferentLocation() throws MetaException {
+    Partition p1 = new PartitionBuilder()
+        .setDbName("DB_NAME")
+        .setTableName(TABLE_NAME)
+        .setLocation("l1")
+        .addCol("a", "int")
+        .addValue("val1")
+        .build(null);
+    Partition p2 = new PartitionBuilder()
+        .setDbName("DB_NAME")
+        .setTableName(TABLE_NAME)
+        .setLocation("l2")
+        .addCol("a", "int")
+        .addValue("val1")
+        .build(null);
+    assertThat(new MetaStoreServerUtils.StorageDescriptorKey("a", p1.getSd()),
+        is(new MetaStoreServerUtils.StorageDescriptorKey("a", p2.getSd())));
+  }
+
+  /**
+   * Two StorageDescriptorKey objects with the same base location
+   * should be equal iff their columns are equal
+   */
+  @Test
+  public void testCompareWithSdSamePrefixDifferentCols() throws MetaException {
+    Partition p1 = new PartitionBuilder()
+        .setDbName("DB_NAME")
+        .setTableName(TABLE_NAME)
+        .setLocation("l1")
+        .addCol("a", "int")
+        .addValue("val1")
+        .build(null);
+    Partition p2 = new PartitionBuilder()
+        .setDbName("DB_NAME")
+        .setTableName(TABLE_NAME)
+        .setLocation("l2")
+        .addCol("b", "int")
+        .addValue("val1")
+        .build(null);
+    Partition p3 = new PartitionBuilder()
+        .setDbName("DB_NAME")
+        .setTableName(TABLE_NAME)
+        .setLocation("l2")
+        .addCol("a", "int")
+        .addValue("val1")
+        .build(null);
+    assertThat(new MetaStoreServerUtils.StorageDescriptorKey("a", p1.getSd()),
+        IsNot.not(new MetaStoreServerUtils.StorageDescriptorKey("a", p2.getSd())));
+    assertThat(new MetaStoreServerUtils.StorageDescriptorKey("a", p1.getSd()),
+        is(new MetaStoreServerUtils.StorageDescriptorKey("a", p3.getSd())));
+  }
+
+  /**
+   * Two StorageDescriptorKey objects with the same base location
+   * should be equal iff their output formats are equal
+   */
+  @Test
+  public void testCompareWithSdSamePrefixDifferentOutputFormat() throws MetaException {
+    Partition p1 = new PartitionBuilder()
+        .setDbName("DB_NAME")
+        .setTableName(TABLE_NAME)
+        .setLocation("l1")
+        .addCol("a", "int")
+        .addValue("val1")
+        .setOutputFormat("foo")
+        .build(null);
+    Partition p2 = new PartitionBuilder()
+        .setDbName("DB_NAME")
+        .setTableName(TABLE_NAME)
+        .setLocation("l2")
+        .addCol("a", "int")
+        .setOutputFormat("bar")
+        .addValue("val1")
+        .build(null);
+    Partition p3 = new PartitionBuilder()
+        .setDbName("DB_NAME")
+        .setTableName(TABLE_NAME)
+        .setLocation("l2")
+        .addCol("a", "int")
+        .setOutputFormat("foo")
+        .addValue("val1")
+        .build(null);
+    assertThat(new MetaStoreServerUtils.StorageDescriptorKey("a", p1.getSd()),
+        IsNot.not(new MetaStoreServerUtils.StorageDescriptorKey("a", p2.getSd())));
+    assertThat(new MetaStoreServerUtils.StorageDescriptorKey("a", p1.getSd()),
+        is(new MetaStoreServerUtils.StorageDescriptorKey("a", p3.getSd())));
+  }
+
+  /**
+   * Two StorageDescriptorKey objects with the same base location
+   * should be equal iff their input formats are equal
+   */
+  @Test
+  public void testCompareWithSdSamePrefixDifferentInputFormat() throws MetaException {
+    Partition p1 = new PartitionBuilder()
+        .setDbName("DB_NAME")
+        .setTableName(TABLE_NAME)
+        .setLocation("l1")
+        .addCol("a", "int")
+        .addValue("val1")
+        .setInputFormat("foo")
+        .build(null);
+    Partition p2 = new PartitionBuilder()
+        .setDbName("DB_NAME")
+        .setTableName(TABLE_NAME)
+        .setLocation("l2")
+        .addCol("a", "int")
+        .setInputFormat("bar")
+        .addValue("val1")
+        .build(null);
+    Partition p3 = new PartitionBuilder()
+        .setDbName("DB_NAME")
+        .setTableName(TABLE_NAME)
+        .setLocation("l1")
+        .addCol("a", "int")
+        .addValue("val1")
+        .setInputFormat("foo")
+        .build(null);
+    assertThat(new MetaStoreServerUtils.StorageDescriptorKey("a", p1.getSd()),
+        IsNot.not(new MetaStoreServerUtils.StorageDescriptorKey("a", p2.getSd())));
+    assertThat(new MetaStoreServerUtils.StorageDescriptorKey("a", p1.getSd()),
+        is(new MetaStoreServerUtils.StorageDescriptorKey("a", p3.getSd())));
+  }
+
+  /**
+   * Test getPartitionspecsGroupedByStorageDescriptor() for partitions with null SDs.
+   */
+  @Test
+  public void testGetPartitionspecsGroupedBySDNullSD() throws MetaException {
+    // Create database and table
+    Table tbl = new TableBuilder()
+        .setDbName(DB_NAME)
+        .setTableName(TABLE_NAME)
+        .addCol("id", "int")
+        .setLocation("/foo")
+        .build(null);
+    Partition p1 = new PartitionBuilder()
+        .setDbName("DB_NAME")
+        .setTableName(TABLE_NAME)
+        .addCol("a", "int")
+        .addValue("val1")
+        .setInputFormat("foo")
+        .build(null);
+    // Set SD to null
+    p1.unsetSd();
+    assertThat(p1.getSd(), is((StorageDescriptor)null));
+    List<PartitionSpec> result =
+        MetaStoreServerUtils.getPartitionspecsGroupedByStorageDescriptor(tbl, Collections.singleton(p1));
+    assertThat(result.size(), is(1));
+    PartitionSpec ps = result.get(0);
+    assertThat(ps.getRootPath(), is((String)null));
+    List<PartitionWithoutSD> partitions = ps.getSharedSDPartitionSpec().getPartitions();
+    assertThat(partitions.size(), is(1));
+    PartitionWithoutSD partition = partitions.get(0);
+    assertThat(partition.getRelativePath(), is((String)null));
+    assertThat(partition.getValues(), is(Collections.singletonList("val1")));
+  }
+
+  /**
+   * Test getPartitionspecsGroupedByStorageDescriptor() for partitions with a single
+   * partition which is located under table location.
+   */
+  @Test
+  public void testGetPartitionspecsGroupedBySDOnePartitionInTable() throws MetaException {
+    // Create database and table
+    Table tbl = new TableBuilder()
+        .setDbName(DB_NAME)
+        .setTableName(TABLE_NAME)
+        .addCol("id", "int")
+        .setLocation("/foo")
+        .build(null);
+    Partition p1 = new PartitionBuilder()
+        .setDbName("DB_NAME")
+        .setTableName(TABLE_NAME)
+        .setLocation("/foo/bar")
+        .addCol("a", "int")
+        .addValue("val1")
+        .setInputFormat("foo")
+        .build(null);
+    List<PartitionSpec> result =
+        MetaStoreServerUtils.getPartitionspecsGroupedByStorageDescriptor(tbl, Collections.singleton(p1));
+    assertThat(result.size(), is(1));
+    PartitionSpec ps = result.get(0);
+    assertThat(ps.getRootPath(), is(tbl.getSd().getLocation()));
+    List<PartitionWithoutSD> partitions = ps.getSharedSDPartitionSpec().getPartitions();
+    assertThat(partitions.size(), is(1));
+    PartitionWithoutSD partition = partitions.get(0);
+    assertThat(partition.getRelativePath(), is("/bar"));
+    assertThat(partition.getValues(), is(Collections.singletonList("val1")));
+  }
+
+  /**
+   * Test getPartitionspecsGroupedByStorageDescriptor() for partitions with a single
+   * partition which is located outside table location.
+   */
+  @Test
+  public void testGetPartitionspecsGroupedBySDonePartitionExternal() throws MetaException {
+    // Create database and table
+    Table tbl = new TableBuilder()
+        .setDbName(DB_NAME)
+        .setTableName(TABLE_NAME)
+        .addCol("id", "int")
+        .setLocation("/foo")
+        .build(null);
+    Partition p1 = new PartitionBuilder()
+        .setDbName("DB_NAME")
+        .setTableName(TABLE_NAME)
+        .setLocation("/a/b")
+        .addCol("a", "int")
+        .addValue("val1")
+        .setInputFormat("foo")
+        .build(null);
+    List<PartitionSpec> result =
+        MetaStoreServerUtils.getPartitionspecsGroupedByStorageDescriptor(tbl, Collections.singleton(p1));
+    assertThat(result.size(), is(1));
+    PartitionSpec ps = result.get(0);
+    assertThat(ps.getRootPath(), is((String)null));
+    List<Partition>partitions = ps.getPartitionList().getPartitions();
+    assertThat(partitions.size(), is(1));
+    Partition partition = partitions.get(0);
+    assertThat(partition.getSd().getLocation(), is("/a/b"));
+    assertThat(partition.getValues(), is(Collections.singletonList("val1")));
+  }
+
+  /**
+   * Test getPartitionspecsGroupedByStorageDescriptor() multiple partitions:
+   * <ul>
+   *   <li>Partition with null SD</li>
+   *   <li>Two partitions under the table location</li>
+   *   <li>One partition outside of table location</li>
+   * </ul>
+   */
+  @Test
+  public void testGetPartitionspecsGroupedBySDonePartitionCombined() throws MetaException {
+    // Create database and table
+    String sharedInputFormat = "foo1";
+
+    Table tbl = new TableBuilder()
+        .setDbName(DB_NAME)
+        .setTableName(TABLE_NAME)
+        .addCol("id", "int")
+        .setLocation("/foo")
+        .build(null);
+    Partition p1 = new PartitionBuilder()
+        .setDbName("DB_NAME")
+        .setTableName(TABLE_NAME)
+        .setLocation("/foo/bar")
+        .addCol("a1", "int")
+        .addValue("val1")
+        .setInputFormat(sharedInputFormat)
+        .build(null);
+    Partition p2 = new PartitionBuilder()
+        .setDbName("DB_NAME")
+        .setTableName(TABLE_NAME)
+        .setLocation("/a/b")
+        .addCol("a2", "int")
+        .addValue("val2")
+        .setInputFormat("foo2")
+        .build(null);
+    Partition p3 = new PartitionBuilder()
+        .setDbName("DB_NAME")
+        .setTableName(TABLE_NAME)
+        .addCol("a3", "int")
+        .addValue("val3")
+        .setInputFormat("foo3")
+        .build(null);
+    Partition p4 = new PartitionBuilder()
+        .setDbName("DB_NAME")
+        .setTableName("TABLE_NAME")
+        .setLocation("/foo/baz")
+        .addCol("a1", "int")
+        .addValue("val4")
+        .setInputFormat(sharedInputFormat)
+        .build(null);
+    p3.unsetSd();
+    List<PartitionSpec> result =
+        MetaStoreServerUtils.getPartitionspecsGroupedByStorageDescriptor(tbl,
+            Arrays.asList(p1, p2, p3, p4));
+    assertThat(result.size(), is(3));
+    PartitionSpec ps1 = result.get(0);
+    assertThat(ps1.getRootPath(), is((String)null));
+    assertThat(ps1.getPartitionList(), is((List<Partition>)null));
+    PartitionSpecWithSharedSD partSpec = ps1.getSharedSDPartitionSpec();
+    List<PartitionWithoutSD> partitions1 = partSpec.getPartitions();
+    assertThat(partitions1.size(), is(1));
+    PartitionWithoutSD partition1 = partitions1.get(0);
+    assertThat(partition1.getRelativePath(), is((String)null));
+    assertThat(partition1.getValues(), is(Collections.singletonList("val3")));
+
+    PartitionSpec ps2 = result.get(1);
+    assertThat(ps2.getRootPath(), is(tbl.getSd().getLocation()));
+    assertThat(ps2.getPartitionList(), is((List<Partition>)null));
+    List<PartitionWithoutSD> partitions2 = ps2.getSharedSDPartitionSpec().getPartitions();
+    assertThat(partitions2.size(), is(2));
+    PartitionWithoutSD partition2_1 = partitions2.get(0);
+    PartitionWithoutSD partition2_2 = partitions2.get(1);
+    if (partition2_1.getRelativePath().equals("baz")) {
+      // Swap p2_1 and p2_2
+      PartitionWithoutSD tmp = partition2_1;
+      partition2_1 = partition2_2;
+      partition2_2 = tmp;
+    }
+    assertThat(partition2_1.getRelativePath(), is("/bar"));
+    assertThat(partition2_1.getValues(), is(Collections.singletonList("val1")));
+    assertThat(partition2_2.getRelativePath(), is("/baz"));
+    assertThat(partition2_2.getValues(), is(Collections.singletonList("val4")));
+
+    PartitionSpec ps4 = result.get(2);
+    assertThat(ps4.getRootPath(), is((String)null));
+    assertThat(ps4.getSharedSDPartitionSpec(), is((PartitionSpecWithSharedSD)null));
+    List<Partition>partitions = ps4.getPartitionList().getPartitions();
+    assertThat(partitions.size(), is(1));
+    Partition partition = partitions.get(0);
+    assertThat(partition.getSd().getLocation(), is("/a/b"));
+    assertThat(partition.getValues(), is(Collections.singletonList("val2")));
+  }
 }
 


[06/13] hive git commit: HIVE-20306 : Implement projection spec for fetching only requested fields from partitions (Vihang Karajgaonkar, reviewed by Aihua Xu, Andrew Sherman and Alexander Kolbasov)

Posted by vi...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/44ef91a6/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
index 0a25b77..af75793 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
@@ -23,8 +23,7 @@ import static org.apache.commons.lang.StringUtils.normalizeSpace;
 import static org.apache.commons.lang.StringUtils.repeat;
 import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_CATALOG_NAME;
 
-import java.sql.Blob;
-import java.sql.Clob;
+import java.net.URL;
 import java.sql.Connection;
 import java.sql.SQLException;
 import java.sql.Statement;
@@ -45,7 +44,7 @@ import javax.jdo.Query;
 import javax.jdo.Transaction;
 import javax.jdo.datastore.JDOConnection;
 
-import org.apache.commons.lang.BooleanUtils;
+import com.google.common.collect.ImmutableMap;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.metastore.AggregateStatsCache.AggrColStats;
@@ -135,8 +134,22 @@ class MetaStoreDirectSql {
    */
   private final boolean isCompatibleDatastore;
   private final boolean isAggregateStatsCacheEnabled;
+  private final ImmutableMap<String, String> fieldnameToTableName;
   private AggregateStatsCache aggrStatsCache;
 
+  /**
+   * This method returns a comma separated string consisting of String values of a given list.
+   * This is used for preparing "SOMETHING_ID in (...)" to use in SQL queries.
+   * @param objectIds the objectId collection
+   * @return The concatenated list
+   * @throws MetaException If the list contains wrong data
+   */
+  public static <T> String getIdListForIn(List<T> objectIds) throws MetaException {
+    return objectIds.stream()
+               .map(i -> i.toString())
+               .collect(Collectors.joining(","));
+  }
+
   @java.lang.annotation.Target(java.lang.annotation.ElementType.FIELD)
   @java.lang.annotation.Retention(java.lang.annotation.RetentionPolicy.RUNTIME)
   private @interface TableName {}
@@ -166,11 +179,15 @@ class MetaStoreDirectSql {
       batchSize = DatabaseProduct.needsInBatching(dbType) ? 1000 : NO_BATCHING;
     }
     this.batchSize = batchSize;
+    ImmutableMap.Builder<String, String> fieldNameToTableNameBuilder =
+        new ImmutableMap.Builder<>();
 
     for (java.lang.reflect.Field f : this.getClass().getDeclaredFields()) {
       if (f.getAnnotation(TableName.class) == null) continue;
       try {
-        f.set(this, getFullyQualifiedName(schema, f.getName()));
+        String value = getFullyQualifiedName(schema, f.getName());
+        f.set(this, value);
+        fieldNameToTableNameBuilder.put(f.getName(), value);
       } catch (IllegalArgumentException | IllegalAccessException e) {
         throw new RuntimeException("Internal error, cannot set " + f.getName());
       }
@@ -198,6 +215,27 @@ class MetaStoreDirectSql {
     if (isAggregateStatsCacheEnabled) {
       aggrStatsCache = AggregateStatsCache.getInstance(conf);
     }
+
+    // now use the tableanames to create the mapping
+    // note that some of the optional single-valued fields are not present
+    fieldnameToTableName =
+        fieldNameToTableNameBuilder
+            .put("createTime", PARTITIONS + ".\"CREATE_TIME\"")
+            .put("lastAccessTime", PARTITIONS + ".\"LAST_ACCESS_TIME\"")
+            .put("writeId", PARTITIONS + ".\"WRITE_ID\"")
+            .put("sd.location", SDS + ".\"LOCATION\"")
+            .put("sd.inputFormat", SDS + ".\"INPUT_FORMAT\"")
+            .put("sd.outputFormat", SDS + ".\"OUTPUT_FORMAT\"")
+            .put("sd.storedAsSubDirectories", SDS + ".\"IS_STOREDASSUBDIRECTORIES\"")
+            .put("sd.compressed", SDS + ".\"IS_COMPRESSED\"")
+            .put("sd.numBuckets", SDS + ".\"NUM_BUCKETS\"")
+            .put("sd.serdeInfo.name", SERDES + ".\"NAME\"")
+            .put("sd.serdeInfo.serializationLib", SERDES + ".\"SLIB\"")
+            .put("PART_ID", PARTITIONS + ".\"PART_ID\"")
+            .put("SD_ID", SDS + ".\"SD_ID\"")
+            .put("SERDE_ID", SERDES + ".\"SERDE_ID\"")
+            .put("CD_ID", SDS + ".\"CD_ID\"")
+            .build();
   }
 
   private static String getFullyQualifiedName(String schema, String tblName) {
@@ -314,7 +352,7 @@ class MetaStoreDirectSql {
       long start = doTrace ? System.nanoTime() : 0;
       statement = ((Connection)jdoConn.getNativeConnection()).createStatement();
       statement.execute(queryText);
-      timingTrace(doTrace, queryText, start, doTrace ? System.nanoTime() : 0);
+      MetastoreDirectSqlUtils.timingTrace(doTrace, queryText, start, doTrace ? System.nanoTime() : 0);
     } finally {
       if(statement != null){
           statement.close();
@@ -355,7 +393,7 @@ class MetaStoreDirectSql {
       }
 
       Object[] dbline = sqlResult.get(0);
-      Long dbid = extractSqlLong(dbline[0]);
+      Long dbid = MetastoreDirectSqlUtils.extractSqlLong(dbline[0]);
 
       String queryTextDbParams = "select \"PARAM_KEY\", \"PARAM_VALUE\" "
           + " from " + DATABASE_PARAMS + " "
@@ -369,22 +407,23 @@ class MetaStoreDirectSql {
       }
 
       Map<String,String> dbParams = new HashMap<String,String>();
-      List<Object[]> sqlResult2 = ensureList(executeWithArray(
+      List<Object[]> sqlResult2 = MetastoreDirectSqlUtils.ensureList(executeWithArray(
           queryDbParams, params, queryTextDbParams));
       if (!sqlResult2.isEmpty()) {
         for (Object[] line : sqlResult2) {
-          dbParams.put(extractSqlString(line[0]), extractSqlString(line[1]));
+          dbParams.put(MetastoreDirectSqlUtils.extractSqlString(line[0]), MetastoreDirectSqlUtils
+              .extractSqlString(line[1]));
         }
       }
       Database db = new Database();
-      db.setName(extractSqlString(dbline[1]));
-      db.setLocationUri(extractSqlString(dbline[2]));
-      db.setDescription(extractSqlString(dbline[3]));
-      db.setOwnerName(extractSqlString(dbline[4]));
-      String type = extractSqlString(dbline[5]);
+      db.setName(MetastoreDirectSqlUtils.extractSqlString(dbline[1]));
+      db.setLocationUri(MetastoreDirectSqlUtils.extractSqlString(dbline[2]));
+      db.setDescription(MetastoreDirectSqlUtils.extractSqlString(dbline[3]));
+      db.setOwnerName(MetastoreDirectSqlUtils.extractSqlString(dbline[4]));
+      String type = MetastoreDirectSqlUtils.extractSqlString(dbline[5]);
       db.setOwnerType(
           (null == type || type.trim().isEmpty()) ? null : PrincipalType.valueOf(type));
-      db.setCatalogName(extractSqlString(dbline[6]));
+      db.setCatalogName(MetastoreDirectSqlUtils.extractSqlString(dbline[6]));
       db.setParameters(MetaStoreServerUtils.trimMapNulls(dbParams,convertMapNullsToEmptyStrings));
       if (LOG.isDebugEnabled()){
         LOG.debug("getDatabase: directsql returning db " + db.getName()
@@ -468,12 +507,12 @@ class MetaStoreDirectSql {
       @Override
       public List<Partition> run(List<String> input) throws MetaException {
         String filter = "" + PARTITIONS + ".\"PART_NAME\" in (" + makeParams(input.size()) + ")";
-        List<Object> partitionIds = getPartitionIdsViaSqlFilter(catName, dbName, tblName,
+        List<Long> partitionIds = getPartitionIdsViaSqlFilter(catName, dbName, tblName,
             filter, input, Collections.<String>emptyList(), null);
         if (partitionIds.isEmpty()) {
           return Collections.emptyList(); // no partitions, bail early.
         }
-        return getPartitionsFromPartitionIds(catName, dbName, tblName, null, partitionIds);
+        return getPartitionsFromPartitionIds(catName, dbName, tblName, null, partitionIds, Collections.emptyList());
       }
     });
   }
@@ -489,17 +528,70 @@ class MetaStoreDirectSql {
     Boolean isViewTable = isViewTable(filter.table);
     String catName = filter.table.isSetCatName() ? filter.table.getCatName() :
         DEFAULT_CATALOG_NAME;
-    List<Object> partitionIds = getPartitionIdsViaSqlFilter(catName,
+    List<Long> partitionIds = getPartitionIdsViaSqlFilter(catName,
         filter.table.getDbName(), filter.table.getTableName(), filter.filter, filter.params,
         filter.joins, max);
     if (partitionIds.isEmpty()) {
       return Collections.emptyList(); // no partitions, bail early.
     }
-    return Batchable.runBatched(batchSize, partitionIds, new Batchable<Object, Partition>() {
+    return Batchable.runBatched(batchSize, partitionIds, new Batchable<Long, Partition>() {
       @Override
-      public List<Partition> run(List<Object> input) throws MetaException {
+      public List<Partition> run(List<Long> input) throws MetaException {
         return getPartitionsFromPartitionIds(catName, filter.table.getDbName(),
-            filter.table.getTableName(), isViewTable, input);
+            filter.table.getTableName(), isViewTable, input, Collections.emptyList());
+      }
+    });
+  }
+
+  /**
+   * This method can be used to return "partially-filled" partitions when clients are only interested in
+   * some fields of the Partition objects. The partitionFields parameter is a list of dot separated
+   * partition field names. For example, if a client is interested in only partition location,
+   * serializationLib, values and parameters it can specify sd.location, sd.serdeInfo.serializationLib,
+   * values, parameters in the partitionFields list. In such a case all the returned partitions will have
+   * only the requested fields set and the rest of the fields will remain unset. The implementation of this method
+   * runs queries only for the fields which are requested and pushes down the projection to the database to improve
+   * performance.
+   *
+   * @param tbl                    Table whose partitions are being requested
+   * @param partitionFields        List of dot separated field names. Each dot separated string represents nested levels. For
+   *                               instance sd.serdeInfo.serializationLib represents the serializationLib field of the StorageDescriptor
+   *                               for a the partition
+   * @param includeParamKeyPattern The SQL regex pattern which is used to include the parameter keys. Can include _ or %
+   *                               When this pattern is set, only the partition parameter key-value pairs where the key matches
+   *                               the pattern will be returned. This is applied in conjunction with excludeParamKeyPattern if it is set.
+   * @param excludeParamKeyPattern The SQL regex paterrn which is used to exclude the parameter keys. Can include _ or %
+   *                               When this pattern is set, all the partition parameters where key is NOT LIKE the pattern
+   *                               are returned. This is applied in conjunction with the includeParamKeyPattern if it is set.
+   * @return
+   * @throws MetaException
+   */
+  public List<Partition> getPartitionSpecsUsingProjection(Table tbl,
+      final List<String> partitionFields, final String includeParamKeyPattern, final String excludeParamKeyPattern)
+      throws MetaException {
+    final String tblName = tbl.getTableName();
+    final String dbName = tbl.getDbName();
+    final String catName = tbl.getCatName();
+    //TODO add support for filter
+    List<Long> partitionIds =
+        getPartitionIdsViaSqlFilter(catName, dbName, tblName, null, Collections.<String>emptyList(),
+            Collections.<String>emptyList(), null);
+    if (partitionIds.isEmpty()) {
+      return Collections.emptyList();
+    }
+    // check if table object has table type as view
+    Boolean isView = isViewTable(tbl);
+    if (isView == null) {
+      isView = isViewTable(catName, dbName, tblName);
+    }
+    PartitionProjectionEvaluator projectionEvaluator =
+        new PartitionProjectionEvaluator(pm, fieldnameToTableName, partitionFields,
+            convertMapNullsToEmptyStrings, isView, includeParamKeyPattern, excludeParamKeyPattern);
+    // Get full objects. For Oracle/etc. do it in batches.
+    return Batchable.runBatched(batchSize, partitionIds, new Batchable<Long, Partition>() {
+      @Override
+      public List<Partition> run(List<Long> input) throws MetaException {
+        return projectionEvaluator.getPartitionsUsingProjectionList(input);
       }
     });
   }
@@ -537,17 +629,17 @@ class MetaStoreDirectSql {
    */
   public List<Partition> getPartitions(String catName,
       String dbName, String tblName, Integer max) throws MetaException {
-    List<Object> partitionIds = getPartitionIdsViaSqlFilter(catName, dbName,
+    List<Long> partitionIds = getPartitionIdsViaSqlFilter(catName, dbName,
         tblName, null, Collections.<String>emptyList(), Collections.<String>emptyList(), max);
     if (partitionIds.isEmpty()) {
       return Collections.emptyList(); // no partitions, bail early.
     }
 
     // Get full objects. For Oracle/etc. do it in batches.
-    List<Partition> result = Batchable.runBatched(batchSize, partitionIds, new Batchable<Object, Partition>() {
+    List<Partition> result = Batchable.runBatched(batchSize, partitionIds, new Batchable<Long, Partition>() {
       @Override
-      public List<Partition> run(List<Object> input) throws MetaException {
-        return getPartitionsFromPartitionIds(catName, dbName, tblName, null, input);
+      public List<Partition> run(List<Long> input) throws MetaException {
+        return getPartitionsFromPartitionIds(catName, dbName, tblName, null, input, Collections.emptyList());
       }
     });
     return result;
@@ -589,7 +681,7 @@ class MetaStoreDirectSql {
    * @param max The maximum number of partitions to return.
    * @return List of partition objects.
    */
-  private List<Object> getPartitionIdsViaSqlFilter(
+  private List<Long> getPartitionIdsViaSqlFilter(
       String catName, String dbName, String tblName, String sqlFilter,
       List<? extends Object> paramsForFilter, List<String> joinsForFilter, Integer max)
       throws MetaException {
@@ -625,14 +717,14 @@ class MetaStoreDirectSql {
     }
     List<Object> sqlResult = executeWithArray(query, params, queryText);
     long queryTime = doTrace ? System.nanoTime() : 0;
-    timingTrace(doTrace, queryText, start, queryTime);
+    MetastoreDirectSqlUtils.timingTrace(doTrace, queryText, start, queryTime);
     if (sqlResult.isEmpty()) {
       return Collections.emptyList(); // no partitions, bail early.
     }
 
-    List<Object> result = new ArrayList<Object>(sqlResult.size());
+    List<Long> result = new ArrayList<>(sqlResult.size());
     for (Object fields : sqlResult) {
-      result.add(extractSqlLong(fields));
+      result.add(MetastoreDirectSqlUtils.extractSqlLong(fields));
     }
     query.closeAll();
     return result;
@@ -640,26 +732,27 @@ class MetaStoreDirectSql {
 
   /** Should be called with the list short enough to not trip up Oracle/etc. */
   private List<Partition> getPartitionsFromPartitionIds(String catName, String dbName, String tblName,
-      Boolean isView, List<Object> partIdList) throws MetaException {
+      Boolean isView, List<Long> partIdList, List<String> projectionFields) throws MetaException {
+
     boolean doTrace = LOG.isDebugEnabled();
 
     int idStringWidth = (int)Math.ceil(Math.log10(partIdList.size())) + 1; // 1 for comma
     int sbCapacity = partIdList.size() * idStringWidth;
-
-    String partIds = getIdListForIn(partIdList);
-
     // Get most of the fields for the IDs provided.
     // Assume db and table names are the same for all partition, as provided in arguments.
+    String partIds = getIdListForIn(partIdList);
     String queryText =
-      "select " + PARTITIONS + ".\"PART_ID\", " + SDS + ".\"SD_ID\", " + SDS + ".\"CD_ID\","
-    + " " + SERDES + ".\"SERDE_ID\", " + PARTITIONS + ".\"CREATE_TIME\","
-    + " " + PARTITIONS + ".\"LAST_ACCESS_TIME\", " + SDS + ".\"INPUT_FORMAT\", " + SDS + ".\"IS_COMPRESSED\","
-    + " " + SDS + ".\"IS_STOREDASSUBDIRECTORIES\", " + SDS + ".\"LOCATION\", " + SDS + ".\"NUM_BUCKETS\","
-    + " " + SDS + ".\"OUTPUT_FORMAT\", " + SERDES + ".\"NAME\", " + SERDES + ".\"SLIB\", " + PARTITIONS
-    + ".\"WRITE_ID\"" + " from " + PARTITIONS + ""
-    + "  left outer join " + SDS + " on " + PARTITIONS + ".\"SD_ID\" = " + SDS + ".\"SD_ID\" "
-    + "  left outer join " + SERDES + " on " + SDS + ".\"SERDE_ID\" = " + SERDES + ".\"SERDE_ID\" "
-    + "where \"PART_ID\" in (" + partIds + ") order by \"PART_NAME\" asc";
+        "select " + PARTITIONS + ".\"PART_ID\", " + SDS + ".\"SD_ID\", " + SDS + ".\"CD_ID\"," + " "
+            + SERDES + ".\"SERDE_ID\", " + PARTITIONS + ".\"CREATE_TIME\"," + " " + PARTITIONS
+            + ".\"LAST_ACCESS_TIME\", " + SDS + ".\"INPUT_FORMAT\", " + SDS + ".\"IS_COMPRESSED\","
+            + " " + SDS + ".\"IS_STOREDASSUBDIRECTORIES\", " + SDS + ".\"LOCATION\", " + SDS
+            + ".\"NUM_BUCKETS\"," + " " + SDS + ".\"OUTPUT_FORMAT\", " + SERDES + ".\"NAME\", "
+            + SERDES + ".\"SLIB\", " + PARTITIONS + ".\"WRITE_ID\"" + " from " + PARTITIONS + ""
+            + "  left outer join " + SDS + " on " + PARTITIONS + ".\"SD_ID\" = " + SDS
+            + ".\"SD_ID\" " + "  left outer join " + SERDES + " on " + SDS + ".\"SERDE_ID\" = "
+            + SERDES + ".\"SERDE_ID\" " + "where \"PART_ID\" in (" + partIds
+            + ") order by \"PART_NAME\" asc";
+
     long start = doTrace ? System.nanoTime() : 0;
     Query query = pm.newQuery("javax.jdo.query.SQL", queryText);
     List<Object[]> sqlResult = executeWithArray(query, null, queryText);
@@ -680,12 +773,13 @@ class MetaStoreDirectSql {
     tblName = tblName.toLowerCase();
     dbName = dbName.toLowerCase();
     catName = normalizeSpace(catName).toLowerCase();
+    partitions.navigableKeySet();
     for (Object[] fields : sqlResult) {
       // Here comes the ugly part...
-      long partitionId = extractSqlLong(fields[0]);
-      Long sdId = extractSqlLong(fields[1]);
-      Long colId = extractSqlLong(fields[2]);
-      Long serdeId = extractSqlLong(fields[3]);
+      long partitionId = MetastoreDirectSqlUtils.extractSqlLong(fields[0]);
+      Long sdId = MetastoreDirectSqlUtils.extractSqlLong(fields[1]);
+      Long colId = MetastoreDirectSqlUtils.extractSqlLong(fields[2]);
+      Long serdeId = MetastoreDirectSqlUtils.extractSqlLong(fields[3]);
       // A partition must have at least sdId and serdeId set, or nothing set if it's a view.
       if (sdId == null || serdeId == null) {
         if (isView == null) {
@@ -693,7 +787,7 @@ class MetaStoreDirectSql {
         }
         if ((sdId != null || colId != null || serdeId != null) || !isView) {
           throw new MetaException("Unexpected null for one of the IDs, SD " + sdId +
-                  ", serde " + serdeId + " for a " + (isView ? "" : "non-") + " view");
+              ", serde " + serdeId + " for a " + (isView ? "" : "non-") + " view");
         }
       }
 
@@ -705,9 +799,9 @@ class MetaStoreDirectSql {
       part.setCatName(catName);
       part.setDbName(dbName);
       part.setTableName(tblName);
-      if (fields[4] != null) part.setCreateTime(extractSqlInt(fields[4]));
-      if (fields[5] != null) part.setLastAccessTime(extractSqlInt(fields[5]));
-      Long writeId = extractSqlLong(fields[14]);
+      if (fields[4] != null) part.setCreateTime(MetastoreDirectSqlUtils.extractSqlInt(fields[4]));
+      if (fields[5] != null) part.setLastAccessTime(MetastoreDirectSqlUtils.extractSqlInt(fields[5]));
+      Long writeId = MetastoreDirectSqlUtils.extractSqlLong(fields[14]);
       if (writeId != null) {
         part.setWriteId(writeId);
       }
@@ -730,12 +824,12 @@ class MetaStoreDirectSql {
       sd.setSkewedInfo(new SkewedInfo(new ArrayList<String>(),
           new ArrayList<List<String>>(), new HashMap<List<String>, String>()));
       sd.setInputFormat((String)fields[6]);
-      Boolean tmpBoolean = extractSqlBoolean(fields[7]);
+      Boolean tmpBoolean = MetastoreDirectSqlUtils.extractSqlBoolean(fields[7]);
       if (tmpBoolean != null) sd.setCompressed(tmpBoolean);
-      tmpBoolean = extractSqlBoolean(fields[8]);
+      tmpBoolean = MetastoreDirectSqlUtils.extractSqlBoolean(fields[8]);
       if (tmpBoolean != null) sd.setStoredAsSubDirectories(tmpBoolean);
       sd.setLocation((String)fields[9]);
-      if (fields[10] != null) sd.setNumBuckets(extractSqlInt(fields[10]));
+      if (fields[10] != null) sd.setNumBuckets(MetastoreDirectSqlUtils.extractSqlInt(fields[10]));
       sd.setOutputFormat((String)fields[11]);
       sdSb.append(sdId).append(",");
       part.setSd(sd);
@@ -766,30 +860,13 @@ class MetaStoreDirectSql {
       Deadline.checkTimeout();
     }
     query.closeAll();
-    timingTrace(doTrace, queryText, start, queryTime);
+    MetastoreDirectSqlUtils.timingTrace(doTrace, queryText, start, queryTime);
 
     // Now get all the one-to-many things. Start with partitions.
-    queryText = "select \"PART_ID\", \"PARAM_KEY\", \"PARAM_VALUE\" from " + PARTITION_PARAMS + ""
-        + " where \"PART_ID\" in (" + partIds + ") and \"PARAM_KEY\" is not null"
-        + " order by \"PART_ID\" asc";
-    loopJoinOrderedResult(partitions, queryText, 0, new ApplyFunc<Partition>() {
-      @Override
-      public void apply(Partition t, Object[] fields) {
-        t.putToParameters((String)fields[1], extractSqlClob(fields[2]));
-      }});
-    // Perform conversion of null map values
-    for (Partition t : partitions.values()) {
-      t.setParameters(MetaStoreServerUtils.trimMapNulls(t.getParameters(), convertMapNullsToEmptyStrings));
-    }
-
-    queryText = "select \"PART_ID\", \"PART_KEY_VAL\" from " + PARTITION_KEY_VALS + ""
-        + " where \"PART_ID\" in (" + partIds + ")"
-        + " order by \"PART_ID\" asc, \"INTEGER_IDX\" asc";
-    loopJoinOrderedResult(partitions, queryText, 0, new ApplyFunc<Partition>() {
-      @Override
-      public void apply(Partition t, Object[] fields) {
-        t.addToValues((String)fields[1]);
-      }});
+    MetastoreDirectSqlUtils
+        .setPartitionParameters(PARTITION_PARAMS, convertMapNullsToEmptyStrings, pm, partIds, partitions);
+
+    MetastoreDirectSqlUtils.setPartitionValues(PARTITION_KEY_VALS, pm, partIds, partitions);
 
     // Prepare IN (blah) lists for the following queries. Cut off the final ','s.
     if (sdSb.length() == 0) {
@@ -802,160 +879,36 @@ class MetaStoreDirectSql {
     String colIds = trimCommaList(colsSb);
 
     // Get all the stuff for SD. Don't do empty-list check - we expect partitions do have SDs.
-    queryText = "select \"SD_ID\", \"PARAM_KEY\", \"PARAM_VALUE\" from " + SD_PARAMS + ""
-        + " where \"SD_ID\" in (" + sdIds + ") and \"PARAM_KEY\" is not null"
-        + " order by \"SD_ID\" asc";
-    loopJoinOrderedResult(sds, queryText, 0, new ApplyFunc<StorageDescriptor>() {
-      @Override
-      public void apply(StorageDescriptor t, Object[] fields) {
-        t.putToParameters((String)fields[1], extractSqlClob(fields[2]));
-      }});
-    // Perform conversion of null map values
-    for (StorageDescriptor t : sds.values()) {
-      t.setParameters(MetaStoreServerUtils.trimMapNulls(t.getParameters(), convertMapNullsToEmptyStrings));
-    }
-
-    queryText = "select \"SD_ID\", \"COLUMN_NAME\", " + SORT_COLS + ".\"ORDER\""
-        + " from " + SORT_COLS + ""
-        + " where \"SD_ID\" in (" + sdIds + ")"
-        + " order by \"SD_ID\" asc, \"INTEGER_IDX\" asc";
-    loopJoinOrderedResult(sds, queryText, 0, new ApplyFunc<StorageDescriptor>() {
-      @Override
-      public void apply(StorageDescriptor t, Object[] fields) {
-        if (fields[2] == null) return;
-        t.addToSortCols(new Order((String)fields[1], extractSqlInt(fields[2])));
-      }});
-
-    queryText = "select \"SD_ID\", \"BUCKET_COL_NAME\" from " + BUCKETING_COLS + ""
-        + " where \"SD_ID\" in (" + sdIds + ")"
-        + " order by \"SD_ID\" asc, \"INTEGER_IDX\" asc";
-    loopJoinOrderedResult(sds, queryText, 0, new ApplyFunc<StorageDescriptor>() {
-      @Override
-      public void apply(StorageDescriptor t, Object[] fields) {
-        t.addToBucketCols((String)fields[1]);
-      }});
+    MetastoreDirectSqlUtils.setSDParameters(SD_PARAMS, convertMapNullsToEmptyStrings, pm, sds, sdIds);
+
+    MetastoreDirectSqlUtils.setSDSortCols(SORT_COLS, pm, sds, sdIds);
+
+    MetastoreDirectSqlUtils.setSDBucketCols(BUCKETING_COLS, pm, sds, sdIds);
 
     // Skewed columns stuff.
-    queryText = "select \"SD_ID\", \"SKEWED_COL_NAME\" from " + SKEWED_COL_NAMES + ""
-        + " where \"SD_ID\" in (" + sdIds + ")"
-        + " order by \"SD_ID\" asc, \"INTEGER_IDX\" asc";
-    boolean hasSkewedColumns =
-      loopJoinOrderedResult(sds, queryText, 0, new ApplyFunc<StorageDescriptor>() {
-        @Override
-        public void apply(StorageDescriptor t, Object[] fields) {
-          if (!t.isSetSkewedInfo()) t.setSkewedInfo(new SkewedInfo());
-          t.getSkewedInfo().addToSkewedColNames((String)fields[1]);
-        }}) > 0;
+    boolean hasSkewedColumns = MetastoreDirectSqlUtils
+        .setSkewedColNames(SKEWED_COL_NAMES, pm, sds, sdIds);
 
     // Assume we don't need to fetch the rest of the skewed column data if we have no columns.
     if (hasSkewedColumns) {
       // We are skipping the SKEWED_STRING_LIST table here, as it seems to be totally useless.
-      queryText =
-            "select " + SKEWED_VALUES + ".\"SD_ID_OID\","
-          + "  " + SKEWED_STRING_LIST_VALUES + ".\"STRING_LIST_ID\","
-          + "  " + SKEWED_STRING_LIST_VALUES + ".\"STRING_LIST_VALUE\" "
-          + "from " + SKEWED_VALUES + " "
-          + "  left outer join " + SKEWED_STRING_LIST_VALUES + " on " + SKEWED_VALUES + "."
-          + "\"STRING_LIST_ID_EID\" = " + SKEWED_STRING_LIST_VALUES + ".\"STRING_LIST_ID\" "
-          + "where " + SKEWED_VALUES + ".\"SD_ID_OID\" in (" + sdIds + ") "
-          + "  and " + SKEWED_VALUES + ".\"STRING_LIST_ID_EID\" is not null "
-          + "  and " + SKEWED_VALUES + ".\"INTEGER_IDX\" >= 0 "
-          + "order by " + SKEWED_VALUES + ".\"SD_ID_OID\" asc, " + SKEWED_VALUES + ".\"INTEGER_IDX\" asc,"
-          + "  " + SKEWED_STRING_LIST_VALUES + ".\"INTEGER_IDX\" asc";
-      loopJoinOrderedResult(sds, queryText, 0, new ApplyFunc<StorageDescriptor>() {
-        private Long currentListId;
-        private List<String> currentList;
-        @Override
-        public void apply(StorageDescriptor t, Object[] fields) throws MetaException {
-          if (!t.isSetSkewedInfo()) t.setSkewedInfo(new SkewedInfo());
-          // Note that this is not a typical list accumulator - there's no call to finalize
-          // the last list. Instead we add list to SD first, as well as locally to add elements.
-          if (fields[1] == null) {
-            currentList = null; // left outer join produced a list with no values
-            currentListId = null;
-            t.getSkewedInfo().addToSkewedColValues(Collections.<String>emptyList());
-          } else {
-            long fieldsListId = extractSqlLong(fields[1]);
-            if (currentListId == null || fieldsListId != currentListId) {
-              currentList = new ArrayList<String>();
-              currentListId = fieldsListId;
-              t.getSkewedInfo().addToSkewedColValues(currentList);
-            }
-            currentList.add((String)fields[2]);
-          }
-        }});
+      MetastoreDirectSqlUtils
+          .setSkewedColValues(SKEWED_STRING_LIST_VALUES, SKEWED_VALUES, pm, sds, sdIds);
 
       // We are skipping the SKEWED_STRING_LIST table here, as it seems to be totally useless.
-      queryText =
-            "select " + SKEWED_COL_VALUE_LOC_MAP + ".\"SD_ID\","
-          + " " + SKEWED_STRING_LIST_VALUES + ".STRING_LIST_ID,"
-          + " " + SKEWED_COL_VALUE_LOC_MAP + ".\"LOCATION\","
-          + " " + SKEWED_STRING_LIST_VALUES + ".\"STRING_LIST_VALUE\" "
-          + "from " + SKEWED_COL_VALUE_LOC_MAP + ""
-          + "  left outer join " + SKEWED_STRING_LIST_VALUES + " on " + SKEWED_COL_VALUE_LOC_MAP + "."
-          + "\"STRING_LIST_ID_KID\" = " + SKEWED_STRING_LIST_VALUES + ".\"STRING_LIST_ID\" "
-          + "where " + SKEWED_COL_VALUE_LOC_MAP + ".\"SD_ID\" in (" + sdIds + ")"
-          + "  and " + SKEWED_COL_VALUE_LOC_MAP + ".\"STRING_LIST_ID_KID\" is not null "
-          + "order by " + SKEWED_COL_VALUE_LOC_MAP + ".\"SD_ID\" asc,"
-          + "  " + SKEWED_STRING_LIST_VALUES + ".\"STRING_LIST_ID\" asc,"
-          + "  " + SKEWED_STRING_LIST_VALUES + ".\"INTEGER_IDX\" asc";
-
-      loopJoinOrderedResult(sds, queryText, 0, new ApplyFunc<StorageDescriptor>() {
-        private Long currentListId;
-        private List<String> currentList;
-        @Override
-        public void apply(StorageDescriptor t, Object[] fields) throws MetaException {
-          if (!t.isSetSkewedInfo()) {
-            SkewedInfo skewedInfo = new SkewedInfo();
-            skewedInfo.setSkewedColValueLocationMaps(new HashMap<List<String>, String>());
-            t.setSkewedInfo(skewedInfo);
-          }
-          Map<List<String>, String> skewMap = t.getSkewedInfo().getSkewedColValueLocationMaps();
-          // Note that this is not a typical list accumulator - there's no call to finalize
-          // the last list. Instead we add list to SD first, as well as locally to add elements.
-          if (fields[1] == null) {
-            currentList = new ArrayList<String>(); // left outer join produced a list with no values
-            currentListId = null;
-          } else {
-            long fieldsListId = extractSqlLong(fields[1]);
-            if (currentListId == null || fieldsListId != currentListId) {
-              currentList = new ArrayList<String>();
-              currentListId = fieldsListId;
-            } else {
-              skewMap.remove(currentList); // value based compare.. remove first
-            }
-            currentList.add((String)fields[3]);
-          }
-          skewMap.put(currentList, (String)fields[2]);
-        }});
+      MetastoreDirectSqlUtils
+          .setSkewedColLocationMaps(SKEWED_COL_VALUE_LOC_MAP, SKEWED_STRING_LIST_VALUES, pm, sds, sdIds);
     } // if (hasSkewedColumns)
 
     // Get FieldSchema stuff if any.
     if (!colss.isEmpty()) {
       // We are skipping the CDS table here, as it seems to be totally useless.
-      queryText = "select \"CD_ID\", \"COMMENT\", \"COLUMN_NAME\", \"TYPE_NAME\""
-          + " from " + COLUMNS_V2 + " where \"CD_ID\" in (" + colIds + ")"
-          + " order by \"CD_ID\" asc, \"INTEGER_IDX\" asc";
-      loopJoinOrderedResult(colss, queryText, 0, new ApplyFunc<List<FieldSchema>>() {
-        @Override
-        public void apply(List<FieldSchema> t, Object[] fields) {
-          t.add(new FieldSchema((String)fields[2], extractSqlClob(fields[3]), (String)fields[1]));
-        }});
+      MetastoreDirectSqlUtils.setSDCols(COLUMNS_V2, pm, colss, colIds);
     }
 
     // Finally, get all the stuff for serdes - just the params.
-    queryText = "select \"SERDE_ID\", \"PARAM_KEY\", \"PARAM_VALUE\" from " + SERDE_PARAMS + ""
-        + " where \"SERDE_ID\" in (" + serdeIds + ") and \"PARAM_KEY\" is not null"
-        + " order by \"SERDE_ID\" asc";
-    loopJoinOrderedResult(serdes, queryText, 0, new ApplyFunc<SerDeInfo>() {
-      @Override
-      public void apply(SerDeInfo t, Object[] fields) {
-        t.putToParameters((String)fields[1], extractSqlClob(fields[2]));
-      }});
-    // Perform conversion of null map values
-    for (SerDeInfo t : serdes.values()) {
-      t.setParameters(MetaStoreServerUtils.trimMapNulls(t.getParameters(), convertMapNullsToEmptyStrings));
-    }
+    MetastoreDirectSqlUtils
+        .setSerdeParams(SERDE_PARAMS, convertMapNullsToEmptyStrings, pm, serdes, serdeIds);
 
     return orderedResult;
   }
@@ -987,124 +940,12 @@ class MetaStoreDirectSql {
     long start = doTrace ? System.nanoTime() : 0;
     Query query = pm.newQuery("javax.jdo.query.SQL", queryText);
     query.setUnique(true);
-    int sqlResult = extractSqlInt(query.executeWithArray(params));
+    int sqlResult = MetastoreDirectSqlUtils.extractSqlInt(query.executeWithArray(params));
     long queryTime = doTrace ? System.nanoTime() : 0;
-    timingTrace(doTrace, queryText, start, queryTime);
+    MetastoreDirectSqlUtils.timingTrace(doTrace, queryText, start, queryTime);
     return sqlResult;
   }
 
-
-  private void timingTrace(boolean doTrace, String queryText, long start, long queryTime) {
-    if (!doTrace) return;
-    LOG.debug("Direct SQL query in " + (queryTime - start) / 1000000.0 + "ms + " +
-        (System.nanoTime() - queryTime) / 1000000.0 + "ms, the query is [" + queryText + "]");
-  }
-
-  static Long extractSqlLong(Object obj) throws MetaException {
-    if (obj == null) return null;
-    if (!(obj instanceof Number)) {
-      throw new MetaException("Expected numeric type but got " + obj.getClass().getName());
-    }
-    return ((Number)obj).longValue();
-  }
-
-  /**
-   * Convert a boolean value returned from the RDBMS to a Java Boolean object.
-   * MySQL has booleans, but e.g. Derby uses 'Y'/'N' mapping.
-   *
-   * @param value
-   *          column value from the database
-   * @return The Boolean value of the database column value, null if the column
-   *         value is null
-   * @throws MetaException
-   *           if the column value cannot be converted into a Boolean object
-   */
-  private static Boolean extractSqlBoolean(Object value) throws MetaException {
-    if (value == null) {
-      return null;
-    }
-    if (value instanceof Boolean) {
-      return (Boolean)value;
-    }
-    if (value instanceof String) {
-      try {
-        return BooleanUtils.toBooleanObject((String) value, "Y", "N", null);
-      } catch (IllegalArgumentException iae) {
-        // NOOP
-      }
-    }
-    throw new MetaException("Cannot extract boolean from column value " + value);
-  }
-
-  private int extractSqlInt(Object field) {
-    return ((Number)field).intValue();
-  }
-
-  private String extractSqlString(Object value) {
-    if (value == null) return null;
-    return value.toString();
-  }
-
-  static Double extractSqlDouble(Object obj) throws MetaException {
-    if (obj == null)
-      return null;
-    if (!(obj instanceof Number)) {
-      throw new MetaException("Expected numeric type but got " + obj.getClass().getName());
-    }
-    return ((Number) obj).doubleValue();
-  }
-
-  private String extractSqlClob(Object value) {
-    if (value == null) return null;
-    try {
-      if (value instanceof Clob) {
-        // we trim the Clob value to a max length an int can hold
-        int maxLength = (((Clob)value).length() < Integer.MAX_VALUE - 2) ? (int)((Clob)value).length() : Integer.MAX_VALUE - 2;
-        return ((Clob)value).getSubString(1L, maxLength);
-      } else {
-        return value.toString();
-      }
-    } catch (SQLException sqle) {
-      return null;
-    }
-  }
-
-  static byte[] extractSqlBlob(Object value) throws MetaException {
-    if (value == null)
-      return null;
-    if (value instanceof Blob) {
-      //derby, oracle
-      try {
-        // getBytes function says: pos the ordinal position of the first byte in
-        // the BLOB value to be extracted; the first byte is at position 1
-        return ((Blob) value).getBytes(1, (int) ((Blob) value).length());
-      } catch (SQLException e) {
-        throw new MetaException("Encounter error while processing blob.");
-      }
-    }
-    else if (value instanceof byte[]) {
-      // mysql, postgres, sql server
-      return (byte[]) value;
-    }
-	else {
-      // this may happen when enablebitvector is false
-      LOG.debug("Expected blob type but got " + value.getClass().getName());
-      return null;
-    }
-  }
-
-  /**
-   * Helper method for preparing for "SOMETHING_ID in (...)" to use in future queries.
-   * @param objectIds the objectId collection
-   * @return The concatenated list
-   * @throws MetaException If the list contains wrong data
-   */
-  private static String getIdListForIn(List<Object> objectIds) throws MetaException {
-    return objectIds.stream()
-               .map(i -> i.toString())
-               .collect(Collectors.joining(","));
-  }
-
   private static String trimCommaList(StringBuilder sb) {
     if (sb.length() > 0) {
       sb.setLength(sb.length() - 1);
@@ -1112,55 +953,6 @@ class MetaStoreDirectSql {
     return sb.toString();
   }
 
-  private abstract class ApplyFunc<Target> {
-    public abstract void apply(Target t, Object[] fields) throws MetaException;
-  }
-
-  /**
-   * Merges applies the result of a PM SQL query into a tree of object.
-   * Essentially it's an object join. DN could do this for us, but it issues queries
-   * separately for every object, which is suboptimal.
-   * @param tree The object tree, by ID.
-   * @param queryText The query text.
-   * @param keyIndex Index of the Long column corresponding to the map ID in query result rows.
-   * @param func The function that is called on each (object,row) pair with the same id.
-   * @return the count of results returned from the query.
-   */
-  private <T> int loopJoinOrderedResult(TreeMap<Long, T> tree,
-      String queryText, int keyIndex, ApplyFunc<T> func) throws MetaException {
-    boolean doTrace = LOG.isDebugEnabled();
-    long start = doTrace ? System.nanoTime() : 0;
-    Query query = pm.newQuery("javax.jdo.query.SQL", queryText);
-    Object result = query.execute();
-    long queryTime = doTrace ? System.nanoTime() : 0;
-    if (result == null) {
-      query.closeAll();
-      return 0;
-    }
-    List<Object[]> list = ensureList(result);
-    Iterator<Object[]> iter = list.iterator();
-    Object[] fields = null;
-    for (Map.Entry<Long, T> entry : tree.entrySet()) {
-      if (fields == null && !iter.hasNext()) break;
-      long id = entry.getKey();
-      while (fields != null || iter.hasNext()) {
-        if (fields == null) {
-          fields = iter.next();
-        }
-        long nestedId = extractSqlLong(fields[keyIndex]);
-        if (nestedId < id) throw new MetaException("Found entries for unknown ID " + nestedId);
-        if (nestedId > id) break; // fields belong to one of the next entries
-        func.apply(entry.getValue(), fields);
-        fields = null;
-      }
-      Deadline.checkTimeout();
-    }
-    int rv = list.size();
-    query.closeAll();
-    timingTrace(doTrace, queryText, start, queryTime);
-    return rv;
-  }
-
   private static class PartitionFilterGenerator extends TreeVisitor {
     private final Table table;
     private final FilterBuilder filterBuffer;
@@ -1416,13 +1208,13 @@ class MetaStoreDirectSql {
         long start = doTrace ? System.nanoTime() : 0;
         Query query = pm.newQuery("javax.jdo.query.SQL", queryText);
         Object qResult = executeWithArray(query, params, queryText);
-        timingTrace(doTrace, queryText0 + "...)", start, (doTrace ? System.nanoTime() : 0));
+        MetastoreDirectSqlUtils.timingTrace(doTrace, queryText0 + "...)", start, (doTrace ? System.nanoTime() : 0));
         if (qResult == null) {
           query.closeAll();
           return null;
         }
         addQueryAfterUse(query);
-        return ensureList(qResult);
+        return MetastoreDirectSqlUtils.ensureList(qResult);
       }
     };
     List<Object[]> list = Batchable.runBatched(batchSize, colNames, b);
@@ -1526,11 +1318,11 @@ class MetaStoreDirectSql {
               Object qResult = executeWithArray(query, prepareParams(
                   catName, dbName, tableName, inputPartNames, inputColName), queryText);
               long end = doTrace ? System.nanoTime() : 0;
-              timingTrace(doTrace, queryText, start, end);
+              MetastoreDirectSqlUtils.timingTrace(doTrace, queryText, start, end);
               ForwardQueryResult<?> fqr = (ForwardQueryResult<?>) qResult;
               Iterator<?> iter = fqr.iterator();
               while (iter.hasNext()) {
-                if (extractSqlLong(iter.next()) == inputColName.size()) {
+                if (MetastoreDirectSqlUtils.extractSqlLong(iter.next()) == inputColName.size()) {
                   partsFound++;
                 }
               }
@@ -1587,8 +1379,8 @@ class MetaStoreDirectSql {
         return colStatsForDB;
       }
       end = doTrace ? System.nanoTime() : 0;
-      timingTrace(doTrace, queryText, start, end);
-      List<Object[]> list = ensureList(qResult);
+      MetastoreDirectSqlUtils.timingTrace(doTrace, queryText, start, end);
+      List<Object[]> list = MetastoreDirectSqlUtils.ensureList(qResult);
       for (Object[] row : list) {
         String tblName = (String) row[0];
         String partName = (String) row[1];
@@ -1677,8 +1469,8 @@ class MetaStoreDirectSql {
         return Collections.emptyList();
       }
       end = doTrace ? System.nanoTime() : 0;
-      timingTrace(doTrace, queryText, start, end);
-      List<Object[]> list = ensureList(qResult);
+      MetastoreDirectSqlUtils.timingTrace(doTrace, queryText, start, end);
+      List<Object[]> list = MetastoreDirectSqlUtils.ensureList(qResult);
       List<ColumnStatisticsObj> colStats = new ArrayList<ColumnStatisticsObj>(list.size());
       for (Object[] row : list) {
         colStats.add(prepareCSObjWithAdjustedNDV(row, 0, useDensityFunctionForNDVEstimation, ndvTuner));
@@ -1702,14 +1494,14 @@ class MetaStoreDirectSql {
       qResult = executeWithArray(query, prepareParams(catName, dbName, tableName, partNames, colNames),
           queryText);
       end = doTrace ? System.nanoTime() : 0;
-      timingTrace(doTrace, queryText, start, end);
+      MetastoreDirectSqlUtils.timingTrace(doTrace, queryText, start, end);
       if (qResult == null) {
         query.closeAll();
         return Collections.emptyList();
       }
       List<String> noExtraColumnNames = new ArrayList<String>();
       Map<String, String[]> extraColumnNameTypeParts = new HashMap<String, String[]>();
-      List<Object[]> list = ensureList(qResult);
+      List<Object[]> list = MetastoreDirectSqlUtils.ensureList(qResult);
       for (Object[] row : list) {
         String colName = (String) row[0];
         String colType = (String) row[1];
@@ -1717,7 +1509,7 @@ class MetaStoreDirectSql {
         // count(\"PARTITION_NAME\")==partNames.size()
         // Or, extrapolation is not possible for this column if
         // count(\"PARTITION_NAME\")<2
-        Long count = extractSqlLong(row[2]);
+        Long count = MetastoreDirectSqlUtils.extractSqlLong(row[2]);
         if (count == partNames.size() || count < 2) {
           noExtraColumnNames.add(colName);
         } else {
@@ -1739,13 +1531,13 @@ class MetaStoreDirectSql {
           query.closeAll();
           return Collections.emptyList();
         }
-        list = ensureList(qResult);
+        list = MetastoreDirectSqlUtils.ensureList(qResult);
         for (Object[] row : list) {
           colStats.add(prepareCSObjWithAdjustedNDV(row, 0, useDensityFunctionForNDVEstimation, ndvTuner));
           Deadline.checkTimeout();
         }
         end = doTrace ? System.nanoTime() : 0;
-        timingTrace(doTrace, queryText, start, end);
+        MetastoreDirectSqlUtils.timingTrace(doTrace, queryText, start, end);
         query.closeAll();
       }
       // Extrapolation is needed for extraColumnNames.
@@ -1772,7 +1564,7 @@ class MetaStoreDirectSql {
           query.closeAll();
           return Collections.emptyList();
         }
-        list = ensureList(qResult);
+        list = MetastoreDirectSqlUtils.ensureList(qResult);
         // see the indexes for colstats in IExtrapolatePartStatus
         Integer[] sumIndex = new Integer[] { 6, 10, 11, 15 };
         for (Object[] row : list) {
@@ -1785,7 +1577,7 @@ class MetaStoreDirectSql {
           Deadline.checkTimeout();
         }
         end = doTrace ? System.nanoTime() : 0;
-        timingTrace(doTrace, queryText, start, end);
+        MetastoreDirectSqlUtils.timingTrace(doTrace, queryText, start, end);
         query.closeAll();
         for (Map.Entry<String, String[]> entry : extraColumnNameTypeParts.entrySet()) {
           Object[] row = new Object[IExtrapolatePartStatus.colStatNames.length + 2];
@@ -1821,7 +1613,7 @@ class MetaStoreDirectSql {
               if (o == null) {
                 row[2 + colStatIndex] = null;
               } else {
-                Long val = extractSqlLong(o);
+                Long val = MetastoreDirectSqlUtils.extractSqlLong(o);
                 row[2 + colStatIndex] = val / sumVal * (partNames.size());
               }
             } else if (IExtrapolatePartStatus.aggrTypes[colStatIndex] == IExtrapolatePartStatus.AggrType.Min
@@ -1853,7 +1645,7 @@ class MetaStoreDirectSql {
               Object[] min = (Object[]) (fqr.get(0));
               Object[] max = (Object[]) (fqr.get(fqr.size() - 1));
               end = doTrace ? System.nanoTime() : 0;
-              timingTrace(doTrace, queryText, start, end);
+              MetastoreDirectSqlUtils.timingTrace(doTrace, queryText, start, end);
               query.closeAll();
               if (min[0] == null || max[0] == null) {
                 row[2 + colStatIndex] = null;
@@ -1884,7 +1676,7 @@ class MetaStoreDirectSql {
               // "AVG_DECIMAL"
               row[2 + colStatIndex] = avg[colStatIndex - 12];
               end = doTrace ? System.nanoTime() : 0;
-              timingTrace(doTrace, queryText, start, end);
+              MetastoreDirectSqlUtils.timingTrace(doTrace, queryText, start, end);
               query.closeAll();
             }
           }
@@ -1959,13 +1751,13 @@ class MetaStoreDirectSql {
             Query query = pm.newQuery("javax.jdo.query.SQL", queryText);
             Object qResult = executeWithArray(query, prepareParams(
                 catName, dbName, tableName, inputPartNames, inputColNames), queryText);
-            timingTrace(doTrace, queryText0, start, (doTrace ? System.nanoTime() : 0));
+            MetastoreDirectSqlUtils.timingTrace(doTrace, queryText0, start, (doTrace ? System.nanoTime() : 0));
             if (qResult == null) {
               query.closeAll();
               return Collections.emptyList();
             }
             addQueryAfterUse(query);
-            return ensureList(qResult);
+            return MetastoreDirectSqlUtils.ensureList(qResult);
           }
         };
         try {
@@ -2018,8 +1810,9 @@ class MetaStoreDirectSql {
       // LastAnalyzed is stored per column but thrift has it per several;
       // get the lowest for now as nobody actually uses this field.
       Object laObj = row[offset + 15];
-      if (laObj != null && (!csd.isSetLastAnalyzed() || csd.getLastAnalyzed() > extractSqlLong(laObj))) {
-        csd.setLastAnalyzed(extractSqlLong(laObj));
+      if (laObj != null && (!csd.isSetLastAnalyzed() || csd.getLastAnalyzed() > MetastoreDirectSqlUtils
+          .extractSqlLong(laObj))) {
+        csd.setLastAnalyzed(MetastoreDirectSqlUtils.extractSqlLong(laObj));
       }
       csos.add(prepareCSObj(row, offset));
       Deadline.checkTimeout();
@@ -2028,14 +1821,6 @@ class MetaStoreDirectSql {
     return result;
   }
 
-  @SuppressWarnings("unchecked")
-  private List<Object[]> ensureList(Object result) throws MetaException {
-    if (!(result instanceof List<?>)) {
-      throw new MetaException("Wrong result type " + result.getClass());
-    }
-    return (List<Object[]>)result;
-  }
-
   private String makeParams(int size) {
     // W/ size 0, query will fail, but at least we'd get to see the query in debug output.
     return (size == 0) ? "" : repeat(",?", size).substring(1);
@@ -2043,21 +1828,7 @@ class MetaStoreDirectSql {
 
   @SuppressWarnings("unchecked")
   private <T> T executeWithArray(Query query, Object[] params, String sql) throws MetaException {
-    try {
-      return (T)((params == null) ? query.execute() : query.executeWithArray(params));
-    } catch (Exception ex) {
-      String error = "Failed to execute [" + sql + "] with parameters [";
-      if (params != null) {
-        boolean isFirst = true;
-        for (Object param : params) {
-          error += (isFirst ? "" : ", ") + param;
-          isFirst = false;
-        }
-      }
-      LOG.warn(error + "]", ex);
-      // We just logged an exception with (in case of JDO) a humongous callstack. Make a new one.
-      throw new MetaException("See previous errors; " + ex.getMessage());
-    }
+    return MetastoreDirectSqlUtils.executeWithArray(query, params, sql);
   }
 
   /**
@@ -2135,27 +1906,27 @@ class MetaStoreDirectSql {
     }
 
     Query queryParams = pm.newQuery("javax.jdo.query.SQL", queryText);
-      List<Object[]> sqlResult = ensureList(executeWithArray(
+      List<Object[]> sqlResult = MetastoreDirectSqlUtils.ensureList(executeWithArray(
         queryParams, pms.toArray(), queryText));
 
     if (!sqlResult.isEmpty()) {
       for (Object[] line : sqlResult) {
-        int enableValidateRely = extractSqlInt(line[11]);
+        int enableValidateRely = MetastoreDirectSqlUtils.extractSqlInt(line[11]);
         boolean enable = (enableValidateRely & 4) != 0;
         boolean validate = (enableValidateRely & 2) != 0;
         boolean rely = (enableValidateRely & 1) != 0;
         SQLForeignKey currKey = new SQLForeignKey(
-          extractSqlString(line[0]),
-          extractSqlString(line[1]),
-          extractSqlString(line[2]),
-          extractSqlString(line[3]),
-          extractSqlString(line[4]),
-          extractSqlString(line[5]),
-          extractSqlInt(line[6]),
-          extractSqlInt(line[7]),
-          extractSqlInt(line[8]),
-          extractSqlString(line[9]),
-          extractSqlString(line[10]),
+          MetastoreDirectSqlUtils.extractSqlString(line[0]),
+          MetastoreDirectSqlUtils.extractSqlString(line[1]),
+          MetastoreDirectSqlUtils.extractSqlString(line[2]),
+          MetastoreDirectSqlUtils.extractSqlString(line[3]),
+          MetastoreDirectSqlUtils.extractSqlString(line[4]),
+          MetastoreDirectSqlUtils.extractSqlString(line[5]),
+          MetastoreDirectSqlUtils.extractSqlInt(line[6]),
+          MetastoreDirectSqlUtils.extractSqlInt(line[7]),
+          MetastoreDirectSqlUtils.extractSqlInt(line[8]),
+          MetastoreDirectSqlUtils.extractSqlString(line[9]),
+          MetastoreDirectSqlUtils.extractSqlString(line[10]),
           enable,
           validate,
           rely
@@ -2202,24 +1973,24 @@ class MetaStoreDirectSql {
     }
 
     Query queryParams = pm.newQuery("javax.jdo.query.SQL", queryText);
-      List<Object[]> sqlResult = ensureList(executeWithArray(
+      List<Object[]> sqlResult = MetastoreDirectSqlUtils.ensureList(executeWithArray(
         queryParams, pms.toArray(), queryText));
 
     if (!sqlResult.isEmpty()) {
       for (Object[] line : sqlResult) {
-          int enableValidateRely = extractSqlInt(line[5]);
+          int enableValidateRely = MetastoreDirectSqlUtils.extractSqlInt(line[5]);
           boolean enable = (enableValidateRely & 4) != 0;
           boolean validate = (enableValidateRely & 2) != 0;
           boolean rely = (enableValidateRely & 1) != 0;
         SQLPrimaryKey currKey = new SQLPrimaryKey(
-          extractSqlString(line[0]),
-          extractSqlString(line[1]),
-          extractSqlString(line[2]),
-          extractSqlInt(line[3]), extractSqlString(line[4]),
+          MetastoreDirectSqlUtils.extractSqlString(line[0]),
+          MetastoreDirectSqlUtils.extractSqlString(line[1]),
+          MetastoreDirectSqlUtils.extractSqlString(line[2]),
+          MetastoreDirectSqlUtils.extractSqlInt(line[3]), MetastoreDirectSqlUtils.extractSqlString(line[4]),
           enable,
           validate,
           rely);
-        currKey.setCatName(extractSqlString(line[6]));
+        currKey.setCatName(MetastoreDirectSqlUtils.extractSqlString(line[6]));
         ret.add(currKey);
       }
     }
@@ -2260,21 +2031,21 @@ class MetaStoreDirectSql {
     }
 
     Query queryParams = pm.newQuery("javax.jdo.query.SQL", queryText);
-      List<Object[]> sqlResult = ensureList(executeWithArray(
+      List<Object[]> sqlResult = MetastoreDirectSqlUtils.ensureList(executeWithArray(
         queryParams, pms.toArray(), queryText));
 
     if (!sqlResult.isEmpty()) {
       for (Object[] line : sqlResult) {
-          int enableValidateRely = extractSqlInt(line[5]);
+          int enableValidateRely = MetastoreDirectSqlUtils.extractSqlInt(line[5]);
           boolean enable = (enableValidateRely & 4) != 0;
           boolean validate = (enableValidateRely & 2) != 0;
           boolean rely = (enableValidateRely & 1) != 0;
         ret.add(new SQLUniqueConstraint(
             catName,
-            extractSqlString(line[0]),
-            extractSqlString(line[1]),
-            extractSqlString(line[2]),
-            extractSqlInt(line[3]), extractSqlString(line[4]),
+            MetastoreDirectSqlUtils.extractSqlString(line[0]),
+            MetastoreDirectSqlUtils.extractSqlString(line[1]),
+            MetastoreDirectSqlUtils.extractSqlString(line[2]),
+            MetastoreDirectSqlUtils.extractSqlInt(line[3]), MetastoreDirectSqlUtils.extractSqlString(line[4]),
             enable,
             validate,
             rely));
@@ -2317,21 +2088,21 @@ class MetaStoreDirectSql {
     }
 
     Query queryParams = pm.newQuery("javax.jdo.query.SQL", queryText);
-      List<Object[]> sqlResult = ensureList(executeWithArray(
+      List<Object[]> sqlResult = MetastoreDirectSqlUtils.ensureList(executeWithArray(
         queryParams, pms.toArray(), queryText));
 
     if (!sqlResult.isEmpty()) {
       for (Object[] line : sqlResult) {
-          int enableValidateRely = extractSqlInt(line[4]);
+          int enableValidateRely = MetastoreDirectSqlUtils.extractSqlInt(line[4]);
           boolean enable = (enableValidateRely & 4) != 0;
           boolean validate = (enableValidateRely & 2) != 0;
           boolean rely = (enableValidateRely & 1) != 0;
         ret.add(new SQLNotNullConstraint(
             catName,
-            extractSqlString(line[0]),
-            extractSqlString(line[1]),
-            extractSqlString(line[2]),
-            extractSqlString(line[3]),
+            MetastoreDirectSqlUtils.extractSqlString(line[0]),
+            MetastoreDirectSqlUtils.extractSqlString(line[1]),
+            MetastoreDirectSqlUtils.extractSqlString(line[2]),
+            MetastoreDirectSqlUtils.extractSqlString(line[3]),
             enable,
             validate,
             rely));
@@ -2378,22 +2149,22 @@ class MetaStoreDirectSql {
     }
 
     Query queryParams = pm.newQuery("javax.jdo.query.SQL", queryText);
-    List<Object[]> sqlResult = ensureList(executeWithArray(
+    List<Object[]> sqlResult = MetastoreDirectSqlUtils.ensureList(executeWithArray(
         queryParams, pms.toArray(), queryText));
 
     if (!sqlResult.isEmpty()) {
       for (Object[] line : sqlResult) {
-        int enableValidateRely = extractSqlInt(line[4]);
+        int enableValidateRely = MetastoreDirectSqlUtils.extractSqlInt(line[4]);
         boolean enable = (enableValidateRely & 4) != 0;
         boolean validate = (enableValidateRely & 2) != 0;
         boolean rely = (enableValidateRely & 1) != 0;
         SQLDefaultConstraint currConstraint = new SQLDefaultConstraint(
             catName,
-            extractSqlString(line[0]),
-            extractSqlString(line[1]),
-            extractSqlString(line[2]),
-            extractSqlString(line[5]),
-            extractSqlString(line[3]),
+            MetastoreDirectSqlUtils.extractSqlString(line[0]),
+            MetastoreDirectSqlUtils.extractSqlString(line[1]),
+            MetastoreDirectSqlUtils.extractSqlString(line[2]),
+            MetastoreDirectSqlUtils.extractSqlString(line[5]),
+            MetastoreDirectSqlUtils.extractSqlString(line[3]),
             enable,
             validate,
             rely);
@@ -2441,22 +2212,22 @@ class MetaStoreDirectSql {
     }
 
     Query queryParams = pm.newQuery("javax.jdo.query.SQL", queryText);
-    List<Object[]> sqlResult = ensureList(executeWithArray(
+    List<Object[]> sqlResult = MetastoreDirectSqlUtils.ensureList(executeWithArray(
         queryParams, pms.toArray(), queryText));
 
     if (!sqlResult.isEmpty()) {
       for (Object[] line : sqlResult) {
-        int enableValidateRely = extractSqlInt(line[4]);
+        int enableValidateRely = MetastoreDirectSqlUtils.extractSqlInt(line[4]);
         boolean enable = (enableValidateRely & 4) != 0;
         boolean validate = (enableValidateRely & 2) != 0;
         boolean rely = (enableValidateRely & 1) != 0;
         SQLCheckConstraint currConstraint = new SQLCheckConstraint(
             catName,
-            extractSqlString(line[0]),
-            extractSqlString(line[1]),
-            extractSqlString(line[2]),
-            extractSqlString(line[5]),
-            extractSqlString(line[3]),
+            MetastoreDirectSqlUtils.extractSqlString(line[0]),
+            MetastoreDirectSqlUtils.extractSqlString(line[1]),
+            MetastoreDirectSqlUtils.extractSqlString(line[2]),
+            MetastoreDirectSqlUtils.extractSqlString(line[5]),
+            MetastoreDirectSqlUtils.extractSqlString(line[3]),
             enable,
             validate,
             rely);
@@ -2486,7 +2257,7 @@ class MetaStoreDirectSql {
       public List<Void> run(List<String> input) throws MetaException {
         String filter = "" + PARTITIONS + ".\"PART_NAME\" in (" + makeParams(input.size()) + ")";
         // Get partition ids
-        List<Object> partitionIds = getPartitionIdsViaSqlFilter(catName, dbName, tblName,
+        List<Long> partitionIds = getPartitionIdsViaSqlFilter(catName, dbName, tblName,
             filter, input, Collections.<String>emptyList(), null);
         if (partitionIds.isEmpty()) {
           return Collections.emptyList(); // no partitions, bail early.
@@ -2504,11 +2275,12 @@ class MetaStoreDirectSql {
    * @throws MetaException If there is an SQL exception during the execution it converted to
    * MetaException
    */
-  private void dropPartitionsByPartitionIds(List<Object> partitionIdList) throws MetaException {
+  private void dropPartitionsByPartitionIds(List<Long> partitionIdList) throws MetaException {
     String queryText;
     if (partitionIdList.isEmpty()) {
       return;
     }
+
     String partitionIds = getIdListForIn(partitionIdList);
 
     // Get the corresponding SD_ID-s, CD_ID-s, SERDE_ID-s
@@ -2519,7 +2291,8 @@ class MetaStoreDirectSql {
             + "WHERE " + PARTITIONS + ".\"PART_ID\" in (" + partitionIds + ")";
 
     Query query = pm.newQuery("javax.jdo.query.SQL", queryText);
-    List<Object[]> sqlResult = ensureList(executeWithArray(query, null, queryText));
+    List<Object[]> sqlResult = MetastoreDirectSqlUtils
+        .ensureList(executeWithArray(query, null, queryText));
 
     List<Object> sdIdList = new ArrayList<>(partitionIdList.size());
     List<Object> columnDescriptorIdList = new ArrayList<>(1);
@@ -2527,12 +2300,12 @@ class MetaStoreDirectSql {
 
     if (!sqlResult.isEmpty()) {
       for (Object[] fields : sqlResult) {
-        sdIdList.add(extractSqlLong(fields[0]));
-        Long colId = extractSqlLong(fields[1]);
+        sdIdList.add(MetastoreDirectSqlUtils.extractSqlLong(fields[0]));
+        Long colId = MetastoreDirectSqlUtils.extractSqlLong(fields[1]);
         if (!columnDescriptorIdList.contains(colId)) {
           columnDescriptorIdList.add(colId);
         }
-        serdeIdList.add(extractSqlLong(fields[2]));
+        serdeIdList.add(MetastoreDirectSqlUtils.extractSqlLong(fields[2]));
       }
     }
     query.closeAll();
@@ -2602,13 +2375,14 @@ class MetaStoreDirectSql {
             + "WHERE " + SKEWED_VALUES + ".\"SD_ID_OID\" in  (" + sdIds + ")";
 
     Query query = pm.newQuery("javax.jdo.query.SQL", queryText);
-    List<Object[]> sqlResult = ensureList(executeWithArray(query, null, queryText));
+    List<Object[]> sqlResult = MetastoreDirectSqlUtils
+        .ensureList(executeWithArray(query, null, queryText));
 
     List<Object> skewedStringListIdList = new ArrayList<>(0);
 
     if (!sqlResult.isEmpty()) {
       for (Object[] fields : sqlResult) {
-        skewedStringListIdList.add(extractSqlLong(fields[0]));
+        skewedStringListIdList.add(MetastoreDirectSqlUtils.extractSqlLong(fields[0]));
       }
     }
     query.closeAll();
@@ -2721,13 +2495,14 @@ class MetaStoreDirectSql {
             + "WHERE " + SDS + ".\"CD_ID\" in (" + colIds + ") "
             + "GROUP BY " + SDS + ".\"CD_ID\"";
     Query query = pm.newQuery("javax.jdo.query.SQL", queryText);
-    List<Object[]> sqlResult = ensureList(executeWithArray(query, null, queryText));
+    List<Object[]> sqlResult = MetastoreDirectSqlUtils
+        .ensureList(executeWithArray(query, null, queryText));
 
     List<Object> danglingColumnDescriptorIdList = new ArrayList<>(columnDescriptorIdList.size());
     if (!sqlResult.isEmpty()) {
       for (Object[] fields : sqlResult) {
-        if (extractSqlInt(fields[1]) == 0) {
-          danglingColumnDescriptorIdList.add(extractSqlLong(fields[0]));
+        if (MetastoreDirectSqlUtils.extractSqlInt(fields[1]) == 0) {
+          danglingColumnDescriptorIdList.add(MetastoreDirectSqlUtils.extractSqlLong(fields[0]));
         }
       }
     }
@@ -2792,14 +2567,14 @@ class MetaStoreDirectSql {
     LOG.debug("Running {}", queryText);
     Query<?> query = pm.newQuery("javax.jdo.query.SQL", queryText);
     try {
-      List<Object[]> sqlResult = ensureList(executeWithArray(
+      List<Object[]> sqlResult = MetastoreDirectSqlUtils.ensureList(executeWithArray(
           query, new Object[] { dbName, catName, tableName }, queryText));
       Map<String, List<String>> result = new HashMap<>();
       String lastPartName = null;
       List<String> cols = null;
       for (Object[] line : sqlResult) {
-        String col = extractSqlString(line[1]);
-        String part = extractSqlString(line[0]);
+        String col = MetastoreDirectSqlUtils.extractSqlString(line[1]);
+        String part = MetastoreDirectSqlUtils.extractSqlString(line[0]);
         if (!part.equals(lastPartName)) {
           if (lastPartName != null) {
             result.put(lastPartName, cols);
@@ -2833,10 +2608,12 @@ class MetaStoreDirectSql {
     LOG.debug("Running {}", queryText);
     Query<?> query = pm.newQuery("javax.jdo.query.SQL", queryText);
     try {
-      List<Object[]> sqlResult = ensureList(executeWithArray(query, STATS_TABLE_TYPES, queryText));
+      List<Object[]> sqlResult = MetastoreDirectSqlUtils
+          .ensureList(executeWithArray(query, STATS_TABLE_TYPES, queryText));
       for (Object[] line : sqlResult) {
         result.add(new org.apache.hadoop.hive.common.TableName(
-            extractSqlString(line[2]), extractSqlString(line[1]), extractSqlString(line[0])));
+            MetastoreDirectSqlUtils.extractSqlString(line[2]), MetastoreDirectSqlUtils
+            .extractSqlString(line[1]), MetastoreDirectSqlUtils.extractSqlString(line[0])));
       }
     } finally {
       query.closeAll();


[04/13] hive git commit: HIVE-20306 : Implement projection spec for fetching only requested fields from partitions (Vihang Karajgaonkar, reviewed by Aihua Xu, Andrew Sherman and Alexander Kolbasov)

Posted by vi...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/44ef91a6/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetastoreDirectSqlUtils.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetastoreDirectSqlUtils.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetastoreDirectSqlUtils.java
new file mode 100644
index 0000000..3905b9e
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetastoreDirectSqlUtils.java
@@ -0,0 +1,571 @@
+/*
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.hive.metastore;
+
+import com.google.common.base.Joiner;
+import org.apache.commons.lang.BooleanUtils;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Order;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.SerDeInfo;
+import org.apache.hadoop.hive.metastore.api.SkewedInfo;
+import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreServerUtils;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.jdo.PersistenceManager;
+import javax.jdo.Query;
+import java.sql.Blob;
+import java.sql.Clob;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.stream.Collectors;
+
+/**
+ * Helper utilities used by DirectSQL code in HiveMetastore.
+ */
+class MetastoreDirectSqlUtils {
+  private static final Logger LOG = LoggerFactory.getLogger(MetastoreDirectSqlUtils.class);
+  private MetastoreDirectSqlUtils() {
+
+  }
+  @SuppressWarnings("unchecked")
+  static <T> T executeWithArray(Query query, Object[] params, String sql) throws MetaException {
+    try {
+      return (T)((params == null) ? query.execute() : query.executeWithArray(params));
+    } catch (Exception ex) {
+      StringBuilder errorBuilder = new StringBuilder("Failed to execute [" + sql + "] with parameters [");
+      if (params != null) {
+        boolean isFirst = true;
+        for (Object param : params) {
+          errorBuilder.append((isFirst ? "" : ", ") + param);
+          isFirst = false;
+        }
+      }
+      LOG.warn(errorBuilder.toString() + "]", ex);
+      // We just logged an exception with (in case of JDO) a humongous callstack. Make a new one.
+      throw new MetaException("See previous errors; " + ex.getMessage());
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  static List<Object[]> ensureList(Object result) throws MetaException {
+    if (!(result instanceof List<?>)) {
+      throw new MetaException("Wrong result type " + result.getClass());
+    }
+    return (List<Object[]>)result;
+  }
+
+  static Long extractSqlLong(Object obj) throws MetaException {
+    if (obj == null) return null;
+    if (!(obj instanceof Number)) {
+      throw new MetaException("Expected numeric type but got " + obj.getClass().getName());
+    }
+    return ((Number)obj).longValue();
+  }
+
+  static void timingTrace(boolean doTrace, String queryText, long start, long queryTime) {
+    if (!doTrace) return;
+    LOG.debug("Direct SQL query in " + (queryTime - start) / 1000000.0 + "ms + " +
+        (System.nanoTime() - queryTime) / 1000000.0 + "ms, the query is [" + queryText + "]");
+  }
+
+  static <T> int loopJoinOrderedResult(PersistenceManager pm, TreeMap<Long, T> tree,
+      String queryText, int keyIndex, ApplyFunc<T> func) throws MetaException {
+    return loopJoinOrderedResult(pm, tree, queryText, null, keyIndex, func);
+  }
+  /**
+   * Merges applies the result of a PM SQL query into a tree of object.
+   * Essentially it's an object join. DN could do this for us, but it issues queries
+   * separately for every object, which is suboptimal.
+   * @param pm
+   * @param tree The object tree, by ID.
+   * @param queryText The query text.
+   * @param keyIndex Index of the Long column corresponding to the map ID in query result rows.
+   * @param func The function that is called on each (object,row) pair with the same id.
+   * @return the count of results returned from the query.
+   */
+  static <T> int loopJoinOrderedResult(PersistenceManager pm, TreeMap<Long, T> tree,
+      String queryText, Object[] parameters, int keyIndex, ApplyFunc<T> func) throws MetaException {
+    boolean doTrace = LOG.isDebugEnabled();
+    long start = doTrace ? System.nanoTime() : 0;
+    Query query = pm.newQuery("javax.jdo.query.SQL", queryText);
+    Object result = null;
+    if (parameters == null || parameters.length == 0) {
+      result = query.execute();
+    } else {
+      result = query.executeWithArray(parameters);
+    }
+    long queryTime = doTrace ? System.nanoTime() : 0;
+    if (result == null) {
+      query.closeAll();
+      return 0;
+    }
+    List<Object[]> list = ensureList(result);
+    Iterator<Object[]> iter = list.iterator();
+    Object[] fields = null;
+    for (Map.Entry<Long, T> entry : tree.entrySet()) {
+      if (fields == null && !iter.hasNext()) break;
+      long id = entry.getKey();
+      while (fields != null || iter.hasNext()) {
+        if (fields == null) {
+          fields = iter.next();
+        }
+        long nestedId = extractSqlLong(fields[keyIndex]);
+        if (nestedId < id) throw new MetaException("Found entries for unknown ID " + nestedId);
+        if (nestedId > id) break; // fields belong to one of the next entries
+        func.apply(entry.getValue(), fields);
+        fields = null;
+      }
+      Deadline.checkTimeout();
+    }
+    int rv = list.size();
+    query.closeAll();
+    timingTrace(doTrace, queryText, start, queryTime);
+    return rv;
+  }
+
+  static void setPartitionParameters(String PARTITION_PARAMS, boolean convertMapNullsToEmptyStrings,
+      PersistenceManager pm, String partIds, TreeMap<Long, Partition> partitions)
+      throws MetaException {
+    String queryText;
+    queryText = "select \"PART_ID\", \"PARAM_KEY\", \"PARAM_VALUE\" from " + PARTITION_PARAMS + ""
+        + " where \"PART_ID\" in (" + partIds + ") and \"PARAM_KEY\" is not null"
+        + " order by \"PART_ID\" asc";
+    loopJoinOrderedResult(pm, partitions, queryText, 0, new ApplyFunc<Partition>() {
+      @Override
+      public void apply(Partition t, Object[] fields) {
+        t.putToParameters((String)fields[1], (String)fields[2]);
+      }});
+    // Perform conversion of null map values
+    for (Partition t : partitions.values()) {
+      t.setParameters(MetaStoreServerUtils.trimMapNulls(t.getParameters(), convertMapNullsToEmptyStrings));
+    }
+  }
+
+  static void setPartitionParametersWithFilter(String PARTITION_PARAMS,
+      boolean convertMapNullsToEmptyStrings, PersistenceManager pm, String partIds,
+      TreeMap<Long, Partition> partitions, String includeParamKeyPattern, String excludeParamKeyPattern)
+      throws MetaException {
+    StringBuilder queryTextBuilder = new StringBuilder("select \"PART_ID\", \"PARAM_KEY\", \"PARAM_VALUE\" from ")
+        .append(PARTITION_PARAMS)
+        .append(" where \"PART_ID\" in (")
+        .append(partIds)
+        .append(") and \"PARAM_KEY\" is not null");
+    List<Object> queryParams = new ArrayList<>(2);;
+    if (includeParamKeyPattern != null && !includeParamKeyPattern.isEmpty()) {
+      queryTextBuilder.append(" and \"PARAM_KEY\" LIKE (?)");
+      queryParams.add(includeParamKeyPattern);
+    }
+    if (excludeParamKeyPattern != null && !excludeParamKeyPattern.isEmpty()) {
+      queryTextBuilder.append(" and \"PARAM_KEY\" NOT LIKE (?)");
+      queryParams.add(excludeParamKeyPattern);
+    }
+
+    queryTextBuilder.append(" order by \"PART_ID\" asc");
+    String queryText = queryTextBuilder.toString();
+    loopJoinOrderedResult(pm, partitions, queryText, queryParams.toArray(), 0, new ApplyFunc<Partition>() {
+      @Override
+      public void apply(Partition t, Object[] fields) {
+        t.putToParameters((String) fields[1], (String) fields[2]);
+      }
+    });
+    // Perform conversion of null map values
+    for (Partition t : partitions.values()) {
+      t.setParameters(MetaStoreServerUtils.trimMapNulls(t.getParameters(), convertMapNullsToEmptyStrings));
+    }
+  }
+
+  static void setPartitionValues(String PARTITION_KEY_VALS, PersistenceManager pm, String partIds,
+      TreeMap<Long, Partition> partitions)
+      throws MetaException {
+    String queryText;
+    queryText = "select \"PART_ID\", \"PART_KEY_VAL\" from " + PARTITION_KEY_VALS + ""
+        + " where \"PART_ID\" in (" + partIds + ")"
+        + " order by \"PART_ID\" asc, \"INTEGER_IDX\" asc";
+    loopJoinOrderedResult(pm, partitions, queryText, 0, new ApplyFunc<Partition>() {
+      @Override
+      public void apply(Partition t, Object[] fields) {
+        t.addToValues((String)fields[1]);
+      }});
+  }
+
+  static String extractSqlClob(Object value) {
+    if (value == null) return null;
+    try {
+      if (value instanceof Clob) {
+        // we trim the Clob value to a max length an int can hold
+        int maxLength = (((Clob)value).length() < Integer.MAX_VALUE - 2) ? (int)((Clob)value).length() : Integer.MAX_VALUE - 2;
+        return ((Clob)value).getSubString(1L, maxLength);
+      } else {
+        return value.toString();
+      }
+    } catch (SQLException sqle) {
+      return null;
+    }
+  }
+
+  static void setSDParameters(String SD_PARAMS, boolean convertMapNullsToEmptyStrings,
+      PersistenceManager pm, TreeMap<Long, StorageDescriptor> sds, String sdIds)
+      throws MetaException {
+    String queryText;
+    queryText = "select \"SD_ID\", \"PARAM_KEY\", \"PARAM_VALUE\" from " + SD_PARAMS + ""
+        + " where \"SD_ID\" in (" + sdIds + ") and \"PARAM_KEY\" is not null"
+        + " order by \"SD_ID\" asc";
+    loopJoinOrderedResult(pm, sds, queryText, 0, new ApplyFunc<StorageDescriptor>() {
+      @Override
+      public void apply(StorageDescriptor t, Object[] fields) {
+        t.putToParameters((String)fields[1], extractSqlClob(fields[2]));
+      }});
+    // Perform conversion of null map values
+    for (StorageDescriptor t : sds.values()) {
+      t.setParameters(MetaStoreServerUtils.trimMapNulls(t.getParameters(), convertMapNullsToEmptyStrings));
+    }
+  }
+
+  static int extractSqlInt(Object field) {
+    return ((Number)field).intValue();
+  }
+
+  static void setSDSortCols(String SORT_COLS, List<String> columnNames, PersistenceManager pm,
+      TreeMap<Long, StorageDescriptor> sds, String sdIds)
+      throws MetaException {
+    StringBuilder queryTextBuilder = new StringBuilder("select \"SD_ID\"");
+    int counter = 0;
+    if (columnNames.contains("col")) {
+      counter++;
+      queryTextBuilder.append(", \"COLUMN_NAME\"");
+    }
+    if (columnNames.contains("order")) {
+      counter++;
+      queryTextBuilder.append(", \"ORDER\"");
+    }
+    queryTextBuilder
+        .append(" from ")
+        .append(SORT_COLS)
+        .append(" where \"SD_ID\" in (")
+        .append(sdIds)
+        .append(") order by \"SD_ID\" asc, \"INTEGER_IDX\" asc");
+    String queryText = queryTextBuilder.toString();
+    final int finalCounter = counter;
+    loopJoinOrderedResult(pm, sds, queryText, 0, new ApplyFunc<StorageDescriptor>() {
+      @Override
+      public void apply(StorageDescriptor t, Object[] fields) {
+        if (finalCounter > 1 && fields[2] == null) {
+          return;
+        }
+        Order order = new Order();
+        if (finalCounter > 0) {
+          order.setCol((String) fields[1]);
+        }
+        if (finalCounter > 1) {
+          order.setOrder(extractSqlInt(fields[2]));
+        }
+        t.addToSortCols(order);
+      }});
+  }
+
+  static void setSDSortCols(String SORT_COLS, PersistenceManager pm,
+      TreeMap<Long, StorageDescriptor> sds, String sdIds)
+      throws MetaException {
+    String queryText;
+    queryText = "select \"SD_ID\", \"COLUMN_NAME\", " + SORT_COLS + ".\"ORDER\""
+        + " from " + SORT_COLS + ""
+        + " where \"SD_ID\" in (" + sdIds + ")"
+        + " order by \"SD_ID\" asc, \"INTEGER_IDX\" asc";
+    loopJoinOrderedResult(pm, sds, queryText, 0, new ApplyFunc<StorageDescriptor>() {
+      @Override
+      public void apply(StorageDescriptor t, Object[] fields) {
+        if (fields[2] == null) return;
+        t.addToSortCols(new Order((String)fields[1], extractSqlInt(fields[2])));
+      }});
+  }
+
+  static void setSDBucketCols(String BUCKETING_COLS, PersistenceManager pm,
+      TreeMap<Long, StorageDescriptor> sds, String sdIds)
+      throws MetaException {
+    String queryText;
+    queryText = "select \"SD_ID\", \"BUCKET_COL_NAME\" from " + BUCKETING_COLS + ""
+        + " where \"SD_ID\" in (" + sdIds + ")"
+        + " order by \"SD_ID\" asc, \"INTEGER_IDX\" asc";
+    loopJoinOrderedResult(pm, sds, queryText, 0, new ApplyFunc<StorageDescriptor>() {
+      @Override
+      public void apply(StorageDescriptor t, Object[] fields) {
+        t.addToBucketCols((String)fields[1]);
+      }});
+  }
+
+  static boolean setSkewedColNames(String SKEWED_COL_NAMES, PersistenceManager pm,
+      TreeMap<Long, StorageDescriptor> sds, String sdIds)
+      throws MetaException {
+    String queryText;
+    queryText = "select \"SD_ID\", \"SKEWED_COL_NAME\" from " + SKEWED_COL_NAMES + ""
+        + " where \"SD_ID\" in (" + sdIds + ")"
+        + " order by \"SD_ID\" asc, \"INTEGER_IDX\" asc";
+    return loopJoinOrderedResult(pm, sds, queryText, 0, new ApplyFunc<StorageDescriptor>() {
+      @Override
+      public void apply(StorageDescriptor t, Object[] fields) {
+        if (!t.isSetSkewedInfo()) t.setSkewedInfo(new SkewedInfo());
+        t.getSkewedInfo().addToSkewedColNames((String)fields[1]);
+      }}) > 0;
+  }
+
+  static void setSkewedColValues(String SKEWED_STRING_LIST_VALUES, String SKEWED_VALUES,
+      PersistenceManager pm, TreeMap<Long, StorageDescriptor> sds, String sdIds)
+      throws MetaException {
+    String queryText;
+    queryText =
+          "select " + SKEWED_VALUES + ".\"SD_ID_OID\","
+        + "  " + SKEWED_STRING_LIST_VALUES + ".\"STRING_LIST_ID\","
+        + "  " + SKEWED_STRING_LIST_VALUES + ".\"STRING_LIST_VALUE\" "
+        + "from " + SKEWED_VALUES + " "
+        + "  left outer join " + SKEWED_STRING_LIST_VALUES + " on " + SKEWED_VALUES + "."
+        + "\"STRING_LIST_ID_EID\" = " + SKEWED_STRING_LIST_VALUES + ".\"STRING_LIST_ID\" "
+        + "where " + SKEWED_VALUES + ".\"SD_ID_OID\" in (" + sdIds + ") "
+        + "  and " + SKEWED_VALUES + ".\"STRING_LIST_ID_EID\" is not null "
+        + "  and " + SKEWED_VALUES + ".\"INTEGER_IDX\" >= 0 "
+        + "order by " + SKEWED_VALUES + ".\"SD_ID_OID\" asc, " + SKEWED_VALUES + ".\"INTEGER_IDX\" asc,"
+        + "  " + SKEWED_STRING_LIST_VALUES + ".\"INTEGER_IDX\" asc";
+    loopJoinOrderedResult(pm, sds, queryText, 0, new ApplyFunc<StorageDescriptor>() {
+      private Long currentListId;
+      private List<String> currentList;
+      @Override
+      public void apply(StorageDescriptor t, Object[] fields) throws MetaException {
+        if (!t.isSetSkewedInfo()) t.setSkewedInfo(new SkewedInfo());
+        // Note that this is not a typical list accumulator - there's no call to finalize
+        // the last list. Instead we add list to SD first, as well as locally to add elements.
+        if (fields[1] == null) {
+          currentList = null; // left outer join produced a list with no values
+          currentListId = null;
+          t.getSkewedInfo().addToSkewedColValues(Collections.<String>emptyList());
+        } else {
+          long fieldsListId = extractSqlLong(fields[1]);
+          if (currentListId == null || fieldsListId != currentListId) {
+            currentList = new ArrayList<String>();
+            currentListId = fieldsListId;
+            t.getSkewedInfo().addToSkewedColValues(currentList);
+          }
+          currentList.add((String)fields[2]);
+        }
+      }});
+  }
+
+  static void setSkewedColLocationMaps(String SKEWED_COL_VALUE_LOC_MAP,
+      String SKEWED_STRING_LIST_VALUES, PersistenceManager pm, TreeMap<Long, StorageDescriptor> sds,
+      String sdIds)
+      throws MetaException {
+    String queryText;
+    queryText =
+          "select " + SKEWED_COL_VALUE_LOC_MAP + ".\"SD_ID\","
+        + " " + SKEWED_STRING_LIST_VALUES + ".STRING_LIST_ID,"
+        + " " + SKEWED_COL_VALUE_LOC_MAP + ".\"LOCATION\","
+        + " " + SKEWED_STRING_LIST_VALUES + ".\"STRING_LIST_VALUE\" "
+        + "from " + SKEWED_COL_VALUE_LOC_MAP + ""
+        + "  left outer join " + SKEWED_STRING_LIST_VALUES + " on " + SKEWED_COL_VALUE_LOC_MAP + "."
+        + "\"STRING_LIST_ID_KID\" = " + SKEWED_STRING_LIST_VALUES + ".\"STRING_LIST_ID\" "
+        + "where " + SKEWED_COL_VALUE_LOC_MAP + ".\"SD_ID\" in (" + sdIds + ")"
+        + "  and " + SKEWED_COL_VALUE_LOC_MAP + ".\"STRING_LIST_ID_KID\" is not null "
+        + "order by " + SKEWED_COL_VALUE_LOC_MAP + ".\"SD_ID\" asc,"
+        + "  " + SKEWED_STRING_LIST_VALUES + ".\"STRING_LIST_ID\" asc,"
+        + "  " + SKEWED_STRING_LIST_VALUES + ".\"INTEGER_IDX\" asc";
+
+    loopJoinOrderedResult(pm, sds, queryText, 0, new ApplyFunc<StorageDescriptor>() {
+      private Long currentListId;
+      private List<String> currentList;
+      @Override
+      public void apply(StorageDescriptor t, Object[] fields) throws MetaException {
+        if (!t.isSetSkewedInfo()) {
+          SkewedInfo skewedInfo = new SkewedInfo();
+          skewedInfo.setSkewedColValueLocationMaps(new HashMap<List<String>, String>());
+          t.setSkewedInfo(skewedInfo);
+        }
+        Map<List<String>, String> skewMap = t.getSkewedInfo().getSkewedColValueLocationMaps();
+        // Note that this is not a typical list accumulator - there's no call to finalize
+        // the last list. Instead we add list to SD first, as well as locally to add elements.
+        if (fields[1] == null) {
+          currentList = new ArrayList<String>(); // left outer join produced a list with no values
+          currentListId = null;
+        } else {
+          long fieldsListId = extractSqlLong(fields[1]);
+          if (currentListId == null || fieldsListId != currentListId) {
+            currentList = new ArrayList<String>();
+            currentListId = fieldsListId;
+          } else {
+            skewMap.remove(currentList); // value based compare.. remove first
+          }
+          currentList.add((String)fields[3]);
+        }
+        skewMap.put(currentList, (String)fields[2]);
+      }});
+  }
+
+  static void setSDCols(String COLUMNS_V2, List<String> columnNames, PersistenceManager pm,
+      TreeMap<Long, List<FieldSchema>> colss, String colIds)
+      throws MetaException {
+    StringBuilder queryTextBuilder = new StringBuilder("select \"CD_ID\"");
+    int counter = 0;
+    if (columnNames.contains("name")) {
+      counter++;
+      queryTextBuilder.append(", \"COLUMN_NAME\"");
+    }
+    if (columnNames.contains("type")) {
+      counter++;
+      queryTextBuilder.append(", \"TYPE_NAME\"");
+    }
+    if (columnNames.contains("comment")) {
+      counter++;
+      queryTextBuilder.append(", \"COMMENT\"");
+    }
+    queryTextBuilder
+        .append(" from ")
+        .append(COLUMNS_V2)
+        .append(" where \"CD_ID\" in (")
+        .append(colIds)
+        .append(") order by \"CD_ID\" asc, \"INTEGER_IDX\" asc");
+    String queryText = queryTextBuilder.toString();
+    int finalCounter = counter;
+    loopJoinOrderedResult(pm, colss, queryText, 0, new ApplyFunc<List<FieldSchema>>() {
+      @Override
+      public void apply(List<FieldSchema> t, Object[] fields) {
+        FieldSchema fieldSchema = new FieldSchema();
+        if (finalCounter > 0) {
+          fieldSchema.setName((String) fields[1]);
+        }
+        if (finalCounter > 1) {
+          fieldSchema.setType(extractSqlClob(fields[2]));
+        }
+        if (finalCounter > 2) {
+          fieldSchema.setComment((String) fields[3]);
+        }
+        t.add(fieldSchema);
+      }});
+  }
+
+  static void setSDCols(String COLUMNS_V2, PersistenceManager pm,
+      TreeMap<Long, List<FieldSchema>> colss, String colIds)
+      throws MetaException {
+    String queryText;
+    queryText = "select \"CD_ID\", \"COMMENT\", \"COLUMN_NAME\", \"TYPE_NAME\""
+        + " from " + COLUMNS_V2 + " where \"CD_ID\" in (" + colIds + ")"
+        + " order by \"CD_ID\" asc, \"INTEGER_IDX\" asc";
+    loopJoinOrderedResult(pm, colss, queryText, 0, new ApplyFunc<List<FieldSchema>>() {
+      @Override
+      public void apply(List<FieldSchema> t, Object[] fields) {
+        t.add(new FieldSchema((String)fields[2], extractSqlClob(fields[3]), (String)fields[1]));
+      }});
+  }
+
+  static void setSerdeParams(String SERDE_PARAMS, boolean convertMapNullsToEmptyStrings,
+      PersistenceManager pm, TreeMap<Long, SerDeInfo> serdes, String serdeIds) throws MetaException {
+    String queryText;
+    queryText = "select \"SERDE_ID\", \"PARAM_KEY\", \"PARAM_VALUE\" from " + SERDE_PARAMS + ""
+        + " where \"SERDE_ID\" in (" + serdeIds + ") and \"PARAM_KEY\" is not null"
+        + " order by \"SERDE_ID\" asc";
+    loopJoinOrderedResult(pm, serdes, queryText, 0, new ApplyFunc<SerDeInfo>() {
+      @Override
+      public void apply(SerDeInfo t, Object[] fields) {
+        t.putToParameters((String)fields[1], extractSqlClob(fields[2]));
+      }});
+    // Perform conversion of null map values
+    for (SerDeInfo t : serdes.values()) {
+      t.setParameters(MetaStoreServerUtils.trimMapNulls(t.getParameters(), convertMapNullsToEmptyStrings));
+    }
+  }
+
+  /**
+   * Convert a boolean value returned from the RDBMS to a Java Boolean object.
+   * MySQL has booleans, but e.g. Derby uses 'Y'/'N' mapping.
+   *
+   * @param value
+   *          column value from the database
+   * @return The Boolean value of the database column value, null if the column
+   *         value is null
+   * @throws MetaException
+   *           if the column value cannot be converted into a Boolean object
+   */
+  static Boolean extractSqlBoolean(Object value) throws MetaException {
+    if (value == null) {
+      return null;
+    }
+    if (value instanceof Boolean) {
+      return (Boolean)value;
+    }
+    if (value instanceof String) {
+      try {
+        return BooleanUtils.toBooleanObject((String) value, "Y", "N", null);
+      } catch (IllegalArgumentException iae) {
+        // NOOP
+      }
+    }
+    throw new MetaException("Cannot extract boolean from column value " + value);
+  }
+
+  static String extractSqlString(Object value) {
+    if (value == null) return null;
+    return value.toString();
+  }
+
+  static Double extractSqlDouble(Object obj) throws MetaException {
+    if (obj == null)
+      return null;
+    if (!(obj instanceof Number)) {
+      throw new MetaException("Expected numeric type but got " + obj.getClass().getName());
+    }
+    return ((Number) obj).doubleValue();
+  }
+
+  static byte[] extractSqlBlob(Object value) throws MetaException {
+    if (value == null)
+      return null;
+    if (value instanceof Blob) {
+      //derby, oracle
+      try {
+        // getBytes function says: pos the ordinal position of the first byte in
+        // the BLOB value to be extracted; the first byte is at position 1
+        return ((Blob) value).getBytes(1, (int) ((Blob) value).length());
+      } catch (SQLException e) {
+        throw new MetaException("Encounter error while processing blob.");
+      }
+    }
+    else if (value instanceof byte[]) {
+      // mysql, postgres, sql server
+      return (byte[]) value;
+    }
+	else {
+      // this may happen when enablebitvector is false
+      LOG.debug("Expected blob type but got " + value.getClass().getName());
+      return null;
+    }
+  }
+
+  @FunctionalInterface
+  static interface ApplyFunc<Target> {
+    void apply(Target t, Object[] fields) throws MetaException;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/44ef91a6/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
index 5372714..66977d7 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
@@ -69,6 +69,7 @@ import javax.jdo.datastore.JDOConnection;
 import javax.jdo.identity.IntIdentity;
 import javax.sql.DataSource;
 
+import com.google.common.base.Joiner;
 import com.google.common.base.Strings;
 
 import org.apache.commons.collections.CollectionUtils;
@@ -2031,8 +2032,11 @@ public class ObjectStore implements RawStore, Configurable {
     return keys;
   }
 
-  private SerDeInfo convertToSerDeInfo(MSerDeInfo ms) throws MetaException {
+  private SerDeInfo convertToSerDeInfo(MSerDeInfo ms, boolean allowNull) throws MetaException {
     if (ms == null) {
+      if (allowNull) {
+        return null;
+      }
       throw new MetaException("Invalid SerDeInfo object");
     }
     SerDeInfo serde =
@@ -2086,7 +2090,7 @@ public class ObjectStore implements RawStore, Configurable {
     StorageDescriptor sd = new StorageDescriptor(noFS ? null : convertToFieldSchemas(mFieldSchemas),
         msd.getLocation(), msd.getInputFormat(), msd.getOutputFormat(), msd
         .isCompressed(), msd.getNumBuckets(), convertToSerDeInfo(msd
-        .getSerDeInfo()), convertList(msd.getBucketCols()), convertToOrders(msd
+        .getSerDeInfo(), true), convertList(msd.getBucketCols()), convertToOrders(msd
         .getSortCols()), convertMap(msd.getParameters()));
     SkewedInfo skewedInfo = new SkewedInfo(convertList(msd.getSkewedColNames()),
         convertToSkewedValues(msd.getSkewedColValues()),
@@ -2607,11 +2611,17 @@ public class ObjectStore implements RawStore, Configurable {
     if (mpart == null) {
       return null;
     }
-    Partition p = new Partition(convertList(mpart.getValues()), mpart.getTable().getDatabase()
-        .getName(), mpart.getTable().getTableName(), mpart.getCreateTime(),
+    //its possible that MPartition is partially filled, do null checks to avoid NPE
+    MTable table = mpart.getTable();
+    String dbName =
+        table == null ? null : table.getDatabase() == null ? null : table.getDatabase().getName();
+    String tableName = table == null ? null : table.getTableName();
+    String catName = table == null ? null :
+        table.getDatabase() == null ? null : table.getDatabase().getCatalogName();
+    Partition p = new Partition(convertList(mpart.getValues()), dbName, tableName, mpart.getCreateTime(),
         mpart.getLastAccessTime(), convertToStorageDescriptor(mpart.getSd()),
         convertMap(mpart.getParameters()));
-    p.setCatName(mpart.getTable().getDatabase().getCatalogName());
+    p.setCatName(catName);
     p.setWriteId(mpart.getWriteId());
     return p;
   }
@@ -3349,6 +3359,64 @@ public class ObjectStore implements RawStore, Configurable {
     return mparts;
   }
 
+  // This code is only executed in JDO code path, not from direct SQL code path.
+  private List<MPartition> listMPartitionsWithProjection(String catName, String dbName, String tblName, int max,
+      QueryWrapper queryWrapper, List<String> fieldNames) throws MetaException {
+    boolean success = false;
+    List<MPartition> mparts = null;
+    try {
+      openTransaction();
+      LOG.debug("Executing listMPartitionsWithProjection");
+      dbName = normalizeIdentifier(dbName);
+      tblName = normalizeIdentifier(tblName);
+      Query query = queryWrapper.query = pm.newQuery(MPartition.class,
+          "table.tableName == t1 && table.database.name == t2 && table.database.catalogName == t3");
+      query.declareParameters("java.lang.String t1, java.lang.String t2, java.lang.String t3");
+      query.setOrdering("partitionName ascending");
+      if (max >= 0) {
+        query.setRange(0, max);
+      }
+      if (fieldNames == null || fieldNames.isEmpty()) {
+        // full fetch of partitions
+        mparts = (List<MPartition>) query.execute(tblName, dbName, catName);
+        pm.retrieveAll(mparts);
+      } else {
+        // fetch partially filled partitions using result clause
+        query.setResult(Joiner.on(',').join(fieldNames));
+        // if more than one fields are in the result class the return type is List<Object[]>
+        if (fieldNames.size() > 1) {
+          List<Object[]> results = (List<Object[]>) query.execute(tblName, dbName, catName);
+          mparts = new ArrayList<>(results.size());
+          for (Object[] row : results) {
+            MPartition mpart = new MPartition();
+            int i = 0;
+            for (Object val : row) {
+              MetaStoreServerUtils.setNestedProperty(mpart, fieldNames.get(i), val, true);
+              i++;
+            }
+            mparts.add(mpart);
+          }
+        } else {
+          // only one field is requested, return type is List<Object>
+          List<Object> results = (List<Object>) query.execute(tblName, dbName, catName);
+          mparts = new ArrayList<>(results.size());
+          for (Object row : results) {
+            MPartition mpart = new MPartition();
+            MetaStoreServerUtils.setNestedProperty(mpart, fieldNames.get(0), row, true);
+            mparts.add(mpart);
+          }
+        }
+      }
+      success = commitTransaction();
+      LOG.debug("Done retrieving {} objects for listMPartitionsWithProjection", mparts.size());
+    } finally {
+      if (!success) {
+        rollbackTransaction();
+      }
+    }
+    return mparts;
+  }
+
   @Override
   public List<Partition> getPartitionsByNames(String catName, String dbName, String tblName,
       List<String> partNames) throws MetaException, NoSuchObjectException {
@@ -3492,7 +3560,6 @@ public class ObjectStore implements RawStore, Configurable {
     return results;
   }
 
-
   private Integer getNumPartitionsViaOrmFilter(Table table, ExpressionTree tree, boolean isValidatedFilter)
     throws MetaException {
     Map<String, Object> params = new HashMap<>();
@@ -3620,17 +3687,23 @@ public class ObjectStore implements RawStore, Configurable {
     private boolean doUseDirectSql;
     private long start;
     private Table table;
+    protected final List<String> partitionFields;
     protected final String catName, dbName, tblName;
     private boolean success = false;
     protected T results = null;
 
     public GetHelper(String catalogName, String dbName, String tblName,
-                     boolean allowSql, boolean allowJdo)
-        throws MetaException {
+        boolean allowSql, boolean allowJdo) throws MetaException {
+      this(catalogName, dbName, tblName, null, allowSql, allowJdo);
+    }
+
+    public GetHelper(String catalogName, String dbName, String tblName,
+        List<String> fields, boolean allowSql, boolean allowJdo) throws MetaException {
       assert allowSql || allowJdo;
       this.allowJdo = allowJdo;
       this.catName = (catalogName != null) ? normalizeIdentifier(catalogName) : null;
       this.dbName = (dbName != null) ? normalizeIdentifier(dbName) : null;
+      this.partitionFields = fields;
       if (tblName != null) {
         this.tblName = normalizeIdentifier(tblName);
       } else {
@@ -3813,7 +3886,12 @@ public class ObjectStore implements RawStore, Configurable {
   private abstract class GetListHelper<T> extends GetHelper<List<T>> {
     public GetListHelper(String catName, String dbName, String tblName, boolean allowSql,
                          boolean allowJdo) throws MetaException {
-      super(catName, dbName, tblName, allowSql, allowJdo);
+      super(catName, dbName, tblName, null, allowSql, allowJdo);
+    }
+
+    public GetListHelper(String catName, String dbName, String tblName, List<String> fields,
+        boolean allowSql, boolean allowJdo) throws MetaException {
+      super(catName, dbName, tblName, fields, allowSql, allowJdo);
     }
 
     @Override
@@ -3961,6 +4039,44 @@ public class ObjectStore implements RawStore, Configurable {
     }.run(true);
   }
 
+  @Override
+  public List<Partition> getPartitionSpecsByFilterAndProjection(String catName, String dbName,
+                                                                String tblName, List<String> fieldList,
+                                                                String includeParamKeyPattern,
+                                                                String excludeParamKeyPattern)
+      throws MetaException, NoSuchObjectException {
+    if (fieldList == null || fieldList.isEmpty()) {
+      // no fields are requested. Fallback to regular getPartitions implementation to return all the fields
+      return getPartitionsInternal(catName, dbName, tblName, -1, true, true);
+    }
+
+    return new GetListHelper<Partition>(catName, dbName, tblName,
+        fieldList, true, true) {
+
+      @Override
+      protected List<Partition> getSqlResult(GetHelper<List<Partition>> ctx) throws MetaException {
+        return directSql
+            .getPartitionSpecsUsingProjection(ctx.getTable(), ctx.partitionFields, includeParamKeyPattern,
+                excludeParamKeyPattern);
+      }
+
+      @Override
+      protected List<Partition> getJdoResult(
+          GetHelper<List<Partition>> ctx) throws MetaException {
+        // For single-valued fields we can use setResult() to implement projection of fields but
+        // JDO doesn't support multi-valued fields in setResult() so currently JDO implementation
+        // fallbacks to full-partition fetch if the requested fields contain multi-valued fields
+        // TODO: Add param filtering logic
+        List<String> fieldNames = PartitionProjectionEvaluator.getMPartitionFieldNames(ctx.partitionFields);
+        try (QueryWrapper queryWrapper = new QueryWrapper()) {
+          return convertToParts(
+              listMPartitionsWithProjection(catName, dbName, tblName, -1, queryWrapper, fieldNames));
+        }
+      }
+    }.run(true);
+
+  }
+
   /**
    * Gets the table object for a given table, throws if anything goes wrong.
    * @param dbName Database name.
@@ -4403,7 +4519,7 @@ public class ObjectStore implements RawStore, Configurable {
     }
 
     oldSd.setBucketCols(newSd.getBucketCols());
-    oldSd.setCompressed(newSd.isCompressed());
+    oldSd.setIsCompressed(newSd.isCompressed());
     oldSd.setInputFormat(newSd.getInputFormat());
     oldSd.setOutputFormat(newSd.getOutputFormat());
     oldSd.setNumBuckets(newSd.getNumBuckets());
@@ -11120,7 +11236,7 @@ public class ObjectStore implements RawStore, Configurable {
       if (mSerDeInfo == null) {
         throw new NoSuchObjectException("No SerDe named " + serDeName);
       }
-      SerDeInfo serde = convertToSerDeInfo(mSerDeInfo);
+      SerDeInfo serde = convertToSerDeInfo(mSerDeInfo, false);
       committed = commitTransaction();
       return serde;
     } finally {
@@ -11238,7 +11354,7 @@ public class ObjectStore implements RawStore, Configurable {
       schemaVersion.setName(mSchemaVersion.getName());
     }
     if (mSchemaVersion.getSerDe() != null) {
-      schemaVersion.setSerDe(convertToSerDeInfo(mSchemaVersion.getSerDe()));
+      schemaVersion.setSerDe(convertToSerDeInfo(mSchemaVersion.getSerDe(), false));
     }
     return schemaVersion;
   }


[05/13] hive git commit: HIVE-20306 : Implement projection spec for fetching only requested fields from partitions (Vihang Karajgaonkar, reviewed by Aihua Xu, Andrew Sherman and Alexander Kolbasov)

Posted by vi...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/44ef91a6/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java.orig
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java.orig b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java.orig
new file mode 100644
index 0000000..0a25b77
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java.orig
@@ -0,0 +1,2845 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.metastore;
+
+import static org.apache.commons.lang.StringUtils.join;
+import static org.apache.commons.lang.StringUtils.normalizeSpace;
+import static org.apache.commons.lang.StringUtils.repeat;
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_CATALOG_NAME;
+
+import java.sql.Blob;
+import java.sql.Clob;
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.text.ParseException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.stream.Collectors;
+
+import javax.jdo.PersistenceManager;
+import javax.jdo.Query;
+import javax.jdo.Transaction;
+import javax.jdo.datastore.JDOConnection;
+
+import org.apache.commons.lang.BooleanUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.AggregateStatsCache.AggrColStats;
+import org.apache.hadoop.hive.metastore.api.AggrStats;
+import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
+import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData;
+import org.apache.hadoop.hive.metastore.api.ColumnStatisticsDesc;
+import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Order;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.PrincipalType;
+import org.apache.hadoop.hive.metastore.api.SQLCheckConstraint;
+import org.apache.hadoop.hive.metastore.api.SQLDefaultConstraint;
+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.SerDeInfo;
+import org.apache.hadoop.hive.metastore.api.SkewedInfo;
+import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
+import org.apache.hadoop.hive.metastore.model.MConstraint;
+import org.apache.hadoop.hive.metastore.model.MCreationMetadata;
+import org.apache.hadoop.hive.metastore.model.MDatabase;
+import org.apache.hadoop.hive.metastore.model.MNotificationLog;
+import org.apache.hadoop.hive.metastore.model.MNotificationNextId;
+import org.apache.hadoop.hive.metastore.model.MPartitionColumnPrivilege;
+import org.apache.hadoop.hive.metastore.model.MPartitionColumnStatistics;
+import org.apache.hadoop.hive.metastore.model.MPartitionPrivilege;
+import org.apache.hadoop.hive.metastore.model.MTableColumnStatistics;
+import org.apache.hadoop.hive.metastore.model.MWMResourcePlan;
+import org.apache.hadoop.hive.metastore.parser.ExpressionTree;
+import org.apache.hadoop.hive.metastore.parser.ExpressionTree.FilterBuilder;
+import org.apache.hadoop.hive.metastore.parser.ExpressionTree.LeafNode;
+import org.apache.hadoop.hive.metastore.parser.ExpressionTree.LogicalOperator;
+import org.apache.hadoop.hive.metastore.parser.ExpressionTree.Operator;
+import org.apache.hadoop.hive.metastore.parser.ExpressionTree.TreeNode;
+import org.apache.hadoop.hive.metastore.parser.ExpressionTree.TreeVisitor;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreServerUtils;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreServerUtils.ColStatsObjWithSourceInfo;
+import org.apache.hive.common.util.BloomFilter;
+import org.datanucleus.store.rdbms.query.ForwardQueryResult;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Lists;
+
+/**
+ * This class contains the optimizations for MetaStore that rely on direct SQL access to
+ * the underlying database. It should use ANSI SQL and be compatible with common databases
+ * such as MySQL (note that MySQL doesn't use full ANSI mode by default), Postgres, etc.
+ *
+ * As of now, only the partition retrieval is done this way to improve job startup time;
+ * JDOQL partition retrieval is still present so as not to limit the ORM solution we have
+ * to SQL stores only. There's always a way to do without direct SQL.
+ */
+class MetaStoreDirectSql {
+  private static final int NO_BATCHING = -1, DETECT_BATCHING = 0;
+
+  private static final Logger LOG = LoggerFactory.getLogger(MetaStoreDirectSql.class);
+  private final PersistenceManager pm;
+  private final Configuration conf;
+  private final String schema;
+
+  /**
+   * We want to avoid db-specific code in this class and stick with ANSI SQL. However:
+   * 1) mysql and postgres are differently ansi-incompatible (mysql by default doesn't support
+   * quoted identifiers, and postgres contravenes ANSI by coercing unquoted ones to lower case).
+   * MySQL's way of working around this is simpler (just set ansi quotes mode on), so we will
+   * use that. MySQL detection is done by actually issuing the set-ansi-quotes command;
+   *
+   * Use sparingly, we don't want to devolve into another DataNucleus...
+   */
+  private final DatabaseProduct dbType;
+  private final int batchSize;
+  private final boolean convertMapNullsToEmptyStrings;
+  private final String defaultPartName;
+
+  /**
+   * Whether direct SQL can be used with the current datastore backing {@link #pm}.
+   */
+  private final boolean isCompatibleDatastore;
+  private final boolean isAggregateStatsCacheEnabled;
+  private AggregateStatsCache aggrStatsCache;
+
+  @java.lang.annotation.Target(java.lang.annotation.ElementType.FIELD)
+  @java.lang.annotation.Retention(java.lang.annotation.RetentionPolicy.RUNTIME)
+  private @interface TableName {}
+
+  // Table names with schema name, if necessary
+  @TableName
+  private String DBS, TBLS, PARTITIONS, DATABASE_PARAMS, PARTITION_PARAMS, SORT_COLS, SD_PARAMS,
+      SDS, SERDES, SKEWED_STRING_LIST_VALUES, SKEWED_VALUES, BUCKETING_COLS, SKEWED_COL_NAMES,
+      SKEWED_COL_VALUE_LOC_MAP, COLUMNS_V2, PARTITION_KEYS, SERDE_PARAMS, PART_COL_STATS, KEY_CONSTRAINTS,
+      TAB_COL_STATS, PARTITION_KEY_VALS, PART_PRIVS, PART_COL_PRIVS, SKEWED_STRING_LIST, CDS;
+
+
+  public MetaStoreDirectSql(PersistenceManager pm, Configuration conf, String schema) {
+    this.pm = pm;
+    this.conf = conf;
+    this.schema = schema;
+    DatabaseProduct dbType = null;
+    try {
+      dbType = DatabaseProduct.determineDatabaseProduct(getProductName(pm));
+    } catch (SQLException e) {
+      LOG.warn("Cannot determine database product; assuming OTHER", e);
+      dbType = DatabaseProduct.OTHER;
+    }
+    this.dbType = dbType;
+    int batchSize = MetastoreConf.getIntVar(conf, ConfVars.DIRECT_SQL_PARTITION_BATCH_SIZE);
+    if (batchSize == DETECT_BATCHING) {
+      batchSize = DatabaseProduct.needsInBatching(dbType) ? 1000 : NO_BATCHING;
+    }
+    this.batchSize = batchSize;
+
+    for (java.lang.reflect.Field f : this.getClass().getDeclaredFields()) {
+      if (f.getAnnotation(TableName.class) == null) continue;
+      try {
+        f.set(this, getFullyQualifiedName(schema, f.getName()));
+      } catch (IllegalArgumentException | IllegalAccessException e) {
+        throw new RuntimeException("Internal error, cannot set " + f.getName());
+      }
+    }
+
+    convertMapNullsToEmptyStrings =
+        MetastoreConf.getBoolVar(conf, ConfVars.ORM_RETRIEVE_MAPNULLS_AS_EMPTY_STRINGS);
+    defaultPartName = MetastoreConf.getVar(conf, ConfVars.DEFAULTPARTITIONNAME);
+
+    String jdoIdFactory = MetastoreConf.getVar(conf, ConfVars.IDENTIFIER_FACTORY);
+    if (! ("datanucleus1".equalsIgnoreCase(jdoIdFactory))){
+      LOG.warn("Underlying metastore does not use 'datanucleus1' for its ORM naming scheme."
+          + " Disabling directSQL as it uses hand-hardcoded SQL with that assumption.");
+      isCompatibleDatastore = false;
+    } else {
+      boolean isInTest = MetastoreConf.getBoolVar(conf, ConfVars.HIVE_IN_TEST);
+      isCompatibleDatastore = (!isInTest || ensureDbInit()) && runTestQuery();
+      if (isCompatibleDatastore) {
+        LOG.debug("Using direct SQL, underlying DB is " + dbType);
+      }
+    }
+
+    isAggregateStatsCacheEnabled = MetastoreConf.getBoolVar(
+        conf, ConfVars.AGGREGATE_STATS_CACHE_ENABLED);
+    if (isAggregateStatsCacheEnabled) {
+      aggrStatsCache = AggregateStatsCache.getInstance(conf);
+    }
+  }
+
+  private static String getFullyQualifiedName(String schema, String tblName) {
+    return ((schema == null || schema.isEmpty()) ? "" : "\"" + schema + "\".\"")
+        + "\"" + tblName + "\"";
+  }
+
+
+  public MetaStoreDirectSql(PersistenceManager pm, Configuration conf) {
+    this(pm, conf, "");
+  }
+
+  static String getProductName(PersistenceManager pm) {
+    JDOConnection jdoConn = pm.getDataStoreConnection();
+    try {
+      return ((Connection)jdoConn.getNativeConnection()).getMetaData().getDatabaseProductName();
+    } catch (Throwable t) {
+      LOG.warn("Error retrieving product name", t);
+      return null;
+    } finally {
+      jdoConn.close(); // We must release the connection before we call other pm methods.
+    }
+  }
+
+  private boolean ensureDbInit() {
+    Transaction tx = pm.currentTransaction();
+    boolean doCommit = false;
+    if (!tx.isActive()) {
+      tx.begin();
+      doCommit = true;
+    }
+    LinkedList<Query> initQueries = new LinkedList<>();
+
+    try {
+      // Force the underlying db to initialize.
+      initQueries.add(pm.newQuery(MDatabase.class, "name == ''"));
+      initQueries.add(pm.newQuery(MTableColumnStatistics.class, "dbName == ''"));
+      initQueries.add(pm.newQuery(MPartitionColumnStatistics.class, "dbName == ''"));
+      initQueries.add(pm.newQuery(MConstraint.class, "childIntegerIndex < 0"));
+      initQueries.add(pm.newQuery(MNotificationLog.class, "dbName == ''"));
+      initQueries.add(pm.newQuery(MNotificationNextId.class, "nextEventId < -1"));
+      initQueries.add(pm.newQuery(MWMResourcePlan.class, "name == ''"));
+      initQueries.add(pm.newQuery(MCreationMetadata.class, "dbName == ''"));
+      initQueries.add(pm.newQuery(MPartitionPrivilege.class, "principalName == ''"));
+      initQueries.add(pm.newQuery(MPartitionColumnPrivilege.class, "principalName == ''"));
+      Query q;
+      while ((q = initQueries.peekFirst()) != null) {
+        q.execute();
+        initQueries.pollFirst();
+      }
+
+      return true;
+    } catch (Exception ex) {
+      doCommit = false;
+      LOG.warn("Database initialization failed; direct SQL is disabled", ex);
+      tx.rollback();
+      return false;
+    } finally {
+      if (doCommit) {
+        tx.commit();
+      }
+      for (Query q : initQueries) {
+        try {
+          q.closeAll();
+        } catch (Throwable t) {
+        }
+      }
+    }
+  }
+
+  private boolean runTestQuery() {
+    Transaction tx = pm.currentTransaction();
+    boolean doCommit = false;
+    if (!tx.isActive()) {
+      tx.begin();
+      doCommit = true;
+    }
+    Query query = null;
+    // Run a self-test query. If it doesn't work, we will self-disable. What a PITA...
+    String selfTestQuery = "select \"DB_ID\" from " + DBS + "";
+    try {
+      prepareTxn();
+      query = pm.newQuery("javax.jdo.query.SQL", selfTestQuery);
+      query.execute();
+      return true;
+    } catch (Throwable t) {
+      doCommit = false;
+      LOG.warn("Self-test query [" + selfTestQuery + "] failed; direct SQL is disabled", t);
+      tx.rollback();
+      return false;
+    } finally {
+      if (doCommit) {
+        tx.commit();
+      }
+      if (query != null) {
+        query.closeAll();
+      }
+    }
+  }
+
+  public String getSchema() {
+    return schema;
+  }
+
+  public boolean isCompatibleDatastore() {
+    return isCompatibleDatastore;
+  }
+
+  private void executeNoResult(final String queryText) throws SQLException {
+    JDOConnection jdoConn = pm.getDataStoreConnection();
+    Statement statement = null;
+    boolean doTrace = LOG.isDebugEnabled();
+    try {
+      long start = doTrace ? System.nanoTime() : 0;
+      statement = ((Connection)jdoConn.getNativeConnection()).createStatement();
+      statement.execute(queryText);
+      timingTrace(doTrace, queryText, start, doTrace ? System.nanoTime() : 0);
+    } finally {
+      if(statement != null){
+          statement.close();
+      }
+      jdoConn.close(); // We must release the connection before we call other pm methods.
+    }
+  }
+
+  public Database getDatabase(String catName, String dbName) throws MetaException{
+    Query queryDbSelector = null;
+    Query queryDbParams = null;
+    try {
+      dbName = dbName.toLowerCase();
+      catName = catName.toLowerCase();
+
+      String queryTextDbSelector= "select "
+          + "\"DB_ID\", \"NAME\", \"DB_LOCATION_URI\", \"DESC\", "
+          + "\"OWNER_NAME\", \"OWNER_TYPE\", \"CTLG_NAME\" "
+          + "FROM "+ DBS
+          + " where \"NAME\" = ? and \"CTLG_NAME\" = ? ";
+      Object[] params = new Object[] { dbName, catName };
+      queryDbSelector = pm.newQuery("javax.jdo.query.SQL", queryTextDbSelector);
+
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("getDatabase:query instantiated : " + queryTextDbSelector
+            + " with param [" + params[0] + "]");
+      }
+
+      List<Object[]> sqlResult = executeWithArray(
+          queryDbSelector, params, queryTextDbSelector);
+      if ((sqlResult == null) || sqlResult.isEmpty()) {
+        return null;
+      }
+
+      assert(sqlResult.size() == 1);
+      if (sqlResult.get(0) == null) {
+        return null;
+      }
+
+      Object[] dbline = sqlResult.get(0);
+      Long dbid = extractSqlLong(dbline[0]);
+
+      String queryTextDbParams = "select \"PARAM_KEY\", \"PARAM_VALUE\" "
+          + " from " + DATABASE_PARAMS + " "
+          + " WHERE \"DB_ID\" = ? "
+          + " AND \"PARAM_KEY\" IS NOT NULL";
+      params[0] = dbid;
+      queryDbParams = pm.newQuery("javax.jdo.query.SQL", queryTextDbParams);
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("getDatabase:query2 instantiated : " + queryTextDbParams
+            + " with param [" + params[0] + "]");
+      }
+
+      Map<String,String> dbParams = new HashMap<String,String>();
+      List<Object[]> sqlResult2 = ensureList(executeWithArray(
+          queryDbParams, params, queryTextDbParams));
+      if (!sqlResult2.isEmpty()) {
+        for (Object[] line : sqlResult2) {
+          dbParams.put(extractSqlString(line[0]), extractSqlString(line[1]));
+        }
+      }
+      Database db = new Database();
+      db.setName(extractSqlString(dbline[1]));
+      db.setLocationUri(extractSqlString(dbline[2]));
+      db.setDescription(extractSqlString(dbline[3]));
+      db.setOwnerName(extractSqlString(dbline[4]));
+      String type = extractSqlString(dbline[5]);
+      db.setOwnerType(
+          (null == type || type.trim().isEmpty()) ? null : PrincipalType.valueOf(type));
+      db.setCatalogName(extractSqlString(dbline[6]));
+      db.setParameters(MetaStoreServerUtils.trimMapNulls(dbParams,convertMapNullsToEmptyStrings));
+      if (LOG.isDebugEnabled()){
+        LOG.debug("getDatabase: directsql returning db " + db.getName()
+            + " locn["+db.getLocationUri()  +"] desc [" +db.getDescription()
+            + "] owner [" + db.getOwnerName() + "] ownertype ["+ db.getOwnerType() +"]");
+      }
+      return db;
+    } finally {
+      if (queryDbSelector != null){
+        queryDbSelector.closeAll();
+      }
+      if (queryDbParams != null){
+        queryDbParams.closeAll();
+      }
+    }
+  }
+
+  /**
+   * Get table names by using direct SQL queries.
+   * @param catName catalog name
+   * @param dbName Metastore database namme
+   * @param tableType Table type, or null if we want to get all tables
+   * @return list of table names
+   */
+  public List<String> getTables(String catName, String dbName, TableType tableType)
+      throws MetaException {
+    String queryText = "SELECT " + TBLS + ".\"TBL_NAME\""
+      + " FROM " + TBLS + " "
+      + " INNER JOIN " + DBS + " ON " + TBLS + ".\"DB_ID\" = " + DBS + ".\"DB_ID\" "
+      + " WHERE " + DBS + ".\"NAME\" = ? AND " + DBS + ".\"CTLG_NAME\" = ? "
+      + (tableType == null ? "" : "AND " + TBLS + ".\"TBL_TYPE\" = ? ") ;
+
+    List<String> pms = new ArrayList<>();
+    pms.add(dbName);
+    pms.add(catName);
+    if (tableType != null) {
+      pms.add(tableType.toString());
+    }
+
+    Query<?> queryParams = pm.newQuery("javax.jdo.query.SQL", queryText);
+    return executeWithArray(
+        queryParams, pms.toArray(), queryText);
+  }
+
+  /**
+   * Get table names by using direct SQL queries.
+   *
+   * @param dbName Metastore database namme
+   * @return list of table names
+   */
+  public List<String> getMaterializedViewsForRewriting(String dbName) throws MetaException {
+    String queryText = "SELECT " + TBLS + ".\"TBL_NAME\""
+      + " FROM " + TBLS + " "
+      + " INNER JOIN " + DBS + " ON " + TBLS + ".\"DB_ID\" = " + DBS + ".\"DB_ID\" "
+      + " WHERE " + DBS + ".\"NAME\" = ? AND " + TBLS + ".\"TBL_TYPE\" = ? " ;
+
+    List<String> pms = new ArrayList<String>();
+    pms.add(dbName);
+    pms.add(TableType.MATERIALIZED_VIEW.toString());
+
+    Query<?> queryParams = pm.newQuery("javax.jdo.query.SQL", queryText);
+    return executeWithArray(
+        queryParams, pms.toArray(), queryText);
+  }
+
+  /**
+   * Gets partitions by using direct SQL queries.
+   * @param catName Metastore catalog name.
+   * @param dbName Metastore db name.
+   * @param tblName Metastore table name.
+   * @param partNames Partition names to get.
+   * @return List of partitions.
+   */
+  public List<Partition> getPartitionsViaSqlFilter(final String catName, final String dbName,
+                                                   final String tblName, List<String> partNames)
+      throws MetaException {
+    if (partNames.isEmpty()) {
+      return Collections.emptyList();
+    }
+    return Batchable.runBatched(batchSize, partNames, new Batchable<String, Partition>() {
+      @Override
+      public List<Partition> run(List<String> input) throws MetaException {
+        String filter = "" + PARTITIONS + ".\"PART_NAME\" in (" + makeParams(input.size()) + ")";
+        List<Object> partitionIds = getPartitionIdsViaSqlFilter(catName, dbName, tblName,
+            filter, input, Collections.<String>emptyList(), null);
+        if (partitionIds.isEmpty()) {
+          return Collections.emptyList(); // no partitions, bail early.
+        }
+        return getPartitionsFromPartitionIds(catName, dbName, tblName, null, partitionIds);
+      }
+    });
+  }
+
+  /**
+   * Gets partitions by using direct SQL queries.
+   * @param filter The filter.
+   * @param max The maximum number of partitions to return.
+   * @return List of partitions.
+   */
+  public List<Partition> getPartitionsViaSqlFilter(
+      SqlFilterForPushdown filter, Integer max) throws MetaException {
+    Boolean isViewTable = isViewTable(filter.table);
+    String catName = filter.table.isSetCatName() ? filter.table.getCatName() :
+        DEFAULT_CATALOG_NAME;
+    List<Object> partitionIds = getPartitionIdsViaSqlFilter(catName,
+        filter.table.getDbName(), filter.table.getTableName(), filter.filter, filter.params,
+        filter.joins, max);
+    if (partitionIds.isEmpty()) {
+      return Collections.emptyList(); // no partitions, bail early.
+    }
+    return Batchable.runBatched(batchSize, partitionIds, new Batchable<Object, Partition>() {
+      @Override
+      public List<Partition> run(List<Object> input) throws MetaException {
+        return getPartitionsFromPartitionIds(catName, filter.table.getDbName(),
+            filter.table.getTableName(), isViewTable, input);
+      }
+    });
+  }
+
+  public static class SqlFilterForPushdown {
+    private final List<Object> params = new ArrayList<>();
+    private final List<String> joins = new ArrayList<>();
+    private String filter;
+    private Table table;
+  }
+
+  public boolean generateSqlFilterForPushdown(
+      Table table, ExpressionTree tree, SqlFilterForPushdown result) throws MetaException {
+    return generateSqlFilterForPushdown(table, tree, null, result);
+  }
+
+  public boolean generateSqlFilterForPushdown(Table table, ExpressionTree tree, String defaultPartitionName,
+                                              SqlFilterForPushdown result) throws MetaException {
+    // Derby and Oracle do not interpret filters ANSI-properly in some cases and need a workaround.
+    boolean dbHasJoinCastBug = DatabaseProduct.hasJoinOperationOrderBug(dbType);
+    result.table = table;
+    result.filter = PartitionFilterGenerator.generateSqlFilter(table, tree, result.params,
+            result.joins, dbHasJoinCastBug, ((defaultPartitionName == null) ? defaultPartName : defaultPartitionName),
+            dbType, schema);
+    return result.filter != null;
+  }
+
+  /**
+   * Gets all partitions of a table by using direct SQL queries.
+   * @param catName Metastore catalog name.
+   * @param dbName Metastore db name.
+   * @param tblName Metastore table name.
+   * @param max The maximum number of partitions to return.
+   * @return List of partitions.
+   */
+  public List<Partition> getPartitions(String catName,
+      String dbName, String tblName, Integer max) throws MetaException {
+    List<Object> partitionIds = getPartitionIdsViaSqlFilter(catName, dbName,
+        tblName, null, Collections.<String>emptyList(), Collections.<String>emptyList(), max);
+    if (partitionIds.isEmpty()) {
+      return Collections.emptyList(); // no partitions, bail early.
+    }
+
+    // Get full objects. For Oracle/etc. do it in batches.
+    List<Partition> result = Batchable.runBatched(batchSize, partitionIds, new Batchable<Object, Partition>() {
+      @Override
+      public List<Partition> run(List<Object> input) throws MetaException {
+        return getPartitionsFromPartitionIds(catName, dbName, tblName, null, input);
+      }
+    });
+    return result;
+  }
+
+  private static Boolean isViewTable(Table t) {
+    return t.isSetTableType() ?
+        t.getTableType().equals(TableType.VIRTUAL_VIEW.toString()) : null;
+  }
+
+  private boolean isViewTable(String catName, String dbName, String tblName) throws MetaException {
+    Query query = null;
+    try {
+      String queryText = "select \"TBL_TYPE\" from " + TBLS + "" +
+          " inner join " + DBS + " on " + TBLS + ".\"DB_ID\" = " + DBS + ".\"DB_ID\" " +
+          " where " + TBLS + ".\"TBL_NAME\" = ? and " + DBS + ".\"NAME\" = ? and " + DBS + ".\"CTLG_NAME\" = ?";
+      Object[] params = new Object[] { tblName, dbName, catName };
+      query = pm.newQuery("javax.jdo.query.SQL", queryText);
+      query.setUnique(true);
+      Object result = executeWithArray(query, params, queryText);
+      return (result != null) && result.toString().equals(TableType.VIRTUAL_VIEW.toString());
+    } finally {
+      if (query != null) {
+        query.closeAll();
+      }
+    }
+  }
+
+  /**
+   * Get partition ids for the query using direct SQL queries, to avoid bazillion
+   * queries created by DN retrieving stuff for each object individually.
+   * @param catName MetaStore catalog name
+   * @param dbName MetaStore db name
+   * @param tblName MetaStore table name
+   * @param sqlFilter SQL filter to use. Better be SQL92-compliant.
+   * @param paramsForFilter params for ?-s in SQL filter text. Params must be in order.
+   * @param joinsForFilter if the filter needs additional join statement, they must be in
+   *                       this list. Better be SQL92-compliant.
+   * @param max The maximum number of partitions to return.
+   * @return List of partition objects.
+   */
+  private List<Object> getPartitionIdsViaSqlFilter(
+      String catName, String dbName, String tblName, String sqlFilter,
+      List<? extends Object> paramsForFilter, List<String> joinsForFilter, Integer max)
+      throws MetaException {
+    boolean doTrace = LOG.isDebugEnabled();
+    final String dbNameLcase = dbName.toLowerCase();
+    final String tblNameLcase = tblName.toLowerCase();
+    final String catNameLcase = normalizeSpace(catName).toLowerCase();
+
+    // We have to be mindful of order during filtering if we are not returning all partitions.
+    String orderForFilter = (max != null) ? " order by \"PART_NAME\" asc" : "";
+
+    String queryText =
+        "select " + PARTITIONS + ".\"PART_ID\" from " + PARTITIONS + ""
+      + "  inner join " + TBLS + " on " + PARTITIONS + ".\"TBL_ID\" = " + TBLS + ".\"TBL_ID\" "
+      + "    and " + TBLS + ".\"TBL_NAME\" = ? "
+      + "  inner join " + DBS + " on " + TBLS + ".\"DB_ID\" = " + DBS + ".\"DB_ID\" "
+      + "     and " + DBS + ".\"NAME\" = ? "
+      + join(joinsForFilter, ' ')
+      + " where " + DBS + ".\"CTLG_NAME\" = ? "
+      + (StringUtils.isBlank(sqlFilter) ? "" : (" and " + sqlFilter)) + orderForFilter;
+    Object[] params = new Object[paramsForFilter.size() + 3];
+    params[0] = tblNameLcase;
+    params[1] = dbNameLcase;
+    params[2] = catNameLcase;
+    for (int i = 0; i < paramsForFilter.size(); ++i) {
+      params[i + 3] = paramsForFilter.get(i);
+    }
+
+    long start = doTrace ? System.nanoTime() : 0;
+    Query query = pm.newQuery("javax.jdo.query.SQL", queryText);
+    if (max != null) {
+      query.setRange(0, max.shortValue());
+    }
+    List<Object> sqlResult = executeWithArray(query, params, queryText);
+    long queryTime = doTrace ? System.nanoTime() : 0;
+    timingTrace(doTrace, queryText, start, queryTime);
+    if (sqlResult.isEmpty()) {
+      return Collections.emptyList(); // no partitions, bail early.
+    }
+
+    List<Object> result = new ArrayList<Object>(sqlResult.size());
+    for (Object fields : sqlResult) {
+      result.add(extractSqlLong(fields));
+    }
+    query.closeAll();
+    return result;
+  }
+
+  /** Should be called with the list short enough to not trip up Oracle/etc. */
+  private List<Partition> getPartitionsFromPartitionIds(String catName, String dbName, String tblName,
+      Boolean isView, List<Object> partIdList) throws MetaException {
+    boolean doTrace = LOG.isDebugEnabled();
+
+    int idStringWidth = (int)Math.ceil(Math.log10(partIdList.size())) + 1; // 1 for comma
+    int sbCapacity = partIdList.size() * idStringWidth;
+
+    String partIds = getIdListForIn(partIdList);
+
+    // Get most of the fields for the IDs provided.
+    // Assume db and table names are the same for all partition, as provided in arguments.
+    String queryText =
+      "select " + PARTITIONS + ".\"PART_ID\", " + SDS + ".\"SD_ID\", " + SDS + ".\"CD_ID\","
+    + " " + SERDES + ".\"SERDE_ID\", " + PARTITIONS + ".\"CREATE_TIME\","
+    + " " + PARTITIONS + ".\"LAST_ACCESS_TIME\", " + SDS + ".\"INPUT_FORMAT\", " + SDS + ".\"IS_COMPRESSED\","
+    + " " + SDS + ".\"IS_STOREDASSUBDIRECTORIES\", " + SDS + ".\"LOCATION\", " + SDS + ".\"NUM_BUCKETS\","
+    + " " + SDS + ".\"OUTPUT_FORMAT\", " + SERDES + ".\"NAME\", " + SERDES + ".\"SLIB\", " + PARTITIONS
+    + ".\"WRITE_ID\"" + " from " + PARTITIONS + ""
+    + "  left outer join " + SDS + " on " + PARTITIONS + ".\"SD_ID\" = " + SDS + ".\"SD_ID\" "
+    + "  left outer join " + SERDES + " on " + SDS + ".\"SERDE_ID\" = " + SERDES + ".\"SERDE_ID\" "
+    + "where \"PART_ID\" in (" + partIds + ") order by \"PART_NAME\" asc";
+    long start = doTrace ? System.nanoTime() : 0;
+    Query query = pm.newQuery("javax.jdo.query.SQL", queryText);
+    List<Object[]> sqlResult = executeWithArray(query, null, queryText);
+    long queryTime = doTrace ? System.nanoTime() : 0;
+    Deadline.checkTimeout();
+
+    // Read all the fields and create partitions, SDs and serdes.
+    TreeMap<Long, Partition> partitions = new TreeMap<Long, Partition>();
+    TreeMap<Long, StorageDescriptor> sds = new TreeMap<Long, StorageDescriptor>();
+    TreeMap<Long, SerDeInfo> serdes = new TreeMap<Long, SerDeInfo>();
+    TreeMap<Long, List<FieldSchema>> colss = new TreeMap<Long, List<FieldSchema>>();
+    // Keep order by name, consistent with JDO.
+    ArrayList<Partition> orderedResult = new ArrayList<Partition>(partIdList.size());
+
+    // Prepare StringBuilder-s for "in (...)" lists to use in one-to-many queries.
+    StringBuilder sdSb = new StringBuilder(sbCapacity), serdeSb = new StringBuilder(sbCapacity);
+    StringBuilder colsSb = new StringBuilder(7); // We expect that there's only one field schema.
+    tblName = tblName.toLowerCase();
+    dbName = dbName.toLowerCase();
+    catName = normalizeSpace(catName).toLowerCase();
+    for (Object[] fields : sqlResult) {
+      // Here comes the ugly part...
+      long partitionId = extractSqlLong(fields[0]);
+      Long sdId = extractSqlLong(fields[1]);
+      Long colId = extractSqlLong(fields[2]);
+      Long serdeId = extractSqlLong(fields[3]);
+      // A partition must have at least sdId and serdeId set, or nothing set if it's a view.
+      if (sdId == null || serdeId == null) {
+        if (isView == null) {
+          isView = isViewTable(catName, dbName, tblName);
+        }
+        if ((sdId != null || colId != null || serdeId != null) || !isView) {
+          throw new MetaException("Unexpected null for one of the IDs, SD " + sdId +
+                  ", serde " + serdeId + " for a " + (isView ? "" : "non-") + " view");
+        }
+      }
+
+      Partition part = new Partition();
+      orderedResult.add(part);
+      // Set the collection fields; some code might not check presence before accessing them.
+      part.setParameters(new HashMap<>());
+      part.setValues(new ArrayList<String>());
+      part.setCatName(catName);
+      part.setDbName(dbName);
+      part.setTableName(tblName);
+      if (fields[4] != null) part.setCreateTime(extractSqlInt(fields[4]));
+      if (fields[5] != null) part.setLastAccessTime(extractSqlInt(fields[5]));
+      Long writeId = extractSqlLong(fields[14]);
+      if (writeId != null) {
+        part.setWriteId(writeId);
+      }
+      partitions.put(partitionId, part);
+
+
+      if (sdId == null) continue; // Probably a view.
+      assert serdeId != null;
+
+      // We assume each partition has an unique SD.
+      StorageDescriptor sd = new StorageDescriptor();
+      StorageDescriptor oldSd = sds.put(sdId, sd);
+      if (oldSd != null) {
+        throw new MetaException("Partitions reuse SDs; we don't expect that");
+      }
+      // Set the collection fields; some code might not check presence before accessing them.
+      sd.setSortCols(new ArrayList<Order>());
+      sd.setBucketCols(new ArrayList<String>());
+      sd.setParameters(new HashMap<String, String>());
+      sd.setSkewedInfo(new SkewedInfo(new ArrayList<String>(),
+          new ArrayList<List<String>>(), new HashMap<List<String>, String>()));
+      sd.setInputFormat((String)fields[6]);
+      Boolean tmpBoolean = extractSqlBoolean(fields[7]);
+      if (tmpBoolean != null) sd.setCompressed(tmpBoolean);
+      tmpBoolean = extractSqlBoolean(fields[8]);
+      if (tmpBoolean != null) sd.setStoredAsSubDirectories(tmpBoolean);
+      sd.setLocation((String)fields[9]);
+      if (fields[10] != null) sd.setNumBuckets(extractSqlInt(fields[10]));
+      sd.setOutputFormat((String)fields[11]);
+      sdSb.append(sdId).append(",");
+      part.setSd(sd);
+
+      if (colId != null) {
+        List<FieldSchema> cols = colss.get(colId);
+        // We expect that colId will be the same for all (or many) SDs.
+        if (cols == null) {
+          cols = new ArrayList<FieldSchema>();
+          colss.put(colId, cols);
+          colsSb.append(colId).append(",");
+        }
+        sd.setCols(cols);
+      }
+
+      // We assume each SD has an unique serde.
+      SerDeInfo serde = new SerDeInfo();
+      SerDeInfo oldSerde = serdes.put(serdeId, serde);
+      if (oldSerde != null) {
+        throw new MetaException("SDs reuse serdes; we don't expect that");
+      }
+      serde.setParameters(new HashMap<String, String>());
+      serde.setName((String)fields[12]);
+      serde.setSerializationLib((String)fields[13]);
+      serdeSb.append(serdeId).append(",");
+      sd.setSerdeInfo(serde);
+
+      Deadline.checkTimeout();
+    }
+    query.closeAll();
+    timingTrace(doTrace, queryText, start, queryTime);
+
+    // Now get all the one-to-many things. Start with partitions.
+    queryText = "select \"PART_ID\", \"PARAM_KEY\", \"PARAM_VALUE\" from " + PARTITION_PARAMS + ""
+        + " where \"PART_ID\" in (" + partIds + ") and \"PARAM_KEY\" is not null"
+        + " order by \"PART_ID\" asc";
+    loopJoinOrderedResult(partitions, queryText, 0, new ApplyFunc<Partition>() {
+      @Override
+      public void apply(Partition t, Object[] fields) {
+        t.putToParameters((String)fields[1], extractSqlClob(fields[2]));
+      }});
+    // Perform conversion of null map values
+    for (Partition t : partitions.values()) {
+      t.setParameters(MetaStoreServerUtils.trimMapNulls(t.getParameters(), convertMapNullsToEmptyStrings));
+    }
+
+    queryText = "select \"PART_ID\", \"PART_KEY_VAL\" from " + PARTITION_KEY_VALS + ""
+        + " where \"PART_ID\" in (" + partIds + ")"
+        + " order by \"PART_ID\" asc, \"INTEGER_IDX\" asc";
+    loopJoinOrderedResult(partitions, queryText, 0, new ApplyFunc<Partition>() {
+      @Override
+      public void apply(Partition t, Object[] fields) {
+        t.addToValues((String)fields[1]);
+      }});
+
+    // Prepare IN (blah) lists for the following queries. Cut off the final ','s.
+    if (sdSb.length() == 0) {
+      assert serdeSb.length() == 0 && colsSb.length() == 0;
+      return orderedResult; // No SDs, probably a view.
+    }
+
+    String sdIds = trimCommaList(sdSb);
+    String serdeIds = trimCommaList(serdeSb);
+    String colIds = trimCommaList(colsSb);
+
+    // Get all the stuff for SD. Don't do empty-list check - we expect partitions do have SDs.
+    queryText = "select \"SD_ID\", \"PARAM_KEY\", \"PARAM_VALUE\" from " + SD_PARAMS + ""
+        + " where \"SD_ID\" in (" + sdIds + ") and \"PARAM_KEY\" is not null"
+        + " order by \"SD_ID\" asc";
+    loopJoinOrderedResult(sds, queryText, 0, new ApplyFunc<StorageDescriptor>() {
+      @Override
+      public void apply(StorageDescriptor t, Object[] fields) {
+        t.putToParameters((String)fields[1], extractSqlClob(fields[2]));
+      }});
+    // Perform conversion of null map values
+    for (StorageDescriptor t : sds.values()) {
+      t.setParameters(MetaStoreServerUtils.trimMapNulls(t.getParameters(), convertMapNullsToEmptyStrings));
+    }
+
+    queryText = "select \"SD_ID\", \"COLUMN_NAME\", " + SORT_COLS + ".\"ORDER\""
+        + " from " + SORT_COLS + ""
+        + " where \"SD_ID\" in (" + sdIds + ")"
+        + " order by \"SD_ID\" asc, \"INTEGER_IDX\" asc";
+    loopJoinOrderedResult(sds, queryText, 0, new ApplyFunc<StorageDescriptor>() {
+      @Override
+      public void apply(StorageDescriptor t, Object[] fields) {
+        if (fields[2] == null) return;
+        t.addToSortCols(new Order((String)fields[1], extractSqlInt(fields[2])));
+      }});
+
+    queryText = "select \"SD_ID\", \"BUCKET_COL_NAME\" from " + BUCKETING_COLS + ""
+        + " where \"SD_ID\" in (" + sdIds + ")"
+        + " order by \"SD_ID\" asc, \"INTEGER_IDX\" asc";
+    loopJoinOrderedResult(sds, queryText, 0, new ApplyFunc<StorageDescriptor>() {
+      @Override
+      public void apply(StorageDescriptor t, Object[] fields) {
+        t.addToBucketCols((String)fields[1]);
+      }});
+
+    // Skewed columns stuff.
+    queryText = "select \"SD_ID\", \"SKEWED_COL_NAME\" from " + SKEWED_COL_NAMES + ""
+        + " where \"SD_ID\" in (" + sdIds + ")"
+        + " order by \"SD_ID\" asc, \"INTEGER_IDX\" asc";
+    boolean hasSkewedColumns =
+      loopJoinOrderedResult(sds, queryText, 0, new ApplyFunc<StorageDescriptor>() {
+        @Override
+        public void apply(StorageDescriptor t, Object[] fields) {
+          if (!t.isSetSkewedInfo()) t.setSkewedInfo(new SkewedInfo());
+          t.getSkewedInfo().addToSkewedColNames((String)fields[1]);
+        }}) > 0;
+
+    // Assume we don't need to fetch the rest of the skewed column data if we have no columns.
+    if (hasSkewedColumns) {
+      // We are skipping the SKEWED_STRING_LIST table here, as it seems to be totally useless.
+      queryText =
+            "select " + SKEWED_VALUES + ".\"SD_ID_OID\","
+          + "  " + SKEWED_STRING_LIST_VALUES + ".\"STRING_LIST_ID\","
+          + "  " + SKEWED_STRING_LIST_VALUES + ".\"STRING_LIST_VALUE\" "
+          + "from " + SKEWED_VALUES + " "
+          + "  left outer join " + SKEWED_STRING_LIST_VALUES + " on " + SKEWED_VALUES + "."
+          + "\"STRING_LIST_ID_EID\" = " + SKEWED_STRING_LIST_VALUES + ".\"STRING_LIST_ID\" "
+          + "where " + SKEWED_VALUES + ".\"SD_ID_OID\" in (" + sdIds + ") "
+          + "  and " + SKEWED_VALUES + ".\"STRING_LIST_ID_EID\" is not null "
+          + "  and " + SKEWED_VALUES + ".\"INTEGER_IDX\" >= 0 "
+          + "order by " + SKEWED_VALUES + ".\"SD_ID_OID\" asc, " + SKEWED_VALUES + ".\"INTEGER_IDX\" asc,"
+          + "  " + SKEWED_STRING_LIST_VALUES + ".\"INTEGER_IDX\" asc";
+      loopJoinOrderedResult(sds, queryText, 0, new ApplyFunc<StorageDescriptor>() {
+        private Long currentListId;
+        private List<String> currentList;
+        @Override
+        public void apply(StorageDescriptor t, Object[] fields) throws MetaException {
+          if (!t.isSetSkewedInfo()) t.setSkewedInfo(new SkewedInfo());
+          // Note that this is not a typical list accumulator - there's no call to finalize
+          // the last list. Instead we add list to SD first, as well as locally to add elements.
+          if (fields[1] == null) {
+            currentList = null; // left outer join produced a list with no values
+            currentListId = null;
+            t.getSkewedInfo().addToSkewedColValues(Collections.<String>emptyList());
+          } else {
+            long fieldsListId = extractSqlLong(fields[1]);
+            if (currentListId == null || fieldsListId != currentListId) {
+              currentList = new ArrayList<String>();
+              currentListId = fieldsListId;
+              t.getSkewedInfo().addToSkewedColValues(currentList);
+            }
+            currentList.add((String)fields[2]);
+          }
+        }});
+
+      // We are skipping the SKEWED_STRING_LIST table here, as it seems to be totally useless.
+      queryText =
+            "select " + SKEWED_COL_VALUE_LOC_MAP + ".\"SD_ID\","
+          + " " + SKEWED_STRING_LIST_VALUES + ".STRING_LIST_ID,"
+          + " " + SKEWED_COL_VALUE_LOC_MAP + ".\"LOCATION\","
+          + " " + SKEWED_STRING_LIST_VALUES + ".\"STRING_LIST_VALUE\" "
+          + "from " + SKEWED_COL_VALUE_LOC_MAP + ""
+          + "  left outer join " + SKEWED_STRING_LIST_VALUES + " on " + SKEWED_COL_VALUE_LOC_MAP + "."
+          + "\"STRING_LIST_ID_KID\" = " + SKEWED_STRING_LIST_VALUES + ".\"STRING_LIST_ID\" "
+          + "where " + SKEWED_COL_VALUE_LOC_MAP + ".\"SD_ID\" in (" + sdIds + ")"
+          + "  and " + SKEWED_COL_VALUE_LOC_MAP + ".\"STRING_LIST_ID_KID\" is not null "
+          + "order by " + SKEWED_COL_VALUE_LOC_MAP + ".\"SD_ID\" asc,"
+          + "  " + SKEWED_STRING_LIST_VALUES + ".\"STRING_LIST_ID\" asc,"
+          + "  " + SKEWED_STRING_LIST_VALUES + ".\"INTEGER_IDX\" asc";
+
+      loopJoinOrderedResult(sds, queryText, 0, new ApplyFunc<StorageDescriptor>() {
+        private Long currentListId;
+        private List<String> currentList;
+        @Override
+        public void apply(StorageDescriptor t, Object[] fields) throws MetaException {
+          if (!t.isSetSkewedInfo()) {
+            SkewedInfo skewedInfo = new SkewedInfo();
+            skewedInfo.setSkewedColValueLocationMaps(new HashMap<List<String>, String>());
+            t.setSkewedInfo(skewedInfo);
+          }
+          Map<List<String>, String> skewMap = t.getSkewedInfo().getSkewedColValueLocationMaps();
+          // Note that this is not a typical list accumulator - there's no call to finalize
+          // the last list. Instead we add list to SD first, as well as locally to add elements.
+          if (fields[1] == null) {
+            currentList = new ArrayList<String>(); // left outer join produced a list with no values
+            currentListId = null;
+          } else {
+            long fieldsListId = extractSqlLong(fields[1]);
+            if (currentListId == null || fieldsListId != currentListId) {
+              currentList = new ArrayList<String>();
+              currentListId = fieldsListId;
+            } else {
+              skewMap.remove(currentList); // value based compare.. remove first
+            }
+            currentList.add((String)fields[3]);
+          }
+          skewMap.put(currentList, (String)fields[2]);
+        }});
+    } // if (hasSkewedColumns)
+
+    // Get FieldSchema stuff if any.
+    if (!colss.isEmpty()) {
+      // We are skipping the CDS table here, as it seems to be totally useless.
+      queryText = "select \"CD_ID\", \"COMMENT\", \"COLUMN_NAME\", \"TYPE_NAME\""
+          + " from " + COLUMNS_V2 + " where \"CD_ID\" in (" + colIds + ")"
+          + " order by \"CD_ID\" asc, \"INTEGER_IDX\" asc";
+      loopJoinOrderedResult(colss, queryText, 0, new ApplyFunc<List<FieldSchema>>() {
+        @Override
+        public void apply(List<FieldSchema> t, Object[] fields) {
+          t.add(new FieldSchema((String)fields[2], extractSqlClob(fields[3]), (String)fields[1]));
+        }});
+    }
+
+    // Finally, get all the stuff for serdes - just the params.
+    queryText = "select \"SERDE_ID\", \"PARAM_KEY\", \"PARAM_VALUE\" from " + SERDE_PARAMS + ""
+        + " where \"SERDE_ID\" in (" + serdeIds + ") and \"PARAM_KEY\" is not null"
+        + " order by \"SERDE_ID\" asc";
+    loopJoinOrderedResult(serdes, queryText, 0, new ApplyFunc<SerDeInfo>() {
+      @Override
+      public void apply(SerDeInfo t, Object[] fields) {
+        t.putToParameters((String)fields[1], extractSqlClob(fields[2]));
+      }});
+    // Perform conversion of null map values
+    for (SerDeInfo t : serdes.values()) {
+      t.setParameters(MetaStoreServerUtils.trimMapNulls(t.getParameters(), convertMapNullsToEmptyStrings));
+    }
+
+    return orderedResult;
+  }
+
+  public int getNumPartitionsViaSqlFilter(SqlFilterForPushdown filter) throws MetaException {
+    boolean doTrace = LOG.isDebugEnabled();
+    String catName = filter.table.getCatName().toLowerCase();
+    String dbName = filter.table.getDbName().toLowerCase();
+    String tblName = filter.table.getTableName().toLowerCase();
+
+    // Get number of partitions by doing count on PART_ID.
+    String queryText = "select count(" + PARTITIONS + ".\"PART_ID\") from " + PARTITIONS + ""
+      + "  inner join " + TBLS + " on " + PARTITIONS + ".\"TBL_ID\" = " + TBLS + ".\"TBL_ID\" "
+      + "    and " + TBLS + ".\"TBL_NAME\" = ? "
+      + "  inner join " + DBS + " on " + TBLS + ".\"DB_ID\" = " + DBS + ".\"DB_ID\" "
+      + "     and " + DBS + ".\"NAME\" = ? "
+      + join(filter.joins, ' ')
+      + " where " + DBS + ".\"CTLG_NAME\" = ? "
+      + (filter.filter == null || filter.filter.trim().isEmpty() ? "" : (" and " + filter.filter));
+
+    Object[] params = new Object[filter.params.size() + 3];
+    params[0] = tblName;
+    params[1] = dbName;
+    params[2] = catName;
+    for (int i = 0; i < filter.params.size(); ++i) {
+      params[i + 3] = filter.params.get(i);
+    }
+
+    long start = doTrace ? System.nanoTime() : 0;
+    Query query = pm.newQuery("javax.jdo.query.SQL", queryText);
+    query.setUnique(true);
+    int sqlResult = extractSqlInt(query.executeWithArray(params));
+    long queryTime = doTrace ? System.nanoTime() : 0;
+    timingTrace(doTrace, queryText, start, queryTime);
+    return sqlResult;
+  }
+
+
+  private void timingTrace(boolean doTrace, String queryText, long start, long queryTime) {
+    if (!doTrace) return;
+    LOG.debug("Direct SQL query in " + (queryTime - start) / 1000000.0 + "ms + " +
+        (System.nanoTime() - queryTime) / 1000000.0 + "ms, the query is [" + queryText + "]");
+  }
+
+  static Long extractSqlLong(Object obj) throws MetaException {
+    if (obj == null) return null;
+    if (!(obj instanceof Number)) {
+      throw new MetaException("Expected numeric type but got " + obj.getClass().getName());
+    }
+    return ((Number)obj).longValue();
+  }
+
+  /**
+   * Convert a boolean value returned from the RDBMS to a Java Boolean object.
+   * MySQL has booleans, but e.g. Derby uses 'Y'/'N' mapping.
+   *
+   * @param value
+   *          column value from the database
+   * @return The Boolean value of the database column value, null if the column
+   *         value is null
+   * @throws MetaException
+   *           if the column value cannot be converted into a Boolean object
+   */
+  private static Boolean extractSqlBoolean(Object value) throws MetaException {
+    if (value == null) {
+      return null;
+    }
+    if (value instanceof Boolean) {
+      return (Boolean)value;
+    }
+    if (value instanceof String) {
+      try {
+        return BooleanUtils.toBooleanObject((String) value, "Y", "N", null);
+      } catch (IllegalArgumentException iae) {
+        // NOOP
+      }
+    }
+    throw new MetaException("Cannot extract boolean from column value " + value);
+  }
+
+  private int extractSqlInt(Object field) {
+    return ((Number)field).intValue();
+  }
+
+  private String extractSqlString(Object value) {
+    if (value == null) return null;
+    return value.toString();
+  }
+
+  static Double extractSqlDouble(Object obj) throws MetaException {
+    if (obj == null)
+      return null;
+    if (!(obj instanceof Number)) {
+      throw new MetaException("Expected numeric type but got " + obj.getClass().getName());
+    }
+    return ((Number) obj).doubleValue();
+  }
+
+  private String extractSqlClob(Object value) {
+    if (value == null) return null;
+    try {
+      if (value instanceof Clob) {
+        // we trim the Clob value to a max length an int can hold
+        int maxLength = (((Clob)value).length() < Integer.MAX_VALUE - 2) ? (int)((Clob)value).length() : Integer.MAX_VALUE - 2;
+        return ((Clob)value).getSubString(1L, maxLength);
+      } else {
+        return value.toString();
+      }
+    } catch (SQLException sqle) {
+      return null;
+    }
+  }
+
+  static byte[] extractSqlBlob(Object value) throws MetaException {
+    if (value == null)
+      return null;
+    if (value instanceof Blob) {
+      //derby, oracle
+      try {
+        // getBytes function says: pos the ordinal position of the first byte in
+        // the BLOB value to be extracted; the first byte is at position 1
+        return ((Blob) value).getBytes(1, (int) ((Blob) value).length());
+      } catch (SQLException e) {
+        throw new MetaException("Encounter error while processing blob.");
+      }
+    }
+    else if (value instanceof byte[]) {
+      // mysql, postgres, sql server
+      return (byte[]) value;
+    }
+	else {
+      // this may happen when enablebitvector is false
+      LOG.debug("Expected blob type but got " + value.getClass().getName());
+      return null;
+    }
+  }
+
+  /**
+   * Helper method for preparing for "SOMETHING_ID in (...)" to use in future queries.
+   * @param objectIds the objectId collection
+   * @return The concatenated list
+   * @throws MetaException If the list contains wrong data
+   */
+  private static String getIdListForIn(List<Object> objectIds) throws MetaException {
+    return objectIds.stream()
+               .map(i -> i.toString())
+               .collect(Collectors.joining(","));
+  }
+
+  private static String trimCommaList(StringBuilder sb) {
+    if (sb.length() > 0) {
+      sb.setLength(sb.length() - 1);
+    }
+    return sb.toString();
+  }
+
+  private abstract class ApplyFunc<Target> {
+    public abstract void apply(Target t, Object[] fields) throws MetaException;
+  }
+
+  /**
+   * Merges applies the result of a PM SQL query into a tree of object.
+   * Essentially it's an object join. DN could do this for us, but it issues queries
+   * separately for every object, which is suboptimal.
+   * @param tree The object tree, by ID.
+   * @param queryText The query text.
+   * @param keyIndex Index of the Long column corresponding to the map ID in query result rows.
+   * @param func The function that is called on each (object,row) pair with the same id.
+   * @return the count of results returned from the query.
+   */
+  private <T> int loopJoinOrderedResult(TreeMap<Long, T> tree,
+      String queryText, int keyIndex, ApplyFunc<T> func) throws MetaException {
+    boolean doTrace = LOG.isDebugEnabled();
+    long start = doTrace ? System.nanoTime() : 0;
+    Query query = pm.newQuery("javax.jdo.query.SQL", queryText);
+    Object result = query.execute();
+    long queryTime = doTrace ? System.nanoTime() : 0;
+    if (result == null) {
+      query.closeAll();
+      return 0;
+    }
+    List<Object[]> list = ensureList(result);
+    Iterator<Object[]> iter = list.iterator();
+    Object[] fields = null;
+    for (Map.Entry<Long, T> entry : tree.entrySet()) {
+      if (fields == null && !iter.hasNext()) break;
+      long id = entry.getKey();
+      while (fields != null || iter.hasNext()) {
+        if (fields == null) {
+          fields = iter.next();
+        }
+        long nestedId = extractSqlLong(fields[keyIndex]);
+        if (nestedId < id) throw new MetaException("Found entries for unknown ID " + nestedId);
+        if (nestedId > id) break; // fields belong to one of the next entries
+        func.apply(entry.getValue(), fields);
+        fields = null;
+      }
+      Deadline.checkTimeout();
+    }
+    int rv = list.size();
+    query.closeAll();
+    timingTrace(doTrace, queryText, start, queryTime);
+    return rv;
+  }
+
+  private static class PartitionFilterGenerator extends TreeVisitor {
+    private final Table table;
+    private final FilterBuilder filterBuffer;
+    private final List<Object> params;
+    private final List<String> joins;
+    private final boolean dbHasJoinCastBug;
+    private final String defaultPartName;
+    private final DatabaseProduct dbType;
+    private final String PARTITION_KEY_VALS, PARTITIONS, DBS, TBLS;
+
+    private PartitionFilterGenerator(Table table, List<Object> params, List<String> joins,
+        boolean dbHasJoinCastBug, String defaultPartName, DatabaseProduct dbType, String schema) {
+      this.table = table;
+      this.params = params;
+      this.joins = joins;
+      this.dbHasJoinCastBug = dbHasJoinCastBug;
+      this.filterBuffer = new FilterBuilder(false);
+      this.defaultPartName = defaultPartName;
+      this.dbType = dbType;
+      this.PARTITION_KEY_VALS = getFullyQualifiedName(schema, "PARTITION_KEY_VALS");
+      this.PARTITIONS = getFullyQualifiedName(schema, "PARTITIONS");
+      this.DBS = getFullyQualifiedName(schema, "DBS");
+      this.TBLS = getFullyQualifiedName(schema, "TBLS");
+    }
+
+    /**
+     * Generate the ANSI SQL92 filter for the given expression tree
+     * @param table the table being queried
+     * @param params the ordered parameters for the resulting expression
+     * @param joins the joins necessary for the resulting expression
+     * @return the string representation of the expression tree
+     */
+    private static String generateSqlFilter(Table table, ExpressionTree tree, List<Object> params,
+        List<String> joins, boolean dbHasJoinCastBug, String defaultPartName,
+        DatabaseProduct dbType, String schema) throws MetaException {
+      assert table != null;
+      if (tree == null) {
+        // consistent with other APIs like makeExpressionTree, null is returned to indicate that
+        // the filter could not pushed down due to parsing issue etc
+        return null;
+      }
+      if (tree.getRoot() == null) {
+        return "";
+      }
+      PartitionFilterGenerator visitor = new PartitionFilterGenerator(
+          table, params, joins, dbHasJoinCastBug, defaultPartName, dbType, schema);
+      tree.accept(visitor);
+      if (visitor.filterBuffer.hasError()) {
+        LOG.info("Unable to push down SQL filter: " + visitor.filterBuffer.getErrorMessage());
+        return null;
+      }
+
+      // Some joins might be null (see processNode for LeafNode), clean them up.
+      for (int i = 0; i < joins.size(); ++i) {
+        if (joins.get(i) != null) continue;
+        joins.remove(i--);
+      }
+      return "(" + visitor.filterBuffer.getFilter() + ")";
+    }
+
+    @Override
+    protected void beginTreeNode(TreeNode node) throws MetaException {
+      filterBuffer.append(" (");
+    }
+
+    @Override
+    protected void midTreeNode(TreeNode node) throws MetaException {
+      filterBuffer.append((node.getAndOr() == LogicalOperator.AND) ? " and " : " or ");
+    }
+
+    @Override
+    protected void endTreeNode(TreeNode node) throws MetaException {
+      filterBuffer.append(") ");
+    }
+
+    @Override
+    protected boolean shouldStop() {
+      return filterBuffer.hasError();
+    }
+
+    private static enum FilterType {
+      Integral,
+      String,
+      Date,
+
+      Invalid;
+
+      static FilterType fromType(String colTypeStr) {
+        if (colTypeStr.equals(ColumnType.STRING_TYPE_NAME)) {
+          return FilterType.String;
+        } else if (colTypeStr.equals(ColumnType.DATE_TYPE_NAME)) {
+          return FilterType.Date;
+        } else if (ColumnType.IntegralTypes.contains(colTypeStr)) {
+          return FilterType.Integral;
+        }
+        return FilterType.Invalid;
+      }
+
+      public static FilterType fromClass(Object value) {
+        if (value instanceof String) {
+          return FilterType.String;
+        } else if (value instanceof Long) {
+          return FilterType.Integral;
+        } else if (value instanceof java.sql.Date) {
+          return FilterType.Date;
+        }
+        return FilterType.Invalid;
+      }
+    }
+
+    @Override
+    public void visit(LeafNode node) throws MetaException {
+      if (node.operator == Operator.LIKE) {
+        filterBuffer.setError("LIKE is not supported for SQL filter pushdown");
+        return;
+      }
+      int partColCount = table.getPartitionKeys().size();
+      int partColIndex = node.getPartColIndexForFilter(table, filterBuffer);
+      if (filterBuffer.hasError()) return;
+
+      // We skipped 'like', other ops should all work as long as the types are right.
+      String colTypeStr = table.getPartitionKeys().get(partColIndex).getType();
+      FilterType colType = FilterType.fromType(colTypeStr);
+      if (colType == FilterType.Invalid) {
+        filterBuffer.setError("Filter pushdown not supported for type " + colTypeStr);
+        return;
+      }
+      FilterType valType = FilterType.fromClass(node.value);
+      Object nodeValue = node.value;
+      if (valType == FilterType.Invalid) {
+        filterBuffer.setError("Filter pushdown not supported for value " + node.value.getClass());
+        return;
+      }
+
+      // if Filter.g does date parsing for quoted strings, we'd need to verify there's no
+      // type mismatch when string col is filtered by a string that looks like date.
+      if (colType == FilterType.Date && valType == FilterType.String) {
+        // Filter.g cannot parse a quoted date; try to parse date here too.
+        try {
+          nodeValue = MetaStoreUtils.PARTITION_DATE_FORMAT.get().parse((String)nodeValue);
+          valType = FilterType.Date;
+        } catch (ParseException pe) { // do nothing, handled below - types will mismatch
+        }
+      }
+
+      // We format it so we are sure we are getting the right value
+      if (valType == FilterType.Date) {
+        // Format
+        nodeValue = MetaStoreUtils.PARTITION_DATE_FORMAT.get().format(nodeValue);
+      }
+
+      boolean isDefaultPartition = (valType == FilterType.String) && defaultPartName.equals(nodeValue);
+      if ((colType != valType) && (!isDefaultPartition)) {
+        // It's not clear how filtering for e.g. "stringCol > 5" should work (which side is
+        // to be coerced?). Let the expression evaluation sort this one out, not metastore.
+        filterBuffer.setError("Cannot push down filter for "
+            + colTypeStr + " column and value " + nodeValue.getClass());
+        return;
+      }
+
+      if (joins.isEmpty()) {
+        // There's a fixed number of partition cols that we might have filters on. To avoid
+        // joining multiple times for one column (if there are several filters on it), we will
+        // keep numCols elements in the list, one for each column; we will fill it with nulls,
+        // put each join at a corresponding index when necessary, and remove nulls in the end.
+        for (int i = 0; i < partColCount; ++i) {
+          joins.add(null);
+        }
+      }
+      if (joins.get(partColIndex) == null) {
+        joins.set(partColIndex, "inner join " + PARTITION_KEY_VALS + " \"FILTER" + partColIndex
+            + "\" on \"FILTER"  + partColIndex + "\".\"PART_ID\" = " + PARTITIONS + ".\"PART_ID\""
+            + " and \"FILTER" + partColIndex + "\".\"INTEGER_IDX\" = " + partColIndex);
+      }
+
+      // Build the filter and add parameters linearly; we are traversing leaf nodes LTR.
+      String tableValue = "\"FILTER" + partColIndex + "\".\"PART_KEY_VAL\"";
+
+      if (node.isReverseOrder) {
+        params.add(nodeValue);
+      }
+      String tableColumn = tableValue;
+      if ((colType != FilterType.String) && (!isDefaultPartition)) {
+        // The underlying database field is varchar, we need to compare numbers.
+        if (colType == FilterType.Integral) {
+          tableValue = "cast(" + tableValue + " as decimal(21,0))";
+        } else if (colType == FilterType.Date) {
+          if (dbType == DatabaseProduct.ORACLE) {
+            // Oracle requires special treatment... as usual.
+            tableValue = "TO_DATE(" + tableValue + ", 'YYYY-MM-DD')";
+          } else {
+            tableValue = "cast(" + tableValue + " as date)";
+          }
+        }
+
+        // Workaround for HIVE_DEFAULT_PARTITION - ignore it like JDO does, for now.
+        String tableValue0 = tableValue;
+        tableValue = "(case when " + tableColumn + " <> ?";
+        params.add(defaultPartName);
+
+        if (dbHasJoinCastBug) {
+          // This is a workaround for DERBY-6358 and Oracle bug; it is pretty horrible.
+          tableValue += (" and " + TBLS + ".\"TBL_NAME\" = ? and " + DBS + ".\"NAME\" = ? and "
+              + DBS + ".\"CTLG_NAME\" = ? and "
+              + "\"FILTER" + partColIndex + "\".\"PART_ID\" = " + PARTITIONS + ".\"PART_ID\" and "
+                + "\"FILTER" + partColIndex + "\".\"INTEGER_IDX\" = " + partColIndex);
+          params.add(table.getTableName().toLowerCase());
+          params.add(table.getDbName().toLowerCase());
+          params.add(table.getCatName().toLowerCase());
+        }
+        tableValue += " then " + tableValue0 + " else null end)";
+      }
+      if (!node.isReverseOrder) {
+        params.add(nodeValue);
+      }
+
+      filterBuffer.append(node.isReverseOrder
+          ? "(? " + node.operator.getSqlOp() + " " + tableValue + ")"
+          : "(" + tableValue + " " + node.operator.getSqlOp() + " ?)");
+    }
+  }
+
+  /**
+   * Retrieve the column statistics for the specified columns of the table. NULL
+   * is returned if the columns are not provided.
+   * @param catName     the catalog name of the table
+   * @param dbName      the database name of the table
+   * @param tableName   the table name
+   * @param colNames    the list of the column names
+   * @return            the column statistics for the specified columns
+   * @throws MetaException
+   */
+  public ColumnStatistics getTableStats(final String catName, final String dbName,
+                                        final String tableName, List<String> colNames,
+                                        boolean enableBitVector) throws MetaException {
+    if (colNames == null || colNames.isEmpty()) {
+      return null;
+    }
+    final boolean doTrace = LOG.isDebugEnabled();
+    final String queryText0 = "select " + getStatsList(enableBitVector) + " from " + TAB_COL_STATS
+          + " where \"CAT_NAME\" = ? and \"DB_NAME\" = ? and \"TABLE_NAME\" = ? and \"COLUMN_NAME\" in (";
+    Batchable<String, Object[]> b = new Batchable<String, Object[]>() {
+      @Override
+      public List<Object[]> run(List<String> input) throws MetaException {
+        String queryText = queryText0 + makeParams(input.size()) + ")";
+        Object[] params = new Object[input.size() + 3];
+        params[0] = catName;
+        params[1] = dbName;
+        params[2] = tableName;
+        for (int i = 0; i < input.size(); ++i) {
+          params[i + 3] = input.get(i);
+        }
+        long start = doTrace ? System.nanoTime() : 0;
+        Query query = pm.newQuery("javax.jdo.query.SQL", queryText);
+        Object qResult = executeWithArray(query, params, queryText);
+        timingTrace(doTrace, queryText0 + "...)", start, (doTrace ? System.nanoTime() : 0));
+        if (qResult == null) {
+          query.closeAll();
+          return null;
+        }
+        addQueryAfterUse(query);
+        return ensureList(qResult);
+      }
+    };
+    List<Object[]> list = Batchable.runBatched(batchSize, colNames, b);
+    if (list.isEmpty()) {
+      return null;
+    }
+    ColumnStatisticsDesc csd = new ColumnStatisticsDesc(true, dbName, tableName);
+    csd.setCatName(catName);
+    ColumnStatistics result = makeColumnStats(list, csd, 0);
+    b.closeAllQueries();
+    return result;
+  }
+
+  public AggrStats aggrColStatsForPartitions(String catName, String dbName, String tableName,
+      List<String> partNames, List<String> colNames, boolean useDensityFunctionForNDVEstimation,
+      double ndvTuner, boolean enableBitVector) throws MetaException {
+    if (colNames.isEmpty() || partNames.isEmpty()) {
+      LOG.debug("Columns is empty or partNames is empty : Short-circuiting stats eval");
+      return new AggrStats(Collections.<ColumnStatisticsObj>emptyList(), 0); // Nothing to aggregate
+    }
+    long partsFound = 0;
+    List<ColumnStatisticsObj> colStatsList;
+    // Try to read from the cache first
+    if (isAggregateStatsCacheEnabled
+        && (partNames.size() < aggrStatsCache.getMaxPartsPerCacheNode())) {
+      AggrColStats colStatsAggrCached;
+      List<ColumnStatisticsObj> colStatsAggrFromDB;
+      int maxPartsPerCacheNode = aggrStatsCache.getMaxPartsPerCacheNode();
+      double fpp = aggrStatsCache.getFalsePositiveProbability();
+      colStatsList = new ArrayList<ColumnStatisticsObj>();
+      // Bloom filter for the new node that we will eventually add to the cache
+      BloomFilter bloomFilter = createPartsBloomFilter(maxPartsPerCacheNode, fpp, partNames);
+      boolean computePartsFound = true;
+      for (String colName : colNames) {
+        // Check the cache first
+        colStatsAggrCached = aggrStatsCache.get(catName, dbName, tableName, colName, partNames);
+        if (colStatsAggrCached != null) {
+          colStatsList.add(colStatsAggrCached.getColStats());
+          partsFound = colStatsAggrCached.getNumPartsCached();
+        } else {
+          if (computePartsFound) {
+            partsFound = partsFoundForPartitions(catName, dbName, tableName, partNames, colNames);
+            computePartsFound = false;
+          }
+          List<String> colNamesForDB = new ArrayList<>();
+          colNamesForDB.add(colName);
+          // Read aggregated stats for one column
+          colStatsAggrFromDB =
+              columnStatisticsObjForPartitions(catName, dbName, tableName, partNames, colNamesForDB,
+                  partsFound, useDensityFunctionForNDVEstimation, ndvTuner, enableBitVector);
+          if (!colStatsAggrFromDB.isEmpty()) {
+            ColumnStatisticsObj colStatsAggr = colStatsAggrFromDB.get(0);
+            colStatsList.add(colStatsAggr);
+            // Update the cache to add this new aggregate node
+            aggrStatsCache.add(catName, dbName, tableName, colName, partsFound, colStatsAggr, bloomFilter);
+          }
+        }
+      }
+    } else {
+      partsFound = partsFoundForPartitions(catName, dbName, tableName, partNames, colNames);
+      colStatsList =
+          columnStatisticsObjForPartitions(catName, dbName, tableName, partNames, colNames, partsFound,
+              useDensityFunctionForNDVEstimation, ndvTuner, enableBitVector);
+    }
+    LOG.info("useDensityFunctionForNDVEstimation = " + useDensityFunctionForNDVEstimation
+        + "\npartsFound = " + partsFound + "\nColumnStatisticsObj = "
+        + Arrays.toString(colStatsList.toArray()));
+    return new AggrStats(colStatsList, partsFound);
+  }
+
+  private BloomFilter createPartsBloomFilter(int maxPartsPerCacheNode, double fpp,
+      List<String> partNames) {
+    BloomFilter bloomFilter = new BloomFilter(maxPartsPerCacheNode, fpp);
+    for (String partName : partNames) {
+      bloomFilter.add(partName.getBytes());
+    }
+    return bloomFilter;
+  }
+
+  private long partsFoundForPartitions(
+      final String catName, final String dbName, final String tableName,
+      final List<String> partNames, List<String> colNames) throws MetaException {
+    assert !colNames.isEmpty() && !partNames.isEmpty();
+    final boolean doTrace = LOG.isDebugEnabled();
+    final String queryText0  = "select count(\"COLUMN_NAME\") from " + PART_COL_STATS + ""
+        + " where \"CAT_NAME\" = ? and \"DB_NAME\" = ? and \"TABLE_NAME\" = ? "
+        + " and \"COLUMN_NAME\" in (%1$s) and \"PARTITION_NAME\" in (%2$s)"
+        + " group by \"PARTITION_NAME\"";
+    List<Long> allCounts = Batchable.runBatched(batchSize, colNames, new Batchable<String, Long>() {
+      @Override
+      public List<Long> run(final List<String> inputColName) throws MetaException {
+        return Batchable.runBatched(batchSize, partNames, new Batchable<String, Long>() {
+          @Override
+          public List<Long> run(List<String> inputPartNames) throws MetaException {
+            long partsFound = 0;
+            String queryText = String.format(queryText0,
+                makeParams(inputColName.size()), makeParams(inputPartNames.size()));
+            long start = doTrace ? System.nanoTime() : 0;
+            Query query = pm.newQuery("javax.jdo.query.SQL", queryText);
+            try {
+              Object qResult = executeWithArray(query, prepareParams(
+                  catName, dbName, tableName, inputPartNames, inputColName), queryText);
+              long end = doTrace ? System.nanoTime() : 0;
+              timingTrace(doTrace, queryText, start, end);
+              ForwardQueryResult<?> fqr = (ForwardQueryResult<?>) qResult;
+              Iterator<?> iter = fqr.iterator();
+              while (iter.hasNext()) {
+                if (extractSqlLong(iter.next()) == inputColName.size()) {
+                  partsFound++;
+                }
+              }
+              return Lists.<Long>newArrayList(partsFound);
+            } finally {
+              query.closeAll();
+            }
+          }
+        });
+      }
+    });
+    long partsFound = 0;
+    for (Long val : allCounts) {
+      partsFound += val;
+    }
+    return partsFound;
+  }
+
+  private List<ColumnStatisticsObj> columnStatisticsObjForPartitions(
+      final String catName, final String dbName,
+    final String tableName, final List<String> partNames, List<String> colNames, long partsFound,
+    final boolean useDensityFunctionForNDVEstimation, final double ndvTuner, final boolean enableBitVector) throws MetaException {
+    final boolean areAllPartsFound = (partsFound == partNames.size());
+    return Batchable.runBatched(batchSize, colNames, new Batchable<String, ColumnStatisticsObj>() {
+      @Override
+      public List<ColumnStatisticsObj> run(final List<String> inputColNames) throws MetaException {
+        return Batchable.runBatched(batchSize, partNames, new Batchable<String, ColumnStatisticsObj>() {
+          @Override
+          public List<ColumnStatisticsObj> run(List<String> inputPartNames) throws MetaException {
+            return columnStatisticsObjForPartitionsBatch(catName, dbName, tableName, inputPartNames,
+                inputColNames, areAllPartsFound, useDensityFunctionForNDVEstimation, ndvTuner, enableBitVector);
+          }
+        });
+      }
+    });
+  }
+
+  public List<ColStatsObjWithSourceInfo> getColStatsForAllTablePartitions(String catName, String dbName,
+      boolean enableBitVector) throws MetaException {
+    String queryText = "select \"TABLE_NAME\", \"PARTITION_NAME\", " + getStatsList(enableBitVector)
+        + " from " + " " + PART_COL_STATS + " where \"DB_NAME\" = ? and \"CAT_NAME\" = ?";
+    long start = 0;
+    long end = 0;
+    Query query = null;
+    boolean doTrace = LOG.isDebugEnabled();
+    Object qResult = null;
+    start = doTrace ? System.nanoTime() : 0;
+    List<ColStatsObjWithSourceInfo> colStatsForDB = new ArrayList<ColStatsObjWithSourceInfo>();
+    try {
+      query = pm.newQuery("javax.jdo.query.SQL", queryText);
+      qResult = executeWithArray(query, new Object[] { dbName, catName }, queryText);
+      if (qResult == null) {
+        query.closeAll();
+        return colStatsForDB;
+      }
+      end = doTrace ? System.nanoTime() : 0;
+      timingTrace(doTrace, queryText, start, end);
+      List<Object[]> list = ensureList(qResult);
+      for (Object[] row : list) {
+        String tblName = (String) row[0];
+        String partName = (String) row[1];
+        ColumnStatisticsObj colStatObj = prepareCSObj(row, 2);
+        colStatsForDB.add(new ColStatsObjWithSourceInfo(colStatObj, catName, dbName, tblName, partName));
+        Deadline.checkTimeout();
+      }
+    } finally {
+      query.closeAll();
+    }
+    return colStatsForDB;
+  }
+
+  /** Should be called with the list short enough to not trip up Oracle/etc. */
+  private List<ColumnStatisticsObj> columnStatisticsObjForPartitionsBatch(String catName, String dbName,
+      String tableName, List<String> partNames, List<String> colNames, boolean areAllPartsFound,
+      boolean useDensityFunctionForNDVEstimation, double ndvTuner, boolean enableBitVector)
+      throws MetaException {
+    if (enableBitVector) {
+      return aggrStatsUseJava(catName, dbName, tableName, partNames, colNames, areAllPartsFound,
+          useDensityFunctionForNDVEstimation, ndvTuner);
+    } else {
+      return aggrStatsUseDB(catName, dbName, tableName, partNames, colNames, areAllPartsFound,
+          useDensityFunctionForNDVEstimation, ndvTuner);
+    }
+  }
+
+  private List<ColumnStatisticsObj> aggrStatsUseJava(String catName, String dbName, String tableName,
+      List<String> partNames, List<String> colNames, boolean areAllPartsFound,
+      boolean useDensityFunctionForNDVEstimation, double ndvTuner) throws MetaException {
+    // 1. get all the stats for colNames in partNames;
+    List<ColumnStatistics> partStats =
+        getPartitionStats(catName, dbName, tableName, partNames, colNames, true);
+    // 2. use util function to aggr stats
+    return MetaStoreServerUtils.aggrPartitionStats(partStats, catName, dbName, tableName, partNames, colNames,
+        areAllPartsFound, useDensityFunctionForNDVEstimation, ndvTuner);
+  }
+
+  private List<ColumnStatisticsObj> aggrStatsUseDB(String catName, String dbName,
+      String tableName, List<String> partNames, List<String> colNames, boolean areAllPartsFound,
+      boolean useDensityFunctionForNDVEstimation, double ndvTuner) throws MetaException {
+    // TODO: all the extrapolation logic should be moved out of this class,
+    // only mechanical data retrieval should remain here.
+    String commonPrefix = "select \"COLUMN_NAME\", \"COLUMN_TYPE\", "
+        + "min(\"LONG_LOW_VALUE\"), max(\"LONG_HIGH_VALUE\"), min(\"DOUBLE_LOW_VALUE\"), max(\"DOUBLE_HIGH_VALUE\"), "
+        + "min(cast(\"BIG_DECIMAL_LOW_VALUE\" as decimal)), max(cast(\"BIG_DECIMAL_HIGH_VALUE\" as decimal)), "
+        + "sum(\"NUM_NULLS\"), max(\"NUM_DISTINCTS\"), "
+        + "max(\"AVG_COL_LEN\"), max(\"MAX_COL_LEN\"), sum(\"NUM_TRUES\"), sum(\"NUM_FALSES\"), "
+        // The following data is used to compute a partitioned table's NDV based
+        // on partitions' NDV when useDensityFunctionForNDVEstimation = true. Global NDVs cannot be
+        // accurately derived from partition NDVs, because the domain of column value two partitions
+        // can overlap. If there is no overlap then global NDV is just the sum
+        // of partition NDVs (UpperBound). But if there is some overlay then
+        // global NDV can be anywhere between sum of partition NDVs (no overlap)
+        // and same as one of the partition NDV (domain of column value in all other
+        // partitions is subset of the domain value in one of the partition)
+        // (LowerBound).But under uniform distribution, we can roughly estimate the global
+        // NDV by leveraging the min/max values.
+        // And, we also guarantee that the estimation makes sense by comparing it to the
+        // UpperBound (calculated by "sum(\"NUM_DISTINCTS\")")
+        // and LowerBound (calculated by "max(\"NUM_DISTINCTS\")")
+        + "avg((\"LONG_HIGH_VALUE\"-\"LONG_LOW_VALUE\")/cast(\"NUM_DISTINCTS\" as decimal)),"
+        + "avg((\"DOUBLE_HIGH_VALUE\"-\"DOUBLE_LOW_VALUE\")/\"NUM_DISTINCTS\"),"
+        + "avg((cast(\"BIG_DECIMAL_HIGH_VALUE\" as decimal)-cast(\"BIG_DECIMAL_LOW_VALUE\" as decimal))/\"NUM_DISTINCTS\"),"
+        + "sum(\"NUM_DISTINCTS\")" + " from " + PART_COL_STATS + ""
+        + " where \"CAT_NAME\" = ? and \"DB_NAME\" = ? and \"TABLE_NAME\" = ? ";
+    String queryText = null;
+    long start = 0;
+    long end = 0;
+    Query query = null;
+    boolean doTrace = LOG.isDebugEnabled();
+    Object qResult = null;
+    ForwardQueryResult<?> fqr = null;
+    // Check if the status of all the columns of all the partitions exists
+    // Extrapolation is not needed.
+    if (areAllPartsFound) {
+      queryText = commonPrefix + " and \"COLUMN_NAME\" in (" + makeParams(colNames.size()) + ")"
+          + " and \"PARTITION_NAME\" in (" + makeParams(partNames.size()) + ")"
+          + " group by \"COLUMN_NAME\", \"COLUMN_TYPE\"";
+      start = doTrace ? System.nanoTime() : 0;
+      query = pm.newQuery("javax.jdo.query.SQL", queryText);
+      qResult = executeWithArray(query, prepareParams(catName, dbName, tableName, partNames, colNames),
+          queryText);
+      if (qResult == null) {
+        query.closeAll();
+        return Collections.emptyList();
+      }
+      end = doTrace ? System.nanoTime() : 0;
+      timingTrace(doTrace, queryText, start, end);
+      List<Object[]> list = ensureList(qResult);
+      List<ColumnStatisticsObj> colStats = new ArrayList<ColumnStatisticsObj>(list.size());
+      for (Object[] row : list) {
+        colStats.add(prepareCSObjWithAdjustedNDV(row, 0, useDensityFunctionForNDVEstimation, ndvTuner));
+        Deadline.checkTimeout();
+      }
+      query.closeAll();
+      return colStats;
+    } else {
+      // Extrapolation is needed for some columns.
+      // In this case, at least a column status for a partition is missing.
+      // We need to extrapolate this partition based on the other partitions
+      List<ColumnStatisticsObj> colStats = new ArrayList<ColumnStatisticsObj>(colNames.size());
+      queryText = "select \"COLUMN_NAME\", \"COLUMN_TYPE\", count(\"PARTITION_NAME\") "
+          + " from " + PART_COL_STATS
+          + " where \"CAT_NAME\" = ? and \"DB_NAME\" = ? and \"TABLE_NAME\" = ? "
+          + " and \"COLUMN_NAME\" in (" + makeParams(colNames.size()) + ")"
+          + " and \"PARTITION_NAME\" in (" + makeParams(partNames.size()) + ")"
+          + " group by \"COLUMN_NAME\", \"COLUMN_TYPE\"";
+      start = doTrace ? System.nanoTime() : 0;
+      query = pm.newQuery("javax.jdo.query.SQL", queryText);
+      qResult = executeWithArray(query, prepareParams(catName, dbName, tableName, partNames, colNames),
+          queryText);
+      end = doTrace ? System.nanoTime() : 0;
+      timingTrace(doTrace, queryText, start, end);
+      if (qResult == null) {
+        query.closeAll();
+        return Collections.emptyList();
+      }
+      List<String> noExtraColumnNames = new ArrayList<String>();
+      Map<String, String[]> extraColumnNameTypeParts = new HashMap<String, String[]>();
+      List<Object[]> list = ensureList(qResult);
+      for (Object[] row : list) {
+        String colName = (String) row[0];
+        String colType = (String) row[1];
+        // Extrapolation is not needed for this column if
+        // count(\"PARTITION_NAME\")==partNames.size()
+        // Or, extrapolation is not possible for this column if
+        // count(\"PARTITION_NAME\")<2
+        Long count = extractSqlLong(row[2]);
+        if (count == partNames.size() || count < 2) {
+          noExtraColumnNames.add(colName);
+        } else {
+          extraColumnNameTypeParts.put(colName, new String[] { colType, String.valueOf(count) });
+        }
+        Deadline.checkTimeout();
+      }
+      query.closeAll();
+      // Extrapolation is not needed for columns noExtraColumnNames
+      if (noExtraColumnNames.size() != 0) {
+        queryText = commonPrefix + " and \"COLUMN_NAME\" in ("
+            + makeParams(noExtraColumnNames.size()) + ")" + " and \"PARTITION_NAME\" in ("
+            + makeParams(partNames.size()) + ")" + " group by \"COLUMN_NAME\", \"COLUMN_TYPE\"";
+        start = doTrace ? System.nanoTime() : 0;
+        query = pm.newQuery("javax.jdo.query.SQL", queryText);
+        qResult = executeWithArray(query,
+            prepareParams(catName, dbName, tableName, partNames, noExtraColumnNames), queryText);
+        if (qResult == null) {
+          query.closeAll();
+          return Collections.emptyList();
+        }
+        list = ensureList(qResult);
+        for (Object[] row : list) {
+          colStats.add(prepareCSObjWithAdjustedNDV(row, 0, useDensityFunctionForNDVEstimation, ndvTuner));
+          Deadline.checkTimeout();
+        }
+        end = doTrace ? System.nanoTime() : 0;
+        timingTrace(doTrace, queryText, start, end);
+        query.closeAll();
+      }
+      // Extrapolation is needed for extraColumnNames.
+      // give a sequence number for all the partitions
+      if (extraColumnNameTypeParts.size() != 0) {
+        Map<String, Integer> indexMap = new HashMap<String, Integer>();
+        for (int index = 0; index < partNames.size(); index++) {
+          indexMap.put(partNames.get(index), index);
+        }
+        // get sum for all columns to reduce the number of queries
+        Map<String, Map<Integer, Object>> sumMap = new HashMap<String, Map<Integer, Object>>();
+        queryText = "select \"COLUMN_NAME\", sum(\"NUM_NULLS\"), sum(\"NUM_TRUES\"), sum(\"NUM_FALSES\"), sum(\"NUM_DISTINCTS\")"
+            + " from " + PART_COL_STATS + " where \"CAT_NAME\" = ? and \"DB_NAME\" = ? and \"TABLE_NAME\" = ? "
+            + " and \"COLUMN_NAME\" in (" + makeParams(extraColumnNameTypeParts.size())
+            + ") and \"PARTITION_NAME\" in (" + makeParams(partNames.size())
+            + ") group by \"COLUMN_NAME\"";
+        start = doTrace ? System.nanoTime() : 0;
+        query = pm.newQuery("javax.jdo.query.SQL", queryText);
+        List<String> extraColumnNames = new ArrayList<String>();
+        extraColumnNames.addAll(extraColumnNameTypeParts.keySet());
+        qResult = executeWithArray(query,
+            prepareParams(catName, dbName, tableName, partNames, extraColumnNames), queryText);
+        if (qResult == null) {
+          query.closeAll();
+          return Collections.emptyList();
+        }
+        list = ensureList(qResult);
+        // see the indexes for colstats in IExtrapolatePartStatus
+        Integer[] sumIndex = new Integer[] { 6, 10, 11, 15 };
+        for (Object[] row : list) {
+          Map<Integer, Object> indexToObject = new HashMap<Integer, Object>();
+          for (int ind = 1; ind < row.length; ind++) {
+            indexToObject.put(sumIndex[ind - 1], row[ind]);
+          }
+          // row[0] is the column name
+          sumMap.put((String) row[0], indexToObject);
+          Deadline.checkTimeout();
+        }
+        end = doTrace ? System.nanoTime() : 0;
+        timingTrace(doTrace, queryText, start, end);
+        query.closeAll();
+        for (Map.Entry<String, String[]> entry : extraColumnNameTypeParts.entrySet()) {
+          Object[] row = new Object[IExtrapolatePartStatus.colStatNames.length + 2];
+          String colName = entry.getKey();
+          String colType = entry.getValue()[0];
+          Long sumVal = Long.parseLong(entry.getValue()[1]);
+          // fill in colname
+          row[0] = colName;
+          // fill in coltype
+          row[1] = colType;
+          // use linear extrapolation. more complicated one can be added in the
+          // future.
+          IExtrapolatePartStatus extrapolateMethod = new LinearExtrapolatePartStatus();
+          // fill in colstatus
+          Integer[] index = null;
+          boolean decimal = false;
+          if (colType.toLowerCase().startsWith("decimal")) {
+            index = IExtrapolatePartStatus.indexMaps.get("decimal");
+            decimal = true;
+          } else {
+            index = IExtrapolatePartStatus.indexMaps.get(colType.toLowerCase());
+          }
+          // if the colType is not the known type, long, double, etc, then get
+          // all index.
+          if (index == null) {
+            index = IExtrapolatePartStatus.indexMaps.get("default");
+          }
+          for (int colStatIndex : index) {
+            String colStatName = IExtrapolatePartStatus.colStatNames[colStatIndex];
+            // if the aggregation type is sum, we do a scale-up
+            if (IExtrapolatePartStatus.aggrTypes[colStatIndex] == IExtrapolatePartStatus.AggrType.Sum) {
+              Object o = sumMap.get(colName).get(colStatIndex);
+              if (o == null) {
+                row[2 + colStatIndex] = null;
+              } else {
+                Long val = extractSqlLong(o);
+                row[2 + colStatIndex] = val / sumVal * (partNames.size());
+              }
+            } else if (IExtrapolatePartStatus.aggrTypes[colStatIndex] == IExtrapolatePartStatus.AggrType.Min
+                || IExtrapolatePartStatus.aggrTypes[colStatIndex] == IExtrapolatePartStatus.AggrType.Max) {
+              // if the aggregation type is min/max, we extrapolate from the
+              // left/right borders
+              if (!decimal) {
+                queryText = "select \"" + colStatName
+                    + "\",\"PARTITION_NAME\" from " + PART_COL_STATS
+                    + " where \"CAT_NAME\" = ? and \"DB_NAME\" = ? and \"TABLE_NAME\" = ?" + " and \"COLUMN_NAME\" = ?"
+                    + " and \"PARTITION_NAME\" in (" + makeParams(partNames.size()) + ")"
+                    + " order by \"" + colStatName + "\"";
+              } else {
+                queryText = "select \"" + colStatName
+                    + "\",\"PARTITION_NAME\" from " + PART_COL_STATS
+                    + " where \"CAT_NAME\" = ? and \"DB_NAME\" = ? and \"TABLE_NAME\" = ?" + " and \"COLUMN_NAME\" = ?"
+                    + " and \"PARTITION_NAME\" in (" + makeParams(partNames.size()) + ")"
+                    + " order by cast(\"" + colStatName + "\" as decimal)";
+              }
+              start = doTrace ? System.nanoTime() : 0;
+              query = pm.newQuery("javax.jdo.query.SQL", queryText);
+              qResult = executeWithArray(query,
+                  prepareParams(catName, dbName, tableName, partNames, Arrays.asList(colName)), queryText);
+              if (qResult == null) {
+                query.closeAll();
+                return Collections.emptyList();
+              }
+              fqr = (ForwardQueryResult<?>) qResult;
+              Object[] min = (Object[]) (fqr.get(0));
+              Object[] max = (Object[]) (fqr.get(fqr.size() - 1));
+              end = doTrace ? System.nanoTime() : 0;
+              timingTrace(doTrace, queryText, start, end);
+              query.closeAll();
+              if (min[0] == null || max[0] == null) {
+                row[2 + colStatIndex] = null;
+              } else {
+                row[2 + colStatIndex] = extrapolateMethod.extrapolate(min, max, colStatIndex,
+                    indexMap);
+              }
+            } else {
+              // if the aggregation type is avg, we use the average on the existing ones.
+              queryText = "select "
+                  + "avg((\"LONG_HIGH_VALUE\"-\"LONG_LOW_VALUE\")/cast(\"NUM_DISTINCTS\" as decimal)),"
+                  + "avg((\"DOUBLE_HIGH_VALUE\"-\"DOUBLE_LOW_VALUE\")/\"NUM_DISTINCTS\"),"
+                  + "avg((cast(\"BIG_DECIMAL_HIGH_VALUE\" as decimal)-cast(\"BIG_DECIMAL_LOW_VALUE\" as decimal))/\"NUM_DISTINCTS\")"
+                  + " from " + PART_COL_STATS + "" + " where \"CAT_NAME\" = ? and \"DB_NAME\" = ? and \"TABLE_NAME\" = ?"
+                  + " and \"COLUMN_NAME\" = ?" + " and \"PARTITION_NAME\" in ("
+                  + makeParams(partNames.size()) + ")" + " group by \"COLUMN_NAME\"";
+              start = doTrace ? System.nanoTime() : 0;
+              query = pm.newQuery("javax.jdo.query.SQL", queryText);
+              qResult = executeWithArray(query,
+                  prepareParams(catName, dbName, tableName, partNames, Arrays.asList(colName)), queryText);
+              if (qResult == null) {
+                query.closeAll();
+                return Collections.emptyList();
+              }
+              fqr = (ForwardQueryResult<?>) qResult;
+              Object[] avg = (Object[]) (fqr.get(0));
+              // colStatIndex=12,13,14 respond to "AVG_LONG", "AVG_DOUBLE",
+              // "AVG_DECIMAL"
+              row[2 + colStatIndex] = avg[colStatIndex - 12];
+              end = doTrace ? System.nanoTime() : 0;
+              timingTrace(doTrace, queryText, start, end);
+              query.closeAll();
+            }
+          }
+          colStats.add(prepareCSObjWithAdjustedNDV(row, 0, useDensityFunctionForNDVEstimation, ndvTuner));
+          Deadline.checkTimeout();
+        }
+      }
+      return colStats;
+    }
+  }
+
+  private ColumnStatisticsObj prepareCSObj (Object[] row, int i) throws MetaException {
+    ColumnStatisticsData data = new ColumnStatisticsData();
+    ColumnStatisticsObj cso = new ColumnStatisticsObj((String)row[i++], (String)row[i++], data);
+    Object llow = row[i++], lhigh = row[i++], dlow = row[i++], dhigh = row[i++],
+        declow = row[i++], dechigh = row[i++], nulls = row[i++], dist = row[i++], bitVector = row[i++],
+        avglen = row[i++], maxlen = row[i++], trues = row[i++], falses = row[i++];
+    StatObjectConverter.fillColumnStatisticsData(cso.getColType(), data,
+        llow, lhigh, dlow, dhigh, declow, dechigh, nulls, dist, bitVector, avglen, maxlen, trues, falses);
+    return cso;
+  }
+
+  p

<TRUNCATED>

[08/13] hive git commit: HIVE-20306 : Implement projection spec for fetching only requested fields from partitions (Vihang Karajgaonkar, reviewed by Aihua Xu, Andrew Sherman and Alexander Kolbasov)

Posted by vi...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/44ef91a6/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py b/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
index 38074ce..9ec1221 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
@@ -1652,6 +1652,13 @@ class Iface(fb303.FacebookService.Iface):
     """
     pass
 
+  def get_partitions_with_specs(self, request):
+    """
+    Parameters:
+     - request
+    """
+    pass
+
 
 class Client(fb303.FacebookService.Client, Iface):
   """
@@ -9297,6 +9304,39 @@ class Client(fb303.FacebookService.Client, Iface):
       raise result.o1
     raise TApplicationException(TApplicationException.MISSING_RESULT, "get_runtime_stats failed: unknown result")
 
+  def get_partitions_with_specs(self, request):
+    """
+    Parameters:
+     - request
+    """
+    self.send_get_partitions_with_specs(request)
+    return self.recv_get_partitions_with_specs()
+
+  def send_get_partitions_with_specs(self, request):
+    self._oprot.writeMessageBegin('get_partitions_with_specs', TMessageType.CALL, self._seqid)
+    args = get_partitions_with_specs_args()
+    args.request = request
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_get_partitions_with_specs(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = get_partitions_with_specs_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_partitions_with_specs failed: unknown result")
+
 
 class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
   def __init__(self, handler):
@@ -9515,6 +9555,7 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
     self._processMap["heartbeat_lock_materialization_rebuild"] = Processor.process_heartbeat_lock_materialization_rebuild
     self._processMap["add_runtime_stats"] = Processor.process_add_runtime_stats
     self._processMap["get_runtime_stats"] = Processor.process_get_runtime_stats
+    self._processMap["get_partitions_with_specs"] = Processor.process_get_partitions_with_specs
 
   def process(self, iprot, oprot):
     (name, type, seqid) = iprot.readMessageBegin()
@@ -14818,6 +14859,28 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
     oprot.writeMessageEnd()
     oprot.trans.flush()
 
+  def process_get_partitions_with_specs(self, seqid, iprot, oprot):
+    args = get_partitions_with_specs_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = get_partitions_with_specs_result()
+    try:
+      result.success = self._handler.get_partitions_with_specs(args.request)
+      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_partitions_with_specs", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
 
 # HELPER FUNCTIONS AND STRUCTURES
 
@@ -16468,10 +16531,10 @@ class get_databases_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype854, _size851) = iprot.readListBegin()
-          for _i855 in xrange(_size851):
-            _elem856 = iprot.readString()
-            self.success.append(_elem856)
+          (_etype882, _size879) = iprot.readListBegin()
+          for _i883 in xrange(_size879):
+            _elem884 = iprot.readString()
+            self.success.append(_elem884)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16494,8 +16557,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 iter857 in self.success:
-        oprot.writeString(iter857)
+      for iter885 in self.success:
+        oprot.writeString(iter885)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -16600,10 +16663,10 @@ class get_all_databases_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype861, _size858) = iprot.readListBegin()
-          for _i862 in xrange(_size858):
-            _elem863 = iprot.readString()
-            self.success.append(_elem863)
+          (_etype889, _size886) = iprot.readListBegin()
+          for _i890 in xrange(_size886):
+            _elem891 = iprot.readString()
+            self.success.append(_elem891)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16626,8 +16689,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 iter864 in self.success:
-        oprot.writeString(iter864)
+      for iter892 in self.success:
+        oprot.writeString(iter892)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -17397,12 +17460,12 @@ class get_type_all_result:
       if fid == 0:
         if ftype == TType.MAP:
           self.success = {}
-          (_ktype866, _vtype867, _size865 ) = iprot.readMapBegin()
-          for _i869 in xrange(_size865):
-            _key870 = iprot.readString()
-            _val871 = Type()
-            _val871.read(iprot)
-            self.success[_key870] = _val871
+          (_ktype894, _vtype895, _size893 ) = iprot.readMapBegin()
+          for _i897 in xrange(_size893):
+            _key898 = iprot.readString()
+            _val899 = Type()
+            _val899.read(iprot)
+            self.success[_key898] = _val899
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -17425,9 +17488,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 kiter872,viter873 in self.success.items():
-        oprot.writeString(kiter872)
-        viter873.write(oprot)
+      for kiter900,viter901 in self.success.items():
+        oprot.writeString(kiter900)
+        viter901.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.o2 is not None:
@@ -17570,11 +17633,11 @@ class get_fields_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype877, _size874) = iprot.readListBegin()
-          for _i878 in xrange(_size874):
-            _elem879 = FieldSchema()
-            _elem879.read(iprot)
-            self.success.append(_elem879)
+          (_etype905, _size902) = iprot.readListBegin()
+          for _i906 in xrange(_size902):
+            _elem907 = FieldSchema()
+            _elem907.read(iprot)
+            self.success.append(_elem907)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17609,8 +17672,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 iter880 in self.success:
-        iter880.write(oprot)
+      for iter908 in self.success:
+        iter908.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -17777,11 +17840,11 @@ class get_fields_with_environment_context_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype884, _size881) = iprot.readListBegin()
-          for _i885 in xrange(_size881):
-            _elem886 = FieldSchema()
-            _elem886.read(iprot)
-            self.success.append(_elem886)
+          (_etype912, _size909) = iprot.readListBegin()
+          for _i913 in xrange(_size909):
+            _elem914 = FieldSchema()
+            _elem914.read(iprot)
+            self.success.append(_elem914)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17816,8 +17879,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 iter887 in self.success:
-        iter887.write(oprot)
+      for iter915 in self.success:
+        iter915.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -17970,11 +18033,11 @@ class get_schema_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype891, _size888) = iprot.readListBegin()
-          for _i892 in xrange(_size888):
-            _elem893 = FieldSchema()
-            _elem893.read(iprot)
-            self.success.append(_elem893)
+          (_etype919, _size916) = iprot.readListBegin()
+          for _i920 in xrange(_size916):
+            _elem921 = FieldSchema()
+            _elem921.read(iprot)
+            self.success.append(_elem921)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18009,8 +18072,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 iter894 in self.success:
-        iter894.write(oprot)
+      for iter922 in self.success:
+        iter922.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -18177,11 +18240,11 @@ class get_schema_with_environment_context_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype898, _size895) = iprot.readListBegin()
-          for _i899 in xrange(_size895):
-            _elem900 = FieldSchema()
-            _elem900.read(iprot)
-            self.success.append(_elem900)
+          (_etype926, _size923) = iprot.readListBegin()
+          for _i927 in xrange(_size923):
+            _elem928 = FieldSchema()
+            _elem928.read(iprot)
+            self.success.append(_elem928)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18216,8 +18279,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 iter901 in self.success:
-        iter901.write(oprot)
+      for iter929 in self.success:
+        iter929.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -18670,66 +18733,66 @@ class create_table_with_constraints_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.primaryKeys = []
-          (_etype905, _size902) = iprot.readListBegin()
-          for _i906 in xrange(_size902):
-            _elem907 = SQLPrimaryKey()
-            _elem907.read(iprot)
-            self.primaryKeys.append(_elem907)
+          (_etype933, _size930) = iprot.readListBegin()
+          for _i934 in xrange(_size930):
+            _elem935 = SQLPrimaryKey()
+            _elem935.read(iprot)
+            self.primaryKeys.append(_elem935)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.LIST:
           self.foreignKeys = []
-          (_etype911, _size908) = iprot.readListBegin()
-          for _i912 in xrange(_size908):
-            _elem913 = SQLForeignKey()
-            _elem913.read(iprot)
-            self.foreignKeys.append(_elem913)
+          (_etype939, _size936) = iprot.readListBegin()
+          for _i940 in xrange(_size936):
+            _elem941 = SQLForeignKey()
+            _elem941.read(iprot)
+            self.foreignKeys.append(_elem941)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 4:
         if ftype == TType.LIST:
           self.uniqueConstraints = []
-          (_etype917, _size914) = iprot.readListBegin()
-          for _i918 in xrange(_size914):
-            _elem919 = SQLUniqueConstraint()
-            _elem919.read(iprot)
-            self.uniqueConstraints.append(_elem919)
+          (_etype945, _size942) = iprot.readListBegin()
+          for _i946 in xrange(_size942):
+            _elem947 = SQLUniqueConstraint()
+            _elem947.read(iprot)
+            self.uniqueConstraints.append(_elem947)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 5:
         if ftype == TType.LIST:
           self.notNullConstraints = []
-          (_etype923, _size920) = iprot.readListBegin()
-          for _i924 in xrange(_size920):
-            _elem925 = SQLNotNullConstraint()
-            _elem925.read(iprot)
-            self.notNullConstraints.append(_elem925)
+          (_etype951, _size948) = iprot.readListBegin()
+          for _i952 in xrange(_size948):
+            _elem953 = SQLNotNullConstraint()
+            _elem953.read(iprot)
+            self.notNullConstraints.append(_elem953)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 6:
         if ftype == TType.LIST:
           self.defaultConstraints = []
-          (_etype929, _size926) = iprot.readListBegin()
-          for _i930 in xrange(_size926):
-            _elem931 = SQLDefaultConstraint()
-            _elem931.read(iprot)
-            self.defaultConstraints.append(_elem931)
+          (_etype957, _size954) = iprot.readListBegin()
+          for _i958 in xrange(_size954):
+            _elem959 = SQLDefaultConstraint()
+            _elem959.read(iprot)
+            self.defaultConstraints.append(_elem959)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 7:
         if ftype == TType.LIST:
           self.checkConstraints = []
-          (_etype935, _size932) = iprot.readListBegin()
-          for _i936 in xrange(_size932):
-            _elem937 = SQLCheckConstraint()
-            _elem937.read(iprot)
-            self.checkConstraints.append(_elem937)
+          (_etype963, _size960) = iprot.readListBegin()
+          for _i964 in xrange(_size960):
+            _elem965 = SQLCheckConstraint()
+            _elem965.read(iprot)
+            self.checkConstraints.append(_elem965)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18750,43 +18813,43 @@ 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 iter938 in self.primaryKeys:
-        iter938.write(oprot)
+      for iter966 in self.primaryKeys:
+        iter966.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 iter939 in self.foreignKeys:
-        iter939.write(oprot)
+      for iter967 in self.foreignKeys:
+        iter967.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 iter940 in self.uniqueConstraints:
-        iter940.write(oprot)
+      for iter968 in self.uniqueConstraints:
+        iter968.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 iter941 in self.notNullConstraints:
-        iter941.write(oprot)
+      for iter969 in self.notNullConstraints:
+        iter969.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 iter942 in self.defaultConstraints:
-        iter942.write(oprot)
+      for iter970 in self.defaultConstraints:
+        iter970.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.checkConstraints is not None:
       oprot.writeFieldBegin('checkConstraints', TType.LIST, 7)
       oprot.writeListBegin(TType.STRUCT, len(self.checkConstraints))
-      for iter943 in self.checkConstraints:
-        iter943.write(oprot)
+      for iter971 in self.checkConstraints:
+        iter971.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -20346,10 +20409,10 @@ class truncate_table_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.partNames = []
-          (_etype947, _size944) = iprot.readListBegin()
-          for _i948 in xrange(_size944):
-            _elem949 = iprot.readString()
-            self.partNames.append(_elem949)
+          (_etype975, _size972) = iprot.readListBegin()
+          for _i976 in xrange(_size972):
+            _elem977 = iprot.readString()
+            self.partNames.append(_elem977)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20374,8 +20437,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 iter950 in self.partNames:
-        oprot.writeString(iter950)
+      for iter978 in self.partNames:
+        oprot.writeString(iter978)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -20720,10 +20783,10 @@ class get_tables_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype954, _size951) = iprot.readListBegin()
-          for _i955 in xrange(_size951):
-            _elem956 = iprot.readString()
-            self.success.append(_elem956)
+          (_etype982, _size979) = iprot.readListBegin()
+          for _i983 in xrange(_size979):
+            _elem984 = iprot.readString()
+            self.success.append(_elem984)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20746,8 +20809,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 iter957 in self.success:
-        oprot.writeString(iter957)
+      for iter985 in self.success:
+        oprot.writeString(iter985)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -20897,10 +20960,10 @@ class get_tables_by_type_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype961, _size958) = iprot.readListBegin()
-          for _i962 in xrange(_size958):
-            _elem963 = iprot.readString()
-            self.success.append(_elem963)
+          (_etype989, _size986) = iprot.readListBegin()
+          for _i990 in xrange(_size986):
+            _elem991 = iprot.readString()
+            self.success.append(_elem991)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20923,8 +20986,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 iter964 in self.success:
-        oprot.writeString(iter964)
+      for iter992 in self.success:
+        oprot.writeString(iter992)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -21048,10 +21111,10 @@ class get_materialized_views_for_rewriting_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype968, _size965) = iprot.readListBegin()
-          for _i969 in xrange(_size965):
-            _elem970 = iprot.readString()
-            self.success.append(_elem970)
+          (_etype996, _size993) = iprot.readListBegin()
+          for _i997 in xrange(_size993):
+            _elem998 = iprot.readString()
+            self.success.append(_elem998)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21074,8 +21137,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 iter971 in self.success:
-        oprot.writeString(iter971)
+      for iter999 in self.success:
+        oprot.writeString(iter999)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -21148,10 +21211,10 @@ class get_table_meta_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.tbl_types = []
-          (_etype975, _size972) = iprot.readListBegin()
-          for _i976 in xrange(_size972):
-            _elem977 = iprot.readString()
-            self.tbl_types.append(_elem977)
+          (_etype1003, _size1000) = iprot.readListBegin()
+          for _i1004 in xrange(_size1000):
+            _elem1005 = iprot.readString()
+            self.tbl_types.append(_elem1005)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21176,8 +21239,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 iter978 in self.tbl_types:
-        oprot.writeString(iter978)
+      for iter1006 in self.tbl_types:
+        oprot.writeString(iter1006)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -21233,11 +21296,11 @@ class get_table_meta_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype982, _size979) = iprot.readListBegin()
-          for _i983 in xrange(_size979):
-            _elem984 = TableMeta()
-            _elem984.read(iprot)
-            self.success.append(_elem984)
+          (_etype1010, _size1007) = iprot.readListBegin()
+          for _i1011 in xrange(_size1007):
+            _elem1012 = TableMeta()
+            _elem1012.read(iprot)
+            self.success.append(_elem1012)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21260,8 +21323,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 iter985 in self.success:
-        iter985.write(oprot)
+      for iter1013 in self.success:
+        iter1013.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -21385,10 +21448,10 @@ class get_all_tables_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype989, _size986) = iprot.readListBegin()
-          for _i990 in xrange(_size986):
-            _elem991 = iprot.readString()
-            self.success.append(_elem991)
+          (_etype1017, _size1014) = iprot.readListBegin()
+          for _i1018 in xrange(_size1014):
+            _elem1019 = iprot.readString()
+            self.success.append(_elem1019)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21411,8 +21474,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 iter992 in self.success:
-        oprot.writeString(iter992)
+      for iter1020 in self.success:
+        oprot.writeString(iter1020)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -21648,10 +21711,10 @@ class get_table_objects_by_name_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.tbl_names = []
-          (_etype996, _size993) = iprot.readListBegin()
-          for _i997 in xrange(_size993):
-            _elem998 = iprot.readString()
-            self.tbl_names.append(_elem998)
+          (_etype1024, _size1021) = iprot.readListBegin()
+          for _i1025 in xrange(_size1021):
+            _elem1026 = iprot.readString()
+            self.tbl_names.append(_elem1026)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21672,8 +21735,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 iter999 in self.tbl_names:
-        oprot.writeString(iter999)
+      for iter1027 in self.tbl_names:
+        oprot.writeString(iter1027)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -21725,11 +21788,11 @@ class get_table_objects_by_name_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1003, _size1000) = iprot.readListBegin()
-          for _i1004 in xrange(_size1000):
-            _elem1005 = Table()
-            _elem1005.read(iprot)
-            self.success.append(_elem1005)
+          (_etype1031, _size1028) = iprot.readListBegin()
+          for _i1032 in xrange(_size1028):
+            _elem1033 = Table()
+            _elem1033.read(iprot)
+            self.success.append(_elem1033)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21746,8 +21809,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 iter1006 in self.success:
-        iter1006.write(oprot)
+      for iter1034 in self.success:
+        iter1034.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -22615,10 +22678,10 @@ class get_table_names_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1010, _size1007) = iprot.readListBegin()
-          for _i1011 in xrange(_size1007):
-            _elem1012 = iprot.readString()
-            self.success.append(_elem1012)
+          (_etype1038, _size1035) = iprot.readListBegin()
+          for _i1039 in xrange(_size1035):
+            _elem1040 = iprot.readString()
+            self.success.append(_elem1040)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22653,8 +22716,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 iter1013 in self.success:
-        oprot.writeString(iter1013)
+      for iter1041 in self.success:
+        oprot.writeString(iter1041)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -23783,11 +23846,11 @@ class add_partitions_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype1017, _size1014) = iprot.readListBegin()
-          for _i1018 in xrange(_size1014):
-            _elem1019 = Partition()
-            _elem1019.read(iprot)
-            self.new_parts.append(_elem1019)
+          (_etype1045, _size1042) = iprot.readListBegin()
+          for _i1046 in xrange(_size1042):
+            _elem1047 = Partition()
+            _elem1047.read(iprot)
+            self.new_parts.append(_elem1047)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23804,8 +23867,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 iter1020 in self.new_parts:
-        iter1020.write(oprot)
+      for iter1048 in self.new_parts:
+        iter1048.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -23963,11 +24026,11 @@ class add_partitions_pspec_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype1024, _size1021) = iprot.readListBegin()
-          for _i1025 in xrange(_size1021):
-            _elem1026 = PartitionSpec()
-            _elem1026.read(iprot)
-            self.new_parts.append(_elem1026)
+          (_etype1052, _size1049) = iprot.readListBegin()
+          for _i1053 in xrange(_size1049):
+            _elem1054 = PartitionSpec()
+            _elem1054.read(iprot)
+            self.new_parts.append(_elem1054)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23984,8 +24047,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 iter1027 in self.new_parts:
-        iter1027.write(oprot)
+      for iter1055 in self.new_parts:
+        iter1055.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -24159,10 +24222,10 @@ class append_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1031, _size1028) = iprot.readListBegin()
-          for _i1032 in xrange(_size1028):
-            _elem1033 = iprot.readString()
-            self.part_vals.append(_elem1033)
+          (_etype1059, _size1056) = iprot.readListBegin()
+          for _i1060 in xrange(_size1056):
+            _elem1061 = iprot.readString()
+            self.part_vals.append(_elem1061)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -24187,8 +24250,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 iter1034 in self.part_vals:
-        oprot.writeString(iter1034)
+      for iter1062 in self.part_vals:
+        oprot.writeString(iter1062)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -24541,10 +24604,10 @@ class append_partition_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1038, _size1035) = iprot.readListBegin()
-          for _i1039 in xrange(_size1035):
-            _elem1040 = iprot.readString()
-            self.part_vals.append(_elem1040)
+          (_etype1066, _size1063) = iprot.readListBegin()
+          for _i1067 in xrange(_size1063):
+            _elem1068 = iprot.readString()
+            self.part_vals.append(_elem1068)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -24575,8 +24638,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 iter1041 in self.part_vals:
-        oprot.writeString(iter1041)
+      for iter1069 in self.part_vals:
+        oprot.writeString(iter1069)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.environment_context is not None:
@@ -25171,10 +25234,10 @@ class drop_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1045, _size1042) = iprot.readListBegin()
-          for _i1046 in xrange(_size1042):
-            _elem1047 = iprot.readString()
-            self.part_vals.append(_elem1047)
+          (_etype1073, _size1070) = iprot.readListBegin()
+          for _i1074 in xrange(_size1070):
+            _elem1075 = iprot.readString()
+            self.part_vals.append(_elem1075)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -25204,8 +25267,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 iter1048 in self.part_vals:
-        oprot.writeString(iter1048)
+      for iter1076 in self.part_vals:
+        oprot.writeString(iter1076)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.deleteData is not None:
@@ -25378,10 +25441,10 @@ class drop_partition_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1052, _size1049) = iprot.readListBegin()
-          for _i1053 in xrange(_size1049):
-            _elem1054 = iprot.readString()
-            self.part_vals.append(_elem1054)
+          (_etype1080, _size1077) = iprot.readListBegin()
+          for _i1081 in xrange(_size1077):
+            _elem1082 = iprot.readString()
+            self.part_vals.append(_elem1082)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -25417,8 +25480,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 iter1055 in self.part_vals:
-        oprot.writeString(iter1055)
+      for iter1083 in self.part_vals:
+        oprot.writeString(iter1083)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.deleteData is not None:
@@ -26155,10 +26218,10 @@ class get_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1059, _size1056) = iprot.readListBegin()
-          for _i1060 in xrange(_size1056):
-            _elem1061 = iprot.readString()
-            self.part_vals.append(_elem1061)
+          (_etype1087, _size1084) = iprot.readListBegin()
+          for _i1088 in xrange(_size1084):
+            _elem1089 = iprot.readString()
+            self.part_vals.append(_elem1089)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26183,8 +26246,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 iter1062 in self.part_vals:
-        oprot.writeString(iter1062)
+      for iter1090 in self.part_vals:
+        oprot.writeString(iter1090)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -26343,11 +26406,11 @@ class exchange_partition_args:
       if fid == 1:
         if ftype == TType.MAP:
           self.partitionSpecs = {}
-          (_ktype1064, _vtype1065, _size1063 ) = iprot.readMapBegin()
-          for _i1067 in xrange(_size1063):
-            _key1068 = iprot.readString()
-            _val1069 = iprot.readString()
-            self.partitionSpecs[_key1068] = _val1069
+          (_ktype1092, _vtype1093, _size1091 ) = iprot.readMapBegin()
+          for _i1095 in xrange(_size1091):
+            _key1096 = iprot.readString()
+            _val1097 = iprot.readString()
+            self.partitionSpecs[_key1096] = _val1097
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -26384,9 +26447,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 kiter1070,viter1071 in self.partitionSpecs.items():
-        oprot.writeString(kiter1070)
-        oprot.writeString(viter1071)
+      for kiter1098,viter1099 in self.partitionSpecs.items():
+        oprot.writeString(kiter1098)
+        oprot.writeString(viter1099)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.source_db is not None:
@@ -26591,11 +26654,11 @@ class exchange_partitions_args:
       if fid == 1:
         if ftype == TType.MAP:
           self.partitionSpecs = {}
-          (_ktype1073, _vtype1074, _size1072 ) = iprot.readMapBegin()
-          for _i1076 in xrange(_size1072):
-            _key1077 = iprot.readString()
-            _val1078 = iprot.readString()
-            self.partitionSpecs[_key1077] = _val1078
+          (_ktype1101, _vtype1102, _size1100 ) = iprot.readMapBegin()
+          for _i1104 in xrange(_size1100):
+            _key1105 = iprot.readString()
+            _val1106 = iprot.readString()
+            self.partitionSpecs[_key1105] = _val1106
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -26632,9 +26695,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 kiter1079,viter1080 in self.partitionSpecs.items():
-        oprot.writeString(kiter1079)
-        oprot.writeString(viter1080)
+      for kiter1107,viter1108 in self.partitionSpecs.items():
+        oprot.writeString(kiter1107)
+        oprot.writeString(viter1108)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.source_db is not None:
@@ -26717,11 +26780,11 @@ class exchange_partitions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1084, _size1081) = iprot.readListBegin()
-          for _i1085 in xrange(_size1081):
-            _elem1086 = Partition()
-            _elem1086.read(iprot)
-            self.success.append(_elem1086)
+          (_etype1112, _size1109) = iprot.readListBegin()
+          for _i1113 in xrange(_size1109):
+            _elem1114 = Partition()
+            _elem1114.read(iprot)
+            self.success.append(_elem1114)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26762,8 +26825,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 iter1087 in self.success:
-        iter1087.write(oprot)
+      for iter1115 in self.success:
+        iter1115.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -26857,10 +26920,10 @@ class get_partition_with_auth_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1091, _size1088) = iprot.readListBegin()
-          for _i1092 in xrange(_size1088):
-            _elem1093 = iprot.readString()
-            self.part_vals.append(_elem1093)
+          (_etype1119, _size1116) = iprot.readListBegin()
+          for _i1120 in xrange(_size1116):
+            _elem1121 = iprot.readString()
+            self.part_vals.append(_elem1121)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26872,10 +26935,10 @@ class get_partition_with_auth_args:
       elif fid == 5:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype1097, _size1094) = iprot.readListBegin()
-          for _i1098 in xrange(_size1094):
-            _elem1099 = iprot.readString()
-            self.group_names.append(_elem1099)
+          (_etype1125, _size1122) = iprot.readListBegin()
+          for _i1126 in xrange(_size1122):
+            _elem1127 = iprot.readString()
+            self.group_names.append(_elem1127)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26900,8 +26963,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 iter1100 in self.part_vals:
-        oprot.writeString(iter1100)
+      for iter1128 in self.part_vals:
+        oprot.writeString(iter1128)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.user_name is not None:
@@ -26911,8 +26974,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 iter1101 in self.group_names:
-        oprot.writeString(iter1101)
+      for iter1129 in self.group_names:
+        oprot.writeString(iter1129)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -27341,11 +27404,11 @@ class get_partitions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1105, _size1102) = iprot.readListBegin()
-          for _i1106 in xrange(_size1102):
-            _elem1107 = Partition()
-            _elem1107.read(iprot)
-            self.success.append(_elem1107)
+          (_etype1133, _size1130) = iprot.readListBegin()
+          for _i1134 in xrange(_size1130):
+            _elem1135 = Partition()
+            _elem1135.read(iprot)
+            self.success.append(_elem1135)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27374,8 +27437,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 iter1108 in self.success:
-        iter1108.write(oprot)
+      for iter1136 in self.success:
+        iter1136.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -27469,10 +27532,10 @@ class get_partitions_with_auth_args:
       elif fid == 5:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype1112, _size1109) = iprot.readListBegin()
-          for _i1113 in xrange(_size1109):
-            _elem1114 = iprot.readString()
-            self.group_names.append(_elem1114)
+          (_etype1140, _size1137) = iprot.readListBegin()
+          for _i1141 in xrange(_size1137):
+            _elem1142 = iprot.readString()
+            self.group_names.append(_elem1142)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27505,8 +27568,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 iter1115 in self.group_names:
-        oprot.writeString(iter1115)
+      for iter1143 in self.group_names:
+        oprot.writeString(iter1143)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -27567,11 +27630,11 @@ class get_partitions_with_auth_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1119, _size1116) = iprot.readListBegin()
-          for _i1120 in xrange(_size1116):
-            _elem1121 = Partition()
-            _elem1121.read(iprot)
-            self.success.append(_elem1121)
+          (_etype1147, _size1144) = iprot.readListBegin()
+          for _i1148 in xrange(_size1144):
+            _elem1149 = Partition()
+            _elem1149.read(iprot)
+            self.success.append(_elem1149)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27600,8 +27663,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 iter1122 in self.success:
-        iter1122.write(oprot)
+      for iter1150 in self.success:
+        iter1150.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -27759,11 +27822,11 @@ class get_partitions_pspec_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1126, _size1123) = iprot.readListBegin()
-          for _i1127 in xrange(_size1123):
-            _elem1128 = PartitionSpec()
-            _elem1128.read(iprot)
-            self.success.append(_elem1128)
+          (_etype1154, _size1151) = iprot.readListBegin()
+          for _i1155 in xrange(_size1151):
+            _elem1156 = PartitionSpec()
+            _elem1156.read(iprot)
+            self.success.append(_elem1156)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27792,8 +27855,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 iter1129 in self.success:
-        iter1129.write(oprot)
+      for iter1157 in self.success:
+        iter1157.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -27951,10 +28014,10 @@ class get_partition_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1133, _size1130) = iprot.readListBegin()
-          for _i1134 in xrange(_size1130):
-            _elem1135 = iprot.readString()
-            self.success.append(_elem1135)
+          (_etype1161, _size1158) = iprot.readListBegin()
+          for _i1162 in xrange(_size1158):
+            _elem1163 = iprot.readString()
+            self.success.append(_elem1163)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27983,8 +28046,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 iter1136 in self.success:
-        oprot.writeString(iter1136)
+      for iter1164 in self.success:
+        oprot.writeString(iter1164)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -28224,10 +28287,10 @@ class get_partitions_ps_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1140, _size1137) = iprot.readListBegin()
-          for _i1141 in xrange(_size1137):
-            _elem1142 = iprot.readString()
-            self.part_vals.append(_elem1142)
+          (_etype1168, _size1165) = iprot.readListBegin()
+          for _i1169 in xrange(_size1165):
+            _elem1170 = iprot.readString()
+            self.part_vals.append(_elem1170)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -28257,8 +28320,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 iter1143 in self.part_vals:
-        oprot.writeString(iter1143)
+      for iter1171 in self.part_vals:
+        oprot.writeString(iter1171)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -28322,11 +28385,11 @@ class get_partitions_ps_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1147, _size1144) = iprot.readListBegin()
-          for _i1148 in xrange(_size1144):
-            _elem1149 = Partition()
-            _elem1149.read(iprot)
-            self.success.append(_elem1149)
+          (_etype1175, _size1172) = iprot.readListBegin()
+          for _i1176 in xrange(_size1172):
+            _elem1177 = Partition()
+            _elem1177.read(iprot)
+            self.success.append(_elem1177)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -28355,8 +28418,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 iter1150 in self.success:
-        iter1150.write(oprot)
+      for iter1178 in self.success:
+        iter1178.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -28443,10 +28506,10 @@ class get_partitions_ps_with_auth_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1154, _size1151) = iprot.readListBegin()
-          for _i1155 in xrange(_size1151):
-            _elem1156 = iprot.readString()
-            self.part_vals.append(_elem1156)
+          (_etype1182, _size1179) = iprot.readListBegin()
+          for _i1183 in xrange(_size1179):
+            _elem1184 = iprot.readString()
+            self.part_vals.append(_elem1184)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -28463,10 +28526,10 @@ class get_partitions_ps_with_auth_args:
       elif fid == 6:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype1160, _size1157) = iprot.readListBegin()
-          for _i1161 in xrange(_size1157):
-            _elem1162 = iprot.readString()
-            self.group_names.append(_elem1162)
+          (_etype1188, _size1185) = iprot.readListBegin()
+          for _i1189 in xrange(_size1185):
+            _elem1190 = iprot.readString()
+            self.group_names.append(_elem1190)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -28491,8 +28554,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 iter1163 in self.part_vals:
-        oprot.writeString(iter1163)
+      for iter1191 in self.part_vals:
+        oprot.writeString(iter1191)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -28506,8 +28569,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 iter1164 in self.group_names:
-        oprot.writeString(iter1164)
+      for iter1192 in self.group_names:
+        oprot.writeString(iter1192)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -28569,11 +28632,11 @@ class get_partitions_ps_with_auth_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1168, _size1165) = iprot.readListBegin()
-          for _i1169 in xrange(_size1165):
-            _elem1170 = Partition()
-            _elem1170.read(iprot)
-            self.success.append(_elem1170)
+          (_etype1196, _size1193) = iprot.readListBegin()
+          for _i1197 in xrange(_size1193):
+            _elem1198 = Partition()
+            _elem1198.read(iprot)
+            self.success.append(_elem1198)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -28602,8 +28665,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 iter1171 in self.success:
-        iter1171.write(oprot)
+      for iter1199 in self.success:
+        iter1199.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -28684,10 +28747,10 @@ class get_partition_names_ps_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1175, _size1172) = iprot.readListBegin()
-          for _i1176 in xrange(_size1172):
-            _elem1177 = iprot.readString()
-            self.part_vals.append(_elem1177)
+          (_etype1203, _size1200) = iprot.readListBegin()
+          for _i1204 in xrange(_size1200):
+            _elem1205 = iprot.readString()
+            self.part_vals.append(_elem1205)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -28717,8 +28780,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 iter1178 in self.part_vals:
-        oprot.writeString(iter1178)
+      for iter1206 in self.part_vals:
+        oprot.writeString(iter1206)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -28782,10 +28845,10 @@ class get_partition_names_ps_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1182, _size1179) = iprot.readListBegin()
-          for _i1183 in xrange(_size1179):
-            _elem1184 = iprot.readString()
-            self.success.append(_elem1184)
+          (_etype1210, _size1207) = iprot.readListBegin()
+          for _i1211 in xrange(_size1207):
+            _elem1212 = iprot.readString()
+            self.success.append(_elem1212)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -28814,8 +28877,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 iter1185 in self.success:
-        oprot.writeString(iter1185)
+      for iter1213 in self.success:
+        oprot.writeString(iter1213)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -28986,11 +29049,11 @@ class get_partitions_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1189, _size1186) = iprot.readListBegin()
-          for _i1190 in xrange(_size1186):
-            _elem1191 = Partition()
-            _elem1191.read(iprot)
-            self.success.append(_elem1191)
+          (_etype1217, _size1214) = iprot.readListBegin()
+          for _i1218 in xrange(_size1214):
+            _elem1219 = Partition()
+            _elem1219.read(iprot)
+            self.success.append(_elem1219)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -29019,8 +29082,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 iter1192 in self.success:
-        iter1192.write(oprot)
+      for iter1220 in self.success:
+        iter1220.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -29191,11 +29254,11 @@ class get_part_specs_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1196, _size1193) = iprot.readListBegin()
-          for _i1197 in xrange(_size1193):
-            _elem1198 = PartitionSpec()
-            _elem1198.read(iprot)
-            self.success.append(_elem1198)
+          (_etype1224, _size1221) = iprot.readListBegin()
+          for _i1225 in xrange(_size1221):
+            _elem1226 = PartitionSpec()
+            _elem1226.read(iprot)
+            self.success.append(_elem1226)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -29224,8 +29287,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 iter1199 in self.success:
-        iter1199.write(oprot)
+      for iter1227 in self.success:
+        iter1227.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -29645,10 +29708,10 @@ class get_partitions_by_names_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.names = []
-          (_etype1203, _size1200) = iprot.readListBegin()
-          for _i1204 in xrange(_size1200):
-            _elem1205 = iprot.readString()
-            self.names.append(_elem1205)
+          (_etype1231, _size1228) = iprot.readListBegin()
+          for _i1232 in xrange(_size1228):
+            _elem1233 = iprot.readString()
+            self.names.append(_elem1233)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -29673,8 +29736,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 iter1206 in self.names:
-        oprot.writeString(iter1206)
+      for iter1234 in self.names:
+        oprot.writeString(iter1234)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -29733,11 +29796,11 @@ class get_partitions_by_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1210, _size1207) = iprot.readListBegin()
-          for _i1211 in xrange(_size1207):
-            _elem1212 = Partition()
-            _elem1212.read(iprot)
-            self.success.append(_elem1212)
+          (_etype1238, _size1235) = iprot.readListBegin()
+          for _i1239 in xrange(_size1235):
+            _elem1240 = Partition()
+            _elem1240.read(iprot)
+            self.success.append(_elem1240)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -29766,8 +29829,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 iter1213 in self.success:
-        iter1213.write(oprot)
+      for iter1241 in self.success:
+        iter1241.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -30017,11 +30080,11 @@ class alter_partitions_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype1217, _size1214) = iprot.readListBegin()
-          for _i1218 in xrange(_size1214):
-            _elem1219 = Partition()
-            _elem1219.read(iprot)
-            self.new_parts.append(_elem1219)
+          (_etype1245, _size1242) = iprot.readListBegin()
+          for _i1246 in xrange(_size1242):
+            _elem1247 = Partition()
+            _elem1247.read(iprot)
+            self.new_parts.append(_elem1247)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -30046,8 +30109,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 iter1220 in self.new_parts:
-        iter1220.write(oprot)
+      for iter1248 in self.new_parts:
+        iter1248.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -30200,11 +30263,11 @@ class alter_partitions_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype1224, _size1221) = iprot.readListBegin()
-          for _i1225 in xrange(_size1221):
-            _elem1226 = Partition()
-            _elem1226.read(iprot)
-            self.new_parts.append(_elem1226)
+          (_etype1252, _size1249) = iprot.readListBegin()
+          for _i1253 in xrange(_size1249):
+            _elem1254 = Partition()
+            _elem1254.read(iprot)
+            self.new_parts.append(_elem1254)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -30235,8 +30298,8 @@ class alter_partitions_with_environment_context_args:
     if self.new_parts is not None:
       oprot.writeFieldBegin('new_parts', TType.LIST, 3)
       oprot.writeListBegin(TType.STRUCT, len(self.new_parts))
-      for iter1227 in self.new_parts:
-        iter1227.write(oprot)
+      for iter1255 in self.new_parts:
+        iter1255.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.environment_context is not None:
@@ -30739,10 +30802,10 @@ class rename_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1231, _size1228) = iprot.readListBegin()
-          for _i1232 in xrange(_size1228):
-            _elem1233 = iprot.readString()
-            self.part_vals.append(_elem1233)
+          (_etype1259, _size1256) = iprot.readListBegin()
+          for _i1260 in xrange(_size1256):
+            _elem1261 = iprot.readString()
+            self.part_vals.append(_elem1261)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -30773,8 +30836,8 @@ class rename_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 iter1234 in self.part_vals:
-        oprot.writeString(iter1234)
+      for iter1262 in self.part_vals:
+        oprot.writeString(iter1262)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.new_part is not None:
@@ -31075,10 +31138,10 @@ class partition_name_has_valid_characters_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1238, _size1235) = iprot.readListBegin()
-          for _i1239 in xrange(_size1235):
-            _elem1240 = iprot.readString()
-            self.part_vals.append(_elem1240)
+          (_etype1266, _size1263) = iprot.readListBegin()
+          for _i1267 in xrange(_size1263):
+            _elem1268 = iprot.readString()
+            self.part_vals.append(_elem1268)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -31100,8 +31163,8 @@ class partition_name_has_valid_characters_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 1)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter1241 in self.part_vals:
-        oprot.writeString(iter1241)
+      for iter1269 in self.part_vals:
+        oprot.writeString(iter1269)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.throw_exception is not None:
@@ -31459,10 +31522,10 @@ class partition_name_to_vals_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1245, _size1242) = iprot.readListBegin()
-          for _i1246 in xrange(_size1242):
-            _elem1247 = iprot.readString()
-            self.success.append(_elem1247)
+          (_etype1273, _size1270) = iprot.readListBegin()
+          for _i1274 in xrange(_size1270):
+            _elem1275 = iprot.readString()
+            self.success.append(_elem1275)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -31485,8 +31548,8 @@ class partition_name_to_vals_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1248 in self.success:
-        oprot.writeString(iter1248)
+      for iter1276 in self.success:
+        oprot.writeString(iter1276)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -31610,11 +31673,11 @@ class partition_name_to_spec_result:
       if fid == 0:
         if ftype == TType.MAP:
           self.success = {}
-          (_ktype1250, _vtype1251, _size1249 ) = iprot.readMapBegin()
-          for _i1253 in xrange(_size1249):
-            _key1254 = iprot.readString()
-            _val1255 = iprot.readString()
-            self.success[_key1254] = _val1255
+          (_ktype1278, _vtype1279, _size1277 ) = iprot.readMapBegin()
+          for _i1281 in xrange(_size1277):
+            _key1282 = iprot.readString()
+            _val1283 = iprot.readString()
+            self.success[_key1282] = _val1283
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -31637,9 +31700,9 @@ class partition_name_to_spec_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.MAP, 0)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.success))
-      for kiter1256,viter1257 in self.success.items():
-        oprot.writeString(kiter1256)
-        oprot.writeString(viter1257)
+      for kiter1284,viter1285 in self.success.items():
+        oprot.writeString(kiter1284)
+        oprot.writeString(viter1285)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -31715,11 +31778,11 @@ class markPartitionForEvent_args:
       elif fid == 3:
         if ftype == TType.MAP:
           self.part_vals = {}
-          (_ktype1259, _vtype1260, _size1258 ) = iprot.readMapBegin()
-          for _i1262 in xrange(_size1258):
-            _key1263 = iprot.readString()
-            _val1264 = iprot.readString()
-            self.part_vals[_key1263] = _val1264
+          (_ktype1287, _vtype1288, _size1286 ) = iprot.readMapBegin()
+          for _i1290 in xrange(_size1286):
+            _key1291 = iprot.readString()
+            _val1292 = iprot.readString()
+            self.part_vals[_key1291] = _val1292
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -31749,9 +31812,9 @@ class markPartitionForEvent_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.MAP, 3)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.part_vals))
-      for kiter1265,viter1266 in self.part_vals.items():
-        oprot.writeString(kiter1265)
-        oprot.writeString(viter1266)
+      for kiter1293,viter1294 in self.part_vals.items():
+        oprot.writeString(kiter1293)
+        oprot.writeString(viter1294)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.eventType is not None:
@@ -31965,11 +32028,11 @@ class isPartitionMarkedForEvent_args:
       elif fid == 3:
         if ftype == TType.MAP:
           self.part_vals = {}
-          (_ktype1268, _vtype1269, _size1267 ) = iprot.readMapBegin()
-          for _i1271 in xrange(_size1267):
-            _key1272 = iprot.readString()
-            _val1273 = iprot.readString()
-            self.part_vals[_key1272] = _val1273
+          (_ktype1296, _vtype1297, _size1295 ) = iprot.readMapBegin()
+          for _i1299 in xrange(_size1295):
+            _key1300 = iprot.readString()
+            _val1301 = iprot.readString()
+            self.part_vals[_key1300] = _val1301
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -31999,9 +32062,9 @@ class isPartitionMarkedForEvent_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.MAP, 3)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.part_vals))
-      for kiter1274,viter1275 in self.part_vals.items():
-        oprot.writeString(kiter1274)
-        oprot.writeString(viter1275)
+      for kiter1302,viter1303 in self.part_vals.items():
+        oprot.writeString(kiter1302)
+        oprot.writeString(viter1303)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.eventType is not None:
@@ -36027,10 +36090,10 @@ class get_functions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1279, _size1276) = iprot.readListBegin()
-          for _i1280 in xrange(_size1276):
-            _elem1281 = iprot.readString()
-            self.success.append(_elem1281)
+          (_etype1307, _size1304) = iprot.readListBegin()
+          for _i1308 in xrange(_size1304):
+            _elem1309 = iprot.readString()
+            self.success.append(_elem1309)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -36053,8 +36116,8 @@ class get_functions_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1282 in self.success:
-        oprot.writeString(iter1282)
+      for iter1310 in self.success:
+        oprot.writeString(iter1310)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -36742,10 +36805,10 @@ class get_role_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1286, _size1283) = iprot.readListBegin()
-          for _i1287 in xrange(_size1283):
-            _elem1288 = iprot.readString()
-            self.success.append(_elem1288)
+          (_etype1314, _size1311) = iprot.readListBegin()
+          for _i1315 in xrange(_size1311):
+            _elem1316 = iprot.readString()
+            self.success.append(_elem1316)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -36768,8 +36831,8 @@ class get_role_names_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1289 in self.success:
-        oprot.writeString(iter1289)
+      for iter1317 in self.success:
+        oprot.writeString(iter1317)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -37283,11 +37346,11 @@ class list_roles_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1293, _size1290) = iprot.readListBegin()
-          for _i1294 in xrange(_size1290):
-            _elem1295 = Role()
-            _elem1295.read(iprot)
-            self.success.append(_elem1295)
+          (_etype1321, _size1318) = iprot.readListBegin()
+          for _i1322 in xrange(_size1318):
+            _elem1323 = Role()
+            _elem1323.read(iprot)
+            self.success.append(_elem1323)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -37310,8 +37373,8 @@ class list_roles_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1296 in self.success:
-        iter1296.write(oprot)
+      for iter1324 in self.success:
+        iter1324.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -37820,10 +37883,10 @@ class get_privilege_set_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype1300, _size1297) = iprot.readListBegin()
-          for _i1301 in xrange(_size1297):
-            _elem1302 = iprot.readString()
-            self.group_names.append(_elem1302)
+          (_etype1328, _size1325) = iprot.readListBegin()
+          for _i1329 in xrange(_size1325):
+            _elem1330 = iprot.readString()
+            self.group_names.append(_elem1330)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -37848,8 +37911,8 @@ class get_privilege_set_args:
     if self.group_names is not None:
       oprot.writeFieldBegin('group_names', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.group_names))
-      for iter1303 in self.group_names:
-        oprot.writeString(iter1303)
+      for iter1331 in self.group_names:
+        oprot.writeString(iter1331)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -38076,11 +38139,11 @@ class list_privileges_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1307, _size1304) = iprot.readListBegin()
-          for _i1308 in xrange(_size1304):
-            _elem1309 = HiveObjectPrivilege()
-            _elem1309.read(iprot)
-            self.success.append(_elem1309)
+          (_etype1335, _size1332) = iprot.readListBegin()
+          for _i1336 in xrange(_size1332):
+            _elem1337 = HiveObjectPrivilege()
+            _elem1337.read(iprot)
+            self.success.append(_elem1337)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -38103,8 +38166,8 @@ class list_privileges_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1310 in self.success:
-        iter1310.write(oprot)
+      for iter1338 in self.success:
+        iter1338.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -38774,10 +38837,10 @@ class set_ugi_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype1314, _size1311) = iprot.readListBegin()
-          for _i1315 in xrange(_size1311):
-            _elem1316 = iprot.readString()
-            self.group_names.append(_elem1316)
+          (_etype1342, _size1339) = iprot.readListBegin()
+          for _i1343 in xrange(_size1339):
+            _elem1344 = iprot.readString()
+            self.group_names.append(_elem1344)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -38798,8 +38861,8 @@ class set_ugi_args:
     if self.group_names is not None:
       oprot.writeFieldBegin('group_names', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.group_names))
-      for iter1317 in self.group_names:
-        oprot.writeString(iter1317)
+      for iter1345 in self.group_names:
+        oprot.writeString(iter1345)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -38854,10 +38917,10 @@ class set_ugi_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1321, _size1318) = iprot.readListBegin()
-          for _i1322 in xrange(_size1318):
-            _elem1323 = iprot.readString()
-            self.success.append(_elem1323)
+          (_etype1349, _size1346) = iprot.readListBegin()
+          for _i1350 in xrange(_size1346):
+            _elem1351 = iprot.readString()
+            self.success.append(_elem1351)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -38880,8 +38943,8 @@ class set_ugi_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1324 in self.success:
-        oprot.writeString(iter1324)
+      for iter1352 in self.success:
+        oprot.writeString(iter1352)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -39813,10 +39876,10 @@ class get_all_token_identifiers_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1328, _size1325) = iprot.readListBegin()
-          for _i1329 in xrange(_size1325):
-            _elem1330 = iprot.readString()
-            self.success.append(_elem1330)
+          (_etype1356, _size1353) = iprot.readListBegin()
+          for _i1357 in xrange(_size1353):
+            _elem1358 = iprot.readString()
+            self.success.append(_elem1358)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -39833,8 +39896,8 @@ class get_all_token_identifiers_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1331 in self.success:
-        oprot.writeString(iter1331)
+      for iter1359 in self.success:
+        oprot.writeString(iter1359)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -40361,10 +40424,10 @@ class get_master_keys_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1335, _size1332) = iprot.readListBegin()
-          for _i1336 in xrange(_size1332):
-            _elem1337 = iprot.readString()
-            self.success.append(_elem1337)
+          (_etype1363, _size1360) = iprot.readListBegin()
+          for _i1364 in xrange(_size1360):
+            _elem1365 = iprot.readString()
+            self.success.append(_elem1365)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -40381,8 +40444,8 @@ class get_master_keys_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1338 in self.success:
-        oprot.writeString(iter1338)
+      for iter1366 in self.success:
+        oprot.writeString(iter1366)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -48799,11 +48862,11 @@ class get_schema_all_versions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1342, _size1339) = iprot.readListBegin()
-          for _i1343 in xrange(_size1339):
-            _elem1344 = SchemaVersion()
-            _elem1344.read(iprot)
-            self.success.append(_elem1344)
+          (_etype1370, _size1367) = iprot.readListBegin()
+          for _i1371 in xrange(_size1367):
+            _elem1372 = SchemaVersion()
+            _elem1372.read(iprot)
+            self.success.append(_elem1372)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -48832,8 +48895,8 @@ class get_schema_all_versions_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1345 in self.success:
-        iter1345.write(oprot)
+      for iter1373 in self.success:
+        iter1373.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -50308,11 +50371,11 @@ class get_runtime_stats_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1349, _size1346) = iprot.readListBegin()
-          for _i1350 in xrange(_size1346):
-            _elem1351 = RuntimeStat()
-            _elem1351.read(iprot)
-            self.success.append(_elem1351)
+          (_etype1377, _size1374) = iprot.readListBegin()
+          for _i1378 in xrange(_size1374):
+            _elem1379 = RuntimeStat()
+            _elem1379.read(iprot)
+            self.success.append(_elem1379)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -50335,8 +50398,8 @@ class get_runtime_stats_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1352 in self.success:
-        iter1352.write(oprot)
+      for iter1380 in self.success:
+        iter1380.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -50366,3 +50429,148 @@ class get_runtime_stats_result:
 
   def __ne__(self, other):
     return not (self == other)
+
+class get_partitions_with_specs_args:
+  """
+  Attributes:
+   - request
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRUCT, 'request', (GetPartitionsRequest, GetPartitionsRequest.thrift_spec), None, ), # 1
+  )
+
+  def __init__(self, request=None,):
+    self.request = request
+
+  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.request = GetPartitionsRequest()
+          self.request.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('get_partitions_with_specs_args')
+    if self.request is not None:
+      oprot.writeFieldBegin('request', TType.STRUCT, 1)
+      self.request.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.request)
+    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 get_partitions_with_specs_result:
+  """
+  Attributes:
+   - success
+   - o1
+  """
+
+  thrift_spec = (
+    (0, TType.STRUCT, 'success', (GetPartitionsResponse, GetPartitionsResponse.thrift_spec), None, ), # 0
+    (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1
+  )
+
+  def __init__(self, success=None, o1=None,):
+    self.success = success
+    self.o1 = o1
+
+  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 == 0:
+        if ftype == TType.STRUCT:
+          self.success = GetPartitionsResponse()
+          self.success.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 1:
+        if ftype == TType.STRUCT:
+          self.o1 = MetaException()
+          self.o1.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('get_partitions_with_specs_result')
+    if self.success is not None:
+      oprot.writeFieldBegin('success', TType.STRUCT, 0)
+      self.success.write(oprot)
+      oprot.writeFieldEnd()
+    if self.o1 is not None:
+      oprot.writeFieldBegin('o1', TType.STRUCT, 1)
+      self.o1.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.success)
+    value = (value * 31) ^ hash(self.o1)
+    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)


[02/13] hive git commit: HIVE-20306 : Implement projection spec for fetching only requested fields from partitions (Vihang Karajgaonkar, reviewed by Aihua Xu, Andrew Sherman and Alexander Kolbasov)

Posted by vi...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/44ef91a6/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/PartitionProjectionEvaluator.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/PartitionProjectionEvaluator.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/PartitionProjectionEvaluator.java
new file mode 100644
index 0000000..e918a33
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/PartitionProjectionEvaluator.java
@@ -0,0 +1,889 @@
+/*
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.hive.metastore;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Joiner;
+import com.google.common.collect.ImmutableBiMap;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+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.utils.MetaStoreServerUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.jdo.PersistenceManager;
+import javax.jdo.Query;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.regex.Pattern;
+
+import static org.apache.hadoop.hive.metastore.MetastoreDirectSqlUtils.extractSqlLong;
+
+/**
+ * Evaluator for partition projection filters which specify parts of the partition that should be
+ * used using dot notation for fields.
+ */
+public class PartitionProjectionEvaluator {
+  private static final Logger LOG = LoggerFactory.getLogger(PartitionProjectionEvaluator.class);
+  private final boolean convertMapNullsToEmptyStrings;
+  private final boolean isView;
+  private final String includeParamKeyPattern;
+  private final String excludeParamKeyPattern;
+  private Set<String> projectionFields;
+
+  interface PartitionFieldValueSetter<T> {
+    void setValue(T part, PartitionFieldNode node, Object value) throws MetaException;
+  }
+
+  private final ImmutableMap<String, MutivaluedFieldSetter> multiValuedFieldSetters =
+      new ImmutableMap.Builder<String, MutivaluedFieldSetter>()
+          .put("values", new PartitionValuesSetter())
+          .put("parameters", new PartitionParametersSetter())
+          .put("sd.cols", new PartitionSDColsSetter())
+          .put("sd.bucketCols", new PartitionSDBucketColsSetter())
+          .put("sd.sortCols", new PartitionSortColsSetter())
+          .put("sd.parameters", new PartitionSDParametersSetter())
+          .put("sd.serdeInfo.parameters", new PartitionSerdeInfoParametersSetter())
+          .put("sd.skewedInfo.skewedColNames", new PartitionSkewedColsNamesSetter())
+          .put("sd.skewedInfo.skewedColValues", new PartitionSkewedColsValuesSetter())
+          .put("sd.skewedInfo.skewedColValueLocationMaps",
+              new PartitionSkewedColValLocationMapSetter()).build();
+
+  private static final String PART_ID = "PART_ID";
+  private static final String SD_ID = "SD_ID";
+  private static final String SERDE_ID = "SERDE_ID";
+  private static final String CD_ID = "CD_ID";
+
+  private static final PartitionFieldNode partIdNode = new PartitionFieldNode(PART_ID);
+  private static final PartitionFieldNode sdIdNode = new PartitionFieldNode(SD_ID);
+  private static final PartitionFieldNode serdeIdNode = new PartitionFieldNode(SERDE_ID);
+  private static final PartitionFieldNode cdIdNode = new PartitionFieldNode(CD_ID);
+
+  private final ImmutableMap<String, String> fieldNameToTableName;
+  private final Set<PartitionFieldNode> roots;
+  private final String PARTITIONS;
+  private final String SDS;
+  private final String SERDES;
+  private final String PARTITION_PARAMS;
+  private final PersistenceManager pm;
+
+  @VisibleForTesting static final String SD_PATTERN = "sd|sd\\.";
+  @VisibleForTesting static final String SERDE_PATTERN = "sd\\.serdeInfo|sd\\.serdeInfo\\.";
+  @VisibleForTesting static final String CD_PATTERN = "sd\\.cols|sd\\.cols\\.";
+
+  private static final int SD_INDEX = 0;
+  private static final int CD_INDEX = 1;
+  private static final int SERDE_INDEX = 2;
+  private static final int PART_INDEX = 3;
+
+  // this map stores all the single valued fields in the Partition class and maps them to the corresponding
+  // single-valued fields from the MPartition class. This map is used to parse the given partition fields
+  // as well as to convert a given partition field list to a JDO setResult string when direct-SQL
+  // is disabled
+  private static final ImmutableMap<String, String> allPartitionSingleValuedFields = new ImmutableMap.Builder<String, String>()
+      .put("dbName", "table.database.name")
+      .put("tableName", "table.tableName")
+      .put("createTime", "createTime")
+      .put("lastAccessTime", "lastAccessTime")
+      .put("sd.location", "sd.location")
+      .put("sd.inputFormat", "sd.inputFormat")
+      .put("sd.outputFormat", "sd.outputFormat")
+      .put("sd.compressed", "sd.isCompressed")
+      .put("sd.numBuckets", "sd.numBuckets")
+      .put("sd.serdeInfo.name", "sd.serDeInfo.name")
+      .put("sd.serdeInfo.serializationLib", "sd.serDeInfo.serializationLib")
+      .put("sd.serdeInfo.description", "sd.serDeInfo.description")
+      .put("sd.serdeInfo.serializerClass", "sd.serDeInfo.serializerClass")
+      .put("sd.serdeInfo.deserializerClass", "sd.serDeInfo.deserializerClass")
+      .put("sd.serdeInfo.serdeType", "sd.serDeInfo.serdeType")
+      .put("catName", "table.database.catalogName")
+      .put("writeId", "writeId")
+      //TODO there is no mapping for isStatsCompliant to JDO MPartition
+      //.put("isStatsCompliant", "isStatsCompliant")
+      .build();
+
+  private static final ImmutableSet<String> allPartitionMultiValuedFields = new ImmutableSet.Builder<String>()
+      .add("values")
+      .add("sd.cols.name")
+      .add("sd.cols.type")
+      .add("sd.cols.comment")
+      .add("sd.serdeInfo.parameters")
+      .add("sd.bucketCols")
+      .add("sd.sortCols.col")
+      .add("sd.sortCols.order")
+      .add("sd.parameters")
+      .add("sd.skewedInfo.skewedColNames")
+      .add("sd.skewedInfo.skewedColValues")
+      .add("sd.skewedInfo.skewedColValueLocationMaps")
+      .add("parameters")
+      .add("privileges.userPrivileges")
+      .add("privileges.groupPrivileges")
+      .add("privileges.rolePrivileges")
+      .build();
+
+  private static final ImmutableSet<String> allPartitionFields = new ImmutableSet.Builder<String>()
+      .addAll(allPartitionSingleValuedFields.keySet())
+      .addAll(allPartitionMultiValuedFields)
+      .build();
+
+  public PartitionProjectionEvaluator(PersistenceManager pm,
+      ImmutableMap<String, String> fieldNameToTableName, List<String> projectionFields,
+      boolean convertMapNullsToEmptyStrings, boolean isView, String includeParamKeyPattern,
+      String excludeParamKeyPattern) throws MetaException {
+    this.pm = pm;
+    this.fieldNameToTableName = fieldNameToTableName;
+    this.convertMapNullsToEmptyStrings = convertMapNullsToEmptyStrings;
+    this.isView = isView;
+    this.includeParamKeyPattern = includeParamKeyPattern;
+    this.excludeParamKeyPattern = excludeParamKeyPattern;
+    this.PARTITIONS =
+        fieldNameToTableName.containsKey("PARTITIONS_TABLE_NAME") ? fieldNameToTableName
+            .get("PARTITIONS_TABLE_NAME") : "PARTITIONS";
+    this.SDS = fieldNameToTableName.containsKey("SDS_TABLE_NAME") ? fieldNameToTableName
+        .get("SDS_TABLE_NAME") : "SDS";
+    this.SERDES = fieldNameToTableName.containsKey("SERDES_TABLE_NAME") ? fieldNameToTableName
+        .get("SERDES_TABLE_NAME") : "SERDES";
+    this.PARTITION_PARAMS =
+        fieldNameToTableName.containsKey("PARTITION_PARAMS") ? fieldNameToTableName
+            .get("PARTITION_PARAMS") : "PARTITION_PARAMS";
+
+    roots = parse(projectionFields);
+
+    // we always query PART_ID
+    roots.add(partIdNode);
+    if (find(SD_PATTERN)) {
+      roots.add(sdIdNode);
+    }
+    if (find(SERDE_PATTERN)) {
+      roots.add(serdeIdNode);
+    }
+    if (find(CD_PATTERN)) {
+      roots.add(cdIdNode);
+    }
+  }
+
+  /**
+   * Given a Java regex string pattern, checks if the the partitionFieldNode tree
+   * has any node whose fieldName matches the given pattern
+   * @param searchField
+   * @return
+   */
+  @VisibleForTesting
+  boolean find(String searchField) {
+    Pattern p = Pattern.compile(searchField);
+    for (PartitionFieldNode node : roots) {
+      if (find(node, p)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  private static boolean find(PartitionFieldNode root, Pattern p) {
+    if (root == null) {
+      return false;
+    }
+    if (p.matcher(root.fieldName).matches()) {
+      return true;
+    }
+    for (PartitionFieldNode child : root.children) {
+      if (find(child, p)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * if top level field name is given expand the top level field such that all the children
+   * of that node are added to the projection list. eg. if only "sd" is provided in the projection
+   * list, it means all the nested fields for sd should be added to the projection fields
+   * @param projectionList
+   * @return
+   */
+  private static Set<String> expand(Collection<String> projectionList) throws MetaException {
+    Set<String> result = new HashSet<>();
+    for (String projectedField : projectionList) {
+      if (allPartitionFields.contains(projectedField)) {
+        result.add(projectedField);
+      } else {
+        boolean found = false;
+        for (String partitionField : allPartitionFields) {
+          if (partitionField.startsWith(projectedField)) {
+            LOG.debug("Found " + partitionField + " included within given projection field "
+                + projectedField);
+            result.add(partitionField);
+            found = true;
+          }
+        }
+        if (!found) {
+          throw new MetaException("Invalid field name " + projectedField);
+        }
+      }
+    }
+    return result;
+  }
+
+  @VisibleForTesting
+  Set<PartitionFieldNode> getRoots() {
+    return roots;
+  }
+
+  private static void validate(Collection<String> projectionFields) throws MetaException {
+    Set<String> verify = new HashSet<>(projectionFields);
+    verify.removeAll(allPartitionFields);
+    if (verify.size() > 0) {
+      throw new MetaException("Invalid partition fields in the projection spec" + Arrays
+          .toString(verify.toArray(new String[verify.size()])));
+    }
+  }
+
+  private Set<PartitionFieldNode> parse(List<String> inputProjectionFields) throws MetaException {
+    // in case of dbName and tableName we rely on table object to get their values
+    this.projectionFields = new HashSet<>(inputProjectionFields);
+    projectionFields.remove("dbName");
+    projectionFields.remove("tableName");
+    projectionFields.remove("catName");
+    if (isView) {
+      // if this is a view SDs are not set so can be skipped
+      projectionFields.removeIf(
+          s -> s.matches(SD_PATTERN) || s.matches(SERDE_PATTERN) || s.matches(CD_PATTERN));
+    }
+    // remove redundant fields
+    projectionFields = PartitionProjectionEvaluator.expand(projectionFields);
+    removeUnsupportedFields();
+    validate(projectionFields);
+
+    Map<String, PartitionFieldNode> nestedNodes = new HashMap<>();
+    Set<PartitionFieldNode> rootNodes = new HashSet<>();
+
+    for (String projectedField : projectionFields) {
+      String[] fields = projectedField.split("\\.");
+      if (fields.length == 0) {
+        LOG.warn("Invalid projected field {}. Ignoring ..", projectedField);
+        continue;
+      }
+      StringBuilder fieldNameBuilder = new StringBuilder(fields[0]);
+      PartitionFieldNode currentNode = createIfNotExists(nestedNodes, fieldNameBuilder.toString());
+      rootNodes.add(currentNode);
+      for (int level = 1; level < fields.length; level++) {
+        final String name = fieldNameBuilder.append(".").append(fields[level]).toString();
+        PartitionFieldNode childNode = createIfNotExists(nestedNodes, name);
+        // all the children of a multi-valued nodes are also multi-valued
+        if (currentNode.isMultiValued) {
+          childNode.setMultiValued();
+        }
+        currentNode.addChild(childNode);
+        currentNode = childNode;
+      }
+    }
+    return rootNodes;
+  }
+
+  // TODO some of the optional partition fields are never set by DirectSQL implementation
+  // Removing such fields to keep it consistent with methods in MetastoreDirectSQL class
+  private void removeUnsupportedFields() {
+    List<String> unsupportedFields = Arrays
+        .asList("sd.serdeInfo.serializerClass", "sd.serdeInfo.deserializerClass",
+            "sd.serdeInfo.serdeType", "sd.serdeInfo.description");
+    for (String unsupportedField : unsupportedFields) {
+      if (projectionFields.contains(unsupportedField)) {
+        LOG.warn("DirectSQL does not return partitions with the optional field" + unsupportedField
+            + " set. Removing it from the projection list");
+        projectionFields.remove(unsupportedField);
+      }
+    }
+  }
+
+  private PartitionFieldNode createIfNotExists(Map<String, PartitionFieldNode> nestedNodes,
+      String fieldName) {
+    PartitionFieldNode currentNode = nestedNodes.computeIfAbsent(fieldName, k -> {
+      if (multiValuedFieldSetters.containsKey(fieldName)) {
+        return new PartitionFieldNode(fieldName, true);
+      } else {
+        return new PartitionFieldNode(fieldName);
+      }
+    });
+    return currentNode;
+  }
+
+  /**
+   * Given a list of partition ids, returns the List of partially filled partitions based on the
+   * projection list used to instantiate this PartitionProjectionEvaluator
+   * @param partitionIds List of partition ids corresponding to the Partitions objects which are requested
+   * @return Partitions where each partition has only the projected fields set
+   * @throws MetaException
+   */
+  public List<Partition> getPartitionsUsingProjectionList(List<Long> partitionIds)
+      throws MetaException {
+    TreeMap<Long, StorageDescriptor> sds = new TreeMap<>();
+    TreeMap<Long, List<FieldSchema>> cds = new TreeMap<>();
+    TreeMap<Long, SerDeInfo> serdes = new TreeMap<>();
+    TreeMap<Long, Partition> partitions = new TreeMap<>();
+    List<Partition> results = setSingleValuedFields(partitionIds, partitions, sds, serdes, cds);
+    setMultivaluedFields(partitions, sds, serdes, cds);
+    return results;
+  }
+
+  private List<Partition> setSingleValuedFields(List<Long> partitionIds,
+      final TreeMap<Long, Partition> partitions, final TreeMap<Long, StorageDescriptor> sdIds,
+      final TreeMap<Long, SerDeInfo> serdeIds, final TreeMap<Long, List<FieldSchema>> cdIds)
+      throws MetaException {
+
+    StringBuilder queryTextBuilder = new StringBuilder();
+    int numColumns = buildQueryForSingleValuedFields(partitionIds, queryTextBuilder);
+    String queryText = queryTextBuilder.toString();
+
+    try (Query query = pm.newQuery("javax.jdo.query.SQL", queryText)) {
+
+      long start = LOG.isDebugEnabled() ? System.nanoTime() : 0;
+      List<Object> sqlResult = MetastoreDirectSqlUtils.executeWithArray(query, null, queryText);
+      long queryTime = LOG.isDebugEnabled() ? System.nanoTime() : 0;
+      MetastoreDirectSqlUtils.timingTrace(LOG.isDebugEnabled(), queryText, start, queryTime);
+      Deadline.checkTimeout();
+      final Long[] ids = new Long[4];
+      Object[] rowVals = new Object[1];
+      // Keep order by name, consistent with JDO.
+      ArrayList<Partition> orderedResult = new ArrayList<Partition>(partitionIds.size());
+      for (Object row : sqlResult) {
+        if (numColumns > 1) {
+          rowVals = (Object[])row;
+        } else {
+          // only one column is selected by query. The result class will be Object
+          rowVals[0] = row;
+        }
+        Partition part = new Partition();
+        for (PartitionFieldNode root : roots) {
+          traverseAndSetValues(part, root, rowVals, new PartitionFieldValueSetter() {
+            @Override
+            public void setValue(Object partition, PartitionFieldNode node, Object value)
+                throws MetaException {
+              if (!node.isMultiValued) {
+                // in case of serdeid and sdId node we just collect the sdIds for further processing
+                if (node.equals(sdIdNode)) {
+                  ids[SD_INDEX] = extractSqlLong(value);
+                } else if (node.equals(serdeIdNode)) {
+                  ids[SERDE_INDEX] = extractSqlLong(value);
+                } else if (node.equals(cdIdNode)) {
+                  ids[CD_INDEX] = extractSqlLong(value);
+                } else if (node.equals(partIdNode)) {
+                  ids[PART_INDEX] = extractSqlLong(value);
+                } else {
+                  // incase of sd.compressed and sd.storedAsSubDirectories we need special code to convert
+                  // string to a boolean value
+                  if (node.fieldName.equals("sd.compressed") || node.fieldName.equals("sd.storedAsSubDirectories")) {
+                    value = MetastoreDirectSqlUtils.extractSqlBoolean(value);
+                  }
+                  MetaStoreServerUtils.setNestedProperty(partition, node.fieldName, value, true);
+                }
+              }
+            }
+          });
+        }
+        // PART_ID is always queried
+        if (ids[PART_INDEX] == null) {
+          throw new MetaException("Could not find PART_ID for partition " + part);
+        }
+        partitions.put(ids[PART_INDEX], part);
+        orderedResult.add(part);
+        ids[PART_INDEX] = null;
+
+        if (ids[SD_INDEX] != null) {
+          // sd object is initialized if any of the sd single-valued fields are in the projection
+          if (part.getSd() == null) {
+            part.setSd(new StorageDescriptor());
+          }
+          sdIds.put(ids[SD_INDEX], part.getSd());
+          ids[SD_INDEX] = null;
+        }
+
+        if (ids[SERDE_INDEX] != null) {
+          // serde object must have already been intialized above in MetaStoreUtils.setNestedProperty call
+          if (part.getSd().getSerdeInfo() == null) {
+            part.getSd().setSerdeInfo(new SerDeInfo());
+          }
+          serdeIds.put(ids[SERDE_INDEX], part.getSd().getSerdeInfo());
+          ids[SERDE_INDEX] = null;
+        }
+
+        if (ids[CD_INDEX] != null) {
+          // common case is all the SDs will reuse the same CD
+          // allocate List<FieldSchema> only when you see a new CD_ID
+          cdIds.putIfAbsent(ids[CD_INDEX], new ArrayList<>(5));
+          if (part.getSd().getCols() == null) {
+            part.getSd().setCols(cdIds.get(ids[CD_INDEX]));
+          }
+          ids[CD_INDEX] = null;
+        }
+        Deadline.checkTimeout();
+      }
+      return orderedResult;
+    } catch (Exception e) {
+      LOG.error("Exception received while getting partitions using projected fields", e);
+      throw new MetaException(e.getMessage());
+    }
+  }
+
+  private void setMultivaluedFields(TreeMap<Long, Partition> partitions,
+      TreeMap<Long, StorageDescriptor> sds, TreeMap<Long, SerDeInfo> serdes,
+      TreeMap<Long, List<FieldSchema>> cds) throws MetaException {
+    for (PartitionFieldNode root : roots) {
+      traverseAndSetMultiValuedFields(root, partitions, sds, serdes, cds);
+    }
+  }
+
+  private void traverseAndSetMultiValuedFields(PartitionFieldNode root,
+      TreeMap<Long, Partition> partitions, TreeMap<Long, StorageDescriptor> sds,
+      TreeMap<Long, SerDeInfo> serdes, TreeMap<Long, List<FieldSchema>> cds) throws MetaException {
+    if (root == null) {
+      return;
+    }
+    // if a multi-valued node is found set its value using its value-setters
+    // note that once a multi-valued node is found the method does not recurse further
+    // this is because the multi-valued setter also sets the values of all its descendents
+    if (root.isMultiValued) {
+      MutivaluedFieldSetter multiValuedFieldSetter = multiValuedFieldSetters.get(root.fieldName);
+      if (multiValuedFieldSetter == null) {
+        throw new MetaException(
+            "Cannot find multi-valued field setter for field " + root.fieldName);
+      }
+      multiValuedFieldSetter.setValue(root, partitions, sds, serdes, cds);
+    } else {
+      for (PartitionFieldNode child : root.children) {
+        traverseAndSetMultiValuedFields(child, partitions, sds, serdes, cds);
+      }
+    }
+  }
+
+  private void traverseAndSetValues(Partition part, PartitionFieldNode root, Object[] row,
+      PartitionFieldValueSetter valueSetter) throws MetaException {
+    // if root is null or is multiValued, do not recurse further
+    // multi-valued fields are set separately in setMultiValuedFields method
+    if (root == null || root.isMultiValued()) {
+      return;
+    }
+    if (root.isLeafNode()) {
+      valueSetter.setValue(part, root, row[root.fieldIndex]);
+      return;
+    }
+    for (PartitionFieldNode child : root.children) {
+      traverseAndSetValues(part, child, row, valueSetter);
+    }
+  }
+
+  private static final String SPACE = " ";
+
+  private int buildQueryForSingleValuedFields(List<Long> partitionIds, StringBuilder queryTextBuilder) {
+    queryTextBuilder.append("select ");
+    // build projection columns using the ProjectedFields
+    // it should not matter if you select all the
+    List<String> columnList = getSingleValuedColumnNames(roots);
+    queryTextBuilder.append(Joiner.on(',').join(columnList));
+    queryTextBuilder.append(SPACE);
+    queryTextBuilder.append("from " + PARTITIONS);
+    // if SD fields are selected add join clause with SDS
+    boolean foundSD = false;
+    if (find(SD_PATTERN)) {
+      queryTextBuilder.append(SPACE);
+      queryTextBuilder.append(
+          "left outer join " + SDS + " on " + PARTITIONS + ".\"SD_ID\" = " + SDS + ".\"SD_ID\"");
+      foundSD = true;
+    }
+    // if serde fields are selected add join clause on serdes
+    if (foundSD || find(SERDE_PATTERN)) {
+      queryTextBuilder.append(SPACE);
+      queryTextBuilder.append(
+          "  left outer join " + SERDES + " on " + SDS + ".\"SERDE_ID\" = " + SERDES
+              + ".\"SERDE_ID\"");
+    }
+    queryTextBuilder.append(SPACE);
+    //add where clause
+    queryTextBuilder.append("where \"PART_ID\" in (" + Joiner.on(',').join(partitionIds)
+        + ") order by \"PART_NAME\" asc");
+    return columnList.size();
+  }
+
+  private int getSingleValuedColumnName(PartitionFieldNode root, int fieldId,
+      final List<String> columnNames) {
+    if (root == null) {
+      return fieldId;
+    }
+    if (root.isLeafNode() && !root.isMultiValued) {
+      if (fieldNameToTableName.containsKey(root.fieldName)) {
+        columnNames.add(fieldNameToTableName.get(root.fieldName));
+        root.setFieldIndex(fieldId++);
+        return fieldId;
+      }
+      throw new RuntimeException(
+          "No column name mapping found for partition field " + root.fieldName);
+    }
+    for (PartitionFieldNode child : root.children) {
+      fieldId = getSingleValuedColumnName(child, fieldId, columnNames);
+    }
+    return fieldId;
+  }
+
+  private List<String> getSingleValuedColumnNames(Set<PartitionFieldNode> roots) {
+    List<String> columnNames = new ArrayList<>();
+    int fieldIndex = 0;
+    for (PartitionFieldNode node : roots) {
+      fieldIndex = getSingleValuedColumnName(node, fieldIndex, columnNames);
+    }
+    return columnNames;
+  }
+
+
+  private static void getNestedFieldName(JsonNode jsonNode, String fieldName,
+      Collection<String> results) {
+    if (jsonNode instanceof ArrayNode) {
+      Iterator<JsonNode> elements = ((ArrayNode) jsonNode).elements();
+      if (!elements.hasNext()) {
+        results.add(fieldName);
+        return;
+      }
+      while (elements.hasNext()) {
+        JsonNode element = elements.next();
+        getNestedFieldName(element, fieldName, results);
+      }
+    } else {
+      Iterator<Entry<String, JsonNode>> fields = jsonNode.fields();
+      if (!fields.hasNext()) {
+        results.add(fieldName);
+        return;
+      }
+      while (fields.hasNext()) {
+        Entry<String, JsonNode> fieldKV = fields.next();
+        String key = fieldKV.getKey();
+        getNestedFieldName(fieldKV.getValue(), fieldName.length() == 0 ? key : fieldName + "." + key,
+            results);
+      }
+    }
+  }
+
+  static class PartitionFieldNode {
+    private String fieldName;
+    private Set<PartitionFieldNode> children = new HashSet<>(4);
+    private boolean isMultiValued;
+    private int fieldIndex;
+
+    PartitionFieldNode(String fieldName) {
+      this.fieldName = fieldName;
+      isMultiValued = false;
+    }
+
+    PartitionFieldNode(String fieldName, boolean isMultiValued) {
+      this.fieldName = fieldName;
+      this.isMultiValued = isMultiValued;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o)
+        return true;
+      if (o == null || getClass() != o.getClass())
+        return false;
+      PartitionFieldNode that = (PartitionFieldNode) o;
+      return Objects.equals(fieldName, that.fieldName);
+    }
+
+    boolean isLeafNode() {
+      return children == null || children.isEmpty();
+    }
+
+    void setFieldIndex(int fieldIndex) {
+      this.fieldIndex = fieldIndex;
+    }
+
+    @VisibleForTesting
+    void addChild(PartitionFieldNode child) {
+      children.add(child);
+    }
+
+    @VisibleForTesting
+    String getFieldName() {
+      return fieldName;
+    }
+
+    @VisibleForTesting
+    Set<PartitionFieldNode> getChildren() {
+      return new HashSet<>(children);
+    }
+
+    @VisibleForTesting
+    boolean isMultiValued() {
+      return isMultiValued;
+    }
+
+    @Override
+    public String toString() {
+      return fieldName;
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(fieldName);
+    }
+
+    void setMultiValued() {
+      this.isMultiValued = true;
+    }
+  }
+
+  private interface MutivaluedFieldSetter {
+    void setValue(PartitionFieldNode root, TreeMap<Long, Partition> partitions, TreeMap<Long, StorageDescriptor> sds,
+        TreeMap<Long, SerDeInfo> serdes, TreeMap<Long, List<FieldSchema>> cds) throws MetaException;
+  }
+
+  private class PartitionValuesSetter implements MutivaluedFieldSetter {
+    private PartitionValuesSetter() {
+      //
+    }
+
+    @Override
+    public void setValue(PartitionFieldNode root, TreeMap<Long, Partition> partitions, TreeMap<Long, StorageDescriptor> sds,
+        TreeMap<Long, SerDeInfo> serdes, TreeMap<Long, List<FieldSchema>> cds)
+        throws MetaException {
+      final String tableName =
+          fieldNameToTableName.containsKey("PARTITION_KEY_VALS") ? fieldNameToTableName
+              .get("PARTITION_KEY_VALS") : "PARTITION_KEY_VALS";
+      MetastoreDirectSqlUtils
+          .setPartitionValues(tableName, pm, Joiner.on(',').join(partitions.keySet()), partitions);
+    }
+  }
+
+  private class PartitionParametersSetter implements MutivaluedFieldSetter {
+    private PartitionParametersSetter() {
+      //
+    }
+
+    @Override
+    public void setValue(PartitionFieldNode root, TreeMap<Long, Partition> partitions, TreeMap<Long, StorageDescriptor> sds,
+        TreeMap<Long, SerDeInfo> serdes, TreeMap<Long, List<FieldSchema>> cds)
+        throws MetaException {
+      MetastoreDirectSqlUtils
+          .setPartitionParametersWithFilter(PARTITION_PARAMS, convertMapNullsToEmptyStrings, pm,
+              Joiner.on(',').join(partitions.keySet()), partitions, includeParamKeyPattern,
+              excludeParamKeyPattern);
+    }
+  }
+
+  private class PartitionSDColsSetter implements MutivaluedFieldSetter {
+    private PartitionSDColsSetter() {
+      // prevent instantiation
+    }
+
+    @Override
+    public void setValue(PartitionFieldNode root, TreeMap<Long, Partition> partitions, TreeMap<Long, StorageDescriptor> sds,
+        TreeMap<Long, SerDeInfo> serdes, TreeMap<Long, List<FieldSchema>> cds)
+        throws MetaException {
+      // find the fields which are requested for sd.cols
+      // children field names would be sd.cols.name, sd.cols.type or sd.cols.description
+      List<String> childFields = getChildrenFieldNames(root);
+      final String tableName = fieldNameToTableName.containsKey("COLUMNS_V2") ? fieldNameToTableName
+          .get("COLUMNS_V2") : "COLUMNS_V2";
+      MetastoreDirectSqlUtils
+          .setSDCols(tableName, childFields, pm, cds, Joiner.on(',').join(cds.keySet()));
+    }
+  }
+
+  private class PartitionSDBucketColsSetter implements MutivaluedFieldSetter {
+    private PartitionSDBucketColsSetter() {
+      //
+    }
+
+    @Override
+    public void setValue(PartitionFieldNode root, TreeMap<Long, Partition> partitions, TreeMap<Long, StorageDescriptor> sds,
+        TreeMap<Long, SerDeInfo> serdes, TreeMap<Long, List<FieldSchema>> cds)
+        throws MetaException {
+      final String tableName =
+          fieldNameToTableName.containsKey("BUCKETING_COLS") ? fieldNameToTableName
+              .get("BUCKETING_COLS") : "BUCKETING_COLS";
+      MetastoreDirectSqlUtils
+          .setSDBucketCols(tableName, pm, sds, Joiner.on(',').join(sds.keySet()));
+    }
+  }
+
+  private class PartitionSortColsSetter implements MutivaluedFieldSetter {
+    private PartitionSortColsSetter() {
+      //
+    }
+
+    @Override
+    public void setValue(PartitionFieldNode root, TreeMap<Long, Partition> partitions, TreeMap<Long, StorageDescriptor> sds,
+        TreeMap<Long, SerDeInfo> serdes, TreeMap<Long, List<FieldSchema>> cds)
+        throws MetaException {
+      List<String> childFieldNames = getChildrenFieldNames(root);
+      final String tableName = fieldNameToTableName.containsKey("SORT_COLS") ? fieldNameToTableName
+          .get("SORT_COLS") : "SORT_COLS";
+      MetastoreDirectSqlUtils
+          .setSDSortCols(tableName, childFieldNames, pm, sds, Joiner.on(',').join(sds.keySet()));
+    }
+  }
+
+  private List<String> getChildrenFieldNames(PartitionFieldNode root) throws MetaException {
+    List<String> childFields = new ArrayList<>(3);
+    for (PartitionFieldNode child : root.getChildren()) {
+      if (child.getFieldName().lastIndexOf(".") < 0) {
+        throw new MetaException("Error parsing multi-valued field name " + child.getFieldName());
+      }
+      childFields.add(child.getFieldName().substring(child.getFieldName().lastIndexOf(".") + 1));
+    }
+    return childFields;
+  }
+
+  private class PartitionSDParametersSetter implements MutivaluedFieldSetter {
+    private PartitionSDParametersSetter() {
+      //
+    }
+
+    @Override
+    public void setValue(PartitionFieldNode root, TreeMap<Long, Partition> partitions, TreeMap<Long, StorageDescriptor> sds,
+        TreeMap<Long, SerDeInfo> serdes, TreeMap<Long, List<FieldSchema>> cds)
+        throws MetaException {
+      final String tableName = fieldNameToTableName.containsKey("SD_PARAMS") ? fieldNameToTableName
+          .get("SD_PARAMS") : "SD_PARAMS";
+      MetastoreDirectSqlUtils.setSDParameters(tableName, convertMapNullsToEmptyStrings, pm, sds,
+          Joiner.on(',').join(sds.keySet()));
+    }
+  }
+
+  private class PartitionSerdeInfoParametersSetter implements MutivaluedFieldSetter {
+    private PartitionSerdeInfoParametersSetter() {
+      //
+    }
+
+    @Override
+    public void setValue(PartitionFieldNode root, TreeMap<Long, Partition> partitions, TreeMap<Long, StorageDescriptor> sds,
+        TreeMap<Long, SerDeInfo> serdes, TreeMap<Long, List<FieldSchema>> cds)
+        throws MetaException {
+      final String tableName =
+          fieldNameToTableName.containsKey("SERDE_PARAMS") ? fieldNameToTableName
+              .get("SERDE_PARAMS") : "SERDE_PARAMS";
+      MetastoreDirectSqlUtils.setSerdeParams(tableName, convertMapNullsToEmptyStrings, pm, serdes,
+          Joiner.on(',').join(serdes.keySet()));
+    }
+  }
+
+  private class PartitionSkewedColsNamesSetter implements MutivaluedFieldSetter {
+    private PartitionSkewedColsNamesSetter() {
+      //
+    }
+
+    @Override
+    public void setValue(PartitionFieldNode root, TreeMap<Long, Partition> partitions, TreeMap<Long, StorageDescriptor> sds,
+        TreeMap<Long, SerDeInfo> serdes, TreeMap<Long, List<FieldSchema>> cds)
+        throws MetaException {
+      final String tableName =
+          fieldNameToTableName.containsKey("SKEWED_COL_NAMES") ? fieldNameToTableName
+              .get("SKEWED_COL_NAMES") : "SKEWED_COL_NAMES";
+      MetastoreDirectSqlUtils
+          .setSkewedColNames(tableName, pm, sds, Joiner.on(',').join(sds.keySet()));
+    }
+  }
+
+  private class PartitionSkewedColsValuesSetter implements MutivaluedFieldSetter {
+    private PartitionSkewedColsValuesSetter() {
+      //
+    }
+
+    @Override
+    public void setValue(PartitionFieldNode root, TreeMap<Long, Partition> partitions, TreeMap<Long, StorageDescriptor> sds,
+        TreeMap<Long, SerDeInfo> serdes, TreeMap<Long, List<FieldSchema>> cds)
+        throws MetaException {
+      final String skewedStringListVals =
+          fieldNameToTableName.containsKey("SKEWED_STRING_LIST_VALUES") ? fieldNameToTableName
+              .get("SKEWED_STRING_LIST_VALUES") : "SKEWED_STRING_LIST_VALUES";
+      final String skewedVals =
+          fieldNameToTableName.containsKey("SKEWED_VALUES") ? fieldNameToTableName
+              .get("SKEWED_VALUES") : "SKEWED_VALUES";
+      MetastoreDirectSqlUtils.setSkewedColValues(skewedStringListVals, skewedVals, pm, sds,
+          Joiner.on(',').join(sds.keySet()));
+    }
+  }
+
+  private class PartitionSkewedColValLocationMapSetter implements MutivaluedFieldSetter {
+    private PartitionSkewedColValLocationMapSetter() {
+      //
+    }
+
+    @Override
+    public void setValue(PartitionFieldNode root, TreeMap<Long, Partition> partitions, TreeMap<Long, StorageDescriptor> sds,
+        TreeMap<Long, SerDeInfo> serdes, TreeMap<Long, List<FieldSchema>> cds)
+        throws MetaException {
+      final String skewedStringListVals =
+          fieldNameToTableName.containsKey("SKEWED_STRING_LIST_VALUES") ? fieldNameToTableName
+              .get("SKEWED_STRING_LIST_VALUES") : "SKEWED_STRING_LIST_VALUES";
+      final String skewedColValLocMap =
+          fieldNameToTableName.containsKey("SKEWED_COL_VALUE_LOC_MAP") ? fieldNameToTableName
+              .get("SKEWED_COL_VALUE_LOC_MAP") : "SKEWED_COL_VALUE_LOC_MAP";
+      MetastoreDirectSqlUtils
+          .setSkewedColLocationMaps(skewedColValLocMap, skewedStringListVals, pm, sds,
+              Joiner.on(',').join(sds.keySet()));
+    }
+  }
+
+  /**
+   * Given a list of partition fields, checks if all the fields requested are single-valued. If all
+   * the fields are single-valued returns list of equivalent MPartition fieldnames
+   * which can be used in the setResult clause of a JDO query
+   *
+   * @param partitionFields List of partitionFields in the projection
+   * @return List of JDO field names which can be used in setResult clause
+   * of a JDO query. Returns null if input partitionFields cannot be used in a setResult clause
+   */
+  public static List<String> getMPartitionFieldNames(List<String> partitionFields)
+      throws MetaException {
+    // if there are no partitionFields requested, it means all the fields are requested which include
+    // multi-valued fields.
+    if (partitionFields == null || partitionFields.isEmpty()) {
+      return null;
+    }
+    // throw exception if there are invalid field names
+    PartitionProjectionEvaluator.validate(partitionFields);
+    // else, check if all the fields are single-valued. In case there are multi-valued fields requested
+    // return null since setResult in JDO doesn't support multi-valued fields
+    if (!allPartitionSingleValuedFields.keySet().containsAll(partitionFields)) {
+      return null;
+    }
+    List<String> jdoFields = new ArrayList<>(partitionFields.size());
+    for (String partitionField : partitionFields) {
+      jdoFields.add(allPartitionSingleValuedFields.get(partitionField));
+    }
+    return jdoFields;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/44ef91a6/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java
index b61ee81..a6d9583 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java
@@ -20,6 +20,12 @@ package org.apache.hadoop.hive.metastore;
 
 import org.apache.hadoop.hive.common.TableName;
 import org.apache.hadoop.hive.metastore.api.*;
+import org.apache.hadoop.hive.metastore.api.CreationMetadata;
+import org.apache.hadoop.hive.metastore.api.ISchemaName;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.SchemaVersionDescriptor;
+import org.apache.hadoop.hive.metastore.api.WMFullResourcePlan;
+import org.apache.hadoop.hive.metastore.api.WriteEventInfo;
 
 import java.lang.annotation.ElementType;
 import java.lang.annotation.Retention;
@@ -546,6 +552,38 @@ public interface RawStore extends Configurable {
       throws MetaException, NoSuchObjectException;
 
   /**
+   * Generic Partition request API, providing different kinds of filtering and controlling output.
+   *
+   * @param catName catalog name
+   * @param dbName database name
+   * @param tblName table name
+   * @param fieldList a dot separated strings which represent the fields which must be returned.
+   *                  Any other field which is not in the fieldList may be unset in the returned
+   *                  partitions (it is up to the implementation to decide whether it chooses to
+   *                 include or exclude such fields). E.g. setting the field list to <em>sd.location</em>,
+   *                 <em>serdeInfo.name</em>, <em>sd.cols.name</em>, <em>sd.cols.type</em> will
+   *                 return partitions which will have location field set in the storage descriptor.
+   *                 Also the serdeInf in the returned storage descriptor will only have name field
+   *                 set. This applies to multi-valued fields as well like sd.cols, so in the
+   *                 example above only name and type fields will be set for <em>sd.cols</em>.
+   *                 If the <em>fieldList</em> is empty or not present, all the fields will be set.
+   * @param includeParamKeyPattern SQL-92 compliant regex pattern for param keys to be included
+   *                               _ or % wildcards are supported. '_' represent one character and
+   *                               '%' represents 0 or more characters.
+   * @param excludeParamKeyPattern SQL-92 compliant regex pattern for param keys to be excluded
+   *                               _ or % wildcards are supported. '_' represent one character and
+   *                               '%' represents 0 or more characters
+   * @return List of matching partitions which which may be partially filled according to fieldList.
+   * @throws MetaException in case of errors
+   * @throws NoSuchObjectException when catalog or database or table isn't found
+   */
+  List<Partition> getPartitionSpecsByFilterAndProjection(String catName, String dbName, String tblName,
+                                                         final List<String> fieldList,
+                                                         final String includeParamKeyPattern,
+                                                         final String excludeParamKeyPattern)
+      throws MetaException, NoSuchObjectException;
+
+  /**
    * Get partitions using an already parsed expression.
    * @param catName catalog name.
    * @param dbName database name

http://git-wip-us.apache.org/repos/asf/hive/blob/44ef91a6/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/StatObjectConverter.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/StatObjectConverter.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/StatObjectConverter.java
index 7a0b21b..89f0db8 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/StatObjectConverter.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/StatObjectConverter.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hive.metastore;
 
 import java.math.BigDecimal;
-import java.math.BigInteger;
 import java.nio.ByteBuffer;
 
 import org.apache.hadoop.hive.metastore.api.BinaryColumnStatsData;
@@ -501,74 +500,74 @@ public class StatObjectConverter {
     colType = colType.toLowerCase();
     if (colType.equals("boolean")) {
       BooleanColumnStatsData boolStats = new BooleanColumnStatsData();
-      boolStats.setNumFalses(MetaStoreDirectSql.extractSqlLong(falses));
-      boolStats.setNumTrues(MetaStoreDirectSql.extractSqlLong(trues));
-      boolStats.setNumNulls(MetaStoreDirectSql.extractSqlLong(nulls));
+      boolStats.setNumFalses(MetastoreDirectSqlUtils.extractSqlLong(falses));
+      boolStats.setNumTrues(MetastoreDirectSqlUtils.extractSqlLong(trues));
+      boolStats.setNumNulls(MetastoreDirectSqlUtils.extractSqlLong(nulls));
       data.setBooleanStats(boolStats);
     } else if (colType.equals("string") ||
         colType.startsWith("varchar") || colType.startsWith("char")) {
       StringColumnStatsDataInspector stringStats = new StringColumnStatsDataInspector();
-      stringStats.setNumNulls(MetaStoreDirectSql.extractSqlLong(nulls));
-      stringStats.setAvgColLen(MetaStoreDirectSql.extractSqlDouble(avglen));
-      stringStats.setMaxColLen(MetaStoreDirectSql.extractSqlLong(maxlen));
-      stringStats.setNumDVs(MetaStoreDirectSql.extractSqlLong(dist));
-      stringStats.setBitVectors(MetaStoreDirectSql.extractSqlBlob(bitVector));
+      stringStats.setNumNulls(MetastoreDirectSqlUtils.extractSqlLong(nulls));
+      stringStats.setAvgColLen(MetastoreDirectSqlUtils.extractSqlDouble(avglen));
+      stringStats.setMaxColLen(MetastoreDirectSqlUtils.extractSqlLong(maxlen));
+      stringStats.setNumDVs(MetastoreDirectSqlUtils.extractSqlLong(dist));
+      stringStats.setBitVectors(MetastoreDirectSqlUtils.extractSqlBlob(bitVector));
       data.setStringStats(stringStats);
     } else if (colType.equals("binary")) {
       BinaryColumnStatsData binaryStats = new BinaryColumnStatsData();
-      binaryStats.setNumNulls(MetaStoreDirectSql.extractSqlLong(nulls));
-      binaryStats.setAvgColLen(MetaStoreDirectSql.extractSqlDouble(avglen));
-      binaryStats.setMaxColLen(MetaStoreDirectSql.extractSqlLong(maxlen));
+      binaryStats.setNumNulls(MetastoreDirectSqlUtils.extractSqlLong(nulls));
+      binaryStats.setAvgColLen(MetastoreDirectSqlUtils.extractSqlDouble(avglen));
+      binaryStats.setMaxColLen(MetastoreDirectSqlUtils.extractSqlLong(maxlen));
       data.setBinaryStats(binaryStats);
     } else if (colType.equals("bigint") || colType.equals("int") ||
         colType.equals("smallint") || colType.equals("tinyint") ||
         colType.equals("timestamp")) {
       LongColumnStatsDataInspector longStats = new LongColumnStatsDataInspector();
-      longStats.setNumNulls(MetaStoreDirectSql.extractSqlLong(nulls));
+      longStats.setNumNulls(MetastoreDirectSqlUtils.extractSqlLong(nulls));
       if (lhigh != null) {
-        longStats.setHighValue(MetaStoreDirectSql.extractSqlLong(lhigh));
+        longStats.setHighValue(MetastoreDirectSqlUtils.extractSqlLong(lhigh));
       }
       if (llow != null) {
-        longStats.setLowValue(MetaStoreDirectSql.extractSqlLong(llow));
+        longStats.setLowValue(MetastoreDirectSqlUtils.extractSqlLong(llow));
       }
-      longStats.setNumDVs(MetaStoreDirectSql.extractSqlLong(dist));
-      longStats.setBitVectors(MetaStoreDirectSql.extractSqlBlob(bitVector));
+      longStats.setNumDVs(MetastoreDirectSqlUtils.extractSqlLong(dist));
+      longStats.setBitVectors(MetastoreDirectSqlUtils.extractSqlBlob(bitVector));
       data.setLongStats(longStats);
     } else if (colType.equals("double") || colType.equals("float")) {
       DoubleColumnStatsDataInspector doubleStats = new DoubleColumnStatsDataInspector();
-      doubleStats.setNumNulls(MetaStoreDirectSql.extractSqlLong(nulls));
+      doubleStats.setNumNulls(MetastoreDirectSqlUtils.extractSqlLong(nulls));
       if (dhigh != null) {
-        doubleStats.setHighValue(MetaStoreDirectSql.extractSqlDouble(dhigh));
+        doubleStats.setHighValue(MetastoreDirectSqlUtils.extractSqlDouble(dhigh));
       }
       if (dlow != null) {
-        doubleStats.setLowValue(MetaStoreDirectSql.extractSqlDouble(dlow));
+        doubleStats.setLowValue(MetastoreDirectSqlUtils.extractSqlDouble(dlow));
       }
-      doubleStats.setNumDVs(MetaStoreDirectSql.extractSqlLong(dist));
-      doubleStats.setBitVectors(MetaStoreDirectSql.extractSqlBlob(bitVector));
+      doubleStats.setNumDVs(MetastoreDirectSqlUtils.extractSqlLong(dist));
+      doubleStats.setBitVectors(MetastoreDirectSqlUtils.extractSqlBlob(bitVector));
       data.setDoubleStats(doubleStats);
     } else if (colType.startsWith("decimal")) {
       DecimalColumnStatsDataInspector decimalStats = new DecimalColumnStatsDataInspector();
-      decimalStats.setNumNulls(MetaStoreDirectSql.extractSqlLong(nulls));
+      decimalStats.setNumNulls(MetastoreDirectSqlUtils.extractSqlLong(nulls));
       if (dechigh != null) {
         decimalStats.setHighValue(DecimalUtils.createThriftDecimal((String)dechigh));
       }
       if (declow != null) {
         decimalStats.setLowValue(DecimalUtils.createThriftDecimal((String)declow));
       }
-      decimalStats.setNumDVs(MetaStoreDirectSql.extractSqlLong(dist));
-      decimalStats.setBitVectors(MetaStoreDirectSql.extractSqlBlob(bitVector));
+      decimalStats.setNumDVs(MetastoreDirectSqlUtils.extractSqlLong(dist));
+      decimalStats.setBitVectors(MetastoreDirectSqlUtils.extractSqlBlob(bitVector));
       data.setDecimalStats(decimalStats);
     } else if (colType.equals("date")) {
       DateColumnStatsDataInspector dateStats = new DateColumnStatsDataInspector();
-      dateStats.setNumNulls(MetaStoreDirectSql.extractSqlLong(nulls));
+      dateStats.setNumNulls(MetastoreDirectSqlUtils.extractSqlLong(nulls));
       if (lhigh != null) {
-        dateStats.setHighValue(new Date(MetaStoreDirectSql.extractSqlLong(lhigh)));
+        dateStats.setHighValue(new Date(MetastoreDirectSqlUtils.extractSqlLong(lhigh)));
       }
       if (llow != null) {
-        dateStats.setLowValue(new Date(MetaStoreDirectSql.extractSqlLong(llow)));
+        dateStats.setLowValue(new Date(MetastoreDirectSqlUtils.extractSqlLong(llow)));
       }
-      dateStats.setNumDVs(MetaStoreDirectSql.extractSqlLong(dist));
-      dateStats.setBitVectors(MetaStoreDirectSql.extractSqlBlob(bitVector));
+      dateStats.setNumDVs(MetastoreDirectSqlUtils.extractSqlLong(dist));
+      dateStats.setBitVectors(MetastoreDirectSqlUtils.extractSqlBlob(bitVector));
       data.setDateStats(dateStats);
     }
   }
@@ -582,49 +581,49 @@ public class StatObjectConverter {
     colType = colType.toLowerCase();
     if (colType.equals("boolean")) {
       BooleanColumnStatsData boolStats = new BooleanColumnStatsData();
-      boolStats.setNumFalses(MetaStoreDirectSql.extractSqlLong(falses));
-      boolStats.setNumTrues(MetaStoreDirectSql.extractSqlLong(trues));
-      boolStats.setNumNulls(MetaStoreDirectSql.extractSqlLong(nulls));
+      boolStats.setNumFalses(MetastoreDirectSqlUtils.extractSqlLong(falses));
+      boolStats.setNumTrues(MetastoreDirectSqlUtils.extractSqlLong(trues));
+      boolStats.setNumNulls(MetastoreDirectSqlUtils.extractSqlLong(nulls));
       data.setBooleanStats(boolStats);
     } else if (colType.equals("string") || colType.startsWith("varchar")
         || colType.startsWith("char")) {
       StringColumnStatsDataInspector stringStats = new StringColumnStatsDataInspector();
-      stringStats.setNumNulls(MetaStoreDirectSql.extractSqlLong(nulls));
-      stringStats.setAvgColLen(MetaStoreDirectSql.extractSqlDouble(avglen));
-      stringStats.setMaxColLen(MetaStoreDirectSql.extractSqlLong(maxlen));
-      stringStats.setNumDVs(MetaStoreDirectSql.extractSqlLong(dist));
+      stringStats.setNumNulls(MetastoreDirectSqlUtils.extractSqlLong(nulls));
+      stringStats.setAvgColLen(MetastoreDirectSqlUtils.extractSqlDouble(avglen));
+      stringStats.setMaxColLen(MetastoreDirectSqlUtils.extractSqlLong(maxlen));
+      stringStats.setNumDVs(MetastoreDirectSqlUtils.extractSqlLong(dist));
       data.setStringStats(stringStats);
     } else if (colType.equals("binary")) {
       BinaryColumnStatsData binaryStats = new BinaryColumnStatsData();
-      binaryStats.setNumNulls(MetaStoreDirectSql.extractSqlLong(nulls));
-      binaryStats.setAvgColLen(MetaStoreDirectSql.extractSqlDouble(avglen));
-      binaryStats.setMaxColLen(MetaStoreDirectSql.extractSqlLong(maxlen));
+      binaryStats.setNumNulls(MetastoreDirectSqlUtils.extractSqlLong(nulls));
+      binaryStats.setAvgColLen(MetastoreDirectSqlUtils.extractSqlDouble(avglen));
+      binaryStats.setMaxColLen(MetastoreDirectSqlUtils.extractSqlLong(maxlen));
       data.setBinaryStats(binaryStats);
     } else if (colType.equals("bigint") || colType.equals("int") || colType.equals("smallint")
         || colType.equals("tinyint") || colType.equals("timestamp")) {
       LongColumnStatsDataInspector longStats = new LongColumnStatsDataInspector();
-      longStats.setNumNulls(MetaStoreDirectSql.extractSqlLong(nulls));
+      longStats.setNumNulls(MetastoreDirectSqlUtils.extractSqlLong(nulls));
       if (lhigh != null) {
-        longStats.setHighValue(MetaStoreDirectSql.extractSqlLong(lhigh));
+        longStats.setHighValue(MetastoreDirectSqlUtils.extractSqlLong(lhigh));
       }
       if (llow != null) {
-        longStats.setLowValue(MetaStoreDirectSql.extractSqlLong(llow));
+        longStats.setLowValue(MetastoreDirectSqlUtils.extractSqlLong(llow));
       }
-      long lowerBound = MetaStoreDirectSql.extractSqlLong(dist);
-      long higherBound = MetaStoreDirectSql.extractSqlLong(sumDist);
+      long lowerBound = MetastoreDirectSqlUtils.extractSqlLong(dist);
+      long higherBound = MetastoreDirectSqlUtils.extractSqlLong(sumDist);
       long rangeBound = Long.MAX_VALUE;
       if (lhigh != null && llow != null) {
-        rangeBound = MetaStoreDirectSql.extractSqlLong(lhigh)
-            - MetaStoreDirectSql.extractSqlLong(llow) + 1;
+        rangeBound = MetastoreDirectSqlUtils.extractSqlLong(lhigh)
+            - MetastoreDirectSqlUtils.extractSqlLong(llow) + 1;
       }
       long estimation;
       if (useDensityFunctionForNDVEstimation && lhigh != null && llow != null && avgLong != null
-          && MetaStoreDirectSql.extractSqlDouble(avgLong) != 0.0) {
+          && MetastoreDirectSqlUtils.extractSqlDouble(avgLong) != 0.0) {
         // We have estimation, lowerbound and higherbound. We use estimation if
         // it is between lowerbound and higherbound.
-        estimation = MetaStoreDirectSql
-            .extractSqlLong((MetaStoreDirectSql.extractSqlLong(lhigh) - MetaStoreDirectSql
-                .extractSqlLong(llow)) / MetaStoreDirectSql.extractSqlDouble(avgLong));
+        estimation = MetastoreDirectSqlUtils
+            .extractSqlLong((MetastoreDirectSqlUtils.extractSqlLong(lhigh) - MetastoreDirectSqlUtils
+                .extractSqlLong(llow)) / MetastoreDirectSqlUtils.extractSqlDouble(avgLong));
         if (estimation < lowerBound) {
           estimation = lowerBound;
         } else if (estimation > higherBound) {
@@ -638,28 +637,28 @@ public class StatObjectConverter {
       data.setLongStats(longStats);
     } else if (colType.equals("date")) {
       DateColumnStatsDataInspector dateStats = new DateColumnStatsDataInspector();
-      dateStats.setNumNulls(MetaStoreDirectSql.extractSqlLong(nulls));
+      dateStats.setNumNulls(MetastoreDirectSqlUtils.extractSqlLong(nulls));
       if (lhigh != null) {
-        dateStats.setHighValue(new Date(MetaStoreDirectSql.extractSqlLong(lhigh)));
+        dateStats.setHighValue(new Date(MetastoreDirectSqlUtils.extractSqlLong(lhigh)));
       }
       if (llow != null) {
-        dateStats.setLowValue(new Date(MetaStoreDirectSql.extractSqlLong(llow)));
+        dateStats.setLowValue(new Date(MetastoreDirectSqlUtils.extractSqlLong(llow)));
       }
-      long lowerBound = MetaStoreDirectSql.extractSqlLong(dist);
-      long higherBound = MetaStoreDirectSql.extractSqlLong(sumDist);
+      long lowerBound = MetastoreDirectSqlUtils.extractSqlLong(dist);
+      long higherBound = MetastoreDirectSqlUtils.extractSqlLong(sumDist);
       long rangeBound = Long.MAX_VALUE;
       if (lhigh != null && llow != null) {
-        rangeBound = MetaStoreDirectSql.extractSqlLong(lhigh)
-            - MetaStoreDirectSql.extractSqlLong(llow) + 1;
+        rangeBound = MetastoreDirectSqlUtils.extractSqlLong(lhigh)
+            - MetastoreDirectSqlUtils.extractSqlLong(llow) + 1;
       }
       long estimation;
       if (useDensityFunctionForNDVEstimation && lhigh != null && llow != null && avgLong != null
-          && MetaStoreDirectSql.extractSqlDouble(avgLong) != 0.0) {
+          && MetastoreDirectSqlUtils.extractSqlDouble(avgLong) != 0.0) {
         // We have estimation, lowerbound and higherbound. We use estimation if
         // it is between lowerbound and higherbound.
-        estimation = MetaStoreDirectSql
-            .extractSqlLong((MetaStoreDirectSql.extractSqlLong(lhigh) - MetaStoreDirectSql
-                .extractSqlLong(llow)) / MetaStoreDirectSql.extractSqlDouble(avgLong));
+        estimation = MetastoreDirectSqlUtils
+            .extractSqlLong((MetastoreDirectSqlUtils.extractSqlLong(lhigh) - MetastoreDirectSqlUtils
+                .extractSqlLong(llow)) / MetastoreDirectSqlUtils.extractSqlDouble(avgLong));
         if (estimation < lowerBound) {
           estimation = lowerBound;
         } else if (estimation > higherBound) {
@@ -673,20 +672,20 @@ public class StatObjectConverter {
       data.setDateStats(dateStats);
     } else if (colType.equals("double") || colType.equals("float")) {
       DoubleColumnStatsDataInspector doubleStats = new DoubleColumnStatsDataInspector();
-      doubleStats.setNumNulls(MetaStoreDirectSql.extractSqlLong(nulls));
+      doubleStats.setNumNulls(MetastoreDirectSqlUtils.extractSqlLong(nulls));
       if (dhigh != null) {
-        doubleStats.setHighValue(MetaStoreDirectSql.extractSqlDouble(dhigh));
+        doubleStats.setHighValue(MetastoreDirectSqlUtils.extractSqlDouble(dhigh));
       }
       if (dlow != null) {
-        doubleStats.setLowValue(MetaStoreDirectSql.extractSqlDouble(dlow));
+        doubleStats.setLowValue(MetastoreDirectSqlUtils.extractSqlDouble(dlow));
       }
-      long lowerBound = MetaStoreDirectSql.extractSqlLong(dist);
-      long higherBound = MetaStoreDirectSql.extractSqlLong(sumDist);
+      long lowerBound = MetastoreDirectSqlUtils.extractSqlLong(dist);
+      long higherBound = MetastoreDirectSqlUtils.extractSqlLong(sumDist);
       if (useDensityFunctionForNDVEstimation && dhigh != null && dlow != null && avgDouble != null
-          && MetaStoreDirectSql.extractSqlDouble(avgDouble) != 0.0) {
-        long estimation = MetaStoreDirectSql
-            .extractSqlLong((MetaStoreDirectSql.extractSqlLong(dhigh) - MetaStoreDirectSql
-                .extractSqlLong(dlow)) / MetaStoreDirectSql.extractSqlDouble(avgDouble));
+          && MetastoreDirectSqlUtils.extractSqlDouble(avgDouble) != 0.0) {
+        long estimation = MetastoreDirectSqlUtils
+            .extractSqlLong((MetastoreDirectSqlUtils.extractSqlLong(dhigh) - MetastoreDirectSqlUtils
+                .extractSqlLong(dlow)) / MetastoreDirectSqlUtils.extractSqlDouble(avgDouble));
         if (estimation < lowerBound) {
           doubleStats.setNumDVs(lowerBound);
         } else if (estimation > higherBound) {
@@ -700,7 +699,7 @@ public class StatObjectConverter {
       data.setDoubleStats(doubleStats);
     } else if (colType.startsWith("decimal")) {
       DecimalColumnStatsDataInspector decimalStats = new DecimalColumnStatsDataInspector();
-      decimalStats.setNumNulls(MetaStoreDirectSql.extractSqlLong(nulls));
+      decimalStats.setNumNulls(MetastoreDirectSqlUtils.extractSqlLong(nulls));
       Decimal low = null;
       Decimal high = null;
       BigDecimal blow = null;
@@ -722,12 +721,12 @@ public class StatObjectConverter {
         low = DecimalUtils.createThriftDecimal((String) declow);
       }
       decimalStats.setLowValue(low);
-      long lowerBound = MetaStoreDirectSql.extractSqlLong(dist);
-      long higherBound = MetaStoreDirectSql.extractSqlLong(sumDist);
+      long lowerBound = MetastoreDirectSqlUtils.extractSqlLong(dist);
+      long higherBound = MetastoreDirectSqlUtils.extractSqlLong(sumDist);
       if (useDensityFunctionForNDVEstimation && dechigh != null && declow != null && avgDecimal != null
-          && MetaStoreDirectSql.extractSqlDouble(avgDecimal) != 0.0) {
-        long estimation = MetaStoreDirectSql.extractSqlLong(MetaStoreDirectSql.extractSqlLong(bhigh
-            .subtract(blow).floatValue() / MetaStoreDirectSql.extractSqlDouble(avgDecimal)));
+          && MetastoreDirectSqlUtils.extractSqlDouble(avgDecimal) != 0.0) {
+        long estimation = MetastoreDirectSqlUtils.extractSqlLong(MetastoreDirectSqlUtils.extractSqlLong(bhigh
+            .subtract(blow).floatValue() / MetastoreDirectSqlUtils.extractSqlDouble(avgDecimal)));
         if (estimation < lowerBound) {
           decimalStats.setNumDVs(lowerBound);
         } else if (estimation > higherBound) {

http://git-wip-us.apache.org/repos/asf/hive/blob/44ef91a6/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
index b9a5458..70490f0 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
@@ -1271,6 +1271,19 @@ public class CachedStore implements RawStore, Configurable {
   }
 
   @Override
+  /**
+   * getPartitionSpecsByFilterAndProjection interface is currently non-cacheable.
+   */
+  public List<Partition> getPartitionSpecsByFilterAndProjection(String catName, String dbName,
+                                                                String tblName,
+                                                                List<String> fieldList,
+                                                                String includeParamKeyPattern,
+      String excludeParamKeysPattern) throws MetaException, NoSuchObjectException {
+    return rawStore.getPartitionSpecsByFilterAndProjection(catName, dbName, tblName, fieldList,
+        includeParamKeyPattern, excludeParamKeysPattern);
+  }
+
+  @Override
   public boolean getPartitionsByExpr(String catName, String dbName, String tblName, byte[] expr,
       String defaultPartitionName, short maxParts, List<Partition> result) throws TException {
     catName = StringUtils.normalizeIdentifier(catName);

http://git-wip-us.apache.org/repos/asf/hive/blob/44ef91a6/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/model/MSerDeInfo.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/model/MSerDeInfo.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/model/MSerDeInfo.java
index 68f07e2..b8895df 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/model/MSerDeInfo.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/model/MSerDeInfo.java
@@ -29,6 +29,9 @@ public class MSerDeInfo {
   private String deserializerClass;
   private int serdeType;
 
+  public MSerDeInfo() {
+    //default constructor used for deserialization
+  }
   /**
    *
    * @param name

http://git-wip-us.apache.org/repos/asf/hive/blob/44ef91a6/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/model/MStorageDescriptor.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/model/MStorageDescriptor.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/model/MStorageDescriptor.java
index 4c6ce00..304860b 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/model/MStorageDescriptor.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/model/MStorageDescriptor.java
@@ -98,7 +98,7 @@ public class MStorageDescriptor {
   /**
    * @param isCompressed the isCompressed to set
    */
-  public void setCompressed(boolean isCompressed) {
+  public void setIsCompressed(boolean isCompressed) {
     this.isCompressed = isCompressed;
   }
 
@@ -274,4 +274,11 @@ public class MStorageDescriptor {
     this.isStoredAsSubDirectories = storedAsSubDirectories;
   }
 
+  public MColumnDescriptor getCd() {
+    return cd;
+  }
+
+  public void setCd(MColumnDescriptor cd) {
+    this.cd = cd;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/44ef91a6/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/utils/MetaStoreServerUtils.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/utils/MetaStoreServerUtils.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/utils/MetaStoreServerUtils.java
index 10ff9df..f3b3866 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/utils/MetaStoreServerUtils.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/utils/MetaStoreServerUtils.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hive.metastore.utils;
 
+import java.beans.PropertyDescriptor;
 import java.io.IOException;
 import java.lang.reflect.InvocationTargetException;
 import java.math.BigDecimal;
@@ -32,6 +33,7 @@ import java.util.Collection;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
 import java.util.SortedMap;
 import java.util.SortedSet;
 import java.util.TreeMap;
@@ -43,10 +45,14 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 import java.util.regex.Pattern;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Predicates;
+import com.google.common.collect.ImmutableListMultimap;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
+import com.google.common.collect.Multimaps;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.commons.beanutils.PropertyUtils;
 import org.apache.commons.collections.CollectionUtils;
 import org.apache.commons.collections.ListUtils;
 import org.apache.commons.lang.StringUtils;
@@ -56,7 +62,6 @@ import org.apache.hadoop.hive.common.StatsSetupConst;
 import org.apache.hadoop.hive.common.TableName;
 import org.apache.hadoop.hive.metastore.ColumnType;
 import org.apache.hadoop.hive.metastore.HiveMetaStore;
-import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
@@ -68,6 +73,10 @@ import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.Order;
 import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.PartitionListComposingSpec;
+import org.apache.hadoop.hive.metastore.api.PartitionSpec;
+import org.apache.hadoop.hive.metastore.api.PartitionSpecWithSharedSD;
+import org.apache.hadoop.hive.metastore.api.PartitionWithoutSD;
 import org.apache.hadoop.hive.metastore.api.SerDeInfo;
 import org.apache.hadoop.hive.metastore.api.SkewedInfo;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
@@ -108,8 +117,11 @@ public class MetaStoreServerUtils {
       return org.apache.commons.lang.StringUtils.defaultString(string);
     }
   };
+
   private static final String DELEGATION_TOKEN_STORE_CLS = "hive.cluster.delegation.token.store.class";
 
+  private static final char DOT = '.';
+
   /**
    * We have a need to sanity-check the map before conversion from persisted objects to
    * metadata thrift objects because null values in maps will cause a NPE if we send
@@ -977,6 +989,168 @@ public class MetaStoreServerUtils {
     }
   }
 
+  /**
+   * Coalesce list of partitions belonging to a table into a more compact PartitionSpec
+   * representation.
+   *
+   * @param table Table thrift object
+   * @param partitions List of partition objects
+   * @return collection PartitionSpec objects which is a compressed representation of original
+   * partition list.
+   */
+  public static List<PartitionSpec> getPartitionspecsGroupedByStorageDescriptor(Table table,
+                                                                                Collection<Partition> partitions) {
+    final String tablePath = table.getSd().getLocation();
+
+    ImmutableListMultimap<StorageDescriptorKey, Partition> partitionsWithinTableDirectory =
+        Multimaps.index(partitions, input -> {
+          // if sd is not in the list of projected fields, all the partitions
+          // can be just grouped in PartitionSpec object
+          if (input.getSd() == null) {
+            return StorageDescriptorKey.UNSET_KEY;
+          }
+          // if the partition is within table, use the tableSDKey to group it with other partitions
+          // within the table directory
+          if (input.getSd().getLocation() != null && input.getSd().getLocation()
+              .startsWith(tablePath)) {
+            return new StorageDescriptorKey(tablePath, input.getSd());
+          }
+          // if partitions are located outside table location we treat them as non-standard
+          // and do not perform any grouping
+          // if the location is not set partitions are grouped according to the rest of the SD fields
+          return new StorageDescriptorKey(input.getSd());
+        });
+
+    List<PartitionSpec> partSpecs = new ArrayList<>();
+
+    // Classify partitions based on shared SD properties.
+    Map<StorageDescriptorKey, List<PartitionWithoutSD>> sdToPartList
+        = new HashMap<>();
+    // we don't expect partitions to exist outside directory in most cases
+    List<Partition> partitionsOutsideTableDir = new ArrayList<>(0);
+    for (StorageDescriptorKey key : partitionsWithinTableDirectory.keySet()) {
+      boolean isUnsetKey = key.equals(StorageDescriptorKey.UNSET_KEY);
+      // group the partitions together when
+      // case I : sd is not set because it was not in the requested fields
+      // case II : when sd.location is not set because it was not in the requested fields
+      // case III : when sd.location is set and it is located within table directory
+      if (isUnsetKey || key.baseLocation == null || key.baseLocation.equals(tablePath)) {
+        for (Partition partition : partitionsWithinTableDirectory.get(key)) {
+
+          PartitionWithoutSD partitionWithoutSD
+              = new PartitionWithoutSD();
+          partitionWithoutSD.setValues(partition.getValues());
+          partitionWithoutSD.setCreateTime(partition.getCreateTime());
+          partitionWithoutSD.setLastAccessTime(partition.getLastAccessTime());
+          partitionWithoutSD.setRelativePath(
+              (isUnsetKey || !partition.getSd().isSetLocation()) ? null : partition.getSd()
+                  .getLocation().substring(tablePath.length()));
+          partitionWithoutSD.setParameters(partition.getParameters());
+
+          if (!sdToPartList.containsKey(key)) {
+            sdToPartList.put(key, new ArrayList<>());
+          }
+          sdToPartList.get(key).add(partitionWithoutSD);
+        }
+      } else {
+        // Lump all partitions outside the tablePath into one PartSpec.
+        // if non-standard partitions need not be deDuped create PartitionListComposingSpec
+        // this will be used mostly for keeping backwards compatibility with  some HMS APIs which use
+        // PartitionListComposingSpec for non-standard partitions located outside table
+        partitionsOutsideTableDir.addAll(partitionsWithinTableDirectory.get(key));
+      }
+    }
+    // create sharedSDPartSpec for all the groupings
+    for (Map.Entry<StorageDescriptorKey, List<PartitionWithoutSD>> entry : sdToPartList
+        .entrySet()) {
+      partSpecs.add(getSharedSDPartSpec(table, entry.getKey(), entry.getValue()));
+    }
+    if (!partitionsOutsideTableDir.isEmpty()) {
+      PartitionSpec partListSpec = new PartitionSpec();
+      partListSpec.setDbName(table.getDbName());
+      partListSpec.setTableName(table.getTableName());
+      partListSpec.setPartitionList(new PartitionListComposingSpec(partitionsOutsideTableDir));
+      partSpecs.add(partListSpec);
+    }
+    return partSpecs;
+  }
+
+  /**
+   * Convert list of partitions to a PartitionSpec object.
+   */
+  private static PartitionSpec getSharedSDPartSpec(Table table, StorageDescriptorKey sdKey, List<PartitionWithoutSD> partitions) {
+    StorageDescriptor sd;
+    if (sdKey.getSd() == null) {
+      //sd is not requested set it empty StorageDescriptor in the PartitionSpec
+      sd = new StorageDescriptor();
+    } else {
+      sd = new StorageDescriptor(sdKey.getSd());
+      sd.setLocation(sdKey.baseLocation); // Use table-dir as root-dir.
+    }
+    PartitionSpecWithSharedSD sharedSDPartSpec =
+        new PartitionSpecWithSharedSD();
+    sharedSDPartSpec.setPartitions(partitions);
+    sharedSDPartSpec.setSd(sd);
+
+    PartitionSpec ret = new PartitionSpec();
+    ret.setRootPath(sd.getLocation());
+    ret.setSharedSDPartitionSpec(sharedSDPartSpec);
+    ret.setDbName(table.getDbName());
+    ret.setTableName(table.getTableName());
+
+    return ret;
+  }
+
+  /**
+   * This is a util method to set a nested property of a given object. The nested property is a
+   * dot separated string where each nesting level is separated by a dot. This method makes use of
+   * PropertyUtils methods from apache-commons library and assumes that the field names provided in
+   * the input propertyName have valid setters. eg. the propertyName sd.serdeInfo.inputFormat represents
+   * the inputformat field of the serdeInfo field of the sd field. The argument bean should have these
+   * fields (in this case it should be a Partition object). The value argument is the value to be set
+   * for the nested field. Note that if in case of one of nested levels is null you must set
+   * instantiateMissingFields argument to true otherwise this method could throw a NPE.
+   *
+   * @param bean the object whose nested field needs to be set. This object must have setter methods
+   *             defined for each nested field name in the propertyName
+   * @param propertyName the nested propertyName to be set. Each level of nesting is dot separated
+   * @param value the value to which the nested property is set
+   * @param instantiateMissingFields in case of some nestedFields being nulls, setting this argument
+   *                                 to true will attempt to instantiate the missing fields using the
+   *                                 default constructor. If there is no default constructor available this would throw a MetaException
+   * @throws MetaException
+   */
+  public static void setNestedProperty(Object bean, String propertyName, Object value,
+      boolean instantiateMissingFields) throws MetaException {
+    try {
+      String[] nestedFields = propertyName.split("\\.");
+      //check if there are more than one nested levels
+      if (nestedFields.length > 1 && instantiateMissingFields) {
+        StringBuilder fieldNameBuilder = new StringBuilder();
+        //check if all the nested levels until the given fieldName is set
+        for (int level = 0; level < nestedFields.length - 1; level++) {
+          fieldNameBuilder.append(nestedFields[level]);
+          String currentFieldName = fieldNameBuilder.toString();
+          Object fieldVal = PropertyUtils.getProperty(bean, currentFieldName);
+          if (fieldVal == null) {
+            //one of the nested levels is null. Instantiate it
+            PropertyDescriptor fieldDescriptor =
+                PropertyUtils.getPropertyDescriptor(bean, currentFieldName);
+            //this assumes the MPartition and the nested field objects have a default constructor
+            Object defaultInstance = fieldDescriptor.getPropertyType().newInstance();
+            PropertyUtils.setNestedProperty(bean, currentFieldName, defaultInstance);
+          }
+          //add dot separator for the next level of nesting
+          fieldNameBuilder.append(DOT);
+        }
+      }
+      PropertyUtils.setNestedProperty(bean, propertyName, value);
+    } catch (Exception e) {
+      throw new MetaException(
+          org.apache.hadoop.hive.metastore.utils.StringUtils.stringifyException(e));
+    }
+  }
+
   // ColumnStatisticsObj with info about its db, table, partition (if table is partitioned)
   public static class ColStatsObjWithSourceInfo {
     private final ColumnStatisticsObj colStatsObj;
@@ -1014,4 +1188,110 @@ public class MetaStoreServerUtils {
       return partName;
     }
   }
+
+  /**
+   * This class is used to group the partitions based on a shared storage descriptor.
+   * The following fields are considered for hashing/equality:
+   * <ul>
+   *   <li>location</li>
+   *   <li>serializationLib</li>
+   *   <li>inputFormat</li>
+   *   <li>outputFormat</li>
+   *   <li>columns</li>
+   * </ul>
+   *
+   * For objects that share these can share the same storage descriptor,
+   * significantly reducing on-the-wire cost.
+   *
+   * Check {@link #getPartitionspecsGroupedByStorageDescriptor} for more details
+   */
+  @VisibleForTesting
+  static class StorageDescriptorKey {
+    private final StorageDescriptor sd;
+    private final String baseLocation;
+    private final int hashCode;
+
+    @VisibleForTesting
+    static final StorageDescriptorKey UNSET_KEY = new StorageDescriptorKey();
+
+    StorageDescriptorKey(StorageDescriptor sd) {
+      this(sd.getLocation(), sd);
+    }
+
+    StorageDescriptorKey(String baseLocation, StorageDescriptor sd) {
+      this.sd = sd;
+      this.baseLocation = baseLocation;
+      if (sd == null) {
+        hashCode = Objects.hashCode(baseLocation);
+      } else {
+        // use the baseLocation provided instead of sd.getLocation()
+        hashCode = Objects.hash(sd.getSerdeInfo() == null ? null :
+                sd.getSerdeInfo().getSerializationLib(),
+            sd.getInputFormat(), sd.getOutputFormat(), baseLocation, sd.getCols());
+      }
+    }
+
+    // Set everything to null
+    StorageDescriptorKey() {
+      baseLocation = null;
+      sd = null;
+      hashCode = 0;
+    }
+
+    StorageDescriptor getSd() {
+      return sd;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o)
+        return true;
+      if (o == null || getClass() != o.getClass())
+        return false;
+      StorageDescriptorKey that = (StorageDescriptorKey) o;
+
+      if (!Objects.equals(baseLocation, that.baseLocation)) {
+        return false;
+      }
+
+      if (sd == null && that.sd == null) {
+        return true;
+      }
+
+      if (sd == null || that.sd == null) {
+        return false;
+      }
+
+      if (!Objects.equals(sd.getOutputFormat(), that.sd.getOutputFormat())) {
+        return false;
+      }
+      if (!Objects.equals(sd.getCols(), that.sd.getCols())) {
+        return false;
+      }
+      if (!Objects.equals(sd.getInputFormat(), that.sd.getInputFormat())) {
+        return false;
+      }
+
+      if (!Objects.equals(sd.getSerdeInfo(), that.sd.getSerdeInfo())) {
+        return false;
+      }
+      if (sd.getSerdeInfo() != null && that.sd.getSerdeInfo() == null) {
+        return false;
+      }
+      if (sd.getSerdeInfo() == null && that.sd.getSerdeInfo() != null) {
+        return false;
+      }
+      if (sd.getSerdeInfo() != null && that.sd.getSerdeInfo() != null &&
+          !Objects.equals(sd.getSerdeInfo().getSerializationLib(),
+              that.sd.getSerdeInfo().getSerializationLib())) {
+        return false;
+      }
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      return hashCode;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/44ef91a6/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
index 0934aeb..4dd4edc 100644
--- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
+++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
@@ -18,13 +18,15 @@
 
 package org.apache.hadoop.hive.metastore;
 
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.common.TableName;
 import org.apache.hadoop.hive.metastore.api.AggrStats;
 import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
 import org.apache.hadoop.hive.metastore.api.Catalog;
 import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
-import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId;
 import org.apache.hadoop.hive.metastore.api.CreationMetadata;
+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;
@@ -32,6 +34,7 @@ import org.apache.hadoop.hive.metastore.api.Function;
 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.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;
@@ -49,11 +52,6 @@ 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.WMNullablePool;
-import org.apache.hadoop.hive.metastore.api.WMNullableResourcePlan;
-import org.apache.hadoop.hive.metastore.api.WMResourcePlan;
-import org.apache.hadoop.hive.metastore.api.WMTrigger;
-import org.apache.hadoop.hive.metastore.api.WMValidateResourcePlanResponse;
 import org.apache.hadoop.hive.metastore.api.Role;
 import org.apache.hadoop.hive.metastore.api.RolePrincipalGrant;
 import org.apache.hadoop.hive.metastore.api.RuntimeStat;
@@ -64,6 +62,7 @@ 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.SchemaVersionDescriptor;
 import org.apache.hadoop.hive.metastore.api.SerDeInfo;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.api.TableMeta;
@@ -71,11 +70,18 @@ import org.apache.hadoop.hive.metastore.api.Type;
 import org.apache.hadoop.hive.metastore.api.UnknownDBException;
 import org.apache.hadoop.hive.metastore.api.UnknownPartitionException;
 import org.apache.hadoop.hive.metastore.api.UnknownTableException;
+import org.apache.hadoop.hive.metastore.api.WMFullResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMMapping;
+import org.apache.hadoop.hive.metastore.api.WMNullablePool;
+import org.apache.hadoop.hive.metastore.api.WMNullableResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMPool;
-import org.apache.hadoop.hive.metastore.api.WMFullResourcePlan;
-import org.apache.hadoop.hive.metastore.api.ISchemaName;
-import org.apache.hadoop.hive.metastore.api.SchemaVersionDescriptor;
+import org.apache.hadoop.hive.metastore.api.WMResourcePlan;
+import org.apache.hadoop.hive.metastore.api.WMTrigger;
+import org.apache.hadoop.hive.metastore.api.WMValidateResourcePlanResponse;
+import org.apache.hadoop.hive.metastore.api.WriteEventInfo;
+import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreServerUtils.ColStatsObjWithSourceInfo;
+import org.apache.thrift.TException;
 
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
@@ -83,13 +89,6 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.metastore.api.WriteEventInfo;
-import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
-import org.apache.hadoop.hive.metastore.utils.MetaStoreServerUtils.ColStatsObjWithSourceInfo;
-import org.apache.thrift.TException;
-
 /**
  * A wrapper around {@link org.apache.hadoop.hive.metastore.ObjectStore}
  * with the ability to control the result of commitTransaction().
@@ -377,6 +376,14 @@ public class DummyRawStoreControlledCommit implements RawStore, Configurable {
   }
 
   @Override
+  public List<Partition> getPartitionSpecsByFilterAndProjection(String catName, String dbName, String tblName,
+                                                                List<String> fieldList, String paramKeys, String excludeFlag)
+      throws MetaException, NoSuchObjectException {
+    return objectStore.getPartitionSpecsByFilterAndProjection(catName,
+        dbName, tblName, fieldList, paramKeys, excludeFlag);
+  }
+
+  @Override
   public int getNumPartitionsByFilter(String catName, String dbName, String tblName,
                                       String filter) throws MetaException, NoSuchObjectException {
     return objectStore.getNumPartitionsByFilter(catName, dbName, tblName, filter);

http://git-wip-us.apache.org/repos/asf/hive/blob/44ef91a6/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
index 70a17f5..06f4cbc 100644
--- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
+++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
@@ -380,6 +380,13 @@ public class DummyRawStoreForJdoConnection implements RawStore {
   }
 
   @Override
+  public List<Partition> getPartitionSpecsByFilterAndProjection(String catName, String dbName, String tblName,
+      List<String> fieldList, String paramKeys, String excludeFlag)
+      throws MetaException, NoSuchObjectException {
+    return Collections.emptyList();
+  }
+
+  @Override
   public List<Partition> getPartitionsByNames(String catName, String dbName, String tblName,
       List<String> partNames) throws MetaException, NoSuchObjectException {
 

http://git-wip-us.apache.org/repos/asf/hive/blob/44ef91a6/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java
index ce590d0..2861b6b 100644
--- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java
+++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java
@@ -3537,4 +3537,10 @@ public class HiveMetaStoreClientPreCatalog implements IMetaStoreClient, AutoClos
       throws TException {
     throw new UnsupportedOperationException();
   }
+
+  @Override
+  public GetPartitionsResponse getPartitionsWithSpecs(GetPartitionsRequest request)
+      throws TException {
+    throw new UnsupportedOperationException();
+  }
 }


[03/13] hive git commit: HIVE-20306 : Implement projection spec for fetching only requested fields from partitions (Vihang Karajgaonkar, reviewed by Aihua Xu, Andrew Sherman and Alexander Kolbasov)

Posted by vi...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/44ef91a6/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java.orig
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java.orig b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java.orig
new file mode 100644
index 0000000..5372714
--- /dev/null
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java.orig
@@ -0,0 +1,12514 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.metastore;
+
+import static org.apache.commons.lang.StringUtils.join;
+import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.getDefaultCatalog;
+import static org.apache.hadoop.hive.metastore.utils.StringUtils.normalizeIdentifier;
+
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.net.InetAddress;
+import java.net.URI;
+import java.nio.ByteBuffer;
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.sql.SQLIntegrityConstraintViolationException;
+import java.sql.Statement;
+import java.time.LocalDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Properties;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.regex.Pattern;
+
+import javax.jdo.JDOCanRetryException;
+import javax.jdo.JDODataStoreException;
+import javax.jdo.JDOException;
+import javax.jdo.JDOHelper;
+import javax.jdo.JDOObjectNotFoundException;
+import javax.jdo.PersistenceManager;
+import javax.jdo.PersistenceManagerFactory;
+import javax.jdo.Query;
+import javax.jdo.Transaction;
+import javax.jdo.datastore.DataStoreCache;
+import javax.jdo.datastore.JDOConnection;
+import javax.jdo.identity.IntIdentity;
+import javax.sql.DataSource;
+
+import com.google.common.base.Strings;
+
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.commons.lang.ArrayUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang.exception.ExceptionUtils;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.common.*;
+import org.apache.hadoop.hive.metastore.MetaStoreDirectSql.SqlFilterForPushdown;
+import org.apache.hadoop.hive.metastore.api.*;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
+import org.apache.hadoop.hive.metastore.datasource.DataSourceProvider;
+import org.apache.hadoop.hive.metastore.datasource.DataSourceProviderFactory;
+import org.apache.hadoop.hive.metastore.metrics.Metrics;
+import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
+import org.apache.hadoop.hive.metastore.model.*;
+import org.apache.hadoop.hive.metastore.model.MWMMapping.EntityType;
+import org.apache.hadoop.hive.metastore.model.MWMResourcePlan.Status;
+import org.apache.hadoop.hive.metastore.parser.ExpressionTree;
+import org.apache.hadoop.hive.metastore.parser.ExpressionTree.FilterBuilder;
+import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
+import org.apache.hadoop.hive.metastore.tools.SQLGenerator;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
+import org.apache.hadoop.hive.metastore.utils.FileUtils;
+import org.apache.hadoop.hive.metastore.utils.JavaUtils;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreServerUtils;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.hive.metastore.utils.ObjectPair;
+import org.apache.thrift.TException;
+import org.datanucleus.AbstractNucleusContext;
+import org.datanucleus.ClassLoaderResolver;
+import org.datanucleus.ClassLoaderResolverImpl;
+import org.datanucleus.NucleusContext;
+import org.datanucleus.PropertyNames;
+import org.datanucleus.api.jdo.JDOPersistenceManager;
+import org.datanucleus.api.jdo.JDOPersistenceManagerFactory;
+import org.datanucleus.store.rdbms.exceptions.MissingTableException;
+import org.datanucleus.store.scostore.Store;
+import org.datanucleus.util.WeakValueMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.codahale.metrics.Counter;
+import com.codahale.metrics.MetricRegistry;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+
+/**
+ * This class is the interface between the application logic and the database
+ * store that contains the objects. Refrain putting any logic in mode.M* objects
+ * or in this file as former could be auto generated and this class would need
+ * to be made into a interface that can read both from a database and a
+ * filestore.
+ */
+public class ObjectStore implements RawStore, Configurable {
+  private static Properties prop = null;
+  private static PersistenceManagerFactory pmf = null;
+  private static boolean forTwoMetastoreTesting = false;
+  private int batchSize = Batchable.NO_BATCHING;
+
+  private static final DateTimeFormatter YMDHMS_FORMAT = DateTimeFormatter.ofPattern(
+      "yyyy_MM_dd_HH_mm_ss");
+
+  private static Lock pmfPropLock = new ReentrantLock();
+  /**
+  * Verify the schema only once per JVM since the db connection info is static
+  */
+  private final static AtomicBoolean isSchemaVerified = new AtomicBoolean(false);
+  private static final Logger LOG = LoggerFactory.getLogger(ObjectStore.class);
+
+  private enum TXN_STATUS {
+    NO_STATE, OPEN, COMMITED, ROLLBACK
+  }
+
+  private static final Map<String, Class<?>> PINCLASSMAP;
+  private static final String HOSTNAME;
+  private static final String USER;
+  private static final String JDO_PARAM = ":param";
+  static {
+    Map<String, Class<?>> map = new HashMap<>();
+    map.put("table", MTable.class);
+    map.put("storagedescriptor", MStorageDescriptor.class);
+    map.put("serdeinfo", MSerDeInfo.class);
+    map.put("partition", MPartition.class);
+    map.put("database", MDatabase.class);
+    map.put("type", MType.class);
+    map.put("fieldschema", MFieldSchema.class);
+    map.put("order", MOrder.class);
+    PINCLASSMAP = Collections.unmodifiableMap(map);
+    String hostname = "UNKNOWN";
+    try {
+      InetAddress clientAddr = InetAddress.getLocalHost();
+      hostname = clientAddr.getHostAddress();
+    } catch (IOException e) {
+    }
+    HOSTNAME = hostname;
+    String user = System.getenv("USER");
+    USER = org.apache.commons.lang.StringUtils.defaultString(user, "UNKNOWN");
+  }
+
+
+  private boolean isInitialized = false;
+  private PersistenceManager pm = null;
+  private SQLGenerator sqlGenerator = null;
+  private MetaStoreDirectSql directSql = null;
+  private DatabaseProduct dbType = null;
+  private PartitionExpressionProxy expressionProxy = null;
+  private Configuration conf;
+  private volatile int openTrasactionCalls = 0;
+  private Transaction currentTransaction = null;
+  private TXN_STATUS transactionStatus = TXN_STATUS.NO_STATE;
+  private Pattern partitionValidationPattern;
+  private Counter directSqlErrors;
+  private boolean areTxnStatsSupported = false;
+
+  /**
+   * A Autocloseable wrapper around Query class to pass the Query object to the caller and let the caller release
+   * the resources when the QueryWrapper goes out of scope
+   */
+  public static class QueryWrapper implements AutoCloseable {
+    public Query query;
+
+    /**
+     * Explicitly closes the query object to release the resources
+     */
+    @Override
+    public void close() {
+      if (query != null) {
+        query.closeAll();
+        query = null;
+      }
+    }
+  }
+
+  public ObjectStore() {
+  }
+
+  @Override
+  public Configuration getConf() {
+    return conf;
+  }
+
+  /**
+   * Called whenever this object is instantiated using ReflectionUtils, and also
+   * on connection retries. In cases of connection retries, conf will usually
+   * contain modified values.
+   */
+  @Override
+  @SuppressWarnings("nls")
+  public void setConf(Configuration conf) {
+    // Although an instance of ObjectStore is accessed by one thread, there may
+    // be many threads with ObjectStore instances. So the static variables
+    // pmf and prop need to be protected with locks.
+    pmfPropLock.lock();
+    try {
+      isInitialized = false;
+      this.conf = conf;
+      this.areTxnStatsSupported = MetastoreConf.getBoolVar(conf, ConfVars.HIVE_TXN_STATS_ENABLED);
+      configureSSL(conf);
+      Properties propsFromConf = getDataSourceProps(conf);
+      boolean propsChanged = !propsFromConf.equals(prop);
+
+      if (propsChanged) {
+        if (pmf != null){
+          clearOutPmfClassLoaderCache(pmf);
+          if (!forTwoMetastoreTesting) {
+            // close the underlying connection pool to avoid leaks
+            pmf.close();
+          }
+        }
+        pmf = null;
+        prop = null;
+      }
+
+      assert(!isActiveTransaction());
+      shutdown();
+      // Always want to re-create pm as we don't know if it were created by the
+      // most recent instance of the pmf
+      pm = null;
+      directSql = null;
+      expressionProxy = null;
+      openTrasactionCalls = 0;
+      currentTransaction = null;
+      transactionStatus = TXN_STATUS.NO_STATE;
+
+      initialize(propsFromConf);
+
+      String partitionValidationRegex =
+          MetastoreConf.getVar(this.conf, ConfVars.PARTITION_NAME_WHITELIST_PATTERN);
+      if (partitionValidationRegex != null && !partitionValidationRegex.isEmpty()) {
+        partitionValidationPattern = Pattern.compile(partitionValidationRegex);
+      } else {
+        partitionValidationPattern = null;
+      }
+
+      // Note, if metrics have not been initialized this will return null, which means we aren't
+      // using metrics.  Thus we should always check whether this is non-null before using.
+      MetricRegistry registry = Metrics.getRegistry();
+      if (registry != null) {
+        directSqlErrors = Metrics.getOrCreateCounter(MetricsConstants.DIRECTSQL_ERRORS);
+      }
+
+      this.batchSize = MetastoreConf.getIntVar(conf, ConfVars.RAWSTORE_PARTITION_BATCH_SIZE);
+
+      if (!isInitialized) {
+        throw new RuntimeException(
+        "Unable to create persistence manager. Check dss.log for details");
+      } else {
+        LOG.debug("Initialized ObjectStore");
+      }
+    } finally {
+      pmfPropLock.unlock();
+    }
+  }
+
+  private ClassLoader classLoader;
+  {
+    classLoader = Thread.currentThread().getContextClassLoader();
+    if (classLoader == null) {
+      classLoader = ObjectStore.class.getClassLoader();
+    }
+  }
+
+  @SuppressWarnings("nls")
+  private void initialize(Properties dsProps) {
+    int retryLimit = MetastoreConf.getIntVar(conf, ConfVars.HMS_HANDLER_ATTEMPTS);
+    long retryInterval = MetastoreConf.getTimeVar(conf,
+        ConfVars.HMS_HANDLER_INTERVAL, TimeUnit.MILLISECONDS);
+    int numTries = retryLimit;
+
+    while (numTries > 0){
+      try {
+        initializeHelper(dsProps);
+        return; // If we reach here, we succeed.
+      } catch (Exception e){
+        shutdown();
+        numTries--;
+        boolean retriable = isRetriableException(e);
+        if ((numTries > 0) && retriable){
+          LOG.info("Retriable exception while instantiating ObjectStore, retrying. " +
+              "{} tries left", numTries, e);
+          try {
+            Thread.sleep(retryInterval);
+          } catch (InterruptedException ie) {
+            // Restore the interrupted status, since we do not want to catch it.
+            LOG.debug("Interrupted while sleeping before retrying.", ie);
+            Thread.currentThread().interrupt();
+          }
+          // If we're here, we'll proceed down the next while loop iteration.
+        } else {
+          // we've reached our limit, throw the last one.
+          if (retriable){
+            LOG.warn("Exception retry limit reached, not retrying any longer.",
+              e);
+          } else {
+            LOG.debug("Non-retriable exception during ObjectStore initialize.", e);
+          }
+          throw e;
+        }
+      }
+    }
+  }
+
+  private static final Set<Class<? extends Throwable>> retriableExceptionClasses =
+      new HashSet<>(Arrays.asList(JDOCanRetryException.class));
+  /**
+   * Helper function for initialize to determine if we should retry an exception.
+   * We return true if the exception is of a known type of retriable exceptions, or if one
+   * of its recursive .getCause returns a known type of retriable exception.
+   */
+  private boolean isRetriableException(Throwable e) {
+    if (e == null){
+      return false;
+    }
+    if (retriableExceptionClasses.contains(e.getClass())){
+      return true;
+    }
+    for (Class<? extends Throwable> c : retriableExceptionClasses){
+      if (c.isInstance(e)){
+        return true;
+      }
+    }
+
+    if (e.getCause() == null){
+      return false;
+    }
+    return isRetriableException(e.getCause());
+  }
+
+  /**
+   * private helper to do initialization routine, so we can retry if needed if it fails.
+   * @param dsProps
+   */
+  private void initializeHelper(Properties dsProps) {
+    LOG.debug("ObjectStore, initialize called");
+    prop = dsProps;
+    pm = getPersistenceManager();
+    LOG.info("RawStore: {}, with PersistenceManager: {}" +
+            " created in the thread with id: {}", this, pm, Thread.currentThread().getId());
+    try {
+      String productName = MetaStoreDirectSql.getProductName(pm);
+      sqlGenerator = new SQLGenerator(DatabaseProduct.determineDatabaseProduct(productName), conf);
+    } catch (SQLException e) {
+      LOG.error("error trying to figure out the database product", e);
+      throw new RuntimeException(e);
+    }
+    isInitialized = pm != null;
+    if (isInitialized) {
+      dbType = determineDatabaseProduct();
+      expressionProxy = createExpressionProxy(conf);
+      if (MetastoreConf.getBoolVar(getConf(), ConfVars.TRY_DIRECT_SQL)) {
+        String schema = prop.getProperty("javax.jdo.mapping.Schema");
+        schema = org.apache.commons.lang.StringUtils.defaultIfBlank(schema, null);
+        directSql = new MetaStoreDirectSql(pm, conf, schema);
+      }
+    }
+  }
+
+  private DatabaseProduct determineDatabaseProduct() {
+    try {
+      return DatabaseProduct.determineDatabaseProduct(getProductName(pm));
+    } catch (SQLException e) {
+      LOG.warn("Cannot determine database product; assuming OTHER", e);
+      return DatabaseProduct.OTHER;
+    }
+  }
+
+  private static String getProductName(PersistenceManager pm) {
+    JDOConnection jdoConn = pm.getDataStoreConnection();
+    try {
+      return ((Connection)jdoConn.getNativeConnection()).getMetaData().getDatabaseProductName();
+    } catch (Throwable t) {
+      LOG.warn("Error retrieving product name", t);
+      return null;
+    } finally {
+      jdoConn.close(); // We must release the connection before we call other pm methods.
+    }
+  }
+
+  /**
+   * Creates the proxy used to evaluate expressions. This is here to prevent circular
+   * dependency - ql -&gt; metastore client &lt;-&gt metastore server -&gt ql. If server and
+   * client are split, this can be removed.
+   * @param conf Configuration.
+   * @return The partition expression proxy.
+   */
+  private static PartitionExpressionProxy createExpressionProxy(Configuration conf) {
+    String className = MetastoreConf.getVar(conf, ConfVars.EXPRESSION_PROXY_CLASS);
+    try {
+      Class<? extends PartitionExpressionProxy> clazz =
+           JavaUtils.getClass(className, PartitionExpressionProxy.class);
+      return JavaUtils.newInstance(clazz, new Class<?>[0], new Object[0]);
+    } catch (MetaException e) {
+      LOG.error("Error loading PartitionExpressionProxy", e);
+      throw new RuntimeException("Error loading PartitionExpressionProxy: " + e.getMessage());
+    }
+  }
+
+  /**
+   * Configure the SSL properties of the connection from provided config
+   * @param conf
+   */
+  private static void configureSSL(Configuration conf) {
+    // SSL support
+    String sslPropString = MetastoreConf.getVar(conf, ConfVars.DBACCESS_SSL_PROPS);
+    if (org.apache.commons.lang.StringUtils.isNotEmpty(sslPropString)) {
+      LOG.info("Metastore setting SSL properties of the connection to backed DB");
+      for (String sslProp : sslPropString.split(",")) {
+        String[] pair = sslProp.trim().split("=");
+        if (pair != null && pair.length == 2) {
+          System.setProperty(pair[0].trim(), pair[1].trim());
+        } else {
+          LOG.warn("Invalid metastore property value for {}", ConfVars.DBACCESS_SSL_PROPS);
+        }
+      }
+    }
+  }
+
+  /**
+   * Properties specified in hive-default.xml override the properties specified
+   * in jpox.properties.
+   */
+  @SuppressWarnings("nls")
+  private static Properties getDataSourceProps(Configuration conf) {
+    Properties prop = new Properties();
+    correctAutoStartMechanism(conf);
+
+    // First, go through and set all our values for datanucleus and javax.jdo parameters.  This
+    // has to be a separate first step because we don't set the default values in the config object.
+    for (ConfVars var : MetastoreConf.dataNucleusAndJdoConfs) {
+      String confVal = MetastoreConf.getAsString(conf, var);
+      String varName = var.getVarname();
+      Object prevVal = prop.setProperty(varName, confVal);
+      if (MetastoreConf.isPrintable(varName)) {
+        LOG.debug("Overriding {} value {} from jpox.properties with {}",
+          varName, prevVal, confVal);
+      }
+    }
+
+    // Now, we need to look for any values that the user set that MetastoreConf doesn't know about.
+    // TODO Commenting this out for now, as it breaks because the conf values aren't getting properly
+    // interpolated in case of variables.  See HIVE-17788.
+    /*
+    for (Map.Entry<String, String> e : conf) {
+      if (e.getKey().startsWith("datanucleus.") || e.getKey().startsWith("javax.jdo.")) {
+        // We have to handle this differently depending on whether it is a value known to
+        // MetastoreConf or not.  If it is, we need to get the default value if a value isn't
+        // provided.  If not, we just set whatever the user has set.
+        Object prevVal = prop.setProperty(e.getKey(), e.getValue());
+        if (LOG.isDebugEnabled() && MetastoreConf.isPrintable(e.getKey())) {
+          LOG.debug("Overriding " + e.getKey() + " value " + prevVal
+              + " from  jpox.properties with " + e.getValue());
+        }
+      }
+    }
+    */
+
+    // Password may no longer be in the conf, use getPassword()
+    try {
+      String passwd = MetastoreConf.getPassword(conf, MetastoreConf.ConfVars.PWD);
+      if (org.apache.commons.lang.StringUtils.isNotEmpty(passwd)) {
+        // We can get away with the use of varname here because varname == hiveName for PWD
+        prop.setProperty(ConfVars.PWD.getVarname(), passwd);
+      }
+    } catch (IOException err) {
+      throw new RuntimeException("Error getting metastore password: " + err.getMessage(), err);
+    }
+
+    if (LOG.isDebugEnabled()) {
+      for (Entry<Object, Object> e : prop.entrySet()) {
+        if (MetastoreConf.isPrintable(e.getKey().toString())) {
+          LOG.debug("{} = {}", e.getKey(), e.getValue());
+        }
+      }
+    }
+
+    return prop;
+  }
+
+  /**
+   * Update conf to set datanucleus.autoStartMechanismMode=ignored.
+   * This is necessary to able to use older version of hive against
+   * an upgraded but compatible metastore schema in db from new version
+   * of hive
+   * @param conf
+   */
+  private static void correctAutoStartMechanism(Configuration conf) {
+    final String autoStartKey = "datanucleus.autoStartMechanismMode";
+    final String autoStartIgnore = "ignored";
+    String currentAutoStartVal = conf.get(autoStartKey);
+    if (!autoStartIgnore.equalsIgnoreCase(currentAutoStartVal)) {
+      LOG.warn("{} is set to unsupported value {} . Setting it to value: {}", autoStartKey,
+        conf.get(autoStartKey), autoStartIgnore);
+    }
+    conf.set(autoStartKey, autoStartIgnore);
+  }
+
+  private static synchronized PersistenceManagerFactory getPMF() {
+    if (pmf == null) {
+
+      Configuration conf = MetastoreConf.newMetastoreConf();
+      DataSourceProvider dsp = DataSourceProviderFactory.hasProviderSpecificConfigurations(conf) ?
+              DataSourceProviderFactory.getDataSourceProvider(conf) : null;
+
+      if (dsp == null) {
+        pmf = JDOHelper.getPersistenceManagerFactory(prop);
+      } else {
+        try {
+          DataSource ds = dsp.create(conf);
+          Map<Object, Object> dsProperties = new HashMap<>();
+          //Any preexisting datanucleus property should be passed along
+          dsProperties.putAll(prop);
+          dsProperties.put(PropertyNames.PROPERTY_CONNECTION_FACTORY, ds);
+          dsProperties.put(PropertyNames.PROPERTY_CONNECTION_FACTORY2, ds);
+          dsProperties.put("javax.jdo.PersistenceManagerFactoryClass",
+              "org.datanucleus.api.jdo.JDOPersistenceManagerFactory");
+          pmf = JDOHelper.getPersistenceManagerFactory(dsProperties);
+        } catch (SQLException e) {
+          LOG.warn("Could not create PersistenceManagerFactory using " +
+              "connection pool properties, will fall back", e);
+          pmf = JDOHelper.getPersistenceManagerFactory(prop);
+        }
+      }
+      DataStoreCache dsc = pmf.getDataStoreCache();
+      if (dsc != null) {
+        String objTypes = MetastoreConf.getVar(conf, ConfVars.CACHE_PINOBJTYPES);
+        LOG.info("Setting MetaStore object pin classes with hive.metastore.cache.pinobjtypes=\"{}\"", objTypes);
+        if (org.apache.commons.lang.StringUtils.isNotEmpty(objTypes)) {
+          String[] typeTokens = objTypes.toLowerCase().split(",");
+          for (String type : typeTokens) {
+            type = type.trim();
+            if (PINCLASSMAP.containsKey(type)) {
+              dsc.pinAll(true, PINCLASSMAP.get(type));
+            } else {
+              LOG.warn("{} is not one of the pinnable object types: {}", type,
+                org.apache.commons.lang.StringUtils.join(PINCLASSMAP.keySet(), " "));
+            }
+          }
+        }
+      } else {
+        LOG.warn("PersistenceManagerFactory returned null DataStoreCache object. Unable to initialize object pin types defined by hive.metastore.cache.pinobjtypes");
+      }
+    }
+    return pmf;
+  }
+
+  @InterfaceAudience.LimitedPrivate({"HCATALOG"})
+  @InterfaceStability.Evolving
+  public PersistenceManager getPersistenceManager() {
+    return getPMF().getPersistenceManager();
+  }
+
+  @Override
+  public void shutdown() {
+    LOG.info("RawStore: {}, with PersistenceManager: {} will be shutdown", this, pm);
+    if (pm != null) {
+      pm.close();
+      pm = null;
+    }
+  }
+
+  /**
+   * Opens a new one or the one already created Every call of this function must
+   * have corresponding commit or rollback function call
+   *
+   * @return an active transaction
+   */
+
+  @Override
+  public boolean openTransaction() {
+    openTrasactionCalls++;
+    if (openTrasactionCalls == 1) {
+      currentTransaction = pm.currentTransaction();
+      currentTransaction.begin();
+      transactionStatus = TXN_STATUS.OPEN;
+    } else {
+      // openTransactionCalls > 1 means this is an interior transaction
+      // We should already have a transaction created that is active.
+      if ((currentTransaction == null) || (!currentTransaction.isActive())){
+        throw new RuntimeException("openTransaction called in an interior"
+            + " transaction scope, but currentTransaction is not active.");
+      }
+    }
+
+    boolean result = currentTransaction.isActive();
+    debugLog("Open transaction: count = " + openTrasactionCalls + ", isActive = " + result);
+    return result;
+  }
+
+  /**
+   * if this is the commit of the first open call then an actual commit is
+   * called.
+   *
+   * @return Always returns true
+   */
+  @Override
+  @SuppressWarnings("nls")
+  public boolean commitTransaction() {
+    if (TXN_STATUS.ROLLBACK == transactionStatus) {
+      debugLog("Commit transaction: rollback");
+      return false;
+    }
+    if (openTrasactionCalls <= 0) {
+      RuntimeException e = new RuntimeException("commitTransaction was called but openTransactionCalls = "
+          + openTrasactionCalls + ". This probably indicates that there are unbalanced " +
+          "calls to openTransaction/commitTransaction");
+      LOG.error("Unbalanced calls to open/commit Transaction", e);
+      throw e;
+    }
+    if (!currentTransaction.isActive()) {
+      RuntimeException e = new RuntimeException("commitTransaction was called but openTransactionCalls = "
+          + openTrasactionCalls + ". This probably indicates that there are unbalanced " +
+          "calls to openTransaction/commitTransaction");
+      LOG.error("Unbalanced calls to open/commit Transaction", e);
+      throw e;
+    }
+    openTrasactionCalls--;
+    debugLog("Commit transaction: count = " + openTrasactionCalls + ", isactive "+ currentTransaction.isActive());
+
+    if ((openTrasactionCalls == 0) && currentTransaction.isActive()) {
+      transactionStatus = TXN_STATUS.COMMITED;
+      currentTransaction.commit();
+    }
+    return true;
+  }
+
+  /**
+   * @return true if there is an active transaction. If the current transaction
+   *         is either committed or rolled back it returns false
+   */
+  @Override
+  public boolean isActiveTransaction() {
+    if (currentTransaction == null) {
+      return false;
+    }
+    return currentTransaction.isActive();
+  }
+
+  /**
+   * Rolls back the current transaction if it is active
+   */
+  @Override
+  public void rollbackTransaction() {
+    if (openTrasactionCalls < 1) {
+      debugLog("rolling back transaction: no open transactions: " + openTrasactionCalls);
+      return;
+    }
+    debugLog("Rollback transaction, isActive: " + currentTransaction.isActive());
+    try {
+      if (currentTransaction.isActive()
+          && transactionStatus != TXN_STATUS.ROLLBACK) {
+        currentTransaction.rollback();
+      }
+    } finally {
+      openTrasactionCalls = 0;
+      transactionStatus = TXN_STATUS.ROLLBACK;
+      // remove all detached objects from the cache, since the transaction is
+      // being rolled back they are no longer relevant, and this prevents them
+      // from reattaching in future transactions
+      pm.evictAll();
+    }
+  }
+
+  @Override
+  public void createCatalog(Catalog cat) throws MetaException {
+    LOG.debug("Creating catalog " + cat.getName());
+    boolean committed = false;
+    MCatalog mCat = catToMCat(cat);
+    try {
+      openTransaction();
+      pm.makePersistent(mCat);
+      committed = commitTransaction();
+    } finally {
+      if (!committed) {
+        rollbackTransaction();
+      }
+    }
+  }
+
+  @Override
+  public void alterCatalog(String catName, Catalog cat)
+      throws MetaException, InvalidOperationException {
+    if (!cat.getName().equals(catName)) {
+      throw new InvalidOperationException("You cannot change a catalog's name");
+    }
+    boolean committed = false;
+    try {
+      MCatalog mCat = getMCatalog(catName);
+      if (org.apache.commons.lang.StringUtils.isNotBlank(cat.getLocationUri())) {
+        mCat.setLocationUri(cat.getLocationUri());
+      }
+      if (org.apache.commons.lang.StringUtils.isNotBlank(cat.getDescription())) {
+        mCat.setDescription(cat.getDescription());
+      }
+      openTransaction();
+      pm.makePersistent(mCat);
+      committed = commitTransaction();
+    } finally {
+      if (!committed) {
+        rollbackTransaction();
+      }
+    }
+  }
+
+  @Override
+  public Catalog getCatalog(String catalogName) throws NoSuchObjectException, MetaException {
+    LOG.debug("Fetching catalog " + catalogName);
+    MCatalog mCat = getMCatalog(catalogName);
+    if (mCat == null) {
+      throw new NoSuchObjectException("No catalog " + catalogName);
+    }
+    return mCatToCat(mCat);
+  }
+
+  @Override
+  public List<String> getCatalogs() throws MetaException {
+    LOG.debug("Fetching all catalog names");
+    boolean commited = false;
+    List<String> catalogs = null;
+
+    String queryStr = "select name from org.apache.hadoop.hive.metastore.model.MCatalog";
+    Query query = null;
+
+    openTransaction();
+    try {
+      query = pm.newQuery(queryStr);
+      query.setResult("name");
+      catalogs = new ArrayList<>((Collection<String>) query.execute());
+      commited = commitTransaction();
+    } finally {
+      rollbackAndCleanup(commited, query);
+    }
+    Collections.sort(catalogs);
+    return catalogs;
+  }
+
+  @Override
+  public void dropCatalog(String catalogName) throws NoSuchObjectException, MetaException {
+    LOG.debug("Dropping catalog " + catalogName);
+    boolean committed = false;
+    try {
+      openTransaction();
+      MCatalog mCat = getMCatalog(catalogName);
+      pm.retrieve(mCat);
+      if (mCat == null) {
+        throw new NoSuchObjectException("No catalog " + catalogName);
+      }
+      pm.deletePersistent(mCat);
+      committed = commitTransaction();
+    } finally {
+      if (!committed) {
+        rollbackTransaction();
+      }
+    }
+  }
+
+  private MCatalog getMCatalog(String catalogName) throws MetaException {
+    boolean committed = false;
+    Query query = null;
+    try {
+      openTransaction();
+      catalogName = normalizeIdentifier(catalogName);
+      query = pm.newQuery(MCatalog.class, "name == catname");
+      query.declareParameters("java.lang.String catname");
+      query.setUnique(true);
+      MCatalog mCat = (MCatalog)query.execute(catalogName);
+      pm.retrieve(mCat);
+      committed = commitTransaction();
+      return mCat;
+    } finally {
+      rollbackAndCleanup(committed, query);
+    }
+  }
+
+  private MCatalog catToMCat(Catalog cat) {
+    MCatalog mCat = new MCatalog();
+    mCat.setName(normalizeIdentifier(cat.getName()));
+    if (cat.isSetDescription()) {
+      mCat.setDescription(cat.getDescription());
+    }
+    mCat.setLocationUri(cat.getLocationUri());
+    return mCat;
+  }
+
+  private Catalog mCatToCat(MCatalog mCat) {
+    Catalog cat = new Catalog(mCat.getName(), mCat.getLocationUri());
+    if (mCat.getDescription() != null) {
+      cat.setDescription(mCat.getDescription());
+    }
+    return cat;
+  }
+
+  @Override
+  public void createDatabase(Database db) throws InvalidObjectException, MetaException {
+    boolean commited = false;
+    MDatabase mdb = new MDatabase();
+    assert db.getCatalogName() != null;
+    mdb.setCatalogName(normalizeIdentifier(db.getCatalogName()));
+    assert mdb.getCatalogName() != null;
+    mdb.setName(db.getName().toLowerCase());
+    mdb.setLocationUri(db.getLocationUri());
+    mdb.setDescription(db.getDescription());
+    mdb.setParameters(db.getParameters());
+    mdb.setOwnerName(db.getOwnerName());
+    PrincipalType ownerType = db.getOwnerType();
+    mdb.setOwnerType((null == ownerType ? PrincipalType.USER.name() : ownerType.name()));
+    try {
+      openTransaction();
+      pm.makePersistent(mdb);
+      commited = commitTransaction();
+    } finally {
+      if (!commited) {
+        rollbackTransaction();
+      }
+    }
+  }
+
+  @SuppressWarnings("nls")
+  private MDatabase getMDatabase(String catName, String name) throws NoSuchObjectException {
+    MDatabase mdb = null;
+    boolean commited = false;
+    Query query = null;
+    try {
+      openTransaction();
+      name = normalizeIdentifier(name);
+      catName = normalizeIdentifier(catName);
+      query = pm.newQuery(MDatabase.class, "name == dbname && catalogName == catname");
+      query.declareParameters("java.lang.String dbname, java.lang.String catname");
+      query.setUnique(true);
+      mdb = (MDatabase) query.execute(name, catName);
+      pm.retrieve(mdb);
+      commited = commitTransaction();
+    } finally {
+      rollbackAndCleanup(commited, query);
+    }
+    if (mdb == null) {
+      throw new NoSuchObjectException("There is no database " + catName + "." + name);
+    }
+    return mdb;
+  }
+
+  @Override
+  public Database getDatabase(String catalogName, String name) throws NoSuchObjectException {
+    MetaException ex = null;
+    Database db = null;
+    try {
+      db = getDatabaseInternal(catalogName, name);
+    } catch (MetaException e) {
+      // Signature restriction to NSOE, and NSOE being a flat exception prevents us from
+      // setting the cause of the NSOE as the MetaException. We should not lose the info
+      // we got here, but it's very likely that the MetaException is irrelevant and is
+      // actually an NSOE message, so we should log it and throw an NSOE with the msg.
+      ex = e;
+    }
+    if (db == null) {
+      LOG.warn("Failed to get database {}.{}, returning NoSuchObjectException",
+          catalogName, name, ex);
+      throw new NoSuchObjectException(name + (ex == null ? "" : (": " + ex.getMessage())));
+    }
+    return db;
+  }
+
+  public Database getDatabaseInternal(String catalogName, String name)
+      throws MetaException, NoSuchObjectException {
+    return new GetDbHelper(catalogName, name, true, true) {
+      @Override
+      protected Database getSqlResult(GetHelper<Database> ctx) throws MetaException {
+        return directSql.getDatabase(catalogName, dbName);
+      }
+
+      @Override
+      protected Database getJdoResult(GetHelper<Database> ctx) throws MetaException, NoSuchObjectException {
+        return getJDODatabase(catalogName, dbName);
+      }
+    }.run(false);
+   }
+
+  public Database getJDODatabase(String catName, String name) throws NoSuchObjectException {
+    MDatabase mdb = null;
+    boolean commited = false;
+    try {
+      openTransaction();
+      mdb = getMDatabase(catName, name);
+      commited = commitTransaction();
+    } finally {
+      if (!commited) {
+        rollbackTransaction();
+      }
+    }
+    Database db = new Database();
+    db.setName(mdb.getName());
+    db.setDescription(mdb.getDescription());
+    db.setLocationUri(mdb.getLocationUri());
+    db.setParameters(convertMap(mdb.getParameters()));
+    db.setOwnerName(mdb.getOwnerName());
+    String type = org.apache.commons.lang.StringUtils.defaultIfBlank(mdb.getOwnerType(), null);
+    PrincipalType principalType = (type == null) ? null : PrincipalType.valueOf(type);
+    db.setOwnerType(principalType);
+    db.setCatalogName(catName);
+    return db;
+  }
+
+  /**
+   * Alter the database object in metastore. Currently only the parameters
+   * of the database or the owner can be changed.
+   * @param dbName the database name
+   * @param db the Hive Database object
+   * @throws MetaException
+   * @throws NoSuchObjectException
+   */
+  @Override
+  public boolean alterDatabase(String catName, String dbName, Database db)
+    throws MetaException, NoSuchObjectException {
+
+    MDatabase mdb = null;
+    boolean committed = false;
+    try {
+      mdb = getMDatabase(catName, dbName);
+      mdb.setParameters(db.getParameters());
+      mdb.setOwnerName(db.getOwnerName());
+      if (db.getOwnerType() != null) {
+        mdb.setOwnerType(db.getOwnerType().name());
+      }
+      if (org.apache.commons.lang.StringUtils.isNotBlank(db.getDescription())) {
+        mdb.setDescription(db.getDescription());
+      }
+      if (org.apache.commons.lang.StringUtils.isNotBlank(db.getLocationUri())) {
+        mdb.setLocationUri(db.getLocationUri());
+      }
+      openTransaction();
+      pm.makePersistent(mdb);
+      committed = commitTransaction();
+    } finally {
+      if (!committed) {
+        rollbackTransaction();
+        return false;
+      }
+    }
+    return true;
+  }
+
+  @Override
+  public boolean dropDatabase(String catName, String dbname)
+      throws NoSuchObjectException, MetaException {
+    boolean success = false;
+    LOG.info("Dropping database {}.{} along with all tables", catName, dbname);
+    dbname = normalizeIdentifier(dbname);
+    catName = normalizeIdentifier(catName);
+    QueryWrapper queryWrapper = new QueryWrapper();
+    try {
+      openTransaction();
+
+      // then drop the database
+      MDatabase db = getMDatabase(catName, dbname);
+      pm.retrieve(db);
+      if (db != null) {
+        List<MDBPrivilege> dbGrants = this.listDatabaseGrants(catName, dbname, null, queryWrapper);
+        if (CollectionUtils.isNotEmpty(dbGrants)) {
+          pm.deletePersistentAll(dbGrants);
+        }
+        pm.deletePersistent(db);
+      }
+      success = commitTransaction();
+    } finally {
+      rollbackAndCleanup(success, queryWrapper);
+    }
+    return success;
+  }
+
+  @Override
+  public List<String> getDatabases(String catName, String pattern) throws MetaException {
+    if (pattern == null || pattern.equals("*")) {
+      return getAllDatabases(catName);
+    }
+    boolean commited = false;
+    List<String> databases = null;
+    Query query = null;
+    try {
+      openTransaction();
+      // Take the pattern and split it on the | to get all the composing
+      // patterns
+      String[] subpatterns = pattern.trim().split("\\|");
+      StringBuilder filterBuilder = new StringBuilder();
+      List<String> parameterVals = new ArrayList<>(subpatterns.length);
+      appendSimpleCondition(filterBuilder, "catalogName", new String[] {catName}, parameterVals);
+      appendPatternCondition(filterBuilder, "name", subpatterns, parameterVals);
+      query = pm.newQuery(MDatabase.class, filterBuilder.toString());
+      query.setResult("name");
+      query.setOrdering("name ascending");
+      Collection<String> names = (Collection<String>) query.executeWithArray(parameterVals.toArray(new String[0]));
+      databases = new ArrayList<>(names);
+      commited = commitTransaction();
+    } finally {
+      rollbackAndCleanup(commited, query);
+    }
+    return databases;
+  }
+
+  @Override
+  public List<String> getAllDatabases(String catName) throws MetaException {
+    boolean commited = false;
+    List<String> databases = null;
+
+    Query query = null;
+    catName = normalizeIdentifier(catName);
+
+    openTransaction();
+    try {
+      query = pm.newQuery("select name from org.apache.hadoop.hive.metastore.model.MDatabase " +
+          "where catalogName == catname");
+      query.declareParameters("java.lang.String catname");
+      query.setResult("name");
+      databases = new ArrayList<>((Collection<String>) query.execute(catName));
+      commited = commitTransaction();
+    } finally {
+      rollbackAndCleanup(commited, query);
+    }
+    Collections.sort(databases);
+    return databases;
+  }
+
+  private MType getMType(Type type) {
+    List<MFieldSchema> fields = new ArrayList<>();
+    if (type.getFields() != null) {
+      for (FieldSchema field : type.getFields()) {
+        fields.add(new MFieldSchema(field.getName(), field.getType(), field
+            .getComment()));
+      }
+    }
+    return new MType(type.getName(), type.getType1(), type.getType2(), fields);
+  }
+
+  private Type getType(MType mtype) {
+    List<FieldSchema> fields = new ArrayList<>();
+    if (mtype.getFields() != null) {
+      for (MFieldSchema field : mtype.getFields()) {
+        fields.add(new FieldSchema(field.getName(), field.getType(), field
+            .getComment()));
+      }
+    }
+    Type ret = new Type();
+    ret.setName(mtype.getName());
+    ret.setType1(mtype.getType1());
+    ret.setType2(mtype.getType2());
+    ret.setFields(fields);
+    return ret;
+  }
+
+  @Override
+  public boolean createType(Type type) {
+    boolean success = false;
+    MType mtype = getMType(type);
+    boolean commited = false;
+    try {
+      openTransaction();
+      pm.makePersistent(mtype);
+      commited = commitTransaction();
+      success = true;
+    } finally {
+      if (!commited) {
+        rollbackTransaction();
+      }
+    }
+    return success;
+  }
+
+  @Override
+  public Type getType(String typeName) {
+    Type type = null;
+    boolean commited = false;
+    Query query = null;
+    try {
+      openTransaction();
+      query = pm.newQuery(MType.class, "name == typeName");
+      query.declareParameters("java.lang.String typeName");
+      query.setUnique(true);
+      MType mtype = (MType) query.execute(typeName.trim());
+      pm.retrieve(type);
+      if (mtype != null) {
+        type = getType(mtype);
+      }
+      commited = commitTransaction();
+    } finally {
+      rollbackAndCleanup(commited, query);
+    }
+    return type;
+  }
+
+  @Override
+  public boolean dropType(String typeName) {
+    boolean success = false;
+    Query query = null;
+    try {
+      openTransaction();
+      query = pm.newQuery(MType.class, "name == typeName");
+      query.declareParameters("java.lang.String typeName");
+      query.setUnique(true);
+      MType type = (MType) query.execute(typeName.trim());
+      pm.retrieve(type);
+      if (type != null) {
+        pm.deletePersistent(type);
+      }
+      success = commitTransaction();
+    } catch (JDOObjectNotFoundException e) {
+      success = commitTransaction();
+      LOG.debug("type not found {}", typeName, e);
+    } finally {
+      rollbackAndCleanup(success, query);
+    }
+    return success;
+  }
+
+  @Override
+  public List<String> createTableWithConstraints(Table tbl,
+    List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys,
+    List<SQLUniqueConstraint> uniqueConstraints, List<SQLNotNullConstraint> notNullConstraints,
+    List<SQLDefaultConstraint> defaultConstraints, List<SQLCheckConstraint> checkConstraints)
+    throws InvalidObjectException, MetaException {
+    boolean success = false;
+    try {
+      openTransaction();
+      createTable(tbl);
+      // Add constraints.
+      // We need not do a deep retrieval of the Table Column Descriptor while persisting the
+      // constraints since this transaction involving create table is not yet committed.
+      List<String> constraintNames = new ArrayList<>();
+      if (foreignKeys != null) {
+        constraintNames.addAll(addForeignKeys(foreignKeys, false, primaryKeys, uniqueConstraints));
+      }
+      if (primaryKeys != null) {
+        constraintNames.addAll(addPrimaryKeys(primaryKeys, false));
+      }
+      if (uniqueConstraints != null) {
+        constraintNames.addAll(addUniqueConstraints(uniqueConstraints, false));
+      }
+      if (notNullConstraints != null) {
+        constraintNames.addAll(addNotNullConstraints(notNullConstraints, false));
+      }
+      if (defaultConstraints != null) {
+        constraintNames.addAll(addDefaultConstraints(defaultConstraints, false));
+      }
+      if (checkConstraints != null) {
+        constraintNames.addAll(addCheckConstraints(checkConstraints, false));
+      }
+      success = commitTransaction();
+      return constraintNames;
+    } finally {
+      if (!success) {
+        rollbackTransaction();
+      }
+    }
+  }
+
+  @Override
+  public void createTable(Table tbl) throws InvalidObjectException, MetaException {
+    boolean commited = false;
+    MTable mtbl = null;
+
+    try {
+      openTransaction();
+
+      mtbl = convertToMTable(tbl);
+      if (TxnUtils.isTransactionalTable(tbl)) {
+        mtbl.setWriteId(tbl.getWriteId());
+      }
+      pm.makePersistent(mtbl);
+
+      if (tbl.getCreationMetadata() != null) {
+        MCreationMetadata mcm = convertToMCreationMetadata(tbl.getCreationMetadata());
+        pm.makePersistent(mcm);
+      }
+      tbl.setId(mtbl.getId());
+
+      PrincipalPrivilegeSet principalPrivs = tbl.getPrivileges();
+      List<Object> toPersistPrivObjs = new ArrayList<>();
+      if (principalPrivs != null) {
+        int now = (int)(System.currentTimeMillis()/1000);
+
+        Map<String, List<PrivilegeGrantInfo>> userPrivs = principalPrivs.getUserPrivileges();
+        putPersistentPrivObjects(mtbl, toPersistPrivObjs, now, userPrivs, PrincipalType.USER, "SQL");
+
+        Map<String, List<PrivilegeGrantInfo>> groupPrivs = principalPrivs.getGroupPrivileges();
+        putPersistentPrivObjects(mtbl, toPersistPrivObjs, now, groupPrivs, PrincipalType.GROUP, "SQL");
+
+        Map<String, List<PrivilegeGrantInfo>> rolePrivs = principalPrivs.getRolePrivileges();
+        putPersistentPrivObjects(mtbl, toPersistPrivObjs, now, rolePrivs, PrincipalType.ROLE, "SQL");
+      }
+      pm.makePersistentAll(toPersistPrivObjs);
+      commited = commitTransaction();
+    } finally {
+      if (!commited) {
+        rollbackTransaction();
+      }
+    }
+  }
+
+  /**
+   * Convert PrivilegeGrantInfo from privMap to MTablePrivilege, and add all of
+   * them to the toPersistPrivObjs. These privilege objects will be persisted as
+   * part of createTable.
+   *
+   * @param mtbl
+   * @param toPersistPrivObjs
+   * @param now
+   * @param privMap
+   * @param type
+   */
+  private void putPersistentPrivObjects(MTable mtbl, List<Object> toPersistPrivObjs,
+      int now, Map<String, List<PrivilegeGrantInfo>> privMap, PrincipalType type, String authorizer) {
+    if (privMap != null) {
+      for (Map.Entry<String, List<PrivilegeGrantInfo>> entry : privMap
+          .entrySet()) {
+        String principalName = entry.getKey();
+        List<PrivilegeGrantInfo> privs = entry.getValue();
+        for (int i = 0; i < privs.size(); i++) {
+          PrivilegeGrantInfo priv = privs.get(i);
+          if (priv == null) {
+            continue;
+          }
+          MTablePrivilege mTblSec = new MTablePrivilege(
+              principalName, type.toString(), mtbl, priv.getPrivilege(),
+              now, priv.getGrantor(), priv.getGrantorType().toString(), priv
+                  .isGrantOption(), authorizer);
+          toPersistPrivObjs.add(mTblSec);
+        }
+      }
+    }
+  }
+
+  @Override
+  public boolean dropTable(String catName, String dbName, String tableName)
+      throws MetaException, NoSuchObjectException, InvalidObjectException, InvalidInputException {
+    boolean materializedView = false;
+    boolean success = false;
+    try {
+      openTransaction();
+      MTable tbl = getMTable(catName, dbName, tableName);
+      pm.retrieve(tbl);
+      if (tbl != null) {
+        materializedView = TableType.MATERIALIZED_VIEW.toString().equals(tbl.getTableType());
+        // first remove all the grants
+        List<MTablePrivilege> tabGrants = listAllTableGrants(catName, dbName, tableName);
+        if (CollectionUtils.isNotEmpty(tabGrants)) {
+          pm.deletePersistentAll(tabGrants);
+        }
+        List<MTableColumnPrivilege> tblColGrants = listTableAllColumnGrants(catName, dbName,
+            tableName);
+        if (CollectionUtils.isNotEmpty(tblColGrants)) {
+          pm.deletePersistentAll(tblColGrants);
+        }
+
+        List<MPartitionPrivilege> partGrants = this.listTableAllPartitionGrants(catName, dbName, tableName);
+        if (CollectionUtils.isNotEmpty(partGrants)) {
+          pm.deletePersistentAll(partGrants);
+        }
+
+        List<MPartitionColumnPrivilege> partColGrants = listTableAllPartitionColumnGrants(catName, dbName,
+            tableName);
+        if (CollectionUtils.isNotEmpty(partColGrants)) {
+          pm.deletePersistentAll(partColGrants);
+        }
+        // delete column statistics if present
+        try {
+          deleteTableColumnStatistics(catName, dbName, tableName, null);
+        } catch (NoSuchObjectException e) {
+          LOG.info("Found no table level column statistics associated with {} to delete",
+              TableName.getQualified(catName, dbName, tableName));
+        }
+
+        List<MConstraint> tabConstraints = listAllTableConstraintsWithOptionalConstraintName(
+                                           catName, dbName, tableName, null);
+        if (CollectionUtils.isNotEmpty(tabConstraints)) {
+          pm.deletePersistentAll(tabConstraints);
+        }
+
+        preDropStorageDescriptor(tbl.getSd());
+
+        if (materializedView) {
+          dropCreationMetadata(tbl.getDatabase().getCatalogName(),
+              tbl.getDatabase().getName(), tbl.getTableName());
+        }
+
+        // then remove the table
+        pm.deletePersistentAll(tbl);
+      }
+      success = commitTransaction();
+    } finally {
+      if (!success) {
+        rollbackTransaction();
+      }
+    }
+    return success;
+  }
+
+  private boolean dropCreationMetadata(String catName, String dbName, String tableName) throws MetaException,
+      NoSuchObjectException, InvalidObjectException, InvalidInputException {
+    boolean success = false;
+    try {
+      openTransaction();
+      MCreationMetadata mcm = getCreationMetadata(catName, dbName, tableName);
+      pm.retrieve(mcm);
+      if (mcm != null) {
+        pm.deletePersistentAll(mcm);
+      }
+      success = commitTransaction();
+    } finally {
+      if (!success) {
+        rollbackTransaction();
+      }
+    }
+    return success;
+  }
+
+  private List<MConstraint> listAllTableConstraintsWithOptionalConstraintName(
+      String catName, String dbName, String tableName, String constraintname) {
+    catName = normalizeIdentifier(catName);
+    dbName = normalizeIdentifier(dbName);
+    tableName = normalizeIdentifier(tableName);
+    constraintname = constraintname!=null?normalizeIdentifier(constraintname):null;
+    List<MConstraint> mConstraints = null;
+    List<String> constraintNames = new ArrayList<>();
+    Query query = null;
+
+    try {
+      query = pm.newQuery("select constraintName from org.apache.hadoop.hive.metastore.model.MConstraint  where "
+        + "((parentTable.tableName == ptblname && parentTable.database.name == pdbname && " +
+              "parentTable.database.catalogName == pcatname) || "
+        + "(childTable != null && childTable.tableName == ctblname &&" +
+              "childTable.database.name == cdbname && childTable.database.catalogName == ccatname)) " +
+          (constraintname != null ? " && constraintName == constraintname" : ""));
+      query.declareParameters("java.lang.String ptblname, java.lang.String pdbname,"
+          + "java.lang.String pcatname, java.lang.String ctblname, java.lang.String cdbname," +
+          "java.lang.String ccatname" +
+        (constraintname != null ? ", java.lang.String constraintname" : ""));
+      Collection<?> constraintNamesColl =
+        constraintname != null ?
+          ((Collection<?>) query.
+            executeWithArray(tableName, dbName, catName, tableName, dbName, catName, constraintname)):
+          ((Collection<?>) query.
+            executeWithArray(tableName, dbName, catName, tableName, dbName, catName));
+      for (Iterator<?> i = constraintNamesColl.iterator(); i.hasNext();) {
+        String currName = (String) i.next();
+        constraintNames.add(currName);
+      }
+      query = pm.newQuery(MConstraint.class);
+      query.setFilter("param.contains(constraintName)");
+      query.declareParameters("java.util.Collection param");
+      Collection<?> constraints = (Collection<?>)query.execute(constraintNames);
+      mConstraints = new ArrayList<>();
+      for (Iterator<?> i = constraints.iterator(); i.hasNext();) {
+        MConstraint currConstraint = (MConstraint) i.next();
+        mConstraints.add(currConstraint);
+      }
+    } finally {
+      if (query != null) {
+        query.closeAll();
+      }
+    }
+    return mConstraints;
+  }
+
+  private static String getFullyQualifiedTableName(String dbName, String tblName) {
+    return ((dbName == null || dbName.isEmpty()) ? "" : "\"" + dbName + "\".\"")
+        + "\"" + tblName + "\"";
+  }
+
+  @Override
+  public Table
+  getTable(String catName, String dbName, String tableName)
+      throws MetaException {
+    return getTable(catName, dbName, tableName, null);
+  }
+
+  @Override
+  public Table getTable(String catName, String dbName, String tableName,
+                        String writeIdList)
+      throws MetaException {
+    boolean commited = false;
+    Table tbl = null;
+    try {
+      openTransaction();
+      MTable mtable = getMTable(catName, dbName, tableName);
+      tbl = convertToTable(mtable);
+      // Retrieve creation metadata if needed
+      if (tbl != null && TableType.MATERIALIZED_VIEW.toString().equals(tbl.getTableType())) {
+        tbl.setCreationMetadata(
+                convertToCreationMetadata(getCreationMetadata(catName, dbName, tableName)));
+      }
+
+      // If transactional non partitioned table,
+      // check whether the current version table statistics
+      // in the metastore comply with the client query's snapshot isolation.
+      // Note: a partitioned table has table stats and table snapshot in MPartiiton.
+      if (writeIdList != null) {
+        boolean isTxn = tbl != null && TxnUtils.isTransactionalTable(tbl);
+        if (isTxn && !areTxnStatsSupported) {
+          StatsSetupConst.setBasicStatsState(tbl.getParameters(), StatsSetupConst.FALSE);
+          LOG.info("Removed COLUMN_STATS_ACCURATE from Table's parameters.");
+        } else if (isTxn && tbl.getPartitionKeysSize() == 0) {
+          if (isCurrentStatsValidForTheQuery(mtable, writeIdList, false)) {
+            tbl.setIsStatsCompliant(true);
+          } else {
+            tbl.setIsStatsCompliant(false);
+            // Do not make persistent the following state since it is the query specific (not global).
+            StatsSetupConst.setBasicStatsState(tbl.getParameters(), StatsSetupConst.FALSE);
+            LOG.info("Removed COLUMN_STATS_ACCURATE from Table's parameters.");
+          }
+        }
+      }
+      commited = commitTransaction();
+    } finally {
+      if (!commited) {
+        rollbackTransaction();
+      }
+    }
+    return tbl;
+  }
+
+  @Override
+  public List<String> getTables(String catName, String dbName, String pattern)
+      throws MetaException {
+    return getTables(catName, dbName, pattern, null);
+  }
+
+  @Override
+  public List<String> getTables(String catName, String dbName, String pattern, TableType tableType)
+      throws MetaException {
+    try {
+      // We only support pattern matching via jdo since pattern matching in Java
+      // might be different than the one used by the metastore backends
+      return getTablesInternal(catName, dbName, pattern, tableType,
+          (pattern == null || pattern.equals(".*")), true);
+    } catch (NoSuchObjectException e) {
+      throw new MetaException(ExceptionUtils.getStackTrace(e));
+    }
+  }
+
+  @Override
+  public List<TableName> getTableNamesWithStats() throws MetaException, NoSuchObjectException {
+    return new GetListHelper<TableName>(null, null, null, true, false) {
+      @Override
+      protected List<TableName> getSqlResult(
+          GetHelper<List<TableName>> ctx) throws MetaException {
+        return directSql.getTableNamesWithStats();
+      }
+
+      @Override
+      protected List<TableName> getJdoResult(
+          GetHelper<List<TableName>> ctx) throws MetaException {
+        throw new UnsupportedOperationException("UnsupportedOperationException"); // TODO: implement?
+      }
+    }.run(false);
+  }
+
+  @Override
+  public Map<String, List<String>> getPartitionColsWithStats(String catName, String dbName, String tableName)
+      throws MetaException, NoSuchObjectException {
+    return new GetHelper<Map<String, List<String>>>(catName, dbName, null, true, false) {
+      @Override
+      protected Map<String, List<String>> getSqlResult(
+          GetHelper<Map<String, List<String>>> ctx) throws MetaException {
+        try {
+          return directSql.getColAndPartNamesWithStats(catName, dbName, tableName);
+        } catch (Throwable ex) {
+          LOG.error("DirectSQL failed", ex);
+          throw new MetaException(ex.getMessage());
+        }
+      }
+
+      @Override
+      protected Map<String, List<String>> getJdoResult(
+          GetHelper<Map<String, List<String>>> ctx) throws MetaException {
+        throw new UnsupportedOperationException("UnsupportedOperationException"); // TODO: implement?
+      }
+
+      @Override
+      protected String describeResult() {
+        return results.size() + " partitions";
+      }
+    }.run(false);
+  }
+
+  @Override
+  public List<TableName> getAllTableNamesForStats() throws MetaException, NoSuchObjectException {
+    return new GetListHelper<TableName>(null, null, null, true, false) {
+      @Override
+      protected List<TableName> getSqlResult(
+          GetHelper<List<TableName>> ctx) throws MetaException {
+        return directSql.getAllTableNamesForStats();
+      }
+
+      @Override
+      protected List<TableName> getJdoResult(
+          GetHelper<List<TableName>> ctx) throws MetaException {
+        boolean commited = false;
+        Query query = null;
+        List<TableName> result = new ArrayList<>();
+        openTransaction();
+        try {
+          String paramStr = "", whereStr = "";
+          for (int i = 0; i < MetaStoreDirectSql.STATS_TABLE_TYPES.length; ++i) {
+            if (i != 0) {
+              paramStr += ", ";
+              whereStr += "||";
+            }
+            paramStr += "java.lang.String tt" + i;
+            whereStr += " tableType == tt" + i;
+          }
+          query = pm.newQuery(MTable.class, whereStr);
+          query.declareParameters(paramStr);
+          @SuppressWarnings("unchecked")
+          Collection<MTable> tbls = (Collection<MTable>) query.executeWithArray(
+              query, MetaStoreDirectSql.STATS_TABLE_TYPES);
+          pm.retrieveAll(tbls);
+          for (MTable tbl : tbls) {
+            result.add(new TableName(
+                tbl.getDatabase().getCatalogName(), tbl.getDatabase().getName(), tbl.getTableName()));
+          }
+          commited = commitTransaction();
+        } finally {
+          rollbackAndCleanup(commited, query);
+        }
+        return result;
+      }
+    }.run(false);
+  }
+
+  protected List<String> getTablesInternal(String catName, String dbName, String pattern,
+                                           TableType tableType, boolean allowSql, boolean allowJdo)
+      throws MetaException, NoSuchObjectException {
+    final String db_name = normalizeIdentifier(dbName);
+    final String cat_name = normalizeIdentifier(catName);
+    return new GetListHelper<String>(cat_name, dbName, null, allowSql, allowJdo) {
+      @Override
+      protected List<String> getSqlResult(GetHelper<List<String>> ctx)
+              throws MetaException {
+        return directSql.getTables(cat_name, db_name, tableType);
+      }
+
+      @Override
+      protected List<String> getJdoResult(GetHelper<List<String>> ctx)
+              throws MetaException, NoSuchObjectException {
+        return getTablesInternalViaJdo(cat_name, db_name, pattern, tableType);
+      }
+    }.run(false);
+  }
+
+  private List<String> getTablesInternalViaJdo(String catName, String dbName, String pattern,
+                                               TableType tableType) throws MetaException {
+    boolean commited = false;
+    Query query = null;
+    List<String> tbls = null;
+    try {
+      openTransaction();
+      dbName = normalizeIdentifier(dbName);
+      // Take the pattern and split it on the | to get all the composing
+      // patterns
+      List<String> parameterVals = new ArrayList<>();
+      StringBuilder filterBuilder = new StringBuilder();
+      //adds database.name == dbName to the filter
+      appendSimpleCondition(filterBuilder, "database.name", new String[] {dbName}, parameterVals);
+      appendSimpleCondition(filterBuilder, "database.catalogName", new String[] {catName}, parameterVals);
+      if(pattern != null) {
+        appendPatternCondition(filterBuilder, "tableName", pattern, parameterVals);
+      }
+      if(tableType != null) {
+        appendPatternCondition(filterBuilder, "tableType", new String[] {tableType.toString()}, parameterVals);
+      }
+
+      query = pm.newQuery(MTable.class, filterBuilder.toString());
+      query.setResult("tableName");
+      query.setOrdering("tableName ascending");
+      Collection<String> names = (Collection<String>) query.executeWithArray(parameterVals.toArray(new String[0]));
+      tbls = new ArrayList<>(names);
+      commited = commitTransaction();
+    } finally {
+      rollbackAndCleanup(commited, query);
+    }
+    return tbls;
+  }
+
+  @Override
+  public List<String> getMaterializedViewsForRewriting(String catName, String dbName)
+      throws MetaException, NoSuchObjectException {
+    final String db_name = normalizeIdentifier(dbName);
+    catName = normalizeIdentifier(catName);
+    boolean commited = false;
+    Query<?> query = null;
+    List<String> tbls = null;
+    try {
+      openTransaction();
+      dbName = normalizeIdentifier(dbName);
+      query = pm.newQuery(MTable.class,
+          "database.name == db && database.catalogName == cat && tableType == tt && rewriteEnabled == re");
+      query.declareParameters(
+          "java.lang.String db, java.lang.String cat, java.lang.String tt, boolean re");
+      query.setResult("tableName");
+      Collection<String> names = (Collection<String>) query.executeWithArray(
+          db_name, catName, TableType.MATERIALIZED_VIEW.toString(), true);
+      tbls = new ArrayList<>(names);
+      commited = commitTransaction();
+    } finally {
+      rollbackAndCleanup(commited, query);
+    }
+    return tbls;
+  }
+
+  @Override
+  public int getDatabaseCount() throws MetaException {
+    return getObjectCount("name", MDatabase.class.getName());
+  }
+
+  @Override
+  public int getPartitionCount() throws MetaException {
+    return getObjectCount("partitionName", MPartition.class.getName());
+  }
+
+  @Override
+  public int getTableCount() throws MetaException {
+    return getObjectCount("tableName", MTable.class.getName());
+  }
+
+  private int getObjectCount(String fieldName, String objName) {
+    Long result = 0L;
+    boolean commited = false;
+    Query query = null;
+    try {
+      openTransaction();
+      String queryStr =
+        "select count(" + fieldName + ") from " + objName;
+      query = pm.newQuery(queryStr);
+      result = (Long) query.execute();
+      commited = commitTransaction();
+    } finally {
+      rollbackAndCleanup(commited, query);
+    }
+    return result.intValue();
+  }
+
+  @Override
+  public List<TableMeta> getTableMeta(String catName, String dbNames, String tableNames,
+                                      List<String> tableTypes) throws MetaException {
+
+    boolean commited = false;
+    Query query = null;
+    List<TableMeta> metas = new ArrayList<>();
+    try {
+      openTransaction();
+      // Take the pattern and split it on the | to get all the composing
+      // patterns
+      StringBuilder filterBuilder = new StringBuilder();
+      List<String> parameterVals = new ArrayList<>();
+      appendSimpleCondition(filterBuilder, "database.catalogName", new String[] {catName}, parameterVals);
+      if (dbNames != null && !dbNames.equals("*")) {
+        appendPatternCondition(filterBuilder, "database.name", dbNames, parameterVals);
+      }
+      if (tableNames != null && !tableNames.equals("*")) {
+        appendPatternCondition(filterBuilder, "tableName", tableNames, parameterVals);
+      }
+      if (tableTypes != null && !tableTypes.isEmpty()) {
+        appendSimpleCondition(filterBuilder, "tableType", tableTypes.toArray(new String[0]), parameterVals);
+      }
+
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("getTableMeta with filter " + filterBuilder.toString() + " params: " +
+            StringUtils.join(parameterVals, ", "));
+      }
+      query = pm.newQuery(MTable.class, filterBuilder.toString());
+      Collection<MTable> tables = (Collection<MTable>) query.executeWithArray(parameterVals.toArray(new String[parameterVals.size()]));
+      for (MTable table : tables) {
+        TableMeta metaData = new TableMeta(
+            table.getDatabase().getName(), table.getTableName(), table.getTableType());
+        metaData.setComments(table.getParameters().get("comment"));
+        metas.add(metaData);
+      }
+      commited = commitTransaction();
+    } finally {
+      rollbackAndCleanup(commited, query);
+    }
+    return metas;
+  }
+
+  private StringBuilder appendPatternCondition(StringBuilder filterBuilder, String fieldName,
+      String[] elements, List<String> parameterVals) {
+    return appendCondition(filterBuilder, fieldName, elements, true, parameterVals);
+  }
+
+  private StringBuilder appendPatternCondition(StringBuilder builder,
+      String fieldName, String elements, List<String> parameters) {
+      elements = normalizeIdentifier(elements);
+    return appendCondition(builder, fieldName, elements.split("\\|"), true, parameters);
+  }
+
+  private StringBuilder appendSimpleCondition(StringBuilder builder,
+      String fieldName, String[] elements, List<String> parameters) {
+    return appendCondition(builder, fieldName, elements, false, parameters);
+  }
+
+  private StringBuilder appendCondition(StringBuilder builder,
+      String fieldName, String[] elements, boolean pattern, List<String> parameters) {
+    if (builder.length() > 0) {
+      builder.append(" && ");
+    }
+    builder.append(" (");
+    int length = builder.length();
+    for (String element : elements) {
+      if (pattern) {
+        element = "(?i)" + element.replaceAll("\\*", ".*");
+      }
+      parameters.add(element);
+      if (builder.length() > length) {
+        builder.append(" || ");
+      }
+      builder.append(fieldName);
+      if (pattern) {
+        builder.append(".matches(").append(JDO_PARAM).append(parameters.size()).append(")");
+      } else {
+        builder.append(" == ").append(JDO_PARAM).append(parameters.size());
+      }
+    }
+    builder.append(" )");
+    return builder;
+  }
+
+  @Override
+  public List<String> getAllTables(String catName, String dbName) throws MetaException {
+    return getTables(catName, dbName, ".*");
+  }
+
+  class AttachedMTableInfo {
+    MTable mtbl;
+    MColumnDescriptor mcd;
+
+    public AttachedMTableInfo() {}
+
+    public AttachedMTableInfo(MTable mtbl, MColumnDescriptor mcd) {
+      this.mtbl = mtbl;
+      this.mcd = mcd;
+    }
+  }
+
+  private AttachedMTableInfo getMTable(String catName, String db, String table,
+                                       boolean retrieveCD) {
+    AttachedMTableInfo nmtbl = new AttachedMTableInfo();
+    MTable mtbl = null;
+    boolean commited = false;
+    Query query = null;
+    try {
+      openTransaction();
+      catName = normalizeIdentifier(catName);
+      db = normalizeIdentifier(db);
+      table = normalizeIdentifier(table);
+      query = pm.newQuery(MTable.class,
+          "tableName == table && database.name == db && database.catalogName == catname");
+      query.declareParameters(
+          "java.lang.String table, java.lang.String db, java.lang.String catname");
+      query.setUnique(true);
+      LOG.debug("Executing getMTable for " +
+          TableName.getQualified(catName, db, table));
+      mtbl = (MTable) query.execute(table, db, catName);
+      pm.retrieve(mtbl);
+      // Retrieving CD can be expensive and unnecessary, so do it only when required.
+      if (mtbl != null && retrieveCD) {
+        pm.retrieve(mtbl.getSd());
+        pm.retrieveAll(mtbl.getSd().getCD());
+        nmtbl.mcd = mtbl.getSd().getCD();
+      }
+      commited = commitTransaction();
+    } finally {
+      rollbackAndCleanup(commited, query);
+    }
+    nmtbl.mtbl = mtbl;
+    return nmtbl;
+  }
+
+  private MCreationMetadata getCreationMetadata(String catName, String dbName, String tblName) {
+    boolean commited = false;
+    MCreationMetadata mcm = null;
+    Query query = null;
+    catName = normalizeIdentifier(catName);
+    dbName = normalizeIdentifier(dbName);
+    tblName = normalizeIdentifier(tblName);
+    try {
+      openTransaction();
+      query = pm.newQuery(
+          MCreationMetadata.class, "tblName == table && dbName == db && catalogName == cat");
+      query.declareParameters("java.lang.String table, java.lang.String db, java.lang.String cat");
+      query.setUnique(true);
+      mcm = (MCreationMetadata) query.execute(tblName, dbName, catName);
+      pm.retrieve(mcm);
+      commited = commitTransaction();
+    } finally {
+      rollbackAndCleanup(commited, query);
+    }
+    return mcm;
+  }
+
+  private MTable getMTable(String catName, String db, String table) {
+    AttachedMTableInfo nmtbl = getMTable(catName, db, table, false);
+    return nmtbl.mtbl;
+  }
+
+  @Override
+  public List<Table> getTableObjectsByName(String catName, String db, List<String> tbl_names)
+      throws MetaException, UnknownDBException {
+    List<Table> tables = new ArrayList<>();
+    boolean committed = false;
+    Query dbExistsQuery = null;
+    Query query = null;
+    try {
+      openTransaction();
+      db = normalizeIdentifier(db);
+      catName = normalizeIdentifier(catName);
+
+      List<String> lowered_tbl_names = new ArrayList<>(tbl_names.size());
+      for (String t : tbl_names) {
+        lowered_tbl_names.add(normalizeIdentifier(t));
+      }
+      query = pm.newQuery(MTable.class);
+      query.setFilter("database.name == db && database.catalogName == cat && tbl_names.contains(tableName)");
+      query.declareParameters("java.lang.String db, java.lang.String cat, java.util.Collection tbl_names");
+      Collection mtables = (Collection) query.execute(db, catName, lowered_tbl_names);
+      if (mtables == null || mtables.isEmpty()) {
+        // Need to differentiate between an unmatched pattern and a non-existent database
+        dbExistsQuery = pm.newQuery(MDatabase.class, "name == db && catalogName == cat");
+        dbExistsQuery.declareParameters("java.lang.String db, java.lang.String cat");
+        dbExistsQuery.setUnique(true);
+        dbExistsQuery.setResult("name");
+        String dbNameIfExists = (String) dbExistsQuery.execute(db, catName);
+        if (org.apache.commons.lang.StringUtils.isEmpty(dbNameIfExists)) {
+          throw new UnknownDBException("Could not find database " +
+              DatabaseName.getQualified(catName, db));
+        }
+      } else {
+        for (Iterator iter = mtables.iterator(); iter.hasNext(); ) {
+          Table tbl = convertToTable((MTable) iter.next());
+          // Retrieve creation metadata if needed
+          if (TableType.MATERIALIZED_VIEW.toString().equals(tbl.getTableType())) {
+            tbl.setCreationMetadata(
+                convertToCreationMetadata(
+                    getCreationMetadata(tbl.getCatName(), tbl.getDbName(), tbl.getTableName())));
+          }
+          tables.add(tbl);
+        }
+      }
+      committed = commitTransaction();
+    } finally {
+      rollbackAndCleanup(committed, query);
+      if (dbExistsQuery != null) {
+        dbExistsQuery.closeAll();
+      }
+    }
+    return tables;
+  }
+
+  /** Makes shallow copy of a list to avoid DataNucleus mucking with our objects. */
+  private <T> List<T> convertList(List<T> dnList) {
+    return (dnList == null) ? null : Lists.newArrayList(dnList);
+  }
+
+  /** Makes shallow copy of a map to avoid DataNucleus mucking with our objects. */
+  private Map<String, String> convertMap(Map<String, String> dnMap) {
+    return MetaStoreServerUtils.trimMapNulls(dnMap,
+        MetastoreConf.getBoolVar(getConf(), ConfVars.ORM_RETRIEVE_MAPNULLS_AS_EMPTY_STRINGS));
+  }
+
+  private Table convertToTable(MTable mtbl) throws MetaException {
+    if (mtbl == null) {
+      return null;
+    }
+    String tableType = mtbl.getTableType();
+    if (tableType == null) {
+      // for backwards compatibility with old metastore persistence
+      if (mtbl.getViewOriginalText() != null) {
+        tableType = TableType.VIRTUAL_VIEW.toString();
+      } else if (Boolean.parseBoolean(mtbl.getParameters().get("EXTERNAL"))) {
+        tableType = TableType.EXTERNAL_TABLE.toString();
+      } else {
+        tableType = TableType.MANAGED_TABLE.toString();
+      }
+    }
+    final Table t = new Table(mtbl.getTableName(), mtbl.getDatabase().getName(), mtbl
+        .getOwner(), mtbl.getCreateTime(), mtbl.getLastAccessTime(), mtbl
+        .getRetention(), convertToStorageDescriptor(mtbl.getSd()),
+        convertToFieldSchemas(mtbl.getPartitionKeys()), convertMap(mtbl.getParameters()),
+        mtbl.getViewOriginalText(), mtbl.getViewExpandedText(), tableType);
+
+    if (Strings.isNullOrEmpty(mtbl.getOwnerType())) {
+      // Before the ownerType exists in an old Hive schema, USER was the default type for owner.
+      // Let's set the default to USER to keep backward compatibility.
+      t.setOwnerType(PrincipalType.USER);
+    } else {
+      t.setOwnerType(PrincipalType.valueOf(mtbl.getOwnerType()));
+    }
+
+    t.setId(mtbl.getId());
+    t.setRewriteEnabled(mtbl.isRewriteEnabled());
+    t.setCatName(mtbl.getDatabase().getCatalogName());
+    t.setWriteId(mtbl.getWriteId());
+    return t;
+  }
+
+  private MTable convertToMTable(Table tbl) throws InvalidObjectException,
+      MetaException {
+    // NOTE: we don't set writeId in this method. Write ID is only set after validating the
+    //       existing write ID against the caller's valid list.
+    if (tbl == null) {
+      return null;
+    }
+    MDatabase mdb = null;
+    String catName = tbl.isSetCatName() ? tbl.getCatName() : getDefaultCatalog(conf);
+    try {
+      mdb = getMDatabase(catName, tbl.getDbName());
+    } catch (NoSuchObjectException e) {
+      LOG.error("Could not convert to MTable", e);
+      throw new InvalidObjectException("Database " +
+          DatabaseName.getQualified(catName, tbl.getDbName()) + " doesn't exist.");
+    }
+
+    // If the table has property EXTERNAL set, update table type
+    // accordingly
+    String tableType = tbl.getTableType();
+    boolean isExternal = Boolean.parseBoolean(tbl.getParameters().get("EXTERNAL"));
+    if (TableType.MANAGED_TABLE.toString().equals(tableType)) {
+      if (isExternal) {
+        tableType = TableType.EXTERNAL_TABLE.toString();
+      }
+    }
+    if (TableType.EXTERNAL_TABLE.toString().equals(tableType)) {
+      if (!isExternal) {
+        tableType = TableType.MANAGED_TABLE.toString();
+      }
+    }
+
+    PrincipalType ownerPrincipalType = tbl.getOwnerType();
+    String ownerType = (ownerPrincipalType == null) ? PrincipalType.USER.name() : ownerPrincipalType.name();
+
+    // A new table is always created with a new column descriptor
+    MTable mtable = new MTable(normalizeIdentifier(tbl.getTableName()), mdb,
+        convertToMStorageDescriptor(tbl.getSd()), tbl.getOwner(), ownerType, tbl
+        .getCreateTime(), tbl.getLastAccessTime(), tbl.getRetention(),
+        convertToMFieldSchemas(tbl.getPartitionKeys()), tbl.getParameters(),
+        tbl.getViewOriginalText(), tbl.getViewExpandedText(), tbl.isRewriteEnabled(),
+        tableType);
+    return mtable;
+  }
+
+  private List<MFieldSchema> convertToMFieldSchemas(List<FieldSchema> keys) {
+    List<MFieldSchema> mkeys = null;
+    if (keys != null) {
+      mkeys = new ArrayList<>(keys.size());
+      for (FieldSchema part : keys) {
+        mkeys.add(new MFieldSchema(part.getName().toLowerCase(),
+            part.getType(), part.getComment()));
+      }
+    }
+    return mkeys;
+  }
+
+  private List<FieldSchema> convertToFieldSchemas(List<MFieldSchema> mkeys) {
+    List<FieldSchema> keys = null;
+    if (mkeys != null) {
+      keys = new ArrayList<>(mkeys.size());
+      for (MFieldSchema part : mkeys) {
+        keys.add(new FieldSchema(part.getName(), part.getType(), part
+            .getComment()));
+      }
+    }
+    return keys;
+  }
+
+  private List<MOrder> convertToMOrders(List<Order> keys) {
+    List<MOrder> mkeys = null;
+    if (keys != null) {
+      mkeys = new ArrayList<>(keys.size());
+      for (Order part : keys) {
+        mkeys.add(new MOrder(normalizeIdentifier(part.getCol()), part.getOrder()));
+      }
+    }
+    return mkeys;
+  }
+
+  private List<Order> convertToOrders(List<MOrder> mkeys) {
+    List<Order> keys = null;
+    if (mkeys != null) {
+      keys = new ArrayList<>(mkeys.size());
+      for (MOrder part : mkeys) {
+        keys.add(new Order(part.getCol(), part.getOrder()));
+      }
+    }
+    return keys;
+  }
+
+  private SerDeInfo convertToSerDeInfo(MSerDeInfo ms) throws MetaException {
+    if (ms == null) {
+      throw new MetaException("Invalid SerDeInfo object");
+    }
+    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(),
+        ms.getDescription(), ms.getSerializerClass(), ms.getDeserializerClass(),
+        ms.getSerdeType() == null ? 0 : ms.getSerdeType().getValue());
+  }
+
+  /**
+   * Given a list of model field schemas, create a new model column descriptor.
+   * @param cols the columns the column descriptor contains
+   * @return a new column descriptor db-backed object
+   */
+  private MColumnDescriptor createNewMColumnDescriptor(List<MFieldSchema> cols) {
+    if (cols == null) {
+      return null;
+    }
+    return new MColumnDescriptor(cols);
+  }
+
+  // MSD and SD should be same objects. Not sure how to make then same right now
+  // MSerdeInfo *& SerdeInfo should be same as well
+  private StorageDescriptor convertToStorageDescriptor(
+      MStorageDescriptor msd,
+      boolean noFS) throws MetaException {
+    if (msd == null) {
+      return null;
+    }
+    List<MFieldSchema> mFieldSchemas = msd.getCD() == null ? null : msd.getCD().getCols();
+
+    StorageDescriptor sd = new StorageDescriptor(noFS ? null : convertToFieldSchemas(mFieldSchemas),
+        msd.getLocation(), msd.getInputFormat(), msd.getOutputFormat(), msd
+        .isCompressed(), msd.getNumBuckets(), convertToSerDeInfo(msd
+        .getSerDeInfo()), convertList(msd.getBucketCols()), convertToOrders(msd
+        .getSortCols()), convertMap(msd.getParameters()));
+    SkewedInfo skewedInfo = new SkewedInfo(convertList(msd.getSkewedColNames()),
+        convertToSkewedValues(msd.getSkewedColValues()),
+        covertToSkewedMap(msd.getSkewedColValueLocationMaps()));
+    sd.setSkewedInfo(skewedInfo);
+    sd.setStoredAsSubDirectories(msd.isStoredAsSubDirectories());
+    return sd;
+  }
+
+  private StorageDescriptor convertToStorageDescriptor(MStorageDescriptor msd)
+      throws MetaException {
+    return convertToStorageDescriptor(msd, false);
+  }
+
+  /**
+   * Convert a list of MStringList to a list of list string
+   *
+   * @param mLists
+   * @return
+   */
+  private List<List<String>> convertToSkewedValues(List<MStringList> mLists) {
+    List<List<String>> lists = null;
+    if (mLists != null) {
+      lists = new ArrayList<>(mLists.size());
+      for (MStringList element : mLists) {
+        lists.add(new ArrayList<>(element.getInternalList()));
+      }
+    }
+    return lists;
+  }
+
+  private List<MStringList> convertToMStringLists(List<List<String>> mLists) {
+    List<MStringList> lists = null ;
+    if (null != mLists) {
+      lists = new ArrayList<>();
+      for (List<String> mList : mLists) {
+        lists.add(new MStringList(mList));
+      }
+    }
+    return lists;
+  }
+
+  /**
+   * Convert a MStringList Map to a Map
+   * @param mMap
+   * @return
+   */
+  private Map<List<String>, String> covertToSkewedMap(Map<MStringList, String> mMap) {
+    Map<List<String>, String> map = null;
+    if (mMap != null) {
+      map = new HashMap<>(mMap.size());
+      Set<MStringList> keys = mMap.keySet();
+      for (MStringList key : keys) {
+        map.put(new ArrayList<>(key.getInternalList()), mMap.get(key));
+      }
+    }
+    return map;
+  }
+
+  /**
+   * Covert a Map to a MStringList Map
+   * @param mMap
+   * @return
+   */
+  private Map<MStringList, String> covertToMapMStringList(Map<List<String>, String> mMap) {
+    Map<MStringList, String> map = null;
+    if (mMap != null) {
+      map = new HashMap<>(mMap.size());
+      Set<List<String>> keys = mMap.keySet();
+      for (List<String> key : keys) {
+        map.put(new MStringList(key), mMap.get(key));
+      }
+    }
+    return map;
+  }
+
+  /**
+   * Converts a storage descriptor to a db-backed storage descriptor.  Creates a
+   *   new db-backed column descriptor object for this SD.
+   * @param sd the storage descriptor to wrap in a db-backed object
+   * @return the storage descriptor db-backed object
+   * @throws MetaException
+   */
+  private MStorageDescriptor convertToMStorageDescriptor(StorageDescriptor sd)
+      throws MetaException {
+    if (sd == null) {
+      return null;
+    }
+    MColumnDescriptor mcd = createNewMColumnDescriptor(convertToMFieldSchemas(sd.getCols()));
+    return convertToMStorageDescriptor(sd, mcd);
+  }
+
+  /**
+   * Converts a storage descriptor to a db-backed storage descriptor.  It points the
+   * storage descriptor's column descriptor to the one passed as an argument,
+   * so it does not create a new mcolumn descriptor object.
+   * @param sd the storage descriptor to wrap in a db-backed object
+   * @param mcd the db-backed column descriptor
+   * @return the db-backed storage descriptor object
+   * @throws MetaException
+   */
+  private MStorageDescriptor convertToMStorageDescriptor(StorageDescriptor sd,
+      MColumnDescriptor mcd) throws MetaException {
+    if (sd == null) {
+      return null;
+    }
+    return new MStorageDescriptor(mcd, sd
+        .getLocation(), sd.getInputFormat(), sd.getOutputFormat(), sd
+        .isCompressed(), sd.getNumBuckets(), convertToMSerDeInfo(sd
+        .getSerdeInfo()), sd.getBucketCols(),
+        convertToMOrders(sd.getSortCols()), sd.getParameters(),
+        (null == sd.getSkewedInfo()) ? null
+            : sd.getSkewedInfo().getSkewedColNames(),
+        convertToMStringLists((null == sd.getSkewedInfo()) ? null : sd.getSkewedInfo()
+            .getSkewedColValues()),
+        covertToMapMStringList((null == sd.getSkewedInfo()) ? null : sd.getSkewedInfo()
+            .getSkewedColValueLocationMaps()), sd.isStoredAsSubDirectories());
+  }
+
+  private MCreationMetadata convertToMCreationMetadata(
+      CreationMetadata m) throws MetaException {
+    if (m == null) {
+      return null;
+    }
+    assert !m.isSetMaterializationTime();
+    Set<MTable> tablesUsed = new HashSet<>();
+    for (String fullyQualifiedName : m.getTablesUsed()) {
+      String[] names =  fullyQualifiedName.split("\\.");
+      tablesUsed.add(getMTable(m.getCatName(), names[0], names[1], false).mtbl);
+    }
+    return new MCreationMetadata(normalizeIdentifier(m.getCatName()),
+            normalizeIdentifier(m.getDbName()), normalizeIdentifier(m.getTblName()),
+        tablesUsed, m.getValidTxnList(), System.currentTimeMillis());
+  }
+
+  private CreationMetadata convertToCreationMetadata(
+      MCreationMetadata s) throws MetaException {
+    if (s == null) {
+      return null;
+    }
+    Set<String> tablesUsed = new HashSet<>();
+    for (MTable mtbl : s.getTables()) {
+      tablesUsed.add(
+          Warehouse.getQualifiedName(
+              mtbl.getDatabase().getName(), mtbl.getTableName()));
+    }
+    CreationMetadata r = new CreationMetadata(s.getCatalogName(),
+        s.getDbName(), s.getTblName(), tablesUsed);
+    r.setMaterializationTime(s.getMaterializationTime());
+    if (s.getTxnList() != null) {
+      r.setValidTxnList(s.getTxnList());
+    }
+    return r;
+  }
+
+  @Override
+  public boolean addPartitions(String catName, String dbName, String tblName, List<Partition> parts)
+      throws InvalidObjectException, MetaException {
+    boolean success = false;
+    openTransaction();
+    try {
+      List<MTablePrivilege> tabGrants = null;
+      List<MTableColumnPrivilege> tabColumnGrants = null;
+      MTable table = this.getMTable(catName, dbName, tblName);
+      if ("TRUE".equalsIgnoreCase(table.getParameters().get("PARTITION_LEVEL_PRIVILEGE"))) {
+        tabGrants = this.listAllTableGrants(catName, dbName, tblName);
+        tabColumnGrants = this.listTableAllColumnGrants(catName, dbName, tblName);
+      }
+      List<Object> toPersist = new ArrayList<>();
+      for (Partition part : parts) {
+        if (!part.getTableName().equals(tblName) || !part.getDbName().equals(dbName)) {
+          throw new MetaException("Partition does not belong to target table "
+              + dbName + "." + tblName + ": " + part);
+        }
+        MPartition mpart = convertToMPart(part, table, true);
+
+        toPersist.add(mpart);
+        int now = (int)(System.currentTimeMillis()/1000);
+        if (tabGrants != null) {
+          for (MTablePrivilege tab: tabGrants) {
+            toPersist.add(new MPartitionPrivilege(tab.getPrincipalName(),
+                tab.getPrincipalType(), mpart, tab.getPrivilege(), now,
+                tab.getGrantor(), tab.getGrantorType(), tab.getGrantOption(),
+                tab.getAuthorizer()));
+          }
+        }
+
+        if (tabColumnGrants != null) {
+          for (MTableColumnPrivilege col : tabColumnGrants) {
+            toPersist.add(new MPartitionColumnPrivilege(col.getPrincipalName(),
+                col.getPrincipalType(), mpart, col.getColumnName(), col.getPrivilege(),
+                now, col.getGrantor(), col.getGrantorType(), col.getGrantOption(),
+                col.getAuthorizer()));
+          }
+        }
+      }
+      if (CollectionUtils.isNotEmpty(toPersist)) {
+        pm.makePersistentAll(toPersist);
+        pm.flush();
+      }
+
+      success = commitTransaction();
+    } finally {
+      if (!success) {
+        rollbackTransaction();
+      }
+    }
+    return success;
+  }
+
+  private boolean isValidPartition(
+      Partition part, List<FieldSchema> partitionKeys, boolean ifNotExists) throws MetaException {
+    MetaStoreServerUtils.validatePartitionNameCharacters(part.getValues(),
+        partitionValidationPattern);
+    boolean doesExist = doesPartitionExist(part.getCatName(),
+        part.getDbName(), part.getTableName(), partitionKeys, part.getValues());
+    if (doesExist && !ifNotExists) {
+      throw new MetaException("Partition already exists: " + part);
+    }
+    return !doesExist;
+  }
+
+  @Override
+  public boolean addPartitions(String catName, String dbName, String tblName,
+                               PartitionSpecProxy partitionSpec, boolean ifNotExists)
+      throws InvalidObjectException, MetaException {
+    boolean success = false;
+    openTransaction();
+    try {
+      List<MTablePrivilege> tabGrants = null;
+      List<MTableColumnPrivilege> tabColumnGrants = null;
+      MTable table = this.getMTable(catName, dbName, tblName);
+      if 

<TRUNCATED>

[10/13] hive git commit: HIVE-20306 : Implement projection spec for fetching only requested fields from partitions (Vihang Karajgaonkar, reviewed by Aihua Xu, Andrew Sherman and Alexander Kolbasov)

Posted by vi...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/44ef91a6/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
index 4574c6a..efeaec7 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
@@ -1608,6 +1608,12 @@ interface ThriftHiveMetastoreIf extends \FacebookServiceIf {
    * @throws \metastore\MetaException
    */
   public function get_runtime_stats(\metastore\GetRuntimeStatsRequest $rqst);
+  /**
+   * @param \metastore\GetPartitionsRequest $request
+   * @return \metastore\GetPartitionsResponse
+   * @throws \metastore\MetaException
+   */
+  public function get_partitions_with_specs(\metastore\GetPartitionsRequest $request);
 }
 
 class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metastore\ThriftHiveMetastoreIf {
@@ -13755,6 +13761,60 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas
     throw new \Exception("get_runtime_stats failed: unknown result");
   }
 
+  public function get_partitions_with_specs(\metastore\GetPartitionsRequest $request)
+  {
+    $this->send_get_partitions_with_specs($request);
+    return $this->recv_get_partitions_with_specs();
+  }
+
+  public function send_get_partitions_with_specs(\metastore\GetPartitionsRequest $request)
+  {
+    $args = new \metastore\ThriftHiveMetastore_get_partitions_with_specs_args();
+    $args->request = $request;
+    $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
+    if ($bin_accel)
+    {
+      thrift_protocol_write_binary($this->output_, 'get_partitions_with_specs', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
+    }
+    else
+    {
+      $this->output_->writeMessageBegin('get_partitions_with_specs', TMessageType::CALL, $this->seqid_);
+      $args->write($this->output_);
+      $this->output_->writeMessageEnd();
+      $this->output_->getTransport()->flush();
+    }
+  }
+
+  public function recv_get_partitions_with_specs()
+  {
+    $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_partitions_with_specs_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_partitions_with_specs_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_partitions_with_specs failed: unknown result");
+  }
+
 }
 
 // HELPER FUNCTIONS AND STRUCTURES
@@ -15892,14 +15952,14 @@ class ThriftHiveMetastore_get_databases_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size854 = 0;
-            $_etype857 = 0;
-            $xfer += $input->readListBegin($_etype857, $_size854);
-            for ($_i858 = 0; $_i858 < $_size854; ++$_i858)
+            $_size882 = 0;
+            $_etype885 = 0;
+            $xfer += $input->readListBegin($_etype885, $_size882);
+            for ($_i886 = 0; $_i886 < $_size882; ++$_i886)
             {
-              $elem859 = null;
-              $xfer += $input->readString($elem859);
-              $this->success []= $elem859;
+              $elem887 = null;
+              $xfer += $input->readString($elem887);
+              $this->success []= $elem887;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15935,9 +15995,9 @@ class ThriftHiveMetastore_get_databases_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter860)
+          foreach ($this->success as $iter888)
           {
-            $xfer += $output->writeString($iter860);
+            $xfer += $output->writeString($iter888);
           }
         }
         $output->writeListEnd();
@@ -16068,14 +16128,14 @@ class ThriftHiveMetastore_get_all_databases_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size861 = 0;
-            $_etype864 = 0;
-            $xfer += $input->readListBegin($_etype864, $_size861);
-            for ($_i865 = 0; $_i865 < $_size861; ++$_i865)
+            $_size889 = 0;
+            $_etype892 = 0;
+            $xfer += $input->readListBegin($_etype892, $_size889);
+            for ($_i893 = 0; $_i893 < $_size889; ++$_i893)
             {
-              $elem866 = null;
-              $xfer += $input->readString($elem866);
-              $this->success []= $elem866;
+              $elem894 = null;
+              $xfer += $input->readString($elem894);
+              $this->success []= $elem894;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16111,9 +16171,9 @@ class ThriftHiveMetastore_get_all_databases_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter867)
+          foreach ($this->success as $iter895)
           {
-            $xfer += $output->writeString($iter867);
+            $xfer += $output->writeString($iter895);
           }
         }
         $output->writeListEnd();
@@ -17114,18 +17174,18 @@ class ThriftHiveMetastore_get_type_all_result {
         case 0:
           if ($ftype == TType::MAP) {
             $this->success = array();
-            $_size868 = 0;
-            $_ktype869 = 0;
-            $_vtype870 = 0;
-            $xfer += $input->readMapBegin($_ktype869, $_vtype870, $_size868);
-            for ($_i872 = 0; $_i872 < $_size868; ++$_i872)
+            $_size896 = 0;
+            $_ktype897 = 0;
+            $_vtype898 = 0;
+            $xfer += $input->readMapBegin($_ktype897, $_vtype898, $_size896);
+            for ($_i900 = 0; $_i900 < $_size896; ++$_i900)
             {
-              $key873 = '';
-              $val874 = new \metastore\Type();
-              $xfer += $input->readString($key873);
-              $val874 = new \metastore\Type();
-              $xfer += $val874->read($input);
-              $this->success[$key873] = $val874;
+              $key901 = '';
+              $val902 = new \metastore\Type();
+              $xfer += $input->readString($key901);
+              $val902 = new \metastore\Type();
+              $xfer += $val902->read($input);
+              $this->success[$key901] = $val902;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -17161,10 +17221,10 @@ class ThriftHiveMetastore_get_type_all_result {
       {
         $output->writeMapBegin(TType::STRING, TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $kiter875 => $viter876)
+          foreach ($this->success as $kiter903 => $viter904)
           {
-            $xfer += $output->writeString($kiter875);
-            $xfer += $viter876->write($output);
+            $xfer += $output->writeString($kiter903);
+            $xfer += $viter904->write($output);
           }
         }
         $output->writeMapEnd();
@@ -17368,15 +17428,15 @@ class ThriftHiveMetastore_get_fields_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size877 = 0;
-            $_etype880 = 0;
-            $xfer += $input->readListBegin($_etype880, $_size877);
-            for ($_i881 = 0; $_i881 < $_size877; ++$_i881)
+            $_size905 = 0;
+            $_etype908 = 0;
+            $xfer += $input->readListBegin($_etype908, $_size905);
+            for ($_i909 = 0; $_i909 < $_size905; ++$_i909)
             {
-              $elem882 = null;
-              $elem882 = new \metastore\FieldSchema();
-              $xfer += $elem882->read($input);
-              $this->success []= $elem882;
+              $elem910 = null;
+              $elem910 = new \metastore\FieldSchema();
+              $xfer += $elem910->read($input);
+              $this->success []= $elem910;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17428,9 +17488,9 @@ class ThriftHiveMetastore_get_fields_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter883)
+          foreach ($this->success as $iter911)
           {
-            $xfer += $iter883->write($output);
+            $xfer += $iter911->write($output);
           }
         }
         $output->writeListEnd();
@@ -17672,15 +17732,15 @@ class ThriftHiveMetastore_get_fields_with_environment_context_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size884 = 0;
-            $_etype887 = 0;
-            $xfer += $input->readListBegin($_etype887, $_size884);
-            for ($_i888 = 0; $_i888 < $_size884; ++$_i888)
+            $_size912 = 0;
+            $_etype915 = 0;
+            $xfer += $input->readListBegin($_etype915, $_size912);
+            for ($_i916 = 0; $_i916 < $_size912; ++$_i916)
             {
-              $elem889 = null;
-              $elem889 = new \metastore\FieldSchema();
-              $xfer += $elem889->read($input);
-              $this->success []= $elem889;
+              $elem917 = null;
+              $elem917 = new \metastore\FieldSchema();
+              $xfer += $elem917->read($input);
+              $this->success []= $elem917;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17732,9 +17792,9 @@ class ThriftHiveMetastore_get_fields_with_environment_context_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter890)
+          foreach ($this->success as $iter918)
           {
-            $xfer += $iter890->write($output);
+            $xfer += $iter918->write($output);
           }
         }
         $output->writeListEnd();
@@ -17948,15 +18008,15 @@ class ThriftHiveMetastore_get_schema_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size891 = 0;
-            $_etype894 = 0;
-            $xfer += $input->readListBegin($_etype894, $_size891);
-            for ($_i895 = 0; $_i895 < $_size891; ++$_i895)
+            $_size919 = 0;
+            $_etype922 = 0;
+            $xfer += $input->readListBegin($_etype922, $_size919);
+            for ($_i923 = 0; $_i923 < $_size919; ++$_i923)
             {
-              $elem896 = null;
-              $elem896 = new \metastore\FieldSchema();
-              $xfer += $elem896->read($input);
-              $this->success []= $elem896;
+              $elem924 = null;
+              $elem924 = new \metastore\FieldSchema();
+              $xfer += $elem924->read($input);
+              $this->success []= $elem924;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18008,9 +18068,9 @@ class ThriftHiveMetastore_get_schema_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter897)
+          foreach ($this->success as $iter925)
           {
-            $xfer += $iter897->write($output);
+            $xfer += $iter925->write($output);
           }
         }
         $output->writeListEnd();
@@ -18252,15 +18312,15 @@ class ThriftHiveMetastore_get_schema_with_environment_context_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size898 = 0;
-            $_etype901 = 0;
-            $xfer += $input->readListBegin($_etype901, $_size898);
-            for ($_i902 = 0; $_i902 < $_size898; ++$_i902)
+            $_size926 = 0;
+            $_etype929 = 0;
+            $xfer += $input->readListBegin($_etype929, $_size926);
+            for ($_i930 = 0; $_i930 < $_size926; ++$_i930)
             {
-              $elem903 = null;
-              $elem903 = new \metastore\FieldSchema();
-              $xfer += $elem903->read($input);
-              $this->success []= $elem903;
+              $elem931 = null;
+              $elem931 = new \metastore\FieldSchema();
+              $xfer += $elem931->read($input);
+              $this->success []= $elem931;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18312,9 +18372,9 @@ class ThriftHiveMetastore_get_schema_with_environment_context_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter904)
+          foreach ($this->success as $iter932)
           {
-            $xfer += $iter904->write($output);
+            $xfer += $iter932->write($output);
           }
         }
         $output->writeListEnd();
@@ -18986,15 +19046,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->primaryKeys = array();
-            $_size905 = 0;
-            $_etype908 = 0;
-            $xfer += $input->readListBegin($_etype908, $_size905);
-            for ($_i909 = 0; $_i909 < $_size905; ++$_i909)
+            $_size933 = 0;
+            $_etype936 = 0;
+            $xfer += $input->readListBegin($_etype936, $_size933);
+            for ($_i937 = 0; $_i937 < $_size933; ++$_i937)
             {
-              $elem910 = null;
-              $elem910 = new \metastore\SQLPrimaryKey();
-              $xfer += $elem910->read($input);
-              $this->primaryKeys []= $elem910;
+              $elem938 = null;
+              $elem938 = new \metastore\SQLPrimaryKey();
+              $xfer += $elem938->read($input);
+              $this->primaryKeys []= $elem938;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19004,15 +19064,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->foreignKeys = array();
-            $_size911 = 0;
-            $_etype914 = 0;
-            $xfer += $input->readListBegin($_etype914, $_size911);
-            for ($_i915 = 0; $_i915 < $_size911; ++$_i915)
+            $_size939 = 0;
+            $_etype942 = 0;
+            $xfer += $input->readListBegin($_etype942, $_size939);
+            for ($_i943 = 0; $_i943 < $_size939; ++$_i943)
             {
-              $elem916 = null;
-              $elem916 = new \metastore\SQLForeignKey();
-              $xfer += $elem916->read($input);
-              $this->foreignKeys []= $elem916;
+              $elem944 = null;
+              $elem944 = new \metastore\SQLForeignKey();
+              $xfer += $elem944->read($input);
+              $this->foreignKeys []= $elem944;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19022,15 +19082,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 4:
           if ($ftype == TType::LST) {
             $this->uniqueConstraints = array();
-            $_size917 = 0;
-            $_etype920 = 0;
-            $xfer += $input->readListBegin($_etype920, $_size917);
-            for ($_i921 = 0; $_i921 < $_size917; ++$_i921)
+            $_size945 = 0;
+            $_etype948 = 0;
+            $xfer += $input->readListBegin($_etype948, $_size945);
+            for ($_i949 = 0; $_i949 < $_size945; ++$_i949)
             {
-              $elem922 = null;
-              $elem922 = new \metastore\SQLUniqueConstraint();
-              $xfer += $elem922->read($input);
-              $this->uniqueConstraints []= $elem922;
+              $elem950 = null;
+              $elem950 = new \metastore\SQLUniqueConstraint();
+              $xfer += $elem950->read($input);
+              $this->uniqueConstraints []= $elem950;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19040,15 +19100,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->notNullConstraints = array();
-            $_size923 = 0;
-            $_etype926 = 0;
-            $xfer += $input->readListBegin($_etype926, $_size923);
-            for ($_i927 = 0; $_i927 < $_size923; ++$_i927)
+            $_size951 = 0;
+            $_etype954 = 0;
+            $xfer += $input->readListBegin($_etype954, $_size951);
+            for ($_i955 = 0; $_i955 < $_size951; ++$_i955)
             {
-              $elem928 = null;
-              $elem928 = new \metastore\SQLNotNullConstraint();
-              $xfer += $elem928->read($input);
-              $this->notNullConstraints []= $elem928;
+              $elem956 = null;
+              $elem956 = new \metastore\SQLNotNullConstraint();
+              $xfer += $elem956->read($input);
+              $this->notNullConstraints []= $elem956;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19058,15 +19118,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 6:
           if ($ftype == TType::LST) {
             $this->defaultConstraints = array();
-            $_size929 = 0;
-            $_etype932 = 0;
-            $xfer += $input->readListBegin($_etype932, $_size929);
-            for ($_i933 = 0; $_i933 < $_size929; ++$_i933)
+            $_size957 = 0;
+            $_etype960 = 0;
+            $xfer += $input->readListBegin($_etype960, $_size957);
+            for ($_i961 = 0; $_i961 < $_size957; ++$_i961)
             {
-              $elem934 = null;
-              $elem934 = new \metastore\SQLDefaultConstraint();
-              $xfer += $elem934->read($input);
-              $this->defaultConstraints []= $elem934;
+              $elem962 = null;
+              $elem962 = new \metastore\SQLDefaultConstraint();
+              $xfer += $elem962->read($input);
+              $this->defaultConstraints []= $elem962;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19076,15 +19136,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 7:
           if ($ftype == TType::LST) {
             $this->checkConstraints = array();
-            $_size935 = 0;
-            $_etype938 = 0;
-            $xfer += $input->readListBegin($_etype938, $_size935);
-            for ($_i939 = 0; $_i939 < $_size935; ++$_i939)
+            $_size963 = 0;
+            $_etype966 = 0;
+            $xfer += $input->readListBegin($_etype966, $_size963);
+            for ($_i967 = 0; $_i967 < $_size963; ++$_i967)
             {
-              $elem940 = null;
-              $elem940 = new \metastore\SQLCheckConstraint();
-              $xfer += $elem940->read($input);
-              $this->checkConstraints []= $elem940;
+              $elem968 = null;
+              $elem968 = new \metastore\SQLCheckConstraint();
+              $xfer += $elem968->read($input);
+              $this->checkConstraints []= $elem968;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19120,9 +19180,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->primaryKeys));
         {
-          foreach ($this->primaryKeys as $iter941)
+          foreach ($this->primaryKeys as $iter969)
           {
-            $xfer += $iter941->write($output);
+            $xfer += $iter969->write($output);
           }
         }
         $output->writeListEnd();
@@ -19137,9 +19197,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->foreignKeys));
         {
-          foreach ($this->foreignKeys as $iter942)
+          foreach ($this->foreignKeys as $iter970)
           {
-            $xfer += $iter942->write($output);
+            $xfer += $iter970->write($output);
           }
         }
         $output->writeListEnd();
@@ -19154,9 +19214,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->uniqueConstraints));
         {
-          foreach ($this->uniqueConstraints as $iter943)
+          foreach ($this->uniqueConstraints as $iter971)
           {
-            $xfer += $iter943->write($output);
+            $xfer += $iter971->write($output);
           }
         }
         $output->writeListEnd();
@@ -19171,9 +19231,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->notNullConstraints));
         {
-          foreach ($this->notNullConstraints as $iter944)
+          foreach ($this->notNullConstraints as $iter972)
           {
-            $xfer += $iter944->write($output);
+            $xfer += $iter972->write($output);
           }
         }
         $output->writeListEnd();
@@ -19188,9 +19248,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->defaultConstraints));
         {
-          foreach ($this->defaultConstraints as $iter945)
+          foreach ($this->defaultConstraints as $iter973)
           {
-            $xfer += $iter945->write($output);
+            $xfer += $iter973->write($output);
           }
         }
         $output->writeListEnd();
@@ -19205,9 +19265,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->checkConstraints));
         {
-          foreach ($this->checkConstraints as $iter946)
+          foreach ($this->checkConstraints as $iter974)
           {
-            $xfer += $iter946->write($output);
+            $xfer += $iter974->write($output);
           }
         }
         $output->writeListEnd();
@@ -21207,14 +21267,14 @@ class ThriftHiveMetastore_truncate_table_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->partNames = array();
-            $_size947 = 0;
-            $_etype950 = 0;
-            $xfer += $input->readListBegin($_etype950, $_size947);
-            for ($_i951 = 0; $_i951 < $_size947; ++$_i951)
+            $_size975 = 0;
+            $_etype978 = 0;
+            $xfer += $input->readListBegin($_etype978, $_size975);
+            for ($_i979 = 0; $_i979 < $_size975; ++$_i979)
             {
-              $elem952 = null;
-              $xfer += $input->readString($elem952);
-              $this->partNames []= $elem952;
+              $elem980 = null;
+              $xfer += $input->readString($elem980);
+              $this->partNames []= $elem980;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21252,9 +21312,9 @@ class ThriftHiveMetastore_truncate_table_args {
       {
         $output->writeListBegin(TType::STRING, count($this->partNames));
         {
-          foreach ($this->partNames as $iter953)
+          foreach ($this->partNames as $iter981)
           {
-            $xfer += $output->writeString($iter953);
+            $xfer += $output->writeString($iter981);
           }
         }
         $output->writeListEnd();
@@ -21690,14 +21750,14 @@ class ThriftHiveMetastore_get_tables_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size954 = 0;
-            $_etype957 = 0;
-            $xfer += $input->readListBegin($_etype957, $_size954);
-            for ($_i958 = 0; $_i958 < $_size954; ++$_i958)
+            $_size982 = 0;
+            $_etype985 = 0;
+            $xfer += $input->readListBegin($_etype985, $_size982);
+            for ($_i986 = 0; $_i986 < $_size982; ++$_i986)
             {
-              $elem959 = null;
-              $xfer += $input->readString($elem959);
-              $this->success []= $elem959;
+              $elem987 = null;
+              $xfer += $input->readString($elem987);
+              $this->success []= $elem987;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21733,9 +21793,9 @@ class ThriftHiveMetastore_get_tables_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter960)
+          foreach ($this->success as $iter988)
           {
-            $xfer += $output->writeString($iter960);
+            $xfer += $output->writeString($iter988);
           }
         }
         $output->writeListEnd();
@@ -21937,14 +21997,14 @@ class ThriftHiveMetastore_get_tables_by_type_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size961 = 0;
-            $_etype964 = 0;
-            $xfer += $input->readListBegin($_etype964, $_size961);
-            for ($_i965 = 0; $_i965 < $_size961; ++$_i965)
+            $_size989 = 0;
+            $_etype992 = 0;
+            $xfer += $input->readListBegin($_etype992, $_size989);
+            for ($_i993 = 0; $_i993 < $_size989; ++$_i993)
             {
-              $elem966 = null;
-              $xfer += $input->readString($elem966);
-              $this->success []= $elem966;
+              $elem994 = null;
+              $xfer += $input->readString($elem994);
+              $this->success []= $elem994;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21980,9 +22040,9 @@ class ThriftHiveMetastore_get_tables_by_type_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter967)
+          foreach ($this->success as $iter995)
           {
-            $xfer += $output->writeString($iter967);
+            $xfer += $output->writeString($iter995);
           }
         }
         $output->writeListEnd();
@@ -22138,14 +22198,14 @@ class ThriftHiveMetastore_get_materialized_views_for_rewriting_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size968 = 0;
-            $_etype971 = 0;
-            $xfer += $input->readListBegin($_etype971, $_size968);
-            for ($_i972 = 0; $_i972 < $_size968; ++$_i972)
+            $_size996 = 0;
+            $_etype999 = 0;
+            $xfer += $input->readListBegin($_etype999, $_size996);
+            for ($_i1000 = 0; $_i1000 < $_size996; ++$_i1000)
             {
-              $elem973 = null;
-              $xfer += $input->readString($elem973);
-              $this->success []= $elem973;
+              $elem1001 = null;
+              $xfer += $input->readString($elem1001);
+              $this->success []= $elem1001;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22181,9 +22241,9 @@ class ThriftHiveMetastore_get_materialized_views_for_rewriting_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter974)
+          foreach ($this->success as $iter1002)
           {
-            $xfer += $output->writeString($iter974);
+            $xfer += $output->writeString($iter1002);
           }
         }
         $output->writeListEnd();
@@ -22288,14 +22348,14 @@ class ThriftHiveMetastore_get_table_meta_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->tbl_types = array();
-            $_size975 = 0;
-            $_etype978 = 0;
-            $xfer += $input->readListBegin($_etype978, $_size975);
-            for ($_i979 = 0; $_i979 < $_size975; ++$_i979)
+            $_size1003 = 0;
+            $_etype1006 = 0;
+            $xfer += $input->readListBegin($_etype1006, $_size1003);
+            for ($_i1007 = 0; $_i1007 < $_size1003; ++$_i1007)
             {
-              $elem980 = null;
-              $xfer += $input->readString($elem980);
-              $this->tbl_types []= $elem980;
+              $elem1008 = null;
+              $xfer += $input->readString($elem1008);
+              $this->tbl_types []= $elem1008;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22333,9 +22393,9 @@ class ThriftHiveMetastore_get_table_meta_args {
       {
         $output->writeListBegin(TType::STRING, count($this->tbl_types));
         {
-          foreach ($this->tbl_types as $iter981)
+          foreach ($this->tbl_types as $iter1009)
           {
-            $xfer += $output->writeString($iter981);
+            $xfer += $output->writeString($iter1009);
           }
         }
         $output->writeListEnd();
@@ -22412,15 +22472,15 @@ class ThriftHiveMetastore_get_table_meta_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size982 = 0;
-            $_etype985 = 0;
-            $xfer += $input->readListBegin($_etype985, $_size982);
-            for ($_i986 = 0; $_i986 < $_size982; ++$_i986)
+            $_size1010 = 0;
+            $_etype1013 = 0;
+            $xfer += $input->readListBegin($_etype1013, $_size1010);
+            for ($_i1014 = 0; $_i1014 < $_size1010; ++$_i1014)
             {
-              $elem987 = null;
-              $elem987 = new \metastore\TableMeta();
-              $xfer += $elem987->read($input);
-              $this->success []= $elem987;
+              $elem1015 = null;
+              $elem1015 = new \metastore\TableMeta();
+              $xfer += $elem1015->read($input);
+              $this->success []= $elem1015;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22456,9 +22516,9 @@ class ThriftHiveMetastore_get_table_meta_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter988)
+          foreach ($this->success as $iter1016)
           {
-            $xfer += $iter988->write($output);
+            $xfer += $iter1016->write($output);
           }
         }
         $output->writeListEnd();
@@ -22614,14 +22674,14 @@ class ThriftHiveMetastore_get_all_tables_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size989 = 0;
-            $_etype992 = 0;
-            $xfer += $input->readListBegin($_etype992, $_size989);
-            for ($_i993 = 0; $_i993 < $_size989; ++$_i993)
+            $_size1017 = 0;
+            $_etype1020 = 0;
+            $xfer += $input->readListBegin($_etype1020, $_size1017);
+            for ($_i1021 = 0; $_i1021 < $_size1017; ++$_i1021)
             {
-              $elem994 = null;
-              $xfer += $input->readString($elem994);
-              $this->success []= $elem994;
+              $elem1022 = null;
+              $xfer += $input->readString($elem1022);
+              $this->success []= $elem1022;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22657,9 +22717,9 @@ class ThriftHiveMetastore_get_all_tables_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter995)
+          foreach ($this->success as $iter1023)
           {
-            $xfer += $output->writeString($iter995);
+            $xfer += $output->writeString($iter1023);
           }
         }
         $output->writeListEnd();
@@ -22974,14 +23034,14 @@ class ThriftHiveMetastore_get_table_objects_by_name_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->tbl_names = array();
-            $_size996 = 0;
-            $_etype999 = 0;
-            $xfer += $input->readListBegin($_etype999, $_size996);
-            for ($_i1000 = 0; $_i1000 < $_size996; ++$_i1000)
+            $_size1024 = 0;
+            $_etype1027 = 0;
+            $xfer += $input->readListBegin($_etype1027, $_size1024);
+            for ($_i1028 = 0; $_i1028 < $_size1024; ++$_i1028)
             {
-              $elem1001 = null;
-              $xfer += $input->readString($elem1001);
-              $this->tbl_names []= $elem1001;
+              $elem1029 = null;
+              $xfer += $input->readString($elem1029);
+              $this->tbl_names []= $elem1029;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23014,9 +23074,9 @@ class ThriftHiveMetastore_get_table_objects_by_name_args {
       {
         $output->writeListBegin(TType::STRING, count($this->tbl_names));
         {
-          foreach ($this->tbl_names as $iter1002)
+          foreach ($this->tbl_names as $iter1030)
           {
-            $xfer += $output->writeString($iter1002);
+            $xfer += $output->writeString($iter1030);
           }
         }
         $output->writeListEnd();
@@ -23081,15 +23141,15 @@ class ThriftHiveMetastore_get_table_objects_by_name_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1003 = 0;
-            $_etype1006 = 0;
-            $xfer += $input->readListBegin($_etype1006, $_size1003);
-            for ($_i1007 = 0; $_i1007 < $_size1003; ++$_i1007)
+            $_size1031 = 0;
+            $_etype1034 = 0;
+            $xfer += $input->readListBegin($_etype1034, $_size1031);
+            for ($_i1035 = 0; $_i1035 < $_size1031; ++$_i1035)
             {
-              $elem1008 = null;
-              $elem1008 = new \metastore\Table();
-              $xfer += $elem1008->read($input);
-              $this->success []= $elem1008;
+              $elem1036 = null;
+              $elem1036 = new \metastore\Table();
+              $xfer += $elem1036->read($input);
+              $this->success []= $elem1036;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23117,9 +23177,9 @@ class ThriftHiveMetastore_get_table_objects_by_name_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1009)
+          foreach ($this->success as $iter1037)
           {
-            $xfer += $iter1009->write($output);
+            $xfer += $iter1037->write($output);
           }
         }
         $output->writeListEnd();
@@ -24319,14 +24379,14 @@ class ThriftHiveMetastore_get_table_names_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1010 = 0;
-            $_etype1013 = 0;
-            $xfer += $input->readListBegin($_etype1013, $_size1010);
-            for ($_i1014 = 0; $_i1014 < $_size1010; ++$_i1014)
+            $_size1038 = 0;
+            $_etype1041 = 0;
+            $xfer += $input->readListBegin($_etype1041, $_size1038);
+            for ($_i1042 = 0; $_i1042 < $_size1038; ++$_i1042)
             {
-              $elem1015 = null;
-              $xfer += $input->readString($elem1015);
-              $this->success []= $elem1015;
+              $elem1043 = null;
+              $xfer += $input->readString($elem1043);
+              $this->success []= $elem1043;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24378,9 +24438,9 @@ class ThriftHiveMetastore_get_table_names_by_filter_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1016)
+          foreach ($this->success as $iter1044)
           {
-            $xfer += $output->writeString($iter1016);
+            $xfer += $output->writeString($iter1044);
           }
         }
         $output->writeListEnd();
@@ -25903,15 +25963,15 @@ class ThriftHiveMetastore_add_partitions_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size1017 = 0;
-            $_etype1020 = 0;
-            $xfer += $input->readListBegin($_etype1020, $_size1017);
-            for ($_i1021 = 0; $_i1021 < $_size1017; ++$_i1021)
+            $_size1045 = 0;
+            $_etype1048 = 0;
+            $xfer += $input->readListBegin($_etype1048, $_size1045);
+            for ($_i1049 = 0; $_i1049 < $_size1045; ++$_i1049)
             {
-              $elem1022 = null;
-              $elem1022 = new \metastore\Partition();
-              $xfer += $elem1022->read($input);
-              $this->new_parts []= $elem1022;
+              $elem1050 = null;
+              $elem1050 = new \metastore\Partition();
+              $xfer += $elem1050->read($input);
+              $this->new_parts []= $elem1050;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25939,9 +25999,9 @@ class ThriftHiveMetastore_add_partitions_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter1023)
+          foreach ($this->new_parts as $iter1051)
           {
-            $xfer += $iter1023->write($output);
+            $xfer += $iter1051->write($output);
           }
         }
         $output->writeListEnd();
@@ -26156,15 +26216,15 @@ class ThriftHiveMetastore_add_partitions_pspec_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size1024 = 0;
-            $_etype1027 = 0;
-            $xfer += $input->readListBegin($_etype1027, $_size1024);
-            for ($_i1028 = 0; $_i1028 < $_size1024; ++$_i1028)
+            $_size1052 = 0;
+            $_etype1055 = 0;
+            $xfer += $input->readListBegin($_etype1055, $_size1052);
+            for ($_i1056 = 0; $_i1056 < $_size1052; ++$_i1056)
             {
-              $elem1029 = null;
-              $elem1029 = new \metastore\PartitionSpec();
-              $xfer += $elem1029->read($input);
-              $this->new_parts []= $elem1029;
+              $elem1057 = null;
+              $elem1057 = new \metastore\PartitionSpec();
+              $xfer += $elem1057->read($input);
+              $this->new_parts []= $elem1057;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -26192,9 +26252,9 @@ class ThriftHiveMetastore_add_partitions_pspec_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter1030)
+          foreach ($this->new_parts as $iter1058)
           {
-            $xfer += $iter1030->write($output);
+            $xfer += $iter1058->write($output);
           }
         }
         $output->writeListEnd();
@@ -26444,14 +26504,14 @@ class ThriftHiveMetastore_append_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1031 = 0;
-            $_etype1034 = 0;
-            $xfer += $input->readListBegin($_etype1034, $_size1031);
-            for ($_i1035 = 0; $_i1035 < $_size1031; ++$_i1035)
+            $_size1059 = 0;
+            $_etype1062 = 0;
+            $xfer += $input->readListBegin($_etype1062, $_size1059);
+            for ($_i1063 = 0; $_i1063 < $_size1059; ++$_i1063)
             {
-              $elem1036 = null;
-              $xfer += $input->readString($elem1036);
-              $this->part_vals []= $elem1036;
+              $elem1064 = null;
+              $xfer += $input->readString($elem1064);
+              $this->part_vals []= $elem1064;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -26489,9 +26549,9 @@ class ThriftHiveMetastore_append_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1037)
+          foreach ($this->part_vals as $iter1065)
           {
-            $xfer += $output->writeString($iter1037);
+            $xfer += $output->writeString($iter1065);
           }
         }
         $output->writeListEnd();
@@ -26993,14 +27053,14 @@ class ThriftHiveMetastore_append_partition_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1038 = 0;
-            $_etype1041 = 0;
-            $xfer += $input->readListBegin($_etype1041, $_size1038);
-            for ($_i1042 = 0; $_i1042 < $_size1038; ++$_i1042)
+            $_size1066 = 0;
+            $_etype1069 = 0;
+            $xfer += $input->readListBegin($_etype1069, $_size1066);
+            for ($_i1070 = 0; $_i1070 < $_size1066; ++$_i1070)
             {
-              $elem1043 = null;
-              $xfer += $input->readString($elem1043);
-              $this->part_vals []= $elem1043;
+              $elem1071 = null;
+              $xfer += $input->readString($elem1071);
+              $this->part_vals []= $elem1071;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -27046,9 +27106,9 @@ class ThriftHiveMetastore_append_partition_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1044)
+          foreach ($this->part_vals as $iter1072)
           {
-            $xfer += $output->writeString($iter1044);
+            $xfer += $output->writeString($iter1072);
           }
         }
         $output->writeListEnd();
@@ -27902,14 +27962,14 @@ class ThriftHiveMetastore_drop_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1045 = 0;
-            $_etype1048 = 0;
-            $xfer += $input->readListBegin($_etype1048, $_size1045);
-            for ($_i1049 = 0; $_i1049 < $_size1045; ++$_i1049)
+            $_size1073 = 0;
+            $_etype1076 = 0;
+            $xfer += $input->readListBegin($_etype1076, $_size1073);
+            for ($_i1077 = 0; $_i1077 < $_size1073; ++$_i1077)
             {
-              $elem1050 = null;
-              $xfer += $input->readString($elem1050);
-              $this->part_vals []= $elem1050;
+              $elem1078 = null;
+              $xfer += $input->readString($elem1078);
+              $this->part_vals []= $elem1078;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -27954,9 +28014,9 @@ class ThriftHiveMetastore_drop_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1051)
+          foreach ($this->part_vals as $iter1079)
           {
-            $xfer += $output->writeString($iter1051);
+            $xfer += $output->writeString($iter1079);
           }
         }
         $output->writeListEnd();
@@ -28209,14 +28269,14 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1052 = 0;
-            $_etype1055 = 0;
-            $xfer += $input->readListBegin($_etype1055, $_size1052);
-            for ($_i1056 = 0; $_i1056 < $_size1052; ++$_i1056)
+            $_size1080 = 0;
+            $_etype1083 = 0;
+            $xfer += $input->readListBegin($_etype1083, $_size1080);
+            for ($_i1084 = 0; $_i1084 < $_size1080; ++$_i1084)
             {
-              $elem1057 = null;
-              $xfer += $input->readString($elem1057);
-              $this->part_vals []= $elem1057;
+              $elem1085 = null;
+              $xfer += $input->readString($elem1085);
+              $this->part_vals []= $elem1085;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -28269,9 +28329,9 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1058)
+          foreach ($this->part_vals as $iter1086)
           {
-            $xfer += $output->writeString($iter1058);
+            $xfer += $output->writeString($iter1086);
           }
         }
         $output->writeListEnd();
@@ -29285,14 +29345,14 @@ class ThriftHiveMetastore_get_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1059 = 0;
-            $_etype1062 = 0;
-            $xfer += $input->readListBegin($_etype1062, $_size1059);
-            for ($_i1063 = 0; $_i1063 < $_size1059; ++$_i1063)
+            $_size1087 = 0;
+            $_etype1090 = 0;
+            $xfer += $input->readListBegin($_etype1090, $_size1087);
+            for ($_i1091 = 0; $_i1091 < $_size1087; ++$_i1091)
             {
-              $elem1064 = null;
-              $xfer += $input->readString($elem1064);
-              $this->part_vals []= $elem1064;
+              $elem1092 = null;
+              $xfer += $input->readString($elem1092);
+              $this->part_vals []= $elem1092;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -29330,9 +29390,9 @@ class ThriftHiveMetastore_get_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1065)
+          foreach ($this->part_vals as $iter1093)
           {
-            $xfer += $output->writeString($iter1065);
+            $xfer += $output->writeString($iter1093);
           }
         }
         $output->writeListEnd();
@@ -29574,17 +29634,17 @@ class ThriftHiveMetastore_exchange_partition_args {
         case 1:
           if ($ftype == TType::MAP) {
             $this->partitionSpecs = array();
-            $_size1066 = 0;
-            $_ktype1067 = 0;
-            $_vtype1068 = 0;
-            $xfer += $input->readMapBegin($_ktype1067, $_vtype1068, $_size1066);
-            for ($_i1070 = 0; $_i1070 < $_size1066; ++$_i1070)
+            $_size1094 = 0;
+            $_ktype1095 = 0;
+            $_vtype1096 = 0;
+            $xfer += $input->readMapBegin($_ktype1095, $_vtype1096, $_size1094);
+            for ($_i1098 = 0; $_i1098 < $_size1094; ++$_i1098)
             {
-              $key1071 = '';
-              $val1072 = '';
-              $xfer += $input->readString($key1071);
-              $xfer += $input->readString($val1072);
-              $this->partitionSpecs[$key1071] = $val1072;
+              $key1099 = '';
+              $val1100 = '';
+              $xfer += $input->readString($key1099);
+              $xfer += $input->readString($val1100);
+              $this->partitionSpecs[$key1099] = $val1100;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -29640,10 +29700,10 @@ class ThriftHiveMetastore_exchange_partition_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs));
         {
-          foreach ($this->partitionSpecs as $kiter1073 => $viter1074)
+          foreach ($this->partitionSpecs as $kiter1101 => $viter1102)
           {
-            $xfer += $output->writeString($kiter1073);
-            $xfer += $output->writeString($viter1074);
+            $xfer += $output->writeString($kiter1101);
+            $xfer += $output->writeString($viter1102);
           }
         }
         $output->writeMapEnd();
@@ -29955,17 +30015,17 @@ class ThriftHiveMetastore_exchange_partitions_args {
         case 1:
           if ($ftype == TType::MAP) {
             $this->partitionSpecs = array();
-            $_size1075 = 0;
-            $_ktype1076 = 0;
-            $_vtype1077 = 0;
-            $xfer += $input->readMapBegin($_ktype1076, $_vtype1077, $_size1075);
-            for ($_i1079 = 0; $_i1079 < $_size1075; ++$_i1079)
+            $_size1103 = 0;
+            $_ktype1104 = 0;
+            $_vtype1105 = 0;
+            $xfer += $input->readMapBegin($_ktype1104, $_vtype1105, $_size1103);
+            for ($_i1107 = 0; $_i1107 < $_size1103; ++$_i1107)
             {
-              $key1080 = '';
-              $val1081 = '';
-              $xfer += $input->readString($key1080);
-              $xfer += $input->readString($val1081);
-              $this->partitionSpecs[$key1080] = $val1081;
+              $key1108 = '';
+              $val1109 = '';
+              $xfer += $input->readString($key1108);
+              $xfer += $input->readString($val1109);
+              $this->partitionSpecs[$key1108] = $val1109;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -30021,10 +30081,10 @@ class ThriftHiveMetastore_exchange_partitions_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs));
         {
-          foreach ($this->partitionSpecs as $kiter1082 => $viter1083)
+          foreach ($this->partitionSpecs as $kiter1110 => $viter1111)
           {
-            $xfer += $output->writeString($kiter1082);
-            $xfer += $output->writeString($viter1083);
+            $xfer += $output->writeString($kiter1110);
+            $xfer += $output->writeString($viter1111);
           }
         }
         $output->writeMapEnd();
@@ -30157,15 +30217,15 @@ class ThriftHiveMetastore_exchange_partitions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1084 = 0;
-            $_etype1087 = 0;
-            $xfer += $input->readListBegin($_etype1087, $_size1084);
-            for ($_i1088 = 0; $_i1088 < $_size1084; ++$_i1088)
+            $_size1112 = 0;
+            $_etype1115 = 0;
+            $xfer += $input->readListBegin($_etype1115, $_size1112);
+            for ($_i1116 = 0; $_i1116 < $_size1112; ++$_i1116)
             {
-              $elem1089 = null;
-              $elem1089 = new \metastore\Partition();
-              $xfer += $elem1089->read($input);
-              $this->success []= $elem1089;
+              $elem1117 = null;
+              $elem1117 = new \metastore\Partition();
+              $xfer += $elem1117->read($input);
+              $this->success []= $elem1117;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -30225,9 +30285,9 @@ class ThriftHiveMetastore_exchange_partitions_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1090)
+          foreach ($this->success as $iter1118)
           {
-            $xfer += $iter1090->write($output);
+            $xfer += $iter1118->write($output);
           }
         }
         $output->writeListEnd();
@@ -30373,14 +30433,14 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1091 = 0;
-            $_etype1094 = 0;
-            $xfer += $input->readListBegin($_etype1094, $_size1091);
-            for ($_i1095 = 0; $_i1095 < $_size1091; ++$_i1095)
+            $_size1119 = 0;
+            $_etype1122 = 0;
+            $xfer += $input->readListBegin($_etype1122, $_size1119);
+            for ($_i1123 = 0; $_i1123 < $_size1119; ++$_i1123)
             {
-              $elem1096 = null;
-              $xfer += $input->readString($elem1096);
-              $this->part_vals []= $elem1096;
+              $elem1124 = null;
+              $xfer += $input->readString($elem1124);
+              $this->part_vals []= $elem1124;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -30397,14 +30457,14 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size1097 = 0;
-            $_etype1100 = 0;
-            $xfer += $input->readListBegin($_etype1100, $_size1097);
-            for ($_i1101 = 0; $_i1101 < $_size1097; ++$_i1101)
+            $_size1125 = 0;
+            $_etype1128 = 0;
+            $xfer += $input->readListBegin($_etype1128, $_size1125);
+            for ($_i1129 = 0; $_i1129 < $_size1125; ++$_i1129)
             {
-              $elem1102 = null;
-              $xfer += $input->readString($elem1102);
-              $this->group_names []= $elem1102;
+              $elem1130 = null;
+              $xfer += $input->readString($elem1130);
+              $this->group_names []= $elem1130;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -30442,9 +30502,9 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1103)
+          foreach ($this->part_vals as $iter1131)
           {
-            $xfer += $output->writeString($iter1103);
+            $xfer += $output->writeString($iter1131);
           }
         }
         $output->writeListEnd();
@@ -30464,9 +30524,9 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter1104)
+          foreach ($this->group_names as $iter1132)
           {
-            $xfer += $output->writeString($iter1104);
+            $xfer += $output->writeString($iter1132);
           }
         }
         $output->writeListEnd();
@@ -31057,15 +31117,15 @@ class ThriftHiveMetastore_get_partitions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1105 = 0;
-            $_etype1108 = 0;
-            $xfer += $input->readListBegin($_etype1108, $_size1105);
-            for ($_i1109 = 0; $_i1109 < $_size1105; ++$_i1109)
+            $_size1133 = 0;
+            $_etype1136 = 0;
+            $xfer += $input->readListBegin($_etype1136, $_size1133);
+            for ($_i1137 = 0; $_i1137 < $_size1133; ++$_i1137)
             {
-              $elem1110 = null;
-              $elem1110 = new \metastore\Partition();
-              $xfer += $elem1110->read($input);
-              $this->success []= $elem1110;
+              $elem1138 = null;
+              $elem1138 = new \metastore\Partition();
+              $xfer += $elem1138->read($input);
+              $this->success []= $elem1138;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -31109,9 +31169,9 @@ class ThriftHiveMetastore_get_partitions_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1111)
+          foreach ($this->success as $iter1139)
           {
-            $xfer += $iter1111->write($output);
+            $xfer += $iter1139->write($output);
           }
         }
         $output->writeListEnd();
@@ -31257,14 +31317,14 @@ class ThriftHiveMetastore_get_partitions_with_auth_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size1112 = 0;
-            $_etype1115 = 0;
-            $xfer += $input->readListBegin($_etype1115, $_size1112);
-            for ($_i1116 = 0; $_i1116 < $_size1112; ++$_i1116)
+            $_size1140 = 0;
+            $_etype1143 = 0;
+            $xfer += $input->readListBegin($_etype1143, $_size1140);
+            for ($_i1144 = 0; $_i1144 < $_size1140; ++$_i1144)
             {
-              $elem1117 = null;
-              $xfer += $input->readString($elem1117);
-              $this->group_names []= $elem1117;
+              $elem1145 = null;
+              $xfer += $input->readString($elem1145);
+              $this->group_names []= $elem1145;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -31312,9 +31372,9 @@ class ThriftHiveMetastore_get_partitions_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter1118)
+          foreach ($this->group_names as $iter1146)
           {
-            $xfer += $output->writeString($iter1118);
+            $xfer += $output->writeString($iter1146);
           }
         }
         $output->writeListEnd();
@@ -31403,15 +31463,15 @@ class ThriftHiveMetastore_get_partitions_with_auth_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1119 = 0;
-            $_etype1122 = 0;
-            $xfer += $input->readListBegin($_etype1122, $_size1119);
-            for ($_i1123 = 0; $_i1123 < $_size1119; ++$_i1123)
+            $_size1147 = 0;
+            $_etype1150 = 0;
+            $xfer += $input->readListBegin($_etype1150, $_size1147);
+            for ($_i1151 = 0; $_i1151 < $_size1147; ++$_i1151)
             {
-              $elem1124 = null;
-              $elem1124 = new \metastore\Partition();
-              $xfer += $elem1124->read($input);
-              $this->success []= $elem1124;
+              $elem1152 = null;
+              $elem1152 = new \metastore\Partition();
+              $xfer += $elem1152->read($input);
+              $this->success []= $elem1152;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -31455,9 +31515,9 @@ class ThriftHiveMetastore_get_partitions_with_auth_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1125)
+          foreach ($this->success as $iter1153)
           {
-            $xfer += $iter1125->write($output);
+            $xfer += $iter1153->write($output);
           }
         }
         $output->writeListEnd();
@@ -31677,15 +31737,15 @@ class ThriftHiveMetastore_get_partitions_pspec_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1126 = 0;
-            $_etype1129 = 0;
-            $xfer += $input->readListBegin($_etype1129, $_size1126);
-            for ($_i1130 = 0; $_i1130 < $_size1126; ++$_i1130)
+            $_size1154 = 0;
+            $_etype1157 = 0;
+            $xfer += $input->readListBegin($_etype1157, $_size1154);
+            for ($_i1158 = 0; $_i1158 < $_size1154; ++$_i1158)
             {
-              $elem1131 = null;
-              $elem1131 = new \metastore\PartitionSpec();
-              $xfer += $elem1131->read($input);
-              $this->success []= $elem1131;
+              $elem1159 = null;
+              $elem1159 = new \metastore\PartitionSpec();
+              $xfer += $elem1159->read($input);
+              $this->success []= $elem1159;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -31729,9 +31789,9 @@ class ThriftHiveMetastore_get_partitions_pspec_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1132)
+          foreach ($this->success as $iter1160)
           {
-            $xfer += $iter1132->write($output);
+            $xfer += $iter1160->write($output);
           }
         }
         $output->writeListEnd();
@@ -31950,14 +32010,14 @@ class ThriftHiveMetastore_get_partition_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1133 = 0;
-            $_etype1136 = 0;
-            $xfer += $input->readListBegin($_etype1136, $_size1133);
-            for ($_i1137 = 0; $_i1137 < $_size1133; ++$_i1137)
+            $_size1161 = 0;
+            $_etype1164 = 0;
+            $xfer += $input->readListBegin($_etype1164, $_size1161);
+            for ($_i1165 = 0; $_i1165 < $_size1161; ++$_i1165)
             {
-              $elem1138 = null;
-              $xfer += $input->readString($elem1138);
-              $this->success []= $elem1138;
+              $elem1166 = null;
+              $xfer += $input->readString($elem1166);
+              $this->success []= $elem1166;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -32001,9 +32061,9 @@ class ThriftHiveMetastore_get_partition_names_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1139)
+          foreach ($this->success as $iter1167)
           {
-            $xfer += $output->writeString($iter1139);
+            $xfer += $output->writeString($iter1167);
           }
         }
         $output->writeListEnd();
@@ -32334,14 +32394,14 @@ class ThriftHiveMetastore_get_partitions_ps_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1140 = 0;
-            $_etype1143 = 0;
-            $xfer += $input->readListBegin($_etype1143, $_size1140);
-            for ($_i1144 = 0; $_i1144 < $_size1140; ++$_i1144)
+            $_size1168 = 0;
+            $_etype1171 = 0;
+            $xfer += $input->readListBegin($_etype1171, $_size1168);
+            for ($_i1172 = 0; $_i1172 < $_size1168; ++$_i1172)
             {
-              $elem1145 = null;
-              $xfer += $input->readString($elem1145);
-              $this->part_vals []= $elem1145;
+              $elem1173 = null;
+              $xfer += $input->readString($elem1173);
+              $this->part_vals []= $elem1173;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -32386,9 +32446,9 @@ class ThriftHiveMetastore_get_partitions_ps_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1146)
+          foreach ($this->part_vals as $iter1174)
           {
-            $xfer += $output->writeString($iter1146);
+            $xfer += $output->writeString($iter1174);
           }
         }
         $output->writeListEnd();
@@ -32482,15 +32542,15 @@ class ThriftHiveMetastore_get_partitions_ps_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1147 = 0;
-            $_etype1150 = 0;
-            $xfer += $input->readListBegin($_etype1150, $_size1147);
-            for ($_i1151 = 0; $_i1151 < $_size1147; ++$_i1151)
+            $_size1175 = 0;
+            $_etype1178 = 0;
+            $xfer += $input->readListBegin($_etype1178, $_size1175);
+            for ($_i1179 = 0; $_i1179 < $_size1175; ++$_i1179)
             {
-              $elem1152 = null;
-              $elem1152 = new \metastore\Partition();
-              $xfer += $elem1152->read($input);
-              $this->success []= $elem1152;
+              $elem1180 = null;
+              $elem1180 = new \metastore\Partition();
+              $xfer += $elem1180->read($input);
+              $this->success []= $elem1180;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -32534,9 +32594,9 @@ class ThriftHiveMetastore_get_partitions_ps_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1153)
+          foreach ($this->success as $iter1181)
           {
-            $xfer += $iter1153->write($output);
+            $xfer += $iter1181->write($output);
           }
         }
         $output->writeListEnd();
@@ -32683,14 +32743,14 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1154 = 0;
-            $_etype1157 = 0;
-            $xfer += $input->readListBegin($_etype1157, $_size1154);
-            for ($_i1158 = 0; $_i1158 < $_size1154; ++$_i1158)
+            $_size1182 = 0;
+            $_etype1185 = 0;
+            $xfer += $input->readListBegin($_etype1185, $_size1182);
+            for ($_i1186 = 0; $_i1186 < $_size1182; ++$_i1186)
             {
-              $elem1159 = null;
-              $xfer += $input->readString($elem1159);
-              $this->part_vals []= $elem1159;
+              $elem1187 = null;
+              $xfer += $input->readString($elem1187);
+              $this->part_vals []= $elem1187;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -32714,14 +32774,14 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
         case 6:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size1160 = 0;
-            $_etype1163 = 0;
-            $xfer += $input->readListBegin($_etype1163, $_size1160);
-            for ($_i1164 = 0; $_i1164 < $_size1160; ++$_i1164)
+            $_size1188 = 0;
+            $_etype1191 = 0;
+            $xfer += $input->readListBegin($_etype1191, $_size1188);
+            for ($_i1192 = 0; $_i1192 < $_size1188; ++$_i1192)
             {
-              $elem1165 = null;
-              $xfer += $input->readString($elem1165);
-              $this->group_names []= $elem1165;
+              $elem1193 = null;
+              $xfer += $input->readString($elem1193);
+              $this->group_names []= $elem1193;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -32759,9 +32819,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1166)
+          foreach ($this->part_vals as $iter1194)
           {
-            $xfer += $output->writeString($iter1166);
+            $xfer += $output->writeString($iter1194);
           }
         }
         $output->writeListEnd();
@@ -32786,9 +32846,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter1167)
+          foreach ($this->group_names as $iter1195)
           {
-            $xfer += $output->writeString($iter1167);
+            $xfer += $output->writeString($iter1195);
           }
         }
         $output->writeListEnd();
@@ -32877,15 +32937,15 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1168 = 0;
-            $_etype1171 = 0;
-            $xfer += $input->readListBegin($_etype1171, $_size1168);
-            for ($_i1172 = 0; $_i1172 < $_size1168; ++$_i1172)
+            $_size1196 = 0;
+            $_etype1199 = 0;
+            $xfer += $input->readListBegin($_etype1199, $_size1196);
+            for ($_i1200 = 0; $_i1200 < $_size1196; ++$_i1200)
             {
-              $elem1173 = null;
-              $elem1173 = new \metastore\Partition();
-              $xfer += $elem1173->read($input);
-              $this->success []= $elem1173;
+              $elem1201 = null;
+              $elem1201 = new \metastore\Partition();
+              $xfer += $elem1201->read($input);
+              $this->success []= $elem1201;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -32929,9 +32989,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1174)
+          foreach ($this->success as $iter1202)
           {
-            $xfer += $iter1174->write($output);
+            $xfer += $iter1202->write($output);
           }
         }
         $output->writeListEnd();
@@ -33052,14 +33112,14 @@ class ThriftHiveMetastore_get_partition_names_ps_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1175 = 0;
-            $_etype1178 = 0;
-            $xfer += $input->readListBegin($_etype1178, $_size1175);
-            for ($_i1179 = 0; $_i1179 < $_size1175; ++$_i1179)
+            $_size1203 = 0;
+            $_etype1206 = 0;
+            $xfer += $input->readListBegin($_etype1206, $_size1203);
+            for ($_i1207 = 0; $_i1207 < $_size1203; ++$_i1207)
             {
-              $elem1180 = null;
-              $xfer += $input->readString($elem1180);
-              $this->part_vals []= $elem1180;
+              $elem1208 = null;
+              $xfer += $input->readString($elem1208);
+              $this->part_vals []= $elem1208;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -33104,9 +33164,9 @@ class ThriftHiveMetastore_get_partition_names_ps_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1181)
+          foreach ($this->part_vals as $iter1209)
           {
-            $xfer += $output->writeString($iter1181);
+            $xfer += $output->writeString($iter1209);
           }
         }
         $output->writeListEnd();
@@ -33199,14 +33259,14 @@ class ThriftHiveMetastore_get_partition_names_ps_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1182 = 0;
-            $_etype1185 = 0;
-            $xfer += $input->readListBegin($_etype1185, $_size1182);
-            for ($_i1186 = 0; $_i1186 < $_size1182; ++$_i1186)
+            $_size1210 = 0;
+            $_etype1213 = 0;
+            $xfer += $input->readListBegin($_etype1213, $_size1210);
+            for ($_i1214 = 0; $_i1214 < $_size1210; ++$_i1214)
             {
-              $elem1187 = null;
-              $xfer += $input->readString($elem1187);
-              $this->success []= $elem1187;
+              $elem1215 = null;
+              $xfer += $input->readString($elem1215);
+              $this->success []= $elem1215;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -33250,9 +33310,9 @@ class ThriftHiveMetastore_get_partition_names_ps_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1188)
+          foreach ($this->success as $iter1216)
           {
-            $xfer += $output->writeString($iter1188);
+            $xfer += $output->writeString($iter1216);
           }
         }
         $output->writeListEnd();
@@ -33495,15 +33555,15 @@ class ThriftHiveMetastore_get_partitions_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1189 = 0;
-            $_etype1192 = 0;
-            $xfer += $input->readListBegin($_etype1192, $_size1189);
-            for ($_i1193 = 0; $_i1193 < $_size1189; ++$_i1193)
+            $_size1217 = 0;
+            $_etype1220 = 0;
+            $xfer += $input->readListBegin($_etype1220, $_size1217);
+            for ($_i1221 = 0; $_i1221 < $_size1217; ++$_i1221)
             {
-              $elem1194 = null;
-              $elem1194 = new \metastore\Partition();
-              $xfer += $elem1194->read($input);
-              $this->success []= $elem1194;
+              $elem1222 = null;
+              $elem1222 = new \metastore\Partition();
+              $xfer += $elem1222->read($input);
+              $this->success []= $elem1222;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -33547,9 +33607,9 @@ class ThriftHiveMetastore_get_partitions_by_filter_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1195)
+          foreach ($this->success as $iter1223)
           {
-            $xfer += $iter1195->write($output);
+            $xfer += $iter1223->write($output);
           }
         }
         $output->writeListEnd();
@@ -33792,15 +33852,15 @@ class ThriftHiveMetastore_get_part_specs_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1196 = 0;
-            $_etype1199 = 0;
-            $xfer += $input->readListBegin($_etype1199, $_size1196);
-            for ($_i1200 = 0; $_i1200 < $_size1196; ++$_i1200)
+            $_size1224 = 0;
+            $_etype1227 = 0;
+            $xfer += $input->readListBegin($_etype1227, $_size1224);
+            for ($_i1228 = 0; $_i1228 < $_size1224; ++$_i1228)
             {
-              $elem1201 = null;
-              $elem1201 = new \metastore\PartitionSpec();
-              $xfer += $elem1201->read($input);
-              $this->success []= $elem1201;
+              $elem1229 = null;
+              $elem1229 = new \metastore\PartitionSpec();
+              $xfer += $elem1229->read($input);
+              $this->success []= $elem1229;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -33844,9 +33904,9 @@ class ThriftHiveMetastore_get_part_specs_by_filter_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1202)
+          foreach ($this->success as $iter1230)
           {
-            $xfer += $iter1202->write($output);
+            $xfer += $iter1230->write($output);
           }
         }
         $output->writeListEnd();
@@ -34412,14 +34472,14 @@ class ThriftHiveMetastore_get_partitions_by_names_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->names = array();
-            $_size1203 = 0;
-            $_etype1206 = 0;
-            $xfer += $input->readListBegin($_etype1206, $_size1203);
-            for ($_i1207 = 0; $_i1207 < $_size1203; ++$_i1207)
+            $_size1231 = 0;
+            $_etype1234 = 0;
+            $xfer += $input->readListBegin($_etype1234, $_size1231);
+            for ($_i1235 = 0; $_i1235 < $_size1231; ++$_i1235)
             {
-              $elem1208 = null;
-              $xfer += $input->readString($elem1208);
-              $this->names []= $elem1208;
+              $elem1236 = null;
+              $xfer += $input->readString($elem1236);
+              $this->names []= $elem1236;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -34457,9 +34517,9 @@ class ThriftHiveMetastore_get_partitions_by_names_args {
       {
         $output->writeListBegin(TType::STRING, count($this->names));
         {
-          foreach ($this->names as $iter1209)
+          foreach ($this->names as $iter1237)
           {
-            $xfer += $output->writeString($iter1209);
+            $xfer += $output->writeString($iter1237);
           }
         }
         $output->writeListEnd();
@@ -34548,15 +34608,15 @@ class ThriftHiveMetastore_get_partitions_by_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1210 = 0;
-            $_etype1213 = 0;
-            $xfer += $input->readListBegin($_etype1213, $_size1210);
-            for ($_i1214 = 0; $_i1214 < $_size1210; ++$_i1214)
+            $_size1238 = 0;
+            $_etype1241 = 0;
+            $xfer += $input->readListBegin($_etype1241, $_size1238);
+            for ($_i1242 = 0; $_i1242 < $_size1238; ++$_i1242)
             {
-              $elem1215 = null;
-              $elem1215 = new \metastore\Partition();
-              $xfer += $elem1215->read($input);
-              $this->success []= $elem1215;
+              $elem1243 = null;
+              $elem1243 = new \metastore\Partition();
+              $xfer += $elem1243->read($input);
+              $this->success []= $elem1243;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -34600,9 +34660,9 @@ class ThriftHiveMetastore_get_partitions_by_names_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1216)
+          foreach ($this->success as $iter1244)
           {
-            $xfer += $iter1216->write($output);
+            $xfer += $iter1244->write($output);
           }
         }
         $output->writeListEnd();
@@ -34941,15 +35001,15 @@ class ThriftHiveMetastore_alter_partitions_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size1217 = 0;
-            $_etype1220 = 0;
-            $xfer += $input->readListBegin($_etype1220, $_size1217);
-            for ($_i1221 = 0; $_i1221 < $_size1217; ++$_i1221)
+            $_size1245 = 0;
+            $_etype1248 = 0;
+            $xfer += $input->readListBegin($_etype1248, $_size1245);
+            for ($_i1249 = 0; $_i1249 < $_size1245; ++$_i1249)
             {
-              $elem1222 = null;
-              $elem1222 = new \metastore\Partition();
-              $xfer += $elem1222->read($input);
-              $this->new_parts []= $elem1222;
+              $elem1250 = null;
+              $elem1250 = new \metastore\Partition();
+              $xfer += $elem1250->read($input);
+              $this->new_parts []= $elem1250;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -34987,9 +35047,9 @@ class ThriftHiveMetastore_alter_partitions_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter1223)
+          foreach ($this->new_parts as $iter1251)
           {
-            $xfer += $iter1223->write($output);
+            $xfer += $iter1251->write($output);
           }
         }
         $output->writeListEnd();
@@ -35204,15 +35264,15 @@ class ThriftHiveMetastore_alter_partitions_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size1224 = 0;
-            $_etype1227 = 0;
-            $xfer += $input->readListBegin($_etype1227, $_size1224);
-            for ($_i1228 = 0; $_i1228 < $_size1224; ++$_i1228)
+            $_size1252 = 0;
+            $_etype1255 = 0;
+            $xfer += $input->readListBegin($_etype1255, $_size1252);
+            for ($_i1256 = 0; $_i1256 < $_size1252; ++$_i1256)
             {
-              $elem1229 = null;
-              $elem1229 = new \metastore\Partition();
-              $xfer += $elem1229->read($input);
-              $this->new_parts []= $elem1229;
+              $elem1257 = null;
+              $elem1257 = new \metastore\Partition();
+              $xfer += $elem1257->read($input);
+              $this->new_parts []= $elem1257;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -35258,9 +35318,9 @@ class ThriftHiveMetastore_alter_partitions_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter1230)
+          foreach ($this->new_parts as $iter1258)
           {
-            $xfer += $iter1230->write($output);
+            $xfer += $iter1258->write($output);
           }
         }
         $output->writeListEnd();
@@ -35948,14 +36008,14 @@ class ThriftHiveMetastore_rename_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1231 = 0;
-            $_etype1234 = 0;
-            $xfer += $input->readListBegin($_etype1234, $_size1231);
-            for ($_i1235 = 0; $_i1235 < $_size1231; ++$_i1235)
+            $_size1259 = 0;
+            $_etype1262 = 0;
+            $xfer += $input->readListBegin($_etype1262, $_size1259);
+            for ($_i1263 = 0; $_i1263 < $_size1259; ++$_i1263)
             {
-              $elem1236 = null;
-              $xfer += $input->readString($elem1236);
-              $this->part_vals []= $elem1236;
+              $elem1264 = null;
+              $xfer += $input->readString($elem1264);
+              $this->part_vals []= $elem1264;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -36001,9 +36061,9 @@ class ThriftHiveMetastore_rename_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1237)
+          foreach ($this->part_vals as $iter1265)
           {
-            $xfer += $output->writeString($iter1237);
+            $xfer += $output->writeString($iter1265);
           }
         }
         $output->writeListEnd();
@@ -36398,14 +36458,14 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1238 = 0;
-            $_etype1241 = 0;
-            $xfer += $input->readListBegin($_etype1241, $_size1238);
-            for ($_i1242 = 0; $_i1242 < $_size1238; ++$_i1242)
+            $_size1266 = 0;
+            $_etype1269 = 0;
+            $xfer += $input->readListBegin($_etype1269, $_size1266);
+            for ($_i1270 = 0; $_i1270 < $_size1266; ++$_i1270)
             {
-              $elem1243 = null;
-              $xfer += $input->readString($elem1243);
-              $this->part_vals []= $elem1243;
+              $elem1271 = null;
+              $xfer += $input->readString($elem1271);
+              $this->part_vals []= $elem1271;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -36440,9 +36500,9 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1244)
+          foreach ($this->part_vals as $iter1272)
           {
-            $xfer += $output->writeString($iter1244);
+            $xfer += $output->writeString($iter1272);
           }
         }
         $output->writeListEnd();
@@ -36896,14 +36956,14 @@ class ThriftHiveMetastore_partition_name_to_vals_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1245 = 0;
-            $_etype1248 = 0;
-            $xfer += $input->readListBegin($_etype1248, $_size1245);
-            for ($_i1249 = 0; $_i1249 < $_size1245; ++$_i1249)
+            $_size1273 = 0;
+            $_etype1276 = 0;
+            $xfer += $input->readListBegin($_etype1276, $_size1273);
+            for ($_i1277 = 0; $_i1277 < $_size1273; ++$_i1277)
             {
-              $elem1250 = null;
-              $xfer += $input->readString($elem1250);
-              $this->success []= $elem1250;
+              $elem1278 = null;
+              $xfer += $input->readString($elem1278);
+              $this->success []= $elem1278;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -36939,9 +36999,9 @@ class ThriftHiveMetastore_partition_name_to_vals_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1251)
+          foreach ($this->success as $iter1279)
           {
-            $xfer += $output->writeString($iter1251);
+            $xfer += $output->writeString($iter1279);
           }
         }
         $output->writeListEnd();
@@ -37101,17 +37161,17 @@ class ThriftHiveMetastore_partition_name_to_spec_result {
         case 0:
           if ($ftype == TType::MAP) {
             $this->success = array();
-            $_size1252 = 0;
-            $_ktype1253 = 0;
-            $_vtype1254 = 0;
-            $xfer += $input->readMapBegin($_ktype1253, $_vtype1254, $_size1252);
-            for ($_i1256 = 0; $_i1256 < $_size1252; ++$_i1256)
+            $_size1280 = 0;
+            $_ktype1281 = 0;
+            $_vtype1282 = 0;
+            $xfer += $input->readMapBegin($_ktype1281, $_vtype1282, $_size1280);
+            for ($_i1284 = 0; $_i1284 < $_size1280; ++$_i1284)
             {
-              $key1257 = '';
-              $val1258 = '';
-              $xfer += $input->readString($key1257);
-              $xfer += $input->readString($val1258);
-              $this->success[$key1257] = $val1258;
+              $key1285 = '';
+              $val1286 = '';
+              $xfer += $input->readString($key1285);
+              $xfer += $input->readString($val1286);
+              $this->success[$key1285] = $val1286;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -37147,10 +37207,10 @@ class ThriftHiveMetastore_partition_name_to_spec_result {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->success));
         {
-          foreach ($this->success as $kiter1259 => $viter1260)
+          foreach ($this->success as $kiter1287 => $viter1288)
           {
-            $xfer += $output->writeString($kiter1259);
-            $xfer += $output->writeString($viter1260);
+            $xfer += $output->writeString($kiter1287);
+            $xfer += $output->writeString($viter1288);
           }
         }
         $output->writeMapEnd();
@@ -37270,17 +37330,17 @@ class ThriftHiveMetastore_markPartitionForEvent_args {
         case 3:
           if ($ftype == TType::MAP) {
             $this->part_vals = array();
-            $_size1261 = 0;
-            $_ktype1262 = 0;
-            $_vtype1263 = 0;
-            $xfer += $input->readMapBegin($_ktype1262, $_vtype1263, $_size1261);
-            for ($_i1265 = 0; $_i1265 < $_size1261; ++$_i1265)
+            $_size1289 = 0;
+            $_ktype1290 = 0;
+            $_vtype1291 = 0;
+            $xfer += $input->readMapBegin($_ktype1290, $_vtype1291, $_size1289);
+            for ($_i1293 = 0; $_i1293 < $_size1289; ++$_i1293)
             {
-              $key1266 = '';
-              $val1267 = '';
-              $xfer += $input->readString($key1266);
-              $xfer += $input->readString($val1267);
-              $this->part_vals[$key1266] = $val1267;
+              $key1294 = '';
+              $val1295 = '';
+              $xfer += $input->readString($key1294);
+              $xfer += $input->readString($val1295);
+              $this->part_vals[$key1294] = $val1295;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -37325,10 +37385,10 @@ class ThriftHiveMetastore_markPartitionForEvent_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $kiter1268 => $viter1269)
+          foreach ($this->part_vals as $kiter1296 => $viter1297)
           {
-            $xfer += $output->writeString($kiter1268);
-            $xfer += $output->writeString($viter1269);
+            $xfer += $output->writeString($kiter1296);
+            $xfer += $output->writeString($viter1297);
           }
         }
         $output->writeMapEnd();
@@ -37650,17 +37710,17 @@ class ThriftHiveMetastore_isPartitionMarkedForEvent_args {
         case 3:
           if ($ftype == TType::MAP) {
             $this->part_vals = array();
-            $_size1270 = 0;
-            $_ktype1271 = 0;
-            $_vtype1272 = 0;
-            $xfer += $input->readMapBegin($_ktype1271, $_vtype1272, $_size1270);
-            for ($_i1274 = 0; $_i1274 < $_size1270; ++$_i1274)
+            $_size1298 = 0;
+            $_ktype1299 = 0;
+            $_vtype1300 = 0;
+            $xfer += $input->readMapBegin($_ktype1299, $_vtype1300, $_size1298);
+            for ($_i1302 = 0; $_i1302 < $_size1298; ++$_i1302)
             {
-              $key1275 = '';
-              $val1276 = '';
-              $xfer += $input->readString($key1275);
-              $xfer += $input->readString($val1276);
-              $this->part_vals[$key1275] = $val1276;
+              $key1303 = '';
+              $val1304 = '';
+              $xfer += $input->readString($key1303);
+              $xfer += $input->readString($val1304);
+              $this->part_vals[$key1303] = $val1304;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -37705,10 +37765,10 @@ class ThriftHiveMetastore_isPartitionMarkedForEvent_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $kiter1277 => $vit

<TRUNCATED>

[07/13] hive git commit: HIVE-20306 : Implement projection spec for fetching only requested fields from partitions (Vihang Karajgaonkar, reviewed by Aihua Xu, Andrew Sherman and Alexander Kolbasov)

Posted by vi...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/44ef91a6/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py b/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py
index 958f13c..8678e19 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py
@@ -389,6 +389,23 @@ class WMPoolSchedulingPolicy:
     "FIFO": 2,
   }
 
+class PartitionFilterMode:
+  BY_NAMES = 0
+  BY_VALUES = 1
+  BY_EXPR = 2
+
+  _VALUES_TO_NAMES = {
+    0: "BY_NAMES",
+    1: "BY_VALUES",
+    2: "BY_EXPR",
+  }
+
+  _NAMES_TO_VALUES = {
+    "BY_NAMES": 0,
+    "BY_VALUES": 1,
+    "BY_EXPR": 2,
+  }
+
 
 class Version:
   """
@@ -23117,6 +23134,437 @@ class AlterTableResponse:
   def __ne__(self, other):
     return not (self == other)
 
+class GetPartitionsProjectionSpec:
+  """
+  Attributes:
+   - fieldList
+   - includeParamKeyPattern
+   - excludeParamKeyPattern
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.LIST, 'fieldList', (TType.STRING,None), None, ), # 1
+    (2, TType.STRING, 'includeParamKeyPattern', None, None, ), # 2
+    (3, TType.STRING, 'excludeParamKeyPattern', None, None, ), # 3
+  )
+
+  def __init__(self, fieldList=None, includeParamKeyPattern=None, excludeParamKeyPattern=None,):
+    self.fieldList = fieldList
+    self.includeParamKeyPattern = includeParamKeyPattern
+    self.excludeParamKeyPattern = excludeParamKeyPattern
+
+  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.fieldList = []
+          (_etype854, _size851) = iprot.readListBegin()
+          for _i855 in xrange(_size851):
+            _elem856 = iprot.readString()
+            self.fieldList.append(_elem856)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRING:
+          self.includeParamKeyPattern = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.STRING:
+          self.excludeParamKeyPattern = 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('GetPartitionsProjectionSpec')
+    if self.fieldList is not None:
+      oprot.writeFieldBegin('fieldList', TType.LIST, 1)
+      oprot.writeListBegin(TType.STRING, len(self.fieldList))
+      for iter857 in self.fieldList:
+        oprot.writeString(iter857)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    if self.includeParamKeyPattern is not None:
+      oprot.writeFieldBegin('includeParamKeyPattern', TType.STRING, 2)
+      oprot.writeString(self.includeParamKeyPattern)
+      oprot.writeFieldEnd()
+    if self.excludeParamKeyPattern is not None:
+      oprot.writeFieldBegin('excludeParamKeyPattern', TType.STRING, 3)
+      oprot.writeString(self.excludeParamKeyPattern)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.fieldList)
+    value = (value * 31) ^ hash(self.includeParamKeyPattern)
+    value = (value * 31) ^ hash(self.excludeParamKeyPattern)
+    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 GetPartitionsFilterSpec:
+  """
+  Attributes:
+   - filterMode
+   - filters
+  """
+
+  thrift_spec = (
+    None, # 0
+    None, # 1
+    None, # 2
+    None, # 3
+    None, # 4
+    None, # 5
+    None, # 6
+    (7, TType.I32, 'filterMode', None, None, ), # 7
+    (8, TType.LIST, 'filters', (TType.STRING,None), None, ), # 8
+  )
+
+  def __init__(self, filterMode=None, filters=None,):
+    self.filterMode = filterMode
+    self.filters = filters
+
+  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 == 7:
+        if ftype == TType.I32:
+          self.filterMode = iprot.readI32()
+        else:
+          iprot.skip(ftype)
+      elif fid == 8:
+        if ftype == TType.LIST:
+          self.filters = []
+          (_etype861, _size858) = iprot.readListBegin()
+          for _i862 in xrange(_size858):
+            _elem863 = iprot.readString()
+            self.filters.append(_elem863)
+          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('GetPartitionsFilterSpec')
+    if self.filterMode is not None:
+      oprot.writeFieldBegin('filterMode', TType.I32, 7)
+      oprot.writeI32(self.filterMode)
+      oprot.writeFieldEnd()
+    if self.filters is not None:
+      oprot.writeFieldBegin('filters', TType.LIST, 8)
+      oprot.writeListBegin(TType.STRING, len(self.filters))
+      for iter864 in self.filters:
+        oprot.writeString(iter864)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.filterMode)
+    value = (value * 31) ^ hash(self.filters)
+    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 GetPartitionsResponse:
+  """
+  Attributes:
+   - partitionSpec
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.LIST, 'partitionSpec', (TType.STRUCT,(PartitionSpec, PartitionSpec.thrift_spec)), None, ), # 1
+  )
+
+  def __init__(self, partitionSpec=None,):
+    self.partitionSpec = partitionSpec
+
+  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.partitionSpec = []
+          (_etype868, _size865) = iprot.readListBegin()
+          for _i869 in xrange(_size865):
+            _elem870 = PartitionSpec()
+            _elem870.read(iprot)
+            self.partitionSpec.append(_elem870)
+          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('GetPartitionsResponse')
+    if self.partitionSpec is not None:
+      oprot.writeFieldBegin('partitionSpec', TType.LIST, 1)
+      oprot.writeListBegin(TType.STRUCT, len(self.partitionSpec))
+      for iter871 in self.partitionSpec:
+        iter871.write(oprot)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.partitionSpec)
+    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 GetPartitionsRequest:
+  """
+  Attributes:
+   - catName
+   - dbName
+   - tblName
+   - withAuth
+   - user
+   - groupNames
+   - projectionSpec
+   - filterSpec
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'catName', None, None, ), # 1
+    (2, TType.STRING, 'dbName', None, None, ), # 2
+    (3, TType.STRING, 'tblName', None, None, ), # 3
+    (4, TType.BOOL, 'withAuth', None, None, ), # 4
+    (5, TType.STRING, 'user', None, None, ), # 5
+    (6, TType.LIST, 'groupNames', (TType.STRING,None), None, ), # 6
+    (7, TType.STRUCT, 'projectionSpec', (GetPartitionsProjectionSpec, GetPartitionsProjectionSpec.thrift_spec), None, ), # 7
+    (8, TType.STRUCT, 'filterSpec', (GetPartitionsFilterSpec, GetPartitionsFilterSpec.thrift_spec), None, ), # 8
+  )
+
+  def __init__(self, catName=None, dbName=None, tblName=None, withAuth=None, user=None, groupNames=None, projectionSpec=None, filterSpec=None,):
+    self.catName = catName
+    self.dbName = dbName
+    self.tblName = tblName
+    self.withAuth = withAuth
+    self.user = user
+    self.groupNames = groupNames
+    self.projectionSpec = projectionSpec
+    self.filterSpec = filterSpec
+
+  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.catName = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRING:
+          self.dbName = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.STRING:
+          self.tblName = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.BOOL:
+          self.withAuth = iprot.readBool()
+        else:
+          iprot.skip(ftype)
+      elif fid == 5:
+        if ftype == TType.STRING:
+          self.user = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 6:
+        if ftype == TType.LIST:
+          self.groupNames = []
+          (_etype875, _size872) = iprot.readListBegin()
+          for _i876 in xrange(_size872):
+            _elem877 = iprot.readString()
+            self.groupNames.append(_elem877)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 7:
+        if ftype == TType.STRUCT:
+          self.projectionSpec = GetPartitionsProjectionSpec()
+          self.projectionSpec.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 8:
+        if ftype == TType.STRUCT:
+          self.filterSpec = GetPartitionsFilterSpec()
+          self.filterSpec.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('GetPartitionsRequest')
+    if self.catName is not None:
+      oprot.writeFieldBegin('catName', TType.STRING, 1)
+      oprot.writeString(self.catName)
+      oprot.writeFieldEnd()
+    if self.dbName is not None:
+      oprot.writeFieldBegin('dbName', TType.STRING, 2)
+      oprot.writeString(self.dbName)
+      oprot.writeFieldEnd()
+    if self.tblName is not None:
+      oprot.writeFieldBegin('tblName', TType.STRING, 3)
+      oprot.writeString(self.tblName)
+      oprot.writeFieldEnd()
+    if self.withAuth is not None:
+      oprot.writeFieldBegin('withAuth', TType.BOOL, 4)
+      oprot.writeBool(self.withAuth)
+      oprot.writeFieldEnd()
+    if self.user is not None:
+      oprot.writeFieldBegin('user', TType.STRING, 5)
+      oprot.writeString(self.user)
+      oprot.writeFieldEnd()
+    if self.groupNames is not None:
+      oprot.writeFieldBegin('groupNames', TType.LIST, 6)
+      oprot.writeListBegin(TType.STRING, len(self.groupNames))
+      for iter878 in self.groupNames:
+        oprot.writeString(iter878)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    if self.projectionSpec is not None:
+      oprot.writeFieldBegin('projectionSpec', TType.STRUCT, 7)
+      self.projectionSpec.write(oprot)
+      oprot.writeFieldEnd()
+    if self.filterSpec is not None:
+      oprot.writeFieldBegin('filterSpec', TType.STRUCT, 8)
+      self.filterSpec.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.catName)
+    value = (value * 31) ^ hash(self.dbName)
+    value = (value * 31) ^ hash(self.tblName)
+    value = (value * 31) ^ hash(self.withAuth)
+    value = (value * 31) ^ hash(self.user)
+    value = (value * 31) ^ hash(self.groupNames)
+    value = (value * 31) ^ hash(self.projectionSpec)
+    value = (value * 31) ^ hash(self.filterSpec)
+    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/44ef91a6/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb b/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb
index a3dddf5..ae6ffe6 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb
@@ -183,6 +183,14 @@ module WMPoolSchedulingPolicy
   VALID_VALUES = Set.new([FAIR, FIFO]).freeze
 end
 
+module PartitionFilterMode
+  BY_NAMES = 0
+  BY_VALUES = 1
+  BY_EXPR = 2
+  VALUE_MAP = {0 => "BY_NAMES", 1 => "BY_VALUES", 2 => "BY_EXPR"}
+  VALID_VALUES = Set.new([BY_NAMES, BY_VALUES, BY_EXPR]).freeze
+end
+
 class Version
   include ::Thrift::Struct, ::Thrift::Struct_Union
   VERSION = 1
@@ -5235,6 +5243,93 @@ class AlterTableResponse
   ::Thrift::Struct.generate_accessors self
 end
 
+class GetPartitionsProjectionSpec
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  FIELDLIST = 1
+  INCLUDEPARAMKEYPATTERN = 2
+  EXCLUDEPARAMKEYPATTERN = 3
+
+  FIELDS = {
+    FIELDLIST => {:type => ::Thrift::Types::LIST, :name => 'fieldList', :element => {:type => ::Thrift::Types::STRING}},
+    INCLUDEPARAMKEYPATTERN => {:type => ::Thrift::Types::STRING, :name => 'includeParamKeyPattern'},
+    EXCLUDEPARAMKEYPATTERN => {:type => ::Thrift::Types::STRING, :name => 'excludeParamKeyPattern'}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
+class GetPartitionsFilterSpec
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  FILTERMODE = 7
+  FILTERS = 8
+
+  FIELDS = {
+    FILTERMODE => {:type => ::Thrift::Types::I32, :name => 'filterMode', :optional => true, :enum_class => ::PartitionFilterMode},
+    FILTERS => {:type => ::Thrift::Types::LIST, :name => 'filters', :element => {:type => ::Thrift::Types::STRING}, :optional => true}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+    unless @filterMode.nil? || ::PartitionFilterMode::VALID_VALUES.include?(@filterMode)
+      raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Invalid value of field filterMode!')
+    end
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
+class GetPartitionsResponse
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  PARTITIONSPEC = 1
+
+  FIELDS = {
+    PARTITIONSPEC => {:type => ::Thrift::Types::LIST, :name => 'partitionSpec', :element => {:type => ::Thrift::Types::STRUCT, :class => ::PartitionSpec}}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
+class GetPartitionsRequest
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  CATNAME = 1
+  DBNAME = 2
+  TBLNAME = 3
+  WITHAUTH = 4
+  USER = 5
+  GROUPNAMES = 6
+  PROJECTIONSPEC = 7
+  FILTERSPEC = 8
+
+  FIELDS = {
+    CATNAME => {:type => ::Thrift::Types::STRING, :name => 'catName', :optional => true},
+    DBNAME => {:type => ::Thrift::Types::STRING, :name => 'dbName'},
+    TBLNAME => {:type => ::Thrift::Types::STRING, :name => 'tblName'},
+    WITHAUTH => {:type => ::Thrift::Types::BOOL, :name => 'withAuth', :optional => true},
+    USER => {:type => ::Thrift::Types::STRING, :name => 'user', :optional => true},
+    GROUPNAMES => {:type => ::Thrift::Types::LIST, :name => 'groupNames', :element => {:type => ::Thrift::Types::STRING}, :optional => true},
+    PROJECTIONSPEC => {:type => ::Thrift::Types::STRUCT, :name => 'projectionSpec', :class => ::GetPartitionsProjectionSpec},
+    FILTERSPEC => {:type => ::Thrift::Types::STRUCT, :name => 'filterSpec', :class => ::GetPartitionsFilterSpec}
+  }
+
+  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/44ef91a6/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/thrift_hive_metastore.rb b/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
index e6a7276..a5f976b 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
@@ -3576,6 +3576,22 @@ module ThriftHiveMetastore
       raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'get_runtime_stats failed: unknown result')
     end
 
+    def get_partitions_with_specs(request)
+      send_get_partitions_with_specs(request)
+      return recv_get_partitions_with_specs()
+    end
+
+    def send_get_partitions_with_specs(request)
+      send_message('get_partitions_with_specs', Get_partitions_with_specs_args, :request => request)
+    end
+
+    def recv_get_partitions_with_specs()
+      result = receive_message(Get_partitions_with_specs_result)
+      return result.success unless result.success.nil?
+      raise result.o1 unless result.o1.nil?
+      raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'get_partitions_with_specs failed: unknown result')
+    end
+
   end
 
   class Processor < ::FacebookService::Processor 
@@ -6263,6 +6279,17 @@ module ThriftHiveMetastore
       write_result(result, oprot, 'get_runtime_stats', seqid)
     end
 
+    def process_get_partitions_with_specs(seqid, iprot, oprot)
+      args = read_args(iprot, Get_partitions_with_specs_args)
+      result = Get_partitions_with_specs_result.new()
+      begin
+        result.success = @handler.get_partitions_with_specs(args.request)
+      rescue ::MetaException => o1
+        result.o1 = o1
+      end
+      write_result(result, oprot, 'get_partitions_with_specs', seqid)
+    end
+
   end
 
   # HELPER FUNCTIONS AND STRUCTURES
@@ -14186,5 +14213,39 @@ module ThriftHiveMetastore
     ::Thrift::Struct.generate_accessors self
   end
 
+  class Get_partitions_with_specs_args
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    REQUEST = 1
+
+    FIELDS = {
+      REQUEST => {:type => ::Thrift::Types::STRUCT, :name => 'request', :class => ::GetPartitionsRequest}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Get_partitions_with_specs_result
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    SUCCESS = 0
+    O1 = 1
+
+    FIELDS = {
+      SUCCESS => {:type => ::Thrift::Types::STRUCT, :name => 'success', :class => ::GetPartitionsResponse},
+      O1 => {:type => ::Thrift::Types::STRUCT, :name => 'o1', :class => ::MetaException}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
 end
 

http://git-wip-us.apache.org/repos/asf/hive/blob/44ef91a6/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
index d226db5..a2ec09f 100644
--- a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
+++ b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
@@ -3647,4 +3647,10 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
     req.setMaxCreateTime(maxCreateTime);
     return client.get_runtime_stats(req);
   }
+
+  @Override
+  public GetPartitionsResponse getPartitionsWithSpecs(GetPartitionsRequest request)
+      throws TException {
+    return client.get_partitions_with_specs(request);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/44ef91a6/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
index 54e7eda..422fce8 100644
--- a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
+++ b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
@@ -59,6 +59,8 @@ import org.apache.hadoop.hive.metastore.api.ForeignKeysRequest;
 import org.apache.hadoop.hive.metastore.api.Function;
 import org.apache.hadoop.hive.metastore.api.GetAllFunctionsResponse;
 import org.apache.hadoop.hive.metastore.api.GetOpenTxnsInfoResponse;
+import org.apache.hadoop.hive.metastore.api.GetPartitionsRequest;
+import org.apache.hadoop.hive.metastore.api.GetPartitionsResponse;
 import org.apache.hadoop.hive.metastore.api.GetPrincipalsInRoleRequest;
 import org.apache.hadoop.hive.metastore.api.GetPrincipalsInRoleResponse;
 import org.apache.hadoop.hive.metastore.api.GetRoleGrantsForPrincipalRequest;
@@ -3758,4 +3760,23 @@ public interface IMetaStoreClient {
 
   /** Reads runtime statistics. */
   List<RuntimeStat> getRuntimeStats(int maxWeight, int maxCreateTime) throws TException;
+
+  /**
+   * Generic Partition request API, providing different ways of filtering and controlling output.
+   *
+   * The API entry point is getPartitionsWithSpecs(), which is based on a single
+   * request/response object model.
+   *
+   * The request (GetPartitionsRequest) defines any filtering that should be done for partitions
+   * as well as the list of fields that should be returned (this is called ProjectionSpec).
+   * Projection is simply a list of dot separated strings which represent the fields which should
+   * be returned. Projection may also include whitelist or blacklist of parameters to include in
+   * the partition. When both blacklist and whitelist are present, the blacklist supersedes the
+   * whitelist in case of conflicts.
+   *
+   * Partition filter spec is the generalization of various types of partition filtering.
+   * Partitions can be filtered by names, by values or by partition expressions.
+   */
+  GetPartitionsResponse getPartitionsWithSpecs(GetPartitionsRequest request) throws TException;
+
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/44ef91a6/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/utils/MetaStoreUtils.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/utils/MetaStoreUtils.java b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/utils/MetaStoreUtils.java
index 720ec71..16f4a50 100644
--- a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/utils/MetaStoreUtils.java
+++ b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/utils/MetaStoreUtils.java
@@ -17,6 +17,10 @@
  */
 package org.apache.hadoop.hive.metastore.utils;
 
+import org.apache.commons.beanutils.PropertyUtils;
+import org.apache.hadoop.hive.metastore.api.PartitionSpec;
+import org.apache.hadoop.hive.metastore.api.PartitionSpecWithSharedSD;
+import org.apache.hadoop.hive.metastore.api.PartitionWithoutSD;
 import org.apache.hadoop.hive.metastore.api.WMPoolSchedulingPolicy;
 
 import com.google.common.base.Joiner;
@@ -44,6 +48,7 @@ import org.slf4j.LoggerFactory;
 
 import javax.annotation.Nullable;
 
+import java.beans.PropertyDescriptor;
 import java.io.File;
 import java.net.URL;
 import java.net.URLClassLoader;

http://git-wip-us.apache.org/repos/asf/hive/blob/44ef91a6/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift b/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift
index ad83162..ddc60a3 100644
--- a/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift
+++ b/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift
@@ -1645,6 +1645,67 @@ struct AlterTableRequest {
 struct AlterTableResponse {
 }
 
+/*
+ * Generic Partition request API, providing different kinds of filtering and controlling output.
+ *
+ * The API entry point is get_partitions_with_specs(), which is based on a single
+ * request/response object model.
+ *
+ * The request (GetPartitionsRequest) defines any filtering that should be done for partitions
+ * as well as the list of fields that should be returned (this is called ProjectionSpec).
+ * Projection is simply a list of dot separated strings which represent the fields which should
+ * be returned. Projection may also include whitelist or blacklist of parameters to include in
+ * the partition. When both blacklist and whitelist are present, the blacklist supersedes the
+ * whitelist in case of conflicts.
+ *
+ * Partition filter spec is the generalization of various types of partition filtering.
+ * Partitions can be filtered by names, by values or by partition expressions.
+ */
+
+struct GetPartitionsProjectionSpec {
+   // fieldList is a list of dot separated strings which represent the fields which must be returned.
+   // Any other field which is not in the fieldList may be unset in the returned partitions (it
+   //   is up to the implementation to decide whether it chooses to include or exclude such fields).
+   // E.g. setting the field list to sd.location, serdeInfo.name, sd.cols.name, sd.cols.type will
+   // return partitions which will have location field set in the storage descriptor. Also the serdeInfo
+   // in the returned storage descriptor will only have name field set. This applies to multi-valued
+   // fields as well like sd.cols, so in the example above only name and type fields will be set for sd.cols.
+   // If the fieldList is empty or not present, all the fields will be set
+   1: list<string> fieldList;
+   // SQL-92 compliant regex pattern for param keys to be included
+   // _ or % wildcards are supported. '_' represent one character and '%' represents 0 or more characters
+   2: string includeParamKeyPattern;
+   // SQL-92 compliant regex pattern for param keys to be excluded
+   // _ or % wildcards are supported. '_' represent one character and '%' represents 0 or more characters
+   3: string excludeParamKeyPattern;
+}
+
+enum PartitionFilterMode {
+   BY_NAMES,                 // filter by names
+   BY_VALUES,                // filter by values
+   BY_EXPR                   // filter by expression
+}
+
+struct GetPartitionsFilterSpec {
+   7: optional PartitionFilterMode filterMode,
+   8: optional list<string> filters //used as list of partitionNames or list of values or expressions depending on mode
+}
+
+struct GetPartitionsResponse {
+  1: list<PartitionSpec> partitionSpec
+}
+
+struct GetPartitionsRequest {
+   1: optional string catName,
+   2: string dbName,
+   3: string tblName,
+   4: optional bool withAuth,
+   5: optional string user,
+   6: optional list<string> groupNames,
+   7: GetPartitionsProjectionSpec projectionSpec
+   8: GetPartitionsFilterSpec filterSpec // TODO not yet implemented. Must be present but ignored
+}
+
 // Exceptions.
 
 exception MetaException {
@@ -2326,6 +2387,9 @@ service ThriftHiveMetastore extends fb303.FacebookService
   
   void add_runtime_stats(1: RuntimeStat stat) throws(1:MetaException o1)
   list<RuntimeStat> get_runtime_stats(1: GetRuntimeStatsRequest rqst) throws(1:MetaException o1)
+
+  // get_partitions with filter and projectspec
+  GetPartitionsResponse get_partitions_with_specs(1: GetPartitionsRequest request) throws(1:MetaException o1)
 }
 
 // * Note about the DDL_TIME: When creating or altering a table or a partition,

http://git-wip-us.apache.org/repos/asf/hive/blob/44ef91a6/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
index b5200bd..0861474 100644
--- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@ -66,10 +66,7 @@ import java.util.regex.Pattern;
 import javax.jdo.JDOException;
 
 import com.codahale.metrics.Counter;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableListMultimap;
 import com.google.common.collect.Lists;
-import com.google.common.collect.Multimaps;
 
 import org.apache.commons.cli.OptionBuilder;
 import org.apache.hadoop.conf.Configuration;
@@ -4663,7 +4660,8 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         List<Partition> partitions = get_partitions(db_name, tableName, (short) max_parts);
 
         if (is_partition_spec_grouping_enabled(table)) {
-          partitionSpecs = get_partitionspecs_grouped_by_storage_descriptor(table, partitions);
+          partitionSpecs = MetaStoreServerUtils
+              .getPartitionspecsGroupedByStorageDescriptor(table, partitions);
         }
         else {
           PartitionSpec pSpec = new PartitionSpec();
@@ -4682,121 +4680,53 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       }
     }
 
-    private static class StorageDescriptorKey {
-
-      private final StorageDescriptor sd;
-
-      StorageDescriptorKey(StorageDescriptor sd) { this.sd = sd; }
-
-      StorageDescriptor getSd() {
-        return sd;
-      }
-
-      private String hashCodeKey() {
-        return sd.getInputFormat() + "\t"
-            + sd.getOutputFormat() +  "\t"
-            + sd.getSerdeInfo().getSerializationLib() + "\t"
-            + sd.getCols();
-      }
-
-      @Override
-      public int hashCode() {
-        return hashCodeKey().hashCode();
+    @Override
+    public GetPartitionsResponse get_partitions_with_specs(GetPartitionsRequest request)
+        throws MetaException, TException {
+      String catName = null;
+      if (request.isSetCatName()) {
+        catName = request.getCatName();
       }
-
-      @Override
-      public boolean equals(Object rhs) {
-        if (rhs == this) {
-          return true;
-        }
-
-        if (!(rhs instanceof StorageDescriptorKey)) {
-          return false;
-        }
-
-        return (hashCodeKey().equals(((StorageDescriptorKey) rhs).hashCodeKey()));
+      String[] parsedDbName = parseDbName(request.getDbName(), conf);
+      String tableName = request.getTblName();
+      if (catName == null) {
+        // if catName is not provided in the request use the catName parsed from the dbName
+        catName = parsedDbName[CAT_NAME];
       }
-    }
-
-    private List<PartitionSpec> get_partitionspecs_grouped_by_storage_descriptor(Table table, List<Partition> partitions)
-      throws NoSuchObjectException, MetaException {
-
-      assert is_partition_spec_grouping_enabled(table);
-
-      final String tablePath = table.getSd().getLocation();
-
-      ImmutableListMultimap<Boolean, Partition> partitionsWithinTableDirectory
-          = Multimaps.index(partitions, new com.google.common.base.Function<Partition, Boolean>() {
-
-        @Override
-        public Boolean apply(Partition input) {
-          return input.getSd().getLocation().startsWith(tablePath);
-        }
-      });
-
-      List<PartitionSpec> partSpecs = new ArrayList<>();
-
-      // Classify partitions within the table directory into groups,
-      // based on shared SD properties.
-
-      Map<StorageDescriptorKey, List<PartitionWithoutSD>> sdToPartList
-          = new HashMap<>();
-
-      if (partitionsWithinTableDirectory.containsKey(true)) {
-
-        ImmutableList<Partition> partsWithinTableDir = partitionsWithinTableDirectory.get(true);
-        for (Partition partition : partsWithinTableDir) {
-
-          PartitionWithoutSD partitionWithoutSD
-              = new PartitionWithoutSD( partition.getValues(),
-              partition.getCreateTime(),
-              partition.getLastAccessTime(),
-              partition.getSd().getLocation().substring(tablePath.length()), partition.getParameters());
-
-          StorageDescriptorKey sdKey = new StorageDescriptorKey(partition.getSd());
-          if (!sdToPartList.containsKey(sdKey)) {
-            sdToPartList.put(sdKey, new ArrayList<>());
-          }
-
-          sdToPartList.get(sdKey).add(partitionWithoutSD);
-
-        } // for (partitionsWithinTableDirectory);
-
-        for (Map.Entry<StorageDescriptorKey, List<PartitionWithoutSD>> entry : sdToPartList.entrySet()) {
-          partSpecs.add(getSharedSDPartSpec(table, entry.getKey(), entry.getValue()));
-        }
-
-      } // Done grouping partitions within table-dir.
-
-      // Lump all partitions outside the tablePath into one PartSpec.
-      if (partitionsWithinTableDirectory.containsKey(false)) {
-        List<Partition> partitionsOutsideTableDir = partitionsWithinTableDirectory.get(false);
-        if (!partitionsOutsideTableDir.isEmpty()) {
-          PartitionSpec partListSpec = new PartitionSpec();
-          partListSpec.setDbName(table.getDbName());
-          partListSpec.setTableName(table.getTableName());
-          partListSpec.setPartitionList(new PartitionListComposingSpec(partitionsOutsideTableDir));
-          partSpecs.add(partListSpec);
-        }
-
+      startTableFunction("get_partitions_with_specs", catName, parsedDbName[DB_NAME],
+          tableName);
+      GetPartitionsResponse response = null;
+      Exception ex = null;
+      try {
+        List<String> fieldList = null;
+        String paramkeyPattern = null;
+        String excludeParamKeyPattern = null;
+        if (request.isSetProjectionSpec()) {
+          GetPartitionsProjectionSpec partitionsProjectSpec = request.getProjectionSpec();
+          fieldList = partitionsProjectSpec.getFieldList();
+          if (partitionsProjectSpec.isSetIncludeParamKeyPattern()) {
+            paramkeyPattern = partitionsProjectSpec.getIncludeParamKeyPattern();
+          }
+          if (partitionsProjectSpec.isSetExcludeParamKeyPattern()) {
+            excludeParamKeyPattern = partitionsProjectSpec.getExcludeParamKeyPattern();
+          }
+        }
+        String dbName = parsedDbName[DB_NAME];
+        Table table = get_table_core(catName, dbName, tableName);
+        List<Partition> partitions = getMS()
+            .getPartitionSpecsByFilterAndProjection(catName, dbName, tableName, fieldList, paramkeyPattern,
+                excludeParamKeyPattern);
+        List<PartitionSpec> partitionSpecs =
+            MetaStoreServerUtils.getPartitionspecsGroupedByStorageDescriptor(table, partitions);
+        response = new GetPartitionsResponse();
+        response.setPartitionSpec(partitionSpecs);
+      } catch (Exception e) {
+        ex = e;
+        rethrowException(e);
+      } finally {
+        endFunction("get_partitions_with_specs", response != null, ex, tableName);
       }
-      return partSpecs;
-    }
-
-    private PartitionSpec getSharedSDPartSpec(Table table, StorageDescriptorKey sdKey, List<PartitionWithoutSD> partitions) {
-
-      StorageDescriptor sd = new StorageDescriptor(sdKey.getSd());
-      sd.setLocation(table.getSd().getLocation()); // Use table-dir as root-dir.
-      PartitionSpecWithSharedSD sharedSDPartSpec =
-          new PartitionSpecWithSharedSD(partitions, sd);
-
-      PartitionSpec ret = new PartitionSpec();
-      ret.setRootPath(sd.getLocation());
-      ret.setSharedSDPartitionSpec(sharedSDPartSpec);
-      ret.setDbName(table.getDbName());
-      ret.setTableName(table.getTableName());
-
-      return ret;
+      return response;
     }
 
     private static boolean is_partition_spec_grouping_enabled(Table table) {
@@ -6055,7 +5985,8 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         List<Partition> partitions = get_partitions_by_filter(dbName, tblName, filter, (short) maxParts);
 
         if (is_partition_spec_grouping_enabled(table)) {
-          partitionSpecs = get_partitionspecs_grouped_by_storage_descriptor(table, partitions);
+          partitionSpecs = MetaStoreServerUtils
+              .getPartitionspecsGroupedByStorageDescriptor(table, partitions);
         }
         else {
           PartitionSpec pSpec = new PartitionSpec();


[09/13] hive git commit: HIVE-20306 : Implement projection spec for fetching only requested fields from partitions (Vihang Karajgaonkar, reviewed by Aihua Xu, Andrew Sherman and Alexander Kolbasov)

Posted by vi...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/44ef91a6/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/Types.php
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/Types.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/Types.php
index 936f7c5..5232da1 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/Types.php
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/Types.php
@@ -257,6 +257,17 @@ final class WMPoolSchedulingPolicy {
   );
 }
 
+final class PartitionFilterMode {
+  const BY_NAMES = 0;
+  const BY_VALUES = 1;
+  const BY_EXPR = 2;
+  static public $__names = array(
+    0 => 'BY_NAMES',
+    1 => 'BY_VALUES',
+    2 => 'BY_EXPR',
+  );
+}
+
 class Version {
   static $_TSPEC;
 
@@ -32428,6 +32439,652 @@ class AlterTableResponse {
 
 }
 
+class GetPartitionsProjectionSpec {
+  static $_TSPEC;
+
+  /**
+   * @var string[]
+   */
+  public $fieldList = null;
+  /**
+   * @var string
+   */
+  public $includeParamKeyPattern = null;
+  /**
+   * @var string
+   */
+  public $excludeParamKeyPattern = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'fieldList',
+          'type' => TType::LST,
+          'etype' => TType::STRING,
+          'elem' => array(
+            'type' => TType::STRING,
+            ),
+          ),
+        2 => array(
+          'var' => 'includeParamKeyPattern',
+          'type' => TType::STRING,
+          ),
+        3 => array(
+          'var' => 'excludeParamKeyPattern',
+          'type' => TType::STRING,
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['fieldList'])) {
+        $this->fieldList = $vals['fieldList'];
+      }
+      if (isset($vals['includeParamKeyPattern'])) {
+        $this->includeParamKeyPattern = $vals['includeParamKeyPattern'];
+      }
+      if (isset($vals['excludeParamKeyPattern'])) {
+        $this->excludeParamKeyPattern = $vals['excludeParamKeyPattern'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'GetPartitionsProjectionSpec';
+  }
+
+  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->fieldList = array();
+            $_size854 = 0;
+            $_etype857 = 0;
+            $xfer += $input->readListBegin($_etype857, $_size854);
+            for ($_i858 = 0; $_i858 < $_size854; ++$_i858)
+            {
+              $elem859 = null;
+              $xfer += $input->readString($elem859);
+              $this->fieldList []= $elem859;
+            }
+            $xfer += $input->readListEnd();
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->includeParamKeyPattern);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 3:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->excludeParamKeyPattern);
+          } 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('GetPartitionsProjectionSpec');
+    if ($this->fieldList !== null) {
+      if (!is_array($this->fieldList)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('fieldList', TType::LST, 1);
+      {
+        $output->writeListBegin(TType::STRING, count($this->fieldList));
+        {
+          foreach ($this->fieldList as $iter860)
+          {
+            $xfer += $output->writeString($iter860);
+          }
+        }
+        $output->writeListEnd();
+      }
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->includeParamKeyPattern !== null) {
+      $xfer += $output->writeFieldBegin('includeParamKeyPattern', TType::STRING, 2);
+      $xfer += $output->writeString($this->includeParamKeyPattern);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->excludeParamKeyPattern !== null) {
+      $xfer += $output->writeFieldBegin('excludeParamKeyPattern', TType::STRING, 3);
+      $xfer += $output->writeString($this->excludeParamKeyPattern);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class GetPartitionsFilterSpec {
+  static $_TSPEC;
+
+  /**
+   * @var int
+   */
+  public $filterMode = null;
+  /**
+   * @var string[]
+   */
+  public $filters = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        7 => array(
+          'var' => 'filterMode',
+          'type' => TType::I32,
+          ),
+        8 => array(
+          'var' => 'filters',
+          'type' => TType::LST,
+          'etype' => TType::STRING,
+          'elem' => array(
+            'type' => TType::STRING,
+            ),
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['filterMode'])) {
+        $this->filterMode = $vals['filterMode'];
+      }
+      if (isset($vals['filters'])) {
+        $this->filters = $vals['filters'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'GetPartitionsFilterSpec';
+  }
+
+  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 7:
+          if ($ftype == TType::I32) {
+            $xfer += $input->readI32($this->filterMode);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 8:
+          if ($ftype == TType::LST) {
+            $this->filters = array();
+            $_size861 = 0;
+            $_etype864 = 0;
+            $xfer += $input->readListBegin($_etype864, $_size861);
+            for ($_i865 = 0; $_i865 < $_size861; ++$_i865)
+            {
+              $elem866 = null;
+              $xfer += $input->readString($elem866);
+              $this->filters []= $elem866;
+            }
+            $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('GetPartitionsFilterSpec');
+    if ($this->filterMode !== null) {
+      $xfer += $output->writeFieldBegin('filterMode', TType::I32, 7);
+      $xfer += $output->writeI32($this->filterMode);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->filters !== null) {
+      if (!is_array($this->filters)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('filters', TType::LST, 8);
+      {
+        $output->writeListBegin(TType::STRING, count($this->filters));
+        {
+          foreach ($this->filters as $iter867)
+          {
+            $xfer += $output->writeString($iter867);
+          }
+        }
+        $output->writeListEnd();
+      }
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class GetPartitionsResponse {
+  static $_TSPEC;
+
+  /**
+   * @var \metastore\PartitionSpec[]
+   */
+  public $partitionSpec = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'partitionSpec',
+          'type' => TType::LST,
+          'etype' => TType::STRUCT,
+          'elem' => array(
+            'type' => TType::STRUCT,
+            'class' => '\metastore\PartitionSpec',
+            ),
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['partitionSpec'])) {
+        $this->partitionSpec = $vals['partitionSpec'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'GetPartitionsResponse';
+  }
+
+  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->partitionSpec = array();
+            $_size868 = 0;
+            $_etype871 = 0;
+            $xfer += $input->readListBegin($_etype871, $_size868);
+            for ($_i872 = 0; $_i872 < $_size868; ++$_i872)
+            {
+              $elem873 = null;
+              $elem873 = new \metastore\PartitionSpec();
+              $xfer += $elem873->read($input);
+              $this->partitionSpec []= $elem873;
+            }
+            $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('GetPartitionsResponse');
+    if ($this->partitionSpec !== null) {
+      if (!is_array($this->partitionSpec)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('partitionSpec', TType::LST, 1);
+      {
+        $output->writeListBegin(TType::STRUCT, count($this->partitionSpec));
+        {
+          foreach ($this->partitionSpec as $iter874)
+          {
+            $xfer += $iter874->write($output);
+          }
+        }
+        $output->writeListEnd();
+      }
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class GetPartitionsRequest {
+  static $_TSPEC;
+
+  /**
+   * @var string
+   */
+  public $catName = null;
+  /**
+   * @var string
+   */
+  public $dbName = null;
+  /**
+   * @var string
+   */
+  public $tblName = null;
+  /**
+   * @var bool
+   */
+  public $withAuth = null;
+  /**
+   * @var string
+   */
+  public $user = null;
+  /**
+   * @var string[]
+   */
+  public $groupNames = null;
+  /**
+   * @var \metastore\GetPartitionsProjectionSpec
+   */
+  public $projectionSpec = null;
+  /**
+   * @var \metastore\GetPartitionsFilterSpec
+   */
+  public $filterSpec = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'catName',
+          'type' => TType::STRING,
+          ),
+        2 => array(
+          'var' => 'dbName',
+          'type' => TType::STRING,
+          ),
+        3 => array(
+          'var' => 'tblName',
+          'type' => TType::STRING,
+          ),
+        4 => array(
+          'var' => 'withAuth',
+          'type' => TType::BOOL,
+          ),
+        5 => array(
+          'var' => 'user',
+          'type' => TType::STRING,
+          ),
+        6 => array(
+          'var' => 'groupNames',
+          'type' => TType::LST,
+          'etype' => TType::STRING,
+          'elem' => array(
+            'type' => TType::STRING,
+            ),
+          ),
+        7 => array(
+          'var' => 'projectionSpec',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\GetPartitionsProjectionSpec',
+          ),
+        8 => array(
+          'var' => 'filterSpec',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\GetPartitionsFilterSpec',
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['catName'])) {
+        $this->catName = $vals['catName'];
+      }
+      if (isset($vals['dbName'])) {
+        $this->dbName = $vals['dbName'];
+      }
+      if (isset($vals['tblName'])) {
+        $this->tblName = $vals['tblName'];
+      }
+      if (isset($vals['withAuth'])) {
+        $this->withAuth = $vals['withAuth'];
+      }
+      if (isset($vals['user'])) {
+        $this->user = $vals['user'];
+      }
+      if (isset($vals['groupNames'])) {
+        $this->groupNames = $vals['groupNames'];
+      }
+      if (isset($vals['projectionSpec'])) {
+        $this->projectionSpec = $vals['projectionSpec'];
+      }
+      if (isset($vals['filterSpec'])) {
+        $this->filterSpec = $vals['filterSpec'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'GetPartitionsRequest';
+  }
+
+  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->catName);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->dbName);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 3:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->tblName);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 4:
+          if ($ftype == TType::BOOL) {
+            $xfer += $input->readBool($this->withAuth);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 5:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->user);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 6:
+          if ($ftype == TType::LST) {
+            $this->groupNames = array();
+            $_size875 = 0;
+            $_etype878 = 0;
+            $xfer += $input->readListBegin($_etype878, $_size875);
+            for ($_i879 = 0; $_i879 < $_size875; ++$_i879)
+            {
+              $elem880 = null;
+              $xfer += $input->readString($elem880);
+              $this->groupNames []= $elem880;
+            }
+            $xfer += $input->readListEnd();
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 7:
+          if ($ftype == TType::STRUCT) {
+            $this->projectionSpec = new \metastore\GetPartitionsProjectionSpec();
+            $xfer += $this->projectionSpec->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 8:
+          if ($ftype == TType::STRUCT) {
+            $this->filterSpec = new \metastore\GetPartitionsFilterSpec();
+            $xfer += $this->filterSpec->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('GetPartitionsRequest');
+    if ($this->catName !== null) {
+      $xfer += $output->writeFieldBegin('catName', TType::STRING, 1);
+      $xfer += $output->writeString($this->catName);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->dbName !== null) {
+      $xfer += $output->writeFieldBegin('dbName', TType::STRING, 2);
+      $xfer += $output->writeString($this->dbName);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->tblName !== null) {
+      $xfer += $output->writeFieldBegin('tblName', TType::STRING, 3);
+      $xfer += $output->writeString($this->tblName);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->withAuth !== null) {
+      $xfer += $output->writeFieldBegin('withAuth', TType::BOOL, 4);
+      $xfer += $output->writeBool($this->withAuth);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->user !== null) {
+      $xfer += $output->writeFieldBegin('user', TType::STRING, 5);
+      $xfer += $output->writeString($this->user);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->groupNames !== null) {
+      if (!is_array($this->groupNames)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('groupNames', TType::LST, 6);
+      {
+        $output->writeListBegin(TType::STRING, count($this->groupNames));
+        {
+          foreach ($this->groupNames as $iter881)
+          {
+            $xfer += $output->writeString($iter881);
+          }
+        }
+        $output->writeListEnd();
+      }
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->projectionSpec !== null) {
+      if (!is_object($this->projectionSpec)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('projectionSpec', TType::STRUCT, 7);
+      $xfer += $this->projectionSpec->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->filterSpec !== null) {
+      if (!is_object($this->filterSpec)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('filterSpec', TType::STRUCT, 8);
+      $xfer += $this->filterSpec->write($output);
+      $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/44ef91a6/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote b/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
index a595732..d57de35 100755
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
@@ -238,6 +238,7 @@ if len(sys.argv) <= 1 or sys.argv[1] == '--help':
   print('  bool heartbeat_lock_materialization_rebuild(string dbName, string tableName, i64 txnId)')
   print('  void add_runtime_stats(RuntimeStat stat)')
   print('   get_runtime_stats(GetRuntimeStatsRequest rqst)')
+  print('  GetPartitionsResponse get_partitions_with_specs(GetPartitionsRequest request)')
   print('  string getName()')
   print('  string getVersion()')
   print('  fb_status getStatus()')
@@ -1591,6 +1592,12 @@ elif cmd == 'get_runtime_stats':
     sys.exit(1)
   pp.pprint(client.get_runtime_stats(eval(args[0]),))
 
+elif cmd == 'get_partitions_with_specs':
+  if len(args) != 1:
+    print('get_partitions_with_specs requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.get_partitions_with_specs(eval(args[0]),))
+
 elif cmd == 'getName':
   if len(args) != 0:
     print('getName requires 0 args')


[12/13] hive git commit: HIVE-20306 : Implement projection spec for fetching only requested fields from partitions (Vihang Karajgaonkar, reviewed by Aihua Xu, Andrew Sherman and Alexander Kolbasov)

Posted by vi...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/44ef91a6/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsResponse.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsResponse.java
new file mode 100644
index 0000000..2c374b8
--- /dev/null
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsResponse.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 GetPartitionsResponse implements org.apache.thrift.TBase<GetPartitionsResponse, GetPartitionsResponse._Fields>, java.io.Serializable, Cloneable, Comparable<GetPartitionsResponse> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetPartitionsResponse");
+
+  private static final org.apache.thrift.protocol.TField PARTITION_SPEC_FIELD_DESC = new org.apache.thrift.protocol.TField("partitionSpec", 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 GetPartitionsResponseStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new GetPartitionsResponseTupleSchemeFactory());
+  }
+
+  private List<PartitionSpec> partitionSpec; // 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 {
+    PARTITION_SPEC((short)1, "partitionSpec");
+
+    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: // PARTITION_SPEC
+          return PARTITION_SPEC;
+        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.PARTITION_SPEC, new org.apache.thrift.meta_data.FieldMetaData("partitionSpec", 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, PartitionSpec.class))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetPartitionsResponse.class, metaDataMap);
+  }
+
+  public GetPartitionsResponse() {
+  }
+
+  public GetPartitionsResponse(
+    List<PartitionSpec> partitionSpec)
+  {
+    this();
+    this.partitionSpec = partitionSpec;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public GetPartitionsResponse(GetPartitionsResponse other) {
+    if (other.isSetPartitionSpec()) {
+      List<PartitionSpec> __this__partitionSpec = new ArrayList<PartitionSpec>(other.partitionSpec.size());
+      for (PartitionSpec other_element : other.partitionSpec) {
+        __this__partitionSpec.add(new PartitionSpec(other_element));
+      }
+      this.partitionSpec = __this__partitionSpec;
+    }
+  }
+
+  public GetPartitionsResponse deepCopy() {
+    return new GetPartitionsResponse(this);
+  }
+
+  @Override
+  public void clear() {
+    this.partitionSpec = null;
+  }
+
+  public int getPartitionSpecSize() {
+    return (this.partitionSpec == null) ? 0 : this.partitionSpec.size();
+  }
+
+  public java.util.Iterator<PartitionSpec> getPartitionSpecIterator() {
+    return (this.partitionSpec == null) ? null : this.partitionSpec.iterator();
+  }
+
+  public void addToPartitionSpec(PartitionSpec elem) {
+    if (this.partitionSpec == null) {
+      this.partitionSpec = new ArrayList<PartitionSpec>();
+    }
+    this.partitionSpec.add(elem);
+  }
+
+  public List<PartitionSpec> getPartitionSpec() {
+    return this.partitionSpec;
+  }
+
+  public void setPartitionSpec(List<PartitionSpec> partitionSpec) {
+    this.partitionSpec = partitionSpec;
+  }
+
+  public void unsetPartitionSpec() {
+    this.partitionSpec = null;
+  }
+
+  /** Returns true if field partitionSpec is set (has been assigned a value) and false otherwise */
+  public boolean isSetPartitionSpec() {
+    return this.partitionSpec != null;
+  }
+
+  public void setPartitionSpecIsSet(boolean value) {
+    if (!value) {
+      this.partitionSpec = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case PARTITION_SPEC:
+      if (value == null) {
+        unsetPartitionSpec();
+      } else {
+        setPartitionSpec((List<PartitionSpec>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case PARTITION_SPEC:
+      return getPartitionSpec();
+
+    }
+    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 PARTITION_SPEC:
+      return isSetPartitionSpec();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof GetPartitionsResponse)
+      return this.equals((GetPartitionsResponse)that);
+    return false;
+  }
+
+  public boolean equals(GetPartitionsResponse that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_partitionSpec = true && this.isSetPartitionSpec();
+    boolean that_present_partitionSpec = true && that.isSetPartitionSpec();
+    if (this_present_partitionSpec || that_present_partitionSpec) {
+      if (!(this_present_partitionSpec && that_present_partitionSpec))
+        return false;
+      if (!this.partitionSpec.equals(that.partitionSpec))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_partitionSpec = true && (isSetPartitionSpec());
+    list.add(present_partitionSpec);
+    if (present_partitionSpec)
+      list.add(partitionSpec);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(GetPartitionsResponse other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetPartitionSpec()).compareTo(other.isSetPartitionSpec());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetPartitionSpec()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.partitionSpec, other.partitionSpec);
+      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("GetPartitionsResponse(");
+    boolean first = true;
+
+    sb.append("partitionSpec:");
+    if (this.partitionSpec == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.partitionSpec);
+    }
+    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 GetPartitionsResponseStandardSchemeFactory implements SchemeFactory {
+    public GetPartitionsResponseStandardScheme getScheme() {
+      return new GetPartitionsResponseStandardScheme();
+    }
+  }
+
+  private static class GetPartitionsResponseStandardScheme extends StandardScheme<GetPartitionsResponse> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, GetPartitionsResponse 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: // PARTITION_SPEC
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list984 = iprot.readListBegin();
+                struct.partitionSpec = new ArrayList<PartitionSpec>(_list984.size);
+                PartitionSpec _elem985;
+                for (int _i986 = 0; _i986 < _list984.size; ++_i986)
+                {
+                  _elem985 = new PartitionSpec();
+                  _elem985.read(iprot);
+                  struct.partitionSpec.add(_elem985);
+                }
+                iprot.readListEnd();
+              }
+              struct.setPartitionSpecIsSet(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, GetPartitionsResponse struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.partitionSpec != null) {
+        oprot.writeFieldBegin(PARTITION_SPEC_FIELD_DESC);
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.partitionSpec.size()));
+          for (PartitionSpec _iter987 : struct.partitionSpec)
+          {
+            _iter987.write(oprot);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class GetPartitionsResponseTupleSchemeFactory implements SchemeFactory {
+    public GetPartitionsResponseTupleScheme getScheme() {
+      return new GetPartitionsResponseTupleScheme();
+    }
+  }
+
+  private static class GetPartitionsResponseTupleScheme extends TupleScheme<GetPartitionsResponse> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, GetPartitionsResponse struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.isSetPartitionSpec()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.isSetPartitionSpec()) {
+        {
+          oprot.writeI32(struct.partitionSpec.size());
+          for (PartitionSpec _iter988 : struct.partitionSpec)
+          {
+            _iter988.write(oprot);
+          }
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, GetPartitionsResponse struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        {
+          org.apache.thrift.protocol.TList _list989 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.partitionSpec = new ArrayList<PartitionSpec>(_list989.size);
+          PartitionSpec _elem990;
+          for (int _i991 = 0; _i991 < _list989.size; ++_i991)
+          {
+            _elem990 = new PartitionSpec();
+            _elem990.read(iprot);
+            struct.partitionSpec.add(_elem990);
+          }
+        }
+        struct.setPartitionSpecIsSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/44ef91a6/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionFilterMode.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionFilterMode.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionFilterMode.java
new file mode 100644
index 0000000..aaea917
--- /dev/null
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PartitionFilterMode.java
@@ -0,0 +1,48 @@
+/**
+ * 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 PartitionFilterMode implements org.apache.thrift.TEnum {
+  BY_NAMES(0),
+  BY_VALUES(1),
+  BY_EXPR(2);
+
+  private final int value;
+
+  private PartitionFilterMode(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 PartitionFilterMode findByValue(int value) { 
+    switch (value) {
+      case 0:
+        return BY_NAMES;
+      case 1:
+        return BY_VALUES;
+      case 2:
+        return BY_EXPR;
+      default:
+        return null;
+    }
+  }
+}


[11/13] hive git commit: HIVE-20306 : Implement projection spec for fetching only requested fields from partitions (Vihang Karajgaonkar, reviewed by Aihua Xu, Andrew Sherman and Alexander Kolbasov)

Posted by vi...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/44ef91a6/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
index ae09568..52a2b71 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
@@ -470,6 +470,8 @@ import org.slf4j.LoggerFactory;
 
     public List<RuntimeStat> get_runtime_stats(GetRuntimeStatsRequest rqst) throws MetaException, org.apache.thrift.TException;
 
+    public GetPartitionsResponse get_partitions_with_specs(GetPartitionsRequest request) throws 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 {
@@ -902,6 +904,8 @@ import org.slf4j.LoggerFactory;
 
     public void get_runtime_stats(GetRuntimeStatsRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
+    public void get_partitions_with_specs(GetPartitionsRequest request, 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 {
@@ -7072,6 +7076,32 @@ import org.slf4j.LoggerFactory;
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "get_runtime_stats failed: unknown result");
     }
 
+    public GetPartitionsResponse get_partitions_with_specs(GetPartitionsRequest request) throws MetaException, org.apache.thrift.TException
+    {
+      send_get_partitions_with_specs(request);
+      return recv_get_partitions_with_specs();
+    }
+
+    public void send_get_partitions_with_specs(GetPartitionsRequest request) throws org.apache.thrift.TException
+    {
+      get_partitions_with_specs_args args = new get_partitions_with_specs_args();
+      args.setRequest(request);
+      sendBase("get_partitions_with_specs", args);
+    }
+
+    public GetPartitionsResponse recv_get_partitions_with_specs() throws MetaException, org.apache.thrift.TException
+    {
+      get_partitions_with_specs_result result = new get_partitions_with_specs_result();
+      receiveBase(result, "get_partitions_with_specs");
+      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_partitions_with_specs 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> {
@@ -14430,6 +14460,38 @@ import org.slf4j.LoggerFactory;
       }
     }
 
+    public void get_partitions_with_specs(GetPartitionsRequest request, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      get_partitions_with_specs_call method_call = new get_partitions_with_specs_call(request, 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_partitions_with_specs_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private GetPartitionsRequest request;
+      public get_partitions_with_specs_call(GetPartitionsRequest request, 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.request = request;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("get_partitions_with_specs", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        get_partitions_with_specs_args args = new get_partitions_with_specs_args();
+        args.setRequest(request);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public GetPartitionsResponse 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_partitions_with_specs();
+      }
+    }
+
   }
 
   @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 {
@@ -14657,6 +14719,7 @@ import org.slf4j.LoggerFactory;
       processMap.put("heartbeat_lock_materialization_rebuild", new heartbeat_lock_materialization_rebuild());
       processMap.put("add_runtime_stats", new add_runtime_stats());
       processMap.put("get_runtime_stats", new get_runtime_stats());
+      processMap.put("get_partitions_with_specs", new get_partitions_with_specs());
       return processMap;
     }
 
@@ -20152,6 +20215,30 @@ import org.slf4j.LoggerFactory;
       }
     }
 
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_partitions_with_specs<I extends Iface> extends org.apache.thrift.ProcessFunction<I, get_partitions_with_specs_args> {
+      public get_partitions_with_specs() {
+        super("get_partitions_with_specs");
+      }
+
+      public get_partitions_with_specs_args getEmptyArgsInstance() {
+        return new get_partitions_with_specs_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public get_partitions_with_specs_result getResult(I iface, get_partitions_with_specs_args args) throws org.apache.thrift.TException {
+        get_partitions_with_specs_result result = new get_partitions_with_specs_result();
+        try {
+          result.success = iface.get_partitions_with_specs(args.request);
+        } catch (MetaException o1) {
+          result.o1 = o1;
+        }
+        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> {
@@ -20379,6 +20466,7 @@ import org.slf4j.LoggerFactory;
       processMap.put("heartbeat_lock_materialization_rebuild", new heartbeat_lock_materialization_rebuild());
       processMap.put("add_runtime_stats", new add_runtime_stats());
       processMap.put("get_runtime_stats", new get_runtime_stats());
+      processMap.put("get_partitions_with_specs", new get_partitions_with_specs());
       return processMap;
     }
 
@@ -33491,6 +33579,63 @@ import org.slf4j.LoggerFactory;
       }
     }
 
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_partitions_with_specs<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_partitions_with_specs_args, GetPartitionsResponse> {
+      public get_partitions_with_specs() {
+        super("get_partitions_with_specs");
+      }
+
+      public get_partitions_with_specs_args getEmptyArgsInstance() {
+        return new get_partitions_with_specs_args();
+      }
+
+      public AsyncMethodCallback<GetPartitionsResponse> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<GetPartitionsResponse>() { 
+          public void onComplete(GetPartitionsResponse o) {
+            get_partitions_with_specs_result result = new get_partitions_with_specs_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_partitions_with_specs_result result = new get_partitions_with_specs_result();
+            if (e instanceof MetaException) {
+                        result.o1 = (MetaException) e;
+                        result.setO1IsSet(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_partitions_with_specs_args args, org.apache.thrift.async.AsyncMethodCallback<GetPartitionsResponse> resultHandler) throws TException {
+        iface.get_partitions_with_specs(args.request,resultHandler);
+      }
+    }
+
   }
 
   @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class getMetaConf_args implements org.apache.thrift.TBase<getMetaConf_args, getMetaConf_args._Fields>, java.io.Serializable, Cloneable, Comparable<getMetaConf_args>   {
@@ -43344,13 +43489,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list968 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list968.size);
-                  String _elem969;
-                  for (int _i970 = 0; _i970 < _list968.size; ++_i970)
+                  org.apache.thrift.protocol.TList _list1000 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1000.size);
+                  String _elem1001;
+                  for (int _i1002 = 0; _i1002 < _list1000.size; ++_i1002)
                   {
-                    _elem969 = iprot.readString();
-                    struct.success.add(_elem969);
+                    _elem1001 = iprot.readString();
+                    struct.success.add(_elem1001);
                   }
                   iprot.readListEnd();
                 }
@@ -43385,9 +43530,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter971 : struct.success)
+            for (String _iter1003 : struct.success)
             {
-              oprot.writeString(_iter971);
+              oprot.writeString(_iter1003);
             }
             oprot.writeListEnd();
           }
@@ -43426,9 +43571,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter972 : struct.success)
+            for (String _iter1004 : struct.success)
             {
-              oprot.writeString(_iter972);
+              oprot.writeString(_iter1004);
             }
           }
         }
@@ -43443,13 +43588,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list973 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list973.size);
-            String _elem974;
-            for (int _i975 = 0; _i975 < _list973.size; ++_i975)
+            org.apache.thrift.protocol.TList _list1005 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1005.size);
+            String _elem1006;
+            for (int _i1007 = 0; _i1007 < _list1005.size; ++_i1007)
             {
-              _elem974 = iprot.readString();
-              struct.success.add(_elem974);
+              _elem1006 = iprot.readString();
+              struct.success.add(_elem1006);
             }
           }
           struct.setSuccessIsSet(true);
@@ -44103,13 +44248,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list976 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list976.size);
-                  String _elem977;
-                  for (int _i978 = 0; _i978 < _list976.size; ++_i978)
+                  org.apache.thrift.protocol.TList _list1008 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1008.size);
+                  String _elem1009;
+                  for (int _i1010 = 0; _i1010 < _list1008.size; ++_i1010)
                   {
-                    _elem977 = iprot.readString();
-                    struct.success.add(_elem977);
+                    _elem1009 = iprot.readString();
+                    struct.success.add(_elem1009);
                   }
                   iprot.readListEnd();
                 }
@@ -44144,9 +44289,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter979 : struct.success)
+            for (String _iter1011 : struct.success)
             {
-              oprot.writeString(_iter979);
+              oprot.writeString(_iter1011);
             }
             oprot.writeListEnd();
           }
@@ -44185,9 +44330,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter980 : struct.success)
+            for (String _iter1012 : struct.success)
             {
-              oprot.writeString(_iter980);
+              oprot.writeString(_iter1012);
             }
           }
         }
@@ -44202,13 +44347,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list981 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list981.size);
-            String _elem982;
-            for (int _i983 = 0; _i983 < _list981.size; ++_i983)
+            org.apache.thrift.protocol.TList _list1013 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1013.size);
+            String _elem1014;
+            for (int _i1015 = 0; _i1015 < _list1013.size; ++_i1015)
             {
-              _elem982 = iprot.readString();
-              struct.success.add(_elem982);
+              _elem1014 = iprot.readString();
+              struct.success.add(_elem1014);
             }
           }
           struct.setSuccessIsSet(true);
@@ -48815,16 +48960,16 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map984 = iprot.readMapBegin();
-                  struct.success = new HashMap<String,Type>(2*_map984.size);
-                  String _key985;
-                  Type _val986;
-                  for (int _i987 = 0; _i987 < _map984.size; ++_i987)
+                  org.apache.thrift.protocol.TMap _map1016 = iprot.readMapBegin();
+                  struct.success = new HashMap<String,Type>(2*_map1016.size);
+                  String _key1017;
+                  Type _val1018;
+                  for (int _i1019 = 0; _i1019 < _map1016.size; ++_i1019)
                   {
-                    _key985 = iprot.readString();
-                    _val986 = new Type();
-                    _val986.read(iprot);
-                    struct.success.put(_key985, _val986);
+                    _key1017 = iprot.readString();
+                    _val1018 = new Type();
+                    _val1018.read(iprot);
+                    struct.success.put(_key1017, _val1018);
                   }
                   iprot.readMapEnd();
                 }
@@ -48859,10 +49004,10 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (Map.Entry<String, Type> _iter988 : struct.success.entrySet())
+            for (Map.Entry<String, Type> _iter1020 : struct.success.entrySet())
             {
-              oprot.writeString(_iter988.getKey());
-              _iter988.getValue().write(oprot);
+              oprot.writeString(_iter1020.getKey());
+              _iter1020.getValue().write(oprot);
             }
             oprot.writeMapEnd();
           }
@@ -48901,10 +49046,10 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Map.Entry<String, Type> _iter989 : struct.success.entrySet())
+            for (Map.Entry<String, Type> _iter1021 : struct.success.entrySet())
             {
-              oprot.writeString(_iter989.getKey());
-              _iter989.getValue().write(oprot);
+              oprot.writeString(_iter1021.getKey());
+              _iter1021.getValue().write(oprot);
             }
           }
         }
@@ -48919,16 +49064,16 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TMap _map990 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new HashMap<String,Type>(2*_map990.size);
-            String _key991;
-            Type _val992;
-            for (int _i993 = 0; _i993 < _map990.size; ++_i993)
+            org.apache.thrift.protocol.TMap _map1022 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new HashMap<String,Type>(2*_map1022.size);
+            String _key1023;
+            Type _val1024;
+            for (int _i1025 = 0; _i1025 < _map1022.size; ++_i1025)
             {
-              _key991 = iprot.readString();
-              _val992 = new Type();
-              _val992.read(iprot);
-              struct.success.put(_key991, _val992);
+              _key1023 = iprot.readString();
+              _val1024 = new Type();
+              _val1024.read(iprot);
+              struct.success.put(_key1023, _val1024);
             }
           }
           struct.setSuccessIsSet(true);
@@ -49963,14 +50108,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list994 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list994.size);
-                  FieldSchema _elem995;
-                  for (int _i996 = 0; _i996 < _list994.size; ++_i996)
+                  org.apache.thrift.protocol.TList _list1026 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list1026.size);
+                  FieldSchema _elem1027;
+                  for (int _i1028 = 0; _i1028 < _list1026.size; ++_i1028)
                   {
-                    _elem995 = new FieldSchema();
-                    _elem995.read(iprot);
-                    struct.success.add(_elem995);
+                    _elem1027 = new FieldSchema();
+                    _elem1027.read(iprot);
+                    struct.success.add(_elem1027);
                   }
                   iprot.readListEnd();
                 }
@@ -50023,9 +50168,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter997 : struct.success)
+            for (FieldSchema _iter1029 : struct.success)
             {
-              _iter997.write(oprot);
+              _iter1029.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -50080,9 +50225,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter998 : struct.success)
+            for (FieldSchema _iter1030 : struct.success)
             {
-              _iter998.write(oprot);
+              _iter1030.write(oprot);
             }
           }
         }
@@ -50103,14 +50248,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list999 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list999.size);
-            FieldSchema _elem1000;
-            for (int _i1001 = 0; _i1001 < _list999.size; ++_i1001)
+            org.apache.thrift.protocol.TList _list1031 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list1031.size);
+            FieldSchema _elem1032;
+            for (int _i1033 = 0; _i1033 < _list1031.size; ++_i1033)
             {
-              _elem1000 = new FieldSchema();
-              _elem1000.read(iprot);
-              struct.success.add(_elem1000);
+              _elem1032 = new FieldSchema();
+              _elem1032.read(iprot);
+              struct.success.add(_elem1032);
             }
           }
           struct.setSuccessIsSet(true);
@@ -51264,14 +51409,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1002 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list1002.size);
-                  FieldSchema _elem1003;
-                  for (int _i1004 = 0; _i1004 < _list1002.size; ++_i1004)
+                  org.apache.thrift.protocol.TList _list1034 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list1034.size);
+                  FieldSchema _elem1035;
+                  for (int _i1036 = 0; _i1036 < _list1034.size; ++_i1036)
                   {
-                    _elem1003 = new FieldSchema();
-                    _elem1003.read(iprot);
-                    struct.success.add(_elem1003);
+                    _elem1035 = new FieldSchema();
+                    _elem1035.read(iprot);
+                    struct.success.add(_elem1035);
                   }
                   iprot.readListEnd();
                 }
@@ -51324,9 +51469,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter1005 : struct.success)
+            for (FieldSchema _iter1037 : struct.success)
             {
-              _iter1005.write(oprot);
+              _iter1037.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -51381,9 +51526,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter1006 : struct.success)
+            for (FieldSchema _iter1038 : struct.success)
             {
-              _iter1006.write(oprot);
+              _iter1038.write(oprot);
             }
           }
         }
@@ -51404,14 +51549,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1007 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list1007.size);
-            FieldSchema _elem1008;
-            for (int _i1009 = 0; _i1009 < _list1007.size; ++_i1009)
+            org.apache.thrift.protocol.TList _list1039 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list1039.size);
+            FieldSchema _elem1040;
+            for (int _i1041 = 0; _i1041 < _list1039.size; ++_i1041)
             {
-              _elem1008 = new FieldSchema();
-              _elem1008.read(iprot);
-              struct.success.add(_elem1008);
+              _elem1040 = new FieldSchema();
+              _elem1040.read(iprot);
+              struct.success.add(_elem1040);
             }
           }
           struct.setSuccessIsSet(true);
@@ -52456,14 +52601,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1010 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list1010.size);
-                  FieldSchema _elem1011;
-                  for (int _i1012 = 0; _i1012 < _list1010.size; ++_i1012)
+                  org.apache.thrift.protocol.TList _list1042 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list1042.size);
+                  FieldSchema _elem1043;
+                  for (int _i1044 = 0; _i1044 < _list1042.size; ++_i1044)
                   {
-                    _elem1011 = new FieldSchema();
-                    _elem1011.read(iprot);
-                    struct.success.add(_elem1011);
+                    _elem1043 = new FieldSchema();
+                    _elem1043.read(iprot);
+                    struct.success.add(_elem1043);
                   }
                   iprot.readListEnd();
                 }
@@ -52516,9 +52661,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter1013 : struct.success)
+            for (FieldSchema _iter1045 : struct.success)
             {
-              _iter1013.write(oprot);
+              _iter1045.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -52573,9 +52718,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter1014 : struct.success)
+            for (FieldSchema _iter1046 : struct.success)
             {
-              _iter1014.write(oprot);
+              _iter1046.write(oprot);
             }
           }
         }
@@ -52596,14 +52741,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1015 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list1015.size);
-            FieldSchema _elem1016;
-            for (int _i1017 = 0; _i1017 < _list1015.size; ++_i1017)
+            org.apache.thrift.protocol.TList _list1047 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list1047.size);
+            FieldSchema _elem1048;
+            for (int _i1049 = 0; _i1049 < _list1047.size; ++_i1049)
             {
-              _elem1016 = new FieldSchema();
-              _elem1016.read(iprot);
-              struct.success.add(_elem1016);
+              _elem1048 = new FieldSchema();
+              _elem1048.read(iprot);
+              struct.success.add(_elem1048);
             }
           }
           struct.setSuccessIsSet(true);
@@ -53757,14 +53902,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1018 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list1018.size);
-                  FieldSchema _elem1019;
-                  for (int _i1020 = 0; _i1020 < _list1018.size; ++_i1020)
+                  org.apache.thrift.protocol.TList _list1050 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list1050.size);
+                  FieldSchema _elem1051;
+                  for (int _i1052 = 0; _i1052 < _list1050.size; ++_i1052)
                   {
-                    _elem1019 = new FieldSchema();
-                    _elem1019.read(iprot);
-                    struct.success.add(_elem1019);
+                    _elem1051 = new FieldSchema();
+                    _elem1051.read(iprot);
+                    struct.success.add(_elem1051);
                   }
                   iprot.readListEnd();
                 }
@@ -53817,9 +53962,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter1021 : struct.success)
+            for (FieldSchema _iter1053 : struct.success)
             {
-              _iter1021.write(oprot);
+              _iter1053.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -53874,9 +54019,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter1022 : struct.success)
+            for (FieldSchema _iter1054 : struct.success)
             {
-              _iter1022.write(oprot);
+              _iter1054.write(oprot);
             }
           }
         }
@@ -53897,14 +54042,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1023 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list1023.size);
-            FieldSchema _elem1024;
-            for (int _i1025 = 0; _i1025 < _list1023.size; ++_i1025)
+            org.apache.thrift.protocol.TList _list1055 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list1055.size);
+            FieldSchema _elem1056;
+            for (int _i1057 = 0; _i1057 < _list1055.size; ++_i1057)
             {
-              _elem1024 = new FieldSchema();
-              _elem1024.read(iprot);
-              struct.success.add(_elem1024);
+              _elem1056 = new FieldSchema();
+              _elem1056.read(iprot);
+              struct.success.add(_elem1056);
             }
           }
           struct.setSuccessIsSet(true);
@@ -57033,14 +57178,14 @@ import org.slf4j.LoggerFactory;
             case 2: // PRIMARY_KEYS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1026 = iprot.readListBegin();
-                  struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list1026.size);
-                  SQLPrimaryKey _elem1027;
-                  for (int _i1028 = 0; _i1028 < _list1026.size; ++_i1028)
+                  org.apache.thrift.protocol.TList _list1058 = iprot.readListBegin();
+                  struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list1058.size);
+                  SQLPrimaryKey _elem1059;
+                  for (int _i1060 = 0; _i1060 < _list1058.size; ++_i1060)
                   {
-                    _elem1027 = new SQLPrimaryKey();
-                    _elem1027.read(iprot);
-                    struct.primaryKeys.add(_elem1027);
+                    _elem1059 = new SQLPrimaryKey();
+                    _elem1059.read(iprot);
+                    struct.primaryKeys.add(_elem1059);
                   }
                   iprot.readListEnd();
                 }
@@ -57052,14 +57197,14 @@ import org.slf4j.LoggerFactory;
             case 3: // FOREIGN_KEYS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1029 = iprot.readListBegin();
-                  struct.foreignKeys = new ArrayList<SQLForeignKey>(_list1029.size);
-                  SQLForeignKey _elem1030;
-                  for (int _i1031 = 0; _i1031 < _list1029.size; ++_i1031)
+                  org.apache.thrift.protocol.TList _list1061 = iprot.readListBegin();
+                  struct.foreignKeys = new ArrayList<SQLForeignKey>(_list1061.size);
+                  SQLForeignKey _elem1062;
+                  for (int _i1063 = 0; _i1063 < _list1061.size; ++_i1063)
                   {
-                    _elem1030 = new SQLForeignKey();
-                    _elem1030.read(iprot);
-                    struct.foreignKeys.add(_elem1030);
+                    _elem1062 = new SQLForeignKey();
+                    _elem1062.read(iprot);
+                    struct.foreignKeys.add(_elem1062);
                   }
                   iprot.readListEnd();
                 }
@@ -57071,14 +57216,14 @@ import org.slf4j.LoggerFactory;
             case 4: // UNIQUE_CONSTRAINTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1032 = iprot.readListBegin();
-                  struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list1032.size);
-                  SQLUniqueConstraint _elem1033;
-                  for (int _i1034 = 0; _i1034 < _list1032.size; ++_i1034)
+                  org.apache.thrift.protocol.TList _list1064 = iprot.readListBegin();
+                  struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list1064.size);
+                  SQLUniqueConstraint _elem1065;
+                  for (int _i1066 = 0; _i1066 < _list1064.size; ++_i1066)
                   {
-                    _elem1033 = new SQLUniqueConstraint();
-                    _elem1033.read(iprot);
-                    struct.uniqueConstraints.add(_elem1033);
+                    _elem1065 = new SQLUniqueConstraint();
+                    _elem1065.read(iprot);
+                    struct.uniqueConstraints.add(_elem1065);
                   }
                   iprot.readListEnd();
                 }
@@ -57090,14 +57235,14 @@ import org.slf4j.LoggerFactory;
             case 5: // NOT_NULL_CONSTRAINTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1035 = iprot.readListBegin();
-                  struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list1035.size);
-                  SQLNotNullConstraint _elem1036;
-                  for (int _i1037 = 0; _i1037 < _list1035.size; ++_i1037)
+                  org.apache.thrift.protocol.TList _list1067 = iprot.readListBegin();
+                  struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list1067.size);
+                  SQLNotNullConstraint _elem1068;
+                  for (int _i1069 = 0; _i1069 < _list1067.size; ++_i1069)
                   {
-                    _elem1036 = new SQLNotNullConstraint();
-                    _elem1036.read(iprot);
-                    struct.notNullConstraints.add(_elem1036);
+                    _elem1068 = new SQLNotNullConstraint();
+                    _elem1068.read(iprot);
+                    struct.notNullConstraints.add(_elem1068);
                   }
                   iprot.readListEnd();
                 }
@@ -57109,14 +57254,14 @@ import org.slf4j.LoggerFactory;
             case 6: // DEFAULT_CONSTRAINTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1038 = iprot.readListBegin();
-                  struct.defaultConstraints = new ArrayList<SQLDefaultConstraint>(_list1038.size);
-                  SQLDefaultConstraint _elem1039;
-                  for (int _i1040 = 0; _i1040 < _list1038.size; ++_i1040)
+                  org.apache.thrift.protocol.TList _list1070 = iprot.readListBegin();
+                  struct.defaultConstraints = new ArrayList<SQLDefaultConstraint>(_list1070.size);
+                  SQLDefaultConstraint _elem1071;
+                  for (int _i1072 = 0; _i1072 < _list1070.size; ++_i1072)
                   {
-                    _elem1039 = new SQLDefaultConstraint();
-                    _elem1039.read(iprot);
-                    struct.defaultConstraints.add(_elem1039);
+                    _elem1071 = new SQLDefaultConstraint();
+                    _elem1071.read(iprot);
+                    struct.defaultConstraints.add(_elem1071);
                   }
                   iprot.readListEnd();
                 }
@@ -57128,14 +57273,14 @@ import org.slf4j.LoggerFactory;
             case 7: // CHECK_CONSTRAINTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1041 = iprot.readListBegin();
-                  struct.checkConstraints = new ArrayList<SQLCheckConstraint>(_list1041.size);
-                  SQLCheckConstraint _elem1042;
-                  for (int _i1043 = 0; _i1043 < _list1041.size; ++_i1043)
+                  org.apache.thrift.protocol.TList _list1073 = iprot.readListBegin();
+                  struct.checkConstraints = new ArrayList<SQLCheckConstraint>(_list1073.size);
+                  SQLCheckConstraint _elem1074;
+                  for (int _i1075 = 0; _i1075 < _list1073.size; ++_i1075)
                   {
-                    _elem1042 = new SQLCheckConstraint();
-                    _elem1042.read(iprot);
-                    struct.checkConstraints.add(_elem1042);
+                    _elem1074 = new SQLCheckConstraint();
+                    _elem1074.read(iprot);
+                    struct.checkConstraints.add(_elem1074);
                   }
                   iprot.readListEnd();
                 }
@@ -57166,9 +57311,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PRIMARY_KEYS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.primaryKeys.size()));
-            for (SQLPrimaryKey _iter1044 : struct.primaryKeys)
+            for (SQLPrimaryKey _iter1076 : struct.primaryKeys)
             {
-              _iter1044.write(oprot);
+              _iter1076.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -57178,9 +57323,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(FOREIGN_KEYS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.foreignKeys.size()));
-            for (SQLForeignKey _iter1045 : struct.foreignKeys)
+            for (SQLForeignKey _iter1077 : struct.foreignKeys)
             {
-              _iter1045.write(oprot);
+              _iter1077.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -57190,9 +57335,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(UNIQUE_CONSTRAINTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.uniqueConstraints.size()));
-            for (SQLUniqueConstraint _iter1046 : struct.uniqueConstraints)
+            for (SQLUniqueConstraint _iter1078 : struct.uniqueConstraints)
             {
-              _iter1046.write(oprot);
+              _iter1078.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -57202,9 +57347,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(NOT_NULL_CONSTRAINTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.notNullConstraints.size()));
-            for (SQLNotNullConstraint _iter1047 : struct.notNullConstraints)
+            for (SQLNotNullConstraint _iter1079 : struct.notNullConstraints)
             {
-              _iter1047.write(oprot);
+              _iter1079.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -57214,9 +57359,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(DEFAULT_CONSTRAINTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.defaultConstraints.size()));
-            for (SQLDefaultConstraint _iter1048 : struct.defaultConstraints)
+            for (SQLDefaultConstraint _iter1080 : struct.defaultConstraints)
             {
-              _iter1048.write(oprot);
+              _iter1080.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -57226,9 +57371,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(CHECK_CONSTRAINTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.checkConstraints.size()));
-            for (SQLCheckConstraint _iter1049 : struct.checkConstraints)
+            for (SQLCheckConstraint _iter1081 : struct.checkConstraints)
             {
-              _iter1049.write(oprot);
+              _iter1081.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -57280,54 +57425,54 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetPrimaryKeys()) {
           {
             oprot.writeI32(struct.primaryKeys.size());
-            for (SQLPrimaryKey _iter1050 : struct.primaryKeys)
+            for (SQLPrimaryKey _iter1082 : struct.primaryKeys)
             {
-              _iter1050.write(oprot);
+              _iter1082.write(oprot);
             }
           }
         }
         if (struct.isSetForeignKeys()) {
           {
             oprot.writeI32(struct.foreignKeys.size());
-            for (SQLForeignKey _iter1051 : struct.foreignKeys)
+            for (SQLForeignKey _iter1083 : struct.foreignKeys)
             {
-              _iter1051.write(oprot);
+              _iter1083.write(oprot);
             }
           }
         }
         if (struct.isSetUniqueConstraints()) {
           {
             oprot.writeI32(struct.uniqueConstraints.size());
-            for (SQLUniqueConstraint _iter1052 : struct.uniqueConstraints)
+            for (SQLUniqueConstraint _iter1084 : struct.uniqueConstraints)
             {
-              _iter1052.write(oprot);
+              _iter1084.write(oprot);
             }
           }
         }
         if (struct.isSetNotNullConstraints()) {
           {
             oprot.writeI32(struct.notNullConstraints.size());
-            for (SQLNotNullConstraint _iter1053 : struct.notNullConstraints)
+            for (SQLNotNullConstraint _iter1085 : struct.notNullConstraints)
             {
-              _iter1053.write(oprot);
+              _iter1085.write(oprot);
             }
           }
         }
         if (struct.isSetDefaultConstraints()) {
           {
             oprot.writeI32(struct.defaultConstraints.size());
-            for (SQLDefaultConstraint _iter1054 : struct.defaultConstraints)
+            for (SQLDefaultConstraint _iter1086 : struct.defaultConstraints)
             {
-              _iter1054.write(oprot);
+              _iter1086.write(oprot);
             }
           }
         }
         if (struct.isSetCheckConstraints()) {
           {
             oprot.writeI32(struct.checkConstraints.size());
-            for (SQLCheckConstraint _iter1055 : struct.checkConstraints)
+            for (SQLCheckConstraint _iter1087 : struct.checkConstraints)
             {
-              _iter1055.write(oprot);
+              _iter1087.write(oprot);
             }
           }
         }
@@ -57344,84 +57489,84 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(1)) {
           {
-            org.apache.thrift.protocol.TList _list1056 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list1056.size);
-            SQLPrimaryKey _elem1057;
-            for (int _i1058 = 0; _i1058 < _list1056.size; ++_i1058)
+            org.apache.thrift.protocol.TList _list1088 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list1088.size);
+            SQLPrimaryKey _elem1089;
+            for (int _i1090 = 0; _i1090 < _list1088.size; ++_i1090)
             {
-              _elem1057 = new SQLPrimaryKey();
-              _elem1057.read(iprot);
-              struct.primaryKeys.add(_elem1057);
+              _elem1089 = new SQLPrimaryKey();
+              _elem1089.read(iprot);
+              struct.primaryKeys.add(_elem1089);
             }
           }
           struct.setPrimaryKeysIsSet(true);
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1059 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.foreignKeys = new ArrayList<SQLForeignKey>(_list1059.size);
-            SQLForeignKey _elem1060;
-            for (int _i1061 = 0; _i1061 < _list1059.size; ++_i1061)
+            org.apache.thrift.protocol.TList _list1091 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.foreignKeys = new ArrayList<SQLForeignKey>(_list1091.size);
+            SQLForeignKey _elem1092;
+            for (int _i1093 = 0; _i1093 < _list1091.size; ++_i1093)
             {
-              _elem1060 = new SQLForeignKey();
-              _elem1060.read(iprot);
-              struct.foreignKeys.add(_elem1060);
+              _elem1092 = new SQLForeignKey();
+              _elem1092.read(iprot);
+              struct.foreignKeys.add(_elem1092);
             }
           }
           struct.setForeignKeysIsSet(true);
         }
         if (incoming.get(3)) {
           {
-            org.apache.thrift.protocol.TList _list1062 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list1062.size);
-            SQLUniqueConstraint _elem1063;
-            for (int _i1064 = 0; _i1064 < _list1062.size; ++_i1064)
+            org.apache.thrift.protocol.TList _list1094 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list1094.size);
+            SQLUniqueConstraint _elem1095;
+            for (int _i1096 = 0; _i1096 < _list1094.size; ++_i1096)
             {
-              _elem1063 = new SQLUniqueConstraint();
-              _elem1063.read(iprot);
-              struct.uniqueConstraints.add(_elem1063);
+              _elem1095 = new SQLUniqueConstraint();
+              _elem1095.read(iprot);
+              struct.uniqueConstraints.add(_elem1095);
             }
           }
           struct.setUniqueConstraintsIsSet(true);
         }
         if (incoming.get(4)) {
           {
-            org.apache.thrift.protocol.TList _list1065 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list1065.size);
-            SQLNotNullConstraint _elem1066;
-            for (int _i1067 = 0; _i1067 < _list1065.size; ++_i1067)
+            org.apache.thrift.protocol.TList _list1097 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list1097.size);
+            SQLNotNullConstraint _elem1098;
+            for (int _i1099 = 0; _i1099 < _list1097.size; ++_i1099)
             {
-              _elem1066 = new SQLNotNullConstraint();
-              _elem1066.read(iprot);
-              struct.notNullConstraints.add(_elem1066);
+              _elem1098 = new SQLNotNullConstraint();
+              _elem1098.read(iprot);
+              struct.notNullConstraints.add(_elem1098);
             }
           }
           struct.setNotNullConstraintsIsSet(true);
         }
         if (incoming.get(5)) {
           {
-            org.apache.thrift.protocol.TList _list1068 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.defaultConstraints = new ArrayList<SQLDefaultConstraint>(_list1068.size);
-            SQLDefaultConstraint _elem1069;
-            for (int _i1070 = 0; _i1070 < _list1068.size; ++_i1070)
+            org.apache.thrift.protocol.TList _list1100 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.defaultConstraints = new ArrayList<SQLDefaultConstraint>(_list1100.size);
+            SQLDefaultConstraint _elem1101;
+            for (int _i1102 = 0; _i1102 < _list1100.size; ++_i1102)
             {
-              _elem1069 = new SQLDefaultConstraint();
-              _elem1069.read(iprot);
-              struct.defaultConstraints.add(_elem1069);
+              _elem1101 = new SQLDefaultConstraint();
+              _elem1101.read(iprot);
+              struct.defaultConstraints.add(_elem1101);
             }
           }
           struct.setDefaultConstraintsIsSet(true);
         }
         if (incoming.get(6)) {
           {
-            org.apache.thrift.protocol.TList _list1071 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.checkConstraints = new ArrayList<SQLCheckConstraint>(_list1071.size);
-            SQLCheckConstraint _elem1072;
-            for (int _i1073 = 0; _i1073 < _list1071.size; ++_i1073)
+            org.apache.thrift.protocol.TList _list1103 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.checkConstraints = new ArrayList<SQLCheckConstraint>(_list1103.size);
+            SQLCheckConstraint _elem1104;
+            for (int _i1105 = 0; _i1105 < _list1103.size; ++_i1105)
             {
-              _elem1072 = new SQLCheckConstraint();
-              _elem1072.read(iprot);
-              struct.checkConstraints.add(_elem1072);
+              _elem1104 = new SQLCheckConstraint();
+              _elem1104.read(iprot);
+              struct.checkConstraints.add(_elem1104);
             }
           }
           struct.setCheckConstraintsIsSet(true);
@@ -66571,13 +66716,13 @@ import org.slf4j.LoggerFactory;
             case 3: // PART_NAMES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1074 = iprot.readListBegin();
-                  struct.partNames = new ArrayList<String>(_list1074.size);
-                  String _elem1075;
-                  for (int _i1076 = 0; _i1076 < _list1074.size; ++_i1076)
+                  org.apache.thrift.protocol.TList _list1106 = iprot.readListBegin();
+                  struct.partNames = new ArrayList<String>(_list1106.size);
+                  String _elem1107;
+                  for (int _i1108 = 0; _i1108 < _list1106.size; ++_i1108)
                   {
-                    _elem1075 = iprot.readString();
-                    struct.partNames.add(_elem1075);
+                    _elem1107 = iprot.readString();
+                    struct.partNames.add(_elem1107);
                   }
                   iprot.readListEnd();
                 }
@@ -66613,9 +66758,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PART_NAMES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.partNames.size()));
-            for (String _iter1077 : struct.partNames)
+            for (String _iter1109 : struct.partNames)
             {
-              oprot.writeString(_iter1077);
+              oprot.writeString(_iter1109);
             }
             oprot.writeListEnd();
           }
@@ -66658,9 +66803,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetPartNames()) {
           {
             oprot.writeI32(struct.partNames.size());
-            for (String _iter1078 : struct.partNames)
+            for (String _iter1110 : struct.partNames)
             {
-              oprot.writeString(_iter1078);
+              oprot.writeString(_iter1110);
             }
           }
         }
@@ -66680,13 +66825,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1079 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.partNames = new ArrayList<String>(_list1079.size);
-            String _elem1080;
-            for (int _i1081 = 0; _i1081 < _list1079.size; ++_i1081)
+            org.apache.thrift.protocol.TList _list1111 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.partNames = new ArrayList<String>(_list1111.size);
+            String _elem1112;
+            for (int _i1113 = 0; _i1113 < _list1111.size; ++_i1113)
             {
-              _elem1080 = iprot.readString();
-              struct.partNames.add(_elem1080);
+              _elem1112 = iprot.readString();
+              struct.partNames.add(_elem1112);
             }
           }
           struct.setPartNamesIsSet(true);
@@ -68743,13 +68888,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1082 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1082.size);
-                  String _elem1083;
-                  for (int _i1084 = 0; _i1084 < _list1082.size; ++_i1084)
+                  org.apache.thrift.protocol.TList _list1114 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1114.size);
+                  String _elem1115;
+                  for (int _i1116 = 0; _i1116 < _list1114.size; ++_i1116)
                   {
-                    _elem1083 = iprot.readString();
-                    struct.success.add(_elem1083);
+                    _elem1115 = iprot.readString();
+                    struct.success.add(_elem1115);
                   }
                   iprot.readListEnd();
                 }
@@ -68784,9 +68929,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter1085 : struct.success)
+            for (String _iter1117 : struct.success)
             {
-              oprot.writeString(_iter1085);
+              oprot.writeString(_iter1117);
             }
             oprot.writeListEnd();
           }
@@ -68825,9 +68970,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1086 : struct.success)
+            for (String _iter1118 : struct.success)
             {
-              oprot.writeString(_iter1086);
+              oprot.writeString(_iter1118);
             }
           }
         }
@@ -68842,13 +68987,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1087 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1087.size);
-            String _elem1088;
-            for (int _i1089 = 0; _i1089 < _list1087.size; ++_i1089)
+            org.apache.thrift.protocol.TList _list1119 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1119.size);
+            String _elem1120;
+            for (int _i1121 = 0; _i1121 < _list1119.size; ++_i1121)
             {
-              _elem1088 = iprot.readString();
-              struct.success.add(_elem1088);
+              _elem1120 = iprot.readString();
+              struct.success.add(_elem1120);
             }
           }
           struct.setSuccessIsSet(true);
@@ -69822,13 +69967,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1090 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1090.size);
-                  String _elem1091;
-                  for (int _i1092 = 0; _i1092 < _list1090.size; ++_i1092)
+                  org.apache.thrift.protocol.TList _list1122 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1122.size);
+                  String _elem1123;
+                  for (int _i1124 = 0; _i1124 < _list1122.size; ++_i1124)
                   {
-                    _elem1091 = iprot.readString();
-                    struct.success.add(_elem1091);
+                    _elem1123 = iprot.readString();
+                    struct.success.add(_elem1123);
                   }
                   iprot.readListEnd();
                 }
@@ -69863,9 +70008,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter1093 : struct.success)
+            for (String _iter1125 : struct.success)
             {
-              oprot.writeString(_iter1093);
+              oprot.writeString(_iter1125);
             }
             oprot.writeListEnd();
           }
@@ -69904,9 +70049,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1094 : struct.success)
+            for (String _iter1126 : struct.success)
             {
-              oprot.writeString(_iter1094);
+              oprot.writeString(_iter1126);
             }
           }
         }
@@ -69921,13 +70066,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1095 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1095.size);
-            String _elem1096;
-            for (int _i1097 = 0; _i1097 < _list1095.size; ++_i1097)
+            org.apache.thrift.protocol.TList _list1127 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1127.size);
+            String _elem1128;
+            for (int _i1129 = 0; _i1129 < _list1127.size; ++_i1129)
             {
-              _elem1096 = iprot.readString();
-              struct.success.add(_elem1096);
+              _elem1128 = iprot.readString();
+              struct.success.add(_elem1128);
             }
           }
           struct.setSuccessIsSet(true);
@@ -70693,13 +70838,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1098 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1098.size);
-                  String _elem1099;
-                  for (int _i1100 = 0; _i1100 < _list1098.size; ++_i1100)
+                  org.apache.thrift.protocol.TList _list1130 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1130.size);
+                  String _elem1131;
+                  for (int _i1132 = 0; _i1132 < _list1130.size; ++_i1132)
                   {
-                    _elem1099 = iprot.readString();
-                    struct.success.add(_elem1099);
+                    _elem1131 = iprot.readString();
+                    struct.success.add(_elem1131);
                   }
                   iprot.readListEnd();
                 }
@@ -70734,9 +70879,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter1101 : struct.success)
+            for (String _iter1133 : struct.success)
             {
-              oprot.writeString(_iter1101);
+              oprot.writeString(_iter1133);
             }
             oprot.writeListEnd();
           }
@@ -70775,9 +70920,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1102 : struct.success)
+            for (String _iter1134 : struct.success)
             {
-              oprot.writeString(_iter1102);
+              oprot.writeString(_iter1134);
             }
           }
         }
@@ -70792,13 +70937,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1103 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1103.size);
-            String _elem1104;
-            for (int _i1105 = 0; _i1105 < _list1103.size; ++_i1105)
+            org.apache.thrift.protocol.TList _list1135 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1135.size);
+            String _elem1136;
+            for (int _i1137 = 0; _i1137 < _list1135.size; ++_i1137)
             {
-              _elem1104 = iprot.readString();
-              struct.success.add(_elem1104);
+              _elem1136 = iprot.readString();
+              struct.success.add(_elem1136);
             }
           }
           struct.setSuccessIsSet(true);
@@ -71303,13 +71448,13 @@ import org.slf4j.LoggerFactory;
             case 3: // TBL_TYPES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1106 = iprot.readListBegin();
-                  struct.tbl_types = new ArrayList<String>(_list1106.size);
-                  String _elem1107;
-                  for (int _i1108 = 0; _i1108 < _list1106.size; ++_i1108)
+                  org.apache.thrift.protocol.TList _list1138 = iprot.readListBegin();
+                  struct.tbl_types = new ArrayList<String>(_list1138.size);
+                  String _elem1139;
+                  for (int _i1140 = 0; _i1140 < _list1138.size; ++_i1140)
                   {
-                    _elem1107 = iprot.readString();
-                    struct.tbl_types.add(_elem1107);
+                    _elem1139 = iprot.readString();
+                    struct.tbl_types.add(_elem1139);
                   }
                   iprot.readListEnd();
                 }
@@ -71345,9 +71490,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(TBL_TYPES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.tbl_types.size()));
-            for (String _iter1109 : struct.tbl_types)
+            for (String _iter1141 : struct.tbl_types)
             {
-              oprot.writeString(_iter1109);
+              oprot.writeString(_iter1141);
             }
             oprot.writeListEnd();
           }
@@ -71390,9 +71535,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetTbl_types()) {
           {
             oprot.writeI32(struct.tbl_types.size());
-            for (String _iter1110 : struct.tbl_types)
+            for (String _iter1142 : struct.tbl_types)
             {
-              oprot.writeString(_iter1110);
+              oprot.writeString(_iter1142);
             }
           }
         }
@@ -71412,13 +71557,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1111 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.tbl_types = new ArrayList<String>(_list1111.size);
-            String _elem1112;
-            for (int _i1113 = 0; _i1113 < _list1111.size; ++_i1113)
+            org.apache.thrift.protocol.TList _list1143 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.tbl_types = new ArrayList<String>(_list1143.size);
+            String _elem1144;
+            for (int _i1145 = 0; _i1145 < _list1143.size; ++_i1145)
             {
-              _elem1112 = iprot.readString();
-              struct.tbl_types.add(_elem1112);
+              _elem1144 = iprot.readString();
+              struct.tbl_types.add(_elem1144);
             }
           }
           struct.setTbl_typesIsSet(true);
@@ -71824,14 +71969,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1114 = iprot.readListBegin();
-                  struct.success = new ArrayList<TableMeta>(_list1114.size);
-                  TableMeta _elem1115;
-                  for (int _i1116 = 0; _i1116 < _list1114.size; ++_i1116)
+                  org.apache.thrift.protocol.TList _list1146 = iprot.readListBegin();
+                  struct.success = new ArrayList<TableMeta>(_list1146.size);
+                  TableMeta _elem1147;
+                  for (int _i1148 = 0; _i1148 < _list1146.size; ++_i1148)
                   {
-                    _elem1115 = new TableMeta();
-                    _elem1115.read(iprot);
-                    struct.success.add(_elem1115);
+                    _elem1147 = new TableMeta();
+                    _elem1147.read(iprot);
+                    struct.success.add(_elem1147);
                   }
                   iprot.readListEnd();
                 }
@@ -71866,9 +72011,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (TableMeta _iter1117 : struct.success)
+            for (TableMeta _iter1149 : struct.success)
             {
-              _iter1117.write(oprot);
+              _iter1149.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -71907,9 +72052,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (TableMeta _iter1118 : struct.success)
+            for (TableMeta _iter1150 : struct.success)
             {
-              _iter1118.write(oprot);
+              _iter1150.write(oprot);
             }
           }
         }
@@ -71924,14 +72069,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1119 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<TableMeta>(_list1119.size);
-            TableMeta _elem1120;
-            for (int _i1121 = 0; _i1121 < _list1119.size; ++_i1121)
+            org.apache.thrift.protocol.TList _list1151 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<TableMeta>(_list1151.size);
+            TableMeta _elem1152;
+            for (int _i1153 = 0; _i1153 < _list1151.size; ++_i1153)
             {
-              _elem1120 = new TableMeta();
-              _elem1120.read(iprot);
-              struct.success.add(_elem1120);
+              _elem1152 = new TableMeta();
+              _elem1152.read(iprot);
+              struct.success.add(_elem1152);
             }
           }
           struct.setSuccessIsSet(true);
@@ -72697,13 +72842,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1122 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1122.size);
-                  String _elem1123;
-                  for (int _i1124 = 0; _i1124 < _list1122.size; ++_i1124)
+                  org.apache.thrift.protocol.TList _list1154 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1154.size);
+                  String _elem1155;
+                  for (int _i1156 = 0; _i1156 < _list1154.size; ++_i1156)
                   {
-                    _elem1123 = iprot.readString();
-                    struct.success.add(_elem1123);
+                    _elem1155 = iprot.readString();
+                    struct.success.add(_elem1155);
                   }
                   iprot.readListEnd();
                 }
@@ -72738,9 +72883,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter1125 : struct.success)
+            for (String _iter1157 : struct.success)
             {
-              oprot.writeString(_iter1125);
+              oprot.writeString(_iter1157);
             }
             oprot.writeListEnd();
           }
@@ -72779,9 +72924,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1126 : struct.success)
+            for (String _iter1158 : struct.success)
             {
-              oprot.writeString(_iter1126);
+              oprot.writeString(_iter1158);
             }
           }
         }
@@ -72796,13 +72941,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1127 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1127.size);
-            String _elem1128;
-            for (int _i1129 = 0; _i1129 < _list1127.size; ++_i1129)
+            org.apache.thrift.protocol.TList _list1159 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1159.size);
+            String _elem1160;
+            for (int _i1161 = 0; _i1161 < _list1159.size; ++_i1161)
             {
-              _elem1128 = iprot.readString();
-              struct.success.add(_elem1128);
+              _elem1160 = iprot.readString();
+              struct.success.add(_elem1160);
             }
           }
           struct.setSuccessIsSet(true);
@@ -74255,13 +74400,13 @@ import org.slf4j.LoggerFactory;
             case 2: // TBL_NAMES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1130 = iprot.readListBegin();
-                  struct.tbl_names = new ArrayList<String>(_list1130.size);
-                  String _elem1131;
-                  for (int _i1132 = 0; _i1132 < _list1130.size; ++_i1132)
+                  org.apache.thrift.protocol.TList _list1162 = iprot.readListBegin();
+                  struct.tbl_names = new ArrayList<String>(_list1162.size);
+                  String _elem1163;
+                  for (int _i1164 = 0; _i1164 < _list1162.size; ++_i1164)
                   {
-                    _elem1131 = iprot.readString();
-                    struct.tbl_names.add(_elem1131);
+                    _elem1163 = iprot.readString();
+                    struct.tbl_names.add(_elem1163);
                   }
                   iprot.readListEnd();
                 }
@@ -74292,9 +74437,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(TBL_NAMES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.tbl_names.size()));
-            for (String _iter1133 : struct.tbl_names)
+            for (String _iter1165 : struct.tbl_names)
             {
-              oprot.writeString(_iter1133);
+              oprot.writeString(_iter1165);
             }
             oprot.writeListEnd();
           }
@@ -74331,9 +74476,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetTbl_names()) {
           {
             oprot.writeI32(struct.tbl_names.size());
-            for (String _iter1134 : struct.tbl_names)
+            for (String _iter1166 : struct.tbl_names)
             {
-              oprot.writeString(_iter1134);
+              oprot.writeString(_iter1166);
             }
           }
         }
@@ -74349,13 +74494,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(1)) {
           {
-            org.apache.thrift.protocol.TList _list1135 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.tbl_names = new ArrayList<String>(_list1135.size);
-            String _elem1136;
-            for (int _i1137 = 0; _i1137 < _list1135.size; ++_i1137)
+            org.apache.thrift.protocol.TList _list1167 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.tbl_names = new ArrayList<String>(_list1167.size);
+            String _elem1168;
+            for (int _i1169 = 0; _i1169 < _list1167.size; ++_i1169)
             {
-              _elem1136 = iprot.readString();
-              struct.tbl_names.add(_elem1136);
+              _elem1168 = iprot.readString();
+              struct.tbl_names.add(_elem1168);
             }
           }
           struct.setTbl_namesIsSet(true);
@@ -74680,14 +74825,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1138 = iprot.readListBegin();
-                  struct.success = new ArrayList<Table>(_list1138.size);
-                  Table _elem1139;
-                  for (int _i1140 = 0; _i1140 < _list1138.size; ++_i1140)
+                  org.apache.thrift.protocol.TList _list1170 = iprot.readListBegin();
+                  struct.success = new ArrayList<Table>(_list1170.size);
+                  Table _elem1171;
+                  for (int _i1172 = 0; _i1172 < _list1170.size; ++_i1172)
                   {
-                    _elem1139 = new Table();
-                    _elem1139.read(iprot);
-                    struct.success.add(_elem1139);
+                    _elem1171 = new Table();
+                    _elem1171.read(iprot);
+                    struct.success.add(_elem1171);
                   }
                   iprot.readListEnd();
                 }
@@ -74713,9 +74858,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (Table _iter1141 : struct.success)
+            for (Table _iter1173 : struct.success)
             {
-              _iter1141.write(oprot);
+              _iter1173.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -74746,9 +74891,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Table _iter1142 : struct.success)
+            for (Table _iter1174 : struct.success)
             {
-              _iter1142.write(oprot);
+              _iter1174.write(oprot);
             }
           }
         }
@@ -74760,14 +74905,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1143 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<Table>(_list1143.size);
-            Table _elem1144;
-            for (int _i1145 = 0; _i1145 < _list1143.size; ++_i1145)
+            org.apache.thrift.protocol.TList _list1175 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<Table>(_list1175.size);
+            Table _elem1176;
+            for (int _i1177 = 0; _i1177 < _list1175.size; ++_i1177)
             {
-              _elem1144 = new Table();
-              _elem1144.read(iprot);
-              struct.success.add(_elem1144);
+              _elem1176 = new Table();
+              _elem1176.read(iprot);
+              struct.success.add(_elem1176);
             }
           }
           struct.setSuccessIsSet(true);
@@ -80275,13 +80420,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1146 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1146.size);
-                  String _elem1147;
-                  for (int _i1148 = 0; _i1148 < _list1146.size; ++_i1148)
+                  org.apache.thrift.protocol.TList _list1178 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1178.size);
+                  String _elem1179;
+                  for (int _i1180 = 0; _i1180 < _list1178.size; ++_i1180)
                   {
-                    _elem1147 = iprot.readString();
-                    struct.success.add(_elem1147);
+                    _elem1179 = iprot.readString();
+                    struct.success.add(_elem1179);
                   }
                   iprot.readListEnd();
                 }
@@ -80334,9 +80479,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter1149 : struct.success)
+            for (String _iter1181 : struct.success)
             {
-              oprot.writeString(_iter1149);
+              oprot.writeString(_iter1181);
             }
             oprot.writeListEnd();
           }
@@ -80391,9 +80536,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1150 : struct.success)
+            for (String _iter1182 : struct.success)
             {
-              oprot.writeString(_iter1150);
+              oprot.writeString(_iter1182);
             }
           }
         }
@@ -80414,13 +80559,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1151 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1151.size);
-            String _elem1152;
-            for (int _i1153 = 0; _i1153 < _list1151.size; ++_i1153)
+            org.apache.thrift.protocol.TList _list1183 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1183.size);
+            String _elem1184;
+            for (int _i1185 = 0; _i1185 < _list1183.size; ++_i1185)
             {
-              _elem1152 = iprot.readString();
-              struct.success.add(_elem1152);
+              _elem1184 = iprot.readString();
+              struct.success.add(_elem1184);
             }
           }
           struct.setSuccessIsSet(true);
@@ -87217,14 +87362,14 @@ import org.slf4j.LoggerFactory;
             case 1: // NEW_PARTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1154 = iprot.readListBegin();
-                  struct.new_parts = new ArrayList<Partition>(_list1154.size);
-                  Partition _elem1155;
-                  for (int _i1156 = 0; _i1156 < _list1154.size; ++_i1156)
+                  org.apache.thrift.protocol.TList _list1186 = iprot.readListBegin();
+                  struct.new_parts = new ArrayList<Partition>(_list1186.size);
+                  Partition _elem1187;
+                  for (int _i1188 = 0; _i1188 < _list1186.size; ++_i1188)
                   {
-                    _elem1155 = new Partition();
-                    _elem1155.read(iprot);
-                    struct.new_parts.add(_elem1155);
+                    _elem1187 = new Partition();
+                    _elem1187.read(iprot);
+                    struct.new_parts.add(_elem1187);
                   }
                   iprot.readListEnd();
                 }
@@ -87250,9 +87395,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(NEW_PARTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.new_parts.size()));
-            for (Partition _iter1157 : struct.new_parts)
+            for (Partition _iter1189 : struct.new_parts)
             {
-              _iter1157.write(oprot);
+              _iter1189.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -87283,9 +87428,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetNew_parts()) {
           {
             oprot.writeI32(struct.new_parts.size());
-            for (Partition _iter1158 : struct.new_parts)
+            for (Partition _iter1190 : struct.new_parts)
             {
-              _iter1158.write(oprot);
+              _iter1190.write(oprot);
             }
           }
         }
@@ -87297,14 +87442,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1159 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.new_parts = new ArrayList<Partition>(_list1159.size);
-            Partition _elem1160;
-            for (int _i1161 = 0; _i1161 < _list1159.size; ++_i1161)
+            org.apache.thrift.protocol.TList _list1191 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.new_parts = new ArrayList<Partition>(_list1191.size);
+            Partition _elem1192;
+            for (int _i1193 = 0; _i1193 < _list1191.size; ++_i1193)
             {
-              _elem1160 = new Partition();
-              _elem1160.read(iprot);
-              struct.new_parts.add(_elem1160);
+              _elem1192 = new Partition();
+              _elem1192.read(iprot);
+              struct.new_parts.add(_elem1192);
             }
           }
           struct.setNew_partsIsSet(true);
@@ -88305,14 +88450,14 @@ import org.slf4j.LoggerFactory;
             case 1: // NEW_PARTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1162 = iprot.readListBegin();
-                  struct.new_parts = new ArrayList<PartitionSpec>(_list1162.size);
-                  PartitionSpec _elem1163;
-                  for (int _i1164 = 0; _i1164 < _list1162.size; ++_i1164)


<TRUNCATED>

[13/13] hive git commit: HIVE-20306 : Implement projection spec for fetching only requested fields from partitions (Vihang Karajgaonkar, reviewed by Aihua Xu, Andrew Sherman and Alexander Kolbasov)

Posted by vi...@apache.org.
HIVE-20306 : Implement projection spec for fetching only requested fields from partitions (Vihang Karajgaonkar, reviewed by Aihua Xu, Andrew Sherman and Alexander Kolbasov)


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

Branch: refs/heads/master
Commit: 44ef91a67c8493dbb7bae13ddc1c208473cb093f
Parents: 827f4f9
Author: Vihang Karajgaonkar <vi...@apache.org>
Authored: Sun Oct 7 15:37:25 2018 -0700
Committer: Vihang Karajgaonkar <vi...@apache.org>
Committed: Sun Oct 7 15:37:31 2018 -0700

----------------------------------------------------------------------
 .../listener/DummyRawStoreFailEvent.java        |     9 +
 .../metastore/api/GetPartitionsFilterSpec.java  |   560 +
 .../api/GetPartitionsProjectionSpec.java        |   652 +
 .../metastore/api/GetPartitionsRequest.java     |  1188 ++
 .../metastore/api/GetPartitionsResponse.java    |   449 +
 .../hive/metastore/api/PartitionFilterMode.java |    48 +
 .../hive/metastore/api/ThriftHiveMetastore.java |  4027 ++++--
 .../gen-php/metastore/ThriftHiveMetastore.php   |  1613 ++-
 .../src/gen/thrift/gen-php/metastore/Types.php  |   657 +
 .../hive_metastore/ThriftHiveMetastore-remote   |     7 +
 .../hive_metastore/ThriftHiveMetastore.py       |  1132 +-
 .../gen/thrift/gen-py/hive_metastore/ttypes.py  |   448 +
 .../gen/thrift/gen-rb/hive_metastore_types.rb   |    95 +
 .../gen/thrift/gen-rb/thrift_hive_metastore.rb  |    61 +
 .../hive/metastore/HiveMetaStoreClient.java     |     6 +
 .../hadoop/hive/metastore/IMetaStoreClient.java |    21 +
 .../hive/metastore/utils/MetaStoreUtils.java    |     5 +
 .../src/main/thrift/hive_metastore.thrift       |    64 +
 .../hadoop/hive/metastore/HiveMetaStore.java    |   165 +-
 .../hive/metastore/MetaStoreDirectSql.java      |   741 +-
 .../hive/metastore/MetaStoreDirectSql.java.orig |  2845 ++++
 .../hive/metastore/MetastoreDirectSqlUtils.java |   571 +
 .../hadoop/hive/metastore/ObjectStore.java      |   140 +-
 .../hadoop/hive/metastore/ObjectStore.java.orig | 12514 +++++++++++++++++
 .../metastore/PartitionProjectionEvaluator.java |   889 ++
 .../apache/hadoop/hive/metastore/RawStore.java  |    38 +
 .../hive/metastore/StatObjectConverter.java     |   153 +-
 .../hive/metastore/cache/CachedStore.java       |    13 +
 .../hadoop/hive/metastore/model/MSerDeInfo.java |     3 +
 .../metastore/model/MStorageDescriptor.java     |     9 +-
 .../metastore/utils/MetaStoreServerUtils.java   |   282 +-
 .../DummyRawStoreControlledCommit.java          |    39 +-
 .../DummyRawStoreForJdoConnection.java          |     7 +
 .../HiveMetaStoreClientPreCatalog.java          |     6 +
 .../TestGetPartitionsUsingProjection.java       |   700 +
 .../hive/metastore/TestHiveMetaStore.java       |   141 +-
 .../TestPartitionProjectionEvaluator.java       |   250 +
 .../utils/TestMetaStoreServerUtils.java         |   356 +-
 38 files changed, 27522 insertions(+), 3382 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/44ef91a6/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 0ad2a24..d59d5d8 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
@@ -413,6 +413,15 @@ public class DummyRawStoreFailEvent implements RawStore, Configurable {
   }
 
   @Override
+  public List<Partition> getPartitionSpecsByFilterAndProjection(String catalog,
+                                                                String dbName, String tblName,
+                                                                List<String> fieldList, String includeParamKeyPattern,
+      String excludeParamKeyPattern) throws MetaException, NoSuchObjectException {
+    return objectStore.getPartitionSpecsByFilterAndProjection(catalog, dbName, tblName, fieldList,
+        includeParamKeyPattern, excludeParamKeyPattern);
+  }
+
+  @Override
   public int getNumPartitionsByFilter(String catName, String dbName, String tblName,
                                       String filter) throws MetaException, NoSuchObjectException {
     return objectStore.getNumPartitionsByFilter(catName, dbName, tblName, filter);

http://git-wip-us.apache.org/repos/asf/hive/blob/44ef91a6/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsFilterSpec.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsFilterSpec.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsFilterSpec.java
new file mode 100644
index 0000000..57511ce
--- /dev/null
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsFilterSpec.java
@@ -0,0 +1,560 @@
+/**
+ * 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 GetPartitionsFilterSpec implements org.apache.thrift.TBase<GetPartitionsFilterSpec, GetPartitionsFilterSpec._Fields>, java.io.Serializable, Cloneable, Comparable<GetPartitionsFilterSpec> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetPartitionsFilterSpec");
+
+  private static final org.apache.thrift.protocol.TField FILTER_MODE_FIELD_DESC = new org.apache.thrift.protocol.TField("filterMode", org.apache.thrift.protocol.TType.I32, (short)7);
+  private static final org.apache.thrift.protocol.TField FILTERS_FIELD_DESC = new org.apache.thrift.protocol.TField("filters", org.apache.thrift.protocol.TType.LIST, (short)8);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new GetPartitionsFilterSpecStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new GetPartitionsFilterSpecTupleSchemeFactory());
+  }
+
+  private PartitionFilterMode filterMode; // optional
+  private List<String> filters; // 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 PartitionFilterMode
+     */
+    FILTER_MODE((short)7, "filterMode"),
+    FILTERS((short)8, "filters");
+
+    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 7: // FILTER_MODE
+          return FILTER_MODE;
+        case 8: // FILTERS
+          return FILTERS;
+        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.FILTER_MODE,_Fields.FILTERS};
+  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.FILTER_MODE, new org.apache.thrift.meta_data.FieldMetaData("filterMode", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, PartitionFilterMode.class)));
+    tmpMap.put(_Fields.FILTERS, new org.apache.thrift.meta_data.FieldMetaData("filters", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetPartitionsFilterSpec.class, metaDataMap);
+  }
+
+  public GetPartitionsFilterSpec() {
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public GetPartitionsFilterSpec(GetPartitionsFilterSpec other) {
+    if (other.isSetFilterMode()) {
+      this.filterMode = other.filterMode;
+    }
+    if (other.isSetFilters()) {
+      List<String> __this__filters = new ArrayList<String>(other.filters);
+      this.filters = __this__filters;
+    }
+  }
+
+  public GetPartitionsFilterSpec deepCopy() {
+    return new GetPartitionsFilterSpec(this);
+  }
+
+  @Override
+  public void clear() {
+    this.filterMode = null;
+    this.filters = null;
+  }
+
+  /**
+   * 
+   * @see PartitionFilterMode
+   */
+  public PartitionFilterMode getFilterMode() {
+    return this.filterMode;
+  }
+
+  /**
+   * 
+   * @see PartitionFilterMode
+   */
+  public void setFilterMode(PartitionFilterMode filterMode) {
+    this.filterMode = filterMode;
+  }
+
+  public void unsetFilterMode() {
+    this.filterMode = null;
+  }
+
+  /** Returns true if field filterMode is set (has been assigned a value) and false otherwise */
+  public boolean isSetFilterMode() {
+    return this.filterMode != null;
+  }
+
+  public void setFilterModeIsSet(boolean value) {
+    if (!value) {
+      this.filterMode = null;
+    }
+  }
+
+  public int getFiltersSize() {
+    return (this.filters == null) ? 0 : this.filters.size();
+  }
+
+  public java.util.Iterator<String> getFiltersIterator() {
+    return (this.filters == null) ? null : this.filters.iterator();
+  }
+
+  public void addToFilters(String elem) {
+    if (this.filters == null) {
+      this.filters = new ArrayList<String>();
+    }
+    this.filters.add(elem);
+  }
+
+  public List<String> getFilters() {
+    return this.filters;
+  }
+
+  public void setFilters(List<String> filters) {
+    this.filters = filters;
+  }
+
+  public void unsetFilters() {
+    this.filters = null;
+  }
+
+  /** Returns true if field filters is set (has been assigned a value) and false otherwise */
+  public boolean isSetFilters() {
+    return this.filters != null;
+  }
+
+  public void setFiltersIsSet(boolean value) {
+    if (!value) {
+      this.filters = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case FILTER_MODE:
+      if (value == null) {
+        unsetFilterMode();
+      } else {
+        setFilterMode((PartitionFilterMode)value);
+      }
+      break;
+
+    case FILTERS:
+      if (value == null) {
+        unsetFilters();
+      } else {
+        setFilters((List<String>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case FILTER_MODE:
+      return getFilterMode();
+
+    case FILTERS:
+      return getFilters();
+
+    }
+    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 FILTER_MODE:
+      return isSetFilterMode();
+    case FILTERS:
+      return isSetFilters();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof GetPartitionsFilterSpec)
+      return this.equals((GetPartitionsFilterSpec)that);
+    return false;
+  }
+
+  public boolean equals(GetPartitionsFilterSpec that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_filterMode = true && this.isSetFilterMode();
+    boolean that_present_filterMode = true && that.isSetFilterMode();
+    if (this_present_filterMode || that_present_filterMode) {
+      if (!(this_present_filterMode && that_present_filterMode))
+        return false;
+      if (!this.filterMode.equals(that.filterMode))
+        return false;
+    }
+
+    boolean this_present_filters = true && this.isSetFilters();
+    boolean that_present_filters = true && that.isSetFilters();
+    if (this_present_filters || that_present_filters) {
+      if (!(this_present_filters && that_present_filters))
+        return false;
+      if (!this.filters.equals(that.filters))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_filterMode = true && (isSetFilterMode());
+    list.add(present_filterMode);
+    if (present_filterMode)
+      list.add(filterMode.getValue());
+
+    boolean present_filters = true && (isSetFilters());
+    list.add(present_filters);
+    if (present_filters)
+      list.add(filters);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(GetPartitionsFilterSpec other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetFilterMode()).compareTo(other.isSetFilterMode());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetFilterMode()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.filterMode, other.filterMode);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetFilters()).compareTo(other.isSetFilters());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetFilters()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.filters, other.filters);
+      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("GetPartitionsFilterSpec(");
+    boolean first = true;
+
+    if (isSetFilterMode()) {
+      sb.append("filterMode:");
+      if (this.filterMode == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.filterMode);
+      }
+      first = false;
+    }
+    if (isSetFilters()) {
+      if (!first) sb.append(", ");
+      sb.append("filters:");
+      if (this.filters == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.filters);
+      }
+      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 GetPartitionsFilterSpecStandardSchemeFactory implements SchemeFactory {
+    public GetPartitionsFilterSpecStandardScheme getScheme() {
+      return new GetPartitionsFilterSpecStandardScheme();
+    }
+  }
+
+  private static class GetPartitionsFilterSpecStandardScheme extends StandardScheme<GetPartitionsFilterSpec> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, GetPartitionsFilterSpec 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 7: // FILTER_MODE
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.filterMode = org.apache.hadoop.hive.metastore.api.PartitionFilterMode.findByValue(iprot.readI32());
+              struct.setFilterModeIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 8: // FILTERS
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list976 = iprot.readListBegin();
+                struct.filters = new ArrayList<String>(_list976.size);
+                String _elem977;
+                for (int _i978 = 0; _i978 < _list976.size; ++_i978)
+                {
+                  _elem977 = iprot.readString();
+                  struct.filters.add(_elem977);
+                }
+                iprot.readListEnd();
+              }
+              struct.setFiltersIsSet(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, GetPartitionsFilterSpec struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.filterMode != null) {
+        if (struct.isSetFilterMode()) {
+          oprot.writeFieldBegin(FILTER_MODE_FIELD_DESC);
+          oprot.writeI32(struct.filterMode.getValue());
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.filters != null) {
+        if (struct.isSetFilters()) {
+          oprot.writeFieldBegin(FILTERS_FIELD_DESC);
+          {
+            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.filters.size()));
+            for (String _iter979 : struct.filters)
+            {
+              oprot.writeString(_iter979);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class GetPartitionsFilterSpecTupleSchemeFactory implements SchemeFactory {
+    public GetPartitionsFilterSpecTupleScheme getScheme() {
+      return new GetPartitionsFilterSpecTupleScheme();
+    }
+  }
+
+  private static class GetPartitionsFilterSpecTupleScheme extends TupleScheme<GetPartitionsFilterSpec> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, GetPartitionsFilterSpec struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.isSetFilterMode()) {
+        optionals.set(0);
+      }
+      if (struct.isSetFilters()) {
+        optionals.set(1);
+      }
+      oprot.writeBitSet(optionals, 2);
+      if (struct.isSetFilterMode()) {
+        oprot.writeI32(struct.filterMode.getValue());
+      }
+      if (struct.isSetFilters()) {
+        {
+          oprot.writeI32(struct.filters.size());
+          for (String _iter980 : struct.filters)
+          {
+            oprot.writeString(_iter980);
+          }
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, GetPartitionsFilterSpec struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(2);
+      if (incoming.get(0)) {
+        struct.filterMode = org.apache.hadoop.hive.metastore.api.PartitionFilterMode.findByValue(iprot.readI32());
+        struct.setFilterModeIsSet(true);
+      }
+      if (incoming.get(1)) {
+        {
+          org.apache.thrift.protocol.TList _list981 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.filters = new ArrayList<String>(_list981.size);
+          String _elem982;
+          for (int _i983 = 0; _i983 < _list981.size; ++_i983)
+          {
+            _elem982 = iprot.readString();
+            struct.filters.add(_elem982);
+          }
+        }
+        struct.setFiltersIsSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/44ef91a6/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsProjectionSpec.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsProjectionSpec.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsProjectionSpec.java
new file mode 100644
index 0000000..bf7b6b7
--- /dev/null
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsProjectionSpec.java
@@ -0,0 +1,652 @@
+/**
+ * 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 GetPartitionsProjectionSpec implements org.apache.thrift.TBase<GetPartitionsProjectionSpec, GetPartitionsProjectionSpec._Fields>, java.io.Serializable, Cloneable, Comparable<GetPartitionsProjectionSpec> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetPartitionsProjectionSpec");
+
+  private static final org.apache.thrift.protocol.TField FIELD_LIST_FIELD_DESC = new org.apache.thrift.protocol.TField("fieldList", org.apache.thrift.protocol.TType.LIST, (short)1);
+  private static final org.apache.thrift.protocol.TField INCLUDE_PARAM_KEY_PATTERN_FIELD_DESC = new org.apache.thrift.protocol.TField("includeParamKeyPattern", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField EXCLUDE_PARAM_KEY_PATTERN_FIELD_DESC = new org.apache.thrift.protocol.TField("excludeParamKeyPattern", 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 GetPartitionsProjectionSpecStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new GetPartitionsProjectionSpecTupleSchemeFactory());
+  }
+
+  private List<String> fieldList; // required
+  private String includeParamKeyPattern; // required
+  private String excludeParamKeyPattern; // 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 {
+    FIELD_LIST((short)1, "fieldList"),
+    INCLUDE_PARAM_KEY_PATTERN((short)2, "includeParamKeyPattern"),
+    EXCLUDE_PARAM_KEY_PATTERN((short)3, "excludeParamKeyPattern");
+
+    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: // FIELD_LIST
+          return FIELD_LIST;
+        case 2: // INCLUDE_PARAM_KEY_PATTERN
+          return INCLUDE_PARAM_KEY_PATTERN;
+        case 3: // EXCLUDE_PARAM_KEY_PATTERN
+          return EXCLUDE_PARAM_KEY_PATTERN;
+        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.FIELD_LIST, new org.apache.thrift.meta_data.FieldMetaData("fieldList", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+    tmpMap.put(_Fields.INCLUDE_PARAM_KEY_PATTERN, new org.apache.thrift.meta_data.FieldMetaData("includeParamKeyPattern", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.EXCLUDE_PARAM_KEY_PATTERN, new org.apache.thrift.meta_data.FieldMetaData("excludeParamKeyPattern", 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(GetPartitionsProjectionSpec.class, metaDataMap);
+  }
+
+  public GetPartitionsProjectionSpec() {
+  }
+
+  public GetPartitionsProjectionSpec(
+    List<String> fieldList,
+    String includeParamKeyPattern,
+    String excludeParamKeyPattern)
+  {
+    this();
+    this.fieldList = fieldList;
+    this.includeParamKeyPattern = includeParamKeyPattern;
+    this.excludeParamKeyPattern = excludeParamKeyPattern;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public GetPartitionsProjectionSpec(GetPartitionsProjectionSpec other) {
+    if (other.isSetFieldList()) {
+      List<String> __this__fieldList = new ArrayList<String>(other.fieldList);
+      this.fieldList = __this__fieldList;
+    }
+    if (other.isSetIncludeParamKeyPattern()) {
+      this.includeParamKeyPattern = other.includeParamKeyPattern;
+    }
+    if (other.isSetExcludeParamKeyPattern()) {
+      this.excludeParamKeyPattern = other.excludeParamKeyPattern;
+    }
+  }
+
+  public GetPartitionsProjectionSpec deepCopy() {
+    return new GetPartitionsProjectionSpec(this);
+  }
+
+  @Override
+  public void clear() {
+    this.fieldList = null;
+    this.includeParamKeyPattern = null;
+    this.excludeParamKeyPattern = null;
+  }
+
+  public int getFieldListSize() {
+    return (this.fieldList == null) ? 0 : this.fieldList.size();
+  }
+
+  public java.util.Iterator<String> getFieldListIterator() {
+    return (this.fieldList == null) ? null : this.fieldList.iterator();
+  }
+
+  public void addToFieldList(String elem) {
+    if (this.fieldList == null) {
+      this.fieldList = new ArrayList<String>();
+    }
+    this.fieldList.add(elem);
+  }
+
+  public List<String> getFieldList() {
+    return this.fieldList;
+  }
+
+  public void setFieldList(List<String> fieldList) {
+    this.fieldList = fieldList;
+  }
+
+  public void unsetFieldList() {
+    this.fieldList = null;
+  }
+
+  /** Returns true if field fieldList is set (has been assigned a value) and false otherwise */
+  public boolean isSetFieldList() {
+    return this.fieldList != null;
+  }
+
+  public void setFieldListIsSet(boolean value) {
+    if (!value) {
+      this.fieldList = null;
+    }
+  }
+
+  public String getIncludeParamKeyPattern() {
+    return this.includeParamKeyPattern;
+  }
+
+  public void setIncludeParamKeyPattern(String includeParamKeyPattern) {
+    this.includeParamKeyPattern = includeParamKeyPattern;
+  }
+
+  public void unsetIncludeParamKeyPattern() {
+    this.includeParamKeyPattern = null;
+  }
+
+  /** Returns true if field includeParamKeyPattern is set (has been assigned a value) and false otherwise */
+  public boolean isSetIncludeParamKeyPattern() {
+    return this.includeParamKeyPattern != null;
+  }
+
+  public void setIncludeParamKeyPatternIsSet(boolean value) {
+    if (!value) {
+      this.includeParamKeyPattern = null;
+    }
+  }
+
+  public String getExcludeParamKeyPattern() {
+    return this.excludeParamKeyPattern;
+  }
+
+  public void setExcludeParamKeyPattern(String excludeParamKeyPattern) {
+    this.excludeParamKeyPattern = excludeParamKeyPattern;
+  }
+
+  public void unsetExcludeParamKeyPattern() {
+    this.excludeParamKeyPattern = null;
+  }
+
+  /** Returns true if field excludeParamKeyPattern is set (has been assigned a value) and false otherwise */
+  public boolean isSetExcludeParamKeyPattern() {
+    return this.excludeParamKeyPattern != null;
+  }
+
+  public void setExcludeParamKeyPatternIsSet(boolean value) {
+    if (!value) {
+      this.excludeParamKeyPattern = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case FIELD_LIST:
+      if (value == null) {
+        unsetFieldList();
+      } else {
+        setFieldList((List<String>)value);
+      }
+      break;
+
+    case INCLUDE_PARAM_KEY_PATTERN:
+      if (value == null) {
+        unsetIncludeParamKeyPattern();
+      } else {
+        setIncludeParamKeyPattern((String)value);
+      }
+      break;
+
+    case EXCLUDE_PARAM_KEY_PATTERN:
+      if (value == null) {
+        unsetExcludeParamKeyPattern();
+      } else {
+        setExcludeParamKeyPattern((String)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case FIELD_LIST:
+      return getFieldList();
+
+    case INCLUDE_PARAM_KEY_PATTERN:
+      return getIncludeParamKeyPattern();
+
+    case EXCLUDE_PARAM_KEY_PATTERN:
+      return getExcludeParamKeyPattern();
+
+    }
+    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 FIELD_LIST:
+      return isSetFieldList();
+    case INCLUDE_PARAM_KEY_PATTERN:
+      return isSetIncludeParamKeyPattern();
+    case EXCLUDE_PARAM_KEY_PATTERN:
+      return isSetExcludeParamKeyPattern();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof GetPartitionsProjectionSpec)
+      return this.equals((GetPartitionsProjectionSpec)that);
+    return false;
+  }
+
+  public boolean equals(GetPartitionsProjectionSpec that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_fieldList = true && this.isSetFieldList();
+    boolean that_present_fieldList = true && that.isSetFieldList();
+    if (this_present_fieldList || that_present_fieldList) {
+      if (!(this_present_fieldList && that_present_fieldList))
+        return false;
+      if (!this.fieldList.equals(that.fieldList))
+        return false;
+    }
+
+    boolean this_present_includeParamKeyPattern = true && this.isSetIncludeParamKeyPattern();
+    boolean that_present_includeParamKeyPattern = true && that.isSetIncludeParamKeyPattern();
+    if (this_present_includeParamKeyPattern || that_present_includeParamKeyPattern) {
+      if (!(this_present_includeParamKeyPattern && that_present_includeParamKeyPattern))
+        return false;
+      if (!this.includeParamKeyPattern.equals(that.includeParamKeyPattern))
+        return false;
+    }
+
+    boolean this_present_excludeParamKeyPattern = true && this.isSetExcludeParamKeyPattern();
+    boolean that_present_excludeParamKeyPattern = true && that.isSetExcludeParamKeyPattern();
+    if (this_present_excludeParamKeyPattern || that_present_excludeParamKeyPattern) {
+      if (!(this_present_excludeParamKeyPattern && that_present_excludeParamKeyPattern))
+        return false;
+      if (!this.excludeParamKeyPattern.equals(that.excludeParamKeyPattern))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_fieldList = true && (isSetFieldList());
+    list.add(present_fieldList);
+    if (present_fieldList)
+      list.add(fieldList);
+
+    boolean present_includeParamKeyPattern = true && (isSetIncludeParamKeyPattern());
+    list.add(present_includeParamKeyPattern);
+    if (present_includeParamKeyPattern)
+      list.add(includeParamKeyPattern);
+
+    boolean present_excludeParamKeyPattern = true && (isSetExcludeParamKeyPattern());
+    list.add(present_excludeParamKeyPattern);
+    if (present_excludeParamKeyPattern)
+      list.add(excludeParamKeyPattern);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(GetPartitionsProjectionSpec other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetFieldList()).compareTo(other.isSetFieldList());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetFieldList()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.fieldList, other.fieldList);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetIncludeParamKeyPattern()).compareTo(other.isSetIncludeParamKeyPattern());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetIncludeParamKeyPattern()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.includeParamKeyPattern, other.includeParamKeyPattern);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetExcludeParamKeyPattern()).compareTo(other.isSetExcludeParamKeyPattern());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetExcludeParamKeyPattern()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.excludeParamKeyPattern, other.excludeParamKeyPattern);
+      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("GetPartitionsProjectionSpec(");
+    boolean first = true;
+
+    sb.append("fieldList:");
+    if (this.fieldList == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.fieldList);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("includeParamKeyPattern:");
+    if (this.includeParamKeyPattern == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.includeParamKeyPattern);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("excludeParamKeyPattern:");
+    if (this.excludeParamKeyPattern == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.excludeParamKeyPattern);
+    }
+    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 GetPartitionsProjectionSpecStandardSchemeFactory implements SchemeFactory {
+    public GetPartitionsProjectionSpecStandardScheme getScheme() {
+      return new GetPartitionsProjectionSpecStandardScheme();
+    }
+  }
+
+  private static class GetPartitionsProjectionSpecStandardScheme extends StandardScheme<GetPartitionsProjectionSpec> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, GetPartitionsProjectionSpec 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: // FIELD_LIST
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list968 = iprot.readListBegin();
+                struct.fieldList = new ArrayList<String>(_list968.size);
+                String _elem969;
+                for (int _i970 = 0; _i970 < _list968.size; ++_i970)
+                {
+                  _elem969 = iprot.readString();
+                  struct.fieldList.add(_elem969);
+                }
+                iprot.readListEnd();
+              }
+              struct.setFieldListIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // INCLUDE_PARAM_KEY_PATTERN
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.includeParamKeyPattern = iprot.readString();
+              struct.setIncludeParamKeyPatternIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // EXCLUDE_PARAM_KEY_PATTERN
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.excludeParamKeyPattern = iprot.readString();
+              struct.setExcludeParamKeyPatternIsSet(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, GetPartitionsProjectionSpec struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.fieldList != null) {
+        oprot.writeFieldBegin(FIELD_LIST_FIELD_DESC);
+        {
+          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.fieldList.size()));
+          for (String _iter971 : struct.fieldList)
+          {
+            oprot.writeString(_iter971);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      if (struct.includeParamKeyPattern != null) {
+        oprot.writeFieldBegin(INCLUDE_PARAM_KEY_PATTERN_FIELD_DESC);
+        oprot.writeString(struct.includeParamKeyPattern);
+        oprot.writeFieldEnd();
+      }
+      if (struct.excludeParamKeyPattern != null) {
+        oprot.writeFieldBegin(EXCLUDE_PARAM_KEY_PATTERN_FIELD_DESC);
+        oprot.writeString(struct.excludeParamKeyPattern);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class GetPartitionsProjectionSpecTupleSchemeFactory implements SchemeFactory {
+    public GetPartitionsProjectionSpecTupleScheme getScheme() {
+      return new GetPartitionsProjectionSpecTupleScheme();
+    }
+  }
+
+  private static class GetPartitionsProjectionSpecTupleScheme extends TupleScheme<GetPartitionsProjectionSpec> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, GetPartitionsProjectionSpec struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.isSetFieldList()) {
+        optionals.set(0);
+      }
+      if (struct.isSetIncludeParamKeyPattern()) {
+        optionals.set(1);
+      }
+      if (struct.isSetExcludeParamKeyPattern()) {
+        optionals.set(2);
+      }
+      oprot.writeBitSet(optionals, 3);
+      if (struct.isSetFieldList()) {
+        {
+          oprot.writeI32(struct.fieldList.size());
+          for (String _iter972 : struct.fieldList)
+          {
+            oprot.writeString(_iter972);
+          }
+        }
+      }
+      if (struct.isSetIncludeParamKeyPattern()) {
+        oprot.writeString(struct.includeParamKeyPattern);
+      }
+      if (struct.isSetExcludeParamKeyPattern()) {
+        oprot.writeString(struct.excludeParamKeyPattern);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, GetPartitionsProjectionSpec struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(3);
+      if (incoming.get(0)) {
+        {
+          org.apache.thrift.protocol.TList _list973 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.fieldList = new ArrayList<String>(_list973.size);
+          String _elem974;
+          for (int _i975 = 0; _i975 < _list973.size; ++_i975)
+          {
+            _elem974 = iprot.readString();
+            struct.fieldList.add(_elem974);
+          }
+        }
+        struct.setFieldListIsSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.includeParamKeyPattern = iprot.readString();
+        struct.setIncludeParamKeyPatternIsSet(true);
+      }
+      if (incoming.get(2)) {
+        struct.excludeParamKeyPattern = iprot.readString();
+        struct.setExcludeParamKeyPatternIsSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/44ef91a6/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsRequest.java
new file mode 100644
index 0000000..6aa8de8
--- /dev/null
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetPartitionsRequest.java
@@ -0,0 +1,1188 @@
+/**
+ * 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 GetPartitionsRequest implements org.apache.thrift.TBase<GetPartitionsRequest, GetPartitionsRequest._Fields>, java.io.Serializable, Cloneable, Comparable<GetPartitionsRequest> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetPartitionsRequest");
+
+  private static final org.apache.thrift.protocol.TField CAT_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catName", org.apache.thrift.protocol.TType.STRING, (short)1);
+  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)2);
+  private static final org.apache.thrift.protocol.TField TBL_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tblName", org.apache.thrift.protocol.TType.STRING, (short)3);
+  private static final org.apache.thrift.protocol.TField WITH_AUTH_FIELD_DESC = new org.apache.thrift.protocol.TField("withAuth", org.apache.thrift.protocol.TType.BOOL, (short)4);
+  private static final org.apache.thrift.protocol.TField USER_FIELD_DESC = new org.apache.thrift.protocol.TField("user", org.apache.thrift.protocol.TType.STRING, (short)5);
+  private static final org.apache.thrift.protocol.TField GROUP_NAMES_FIELD_DESC = new org.apache.thrift.protocol.TField("groupNames", org.apache.thrift.protocol.TType.LIST, (short)6);
+  private static final org.apache.thrift.protocol.TField PROJECTION_SPEC_FIELD_DESC = new org.apache.thrift.protocol.TField("projectionSpec", org.apache.thrift.protocol.TType.STRUCT, (short)7);
+  private static final org.apache.thrift.protocol.TField FILTER_SPEC_FIELD_DESC = new org.apache.thrift.protocol.TField("filterSpec", org.apache.thrift.protocol.TType.STRUCT, (short)8);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new GetPartitionsRequestStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new GetPartitionsRequestTupleSchemeFactory());
+  }
+
+  private String catName; // optional
+  private String dbName; // required
+  private String tblName; // required
+  private boolean withAuth; // optional
+  private String user; // optional
+  private List<String> groupNames; // optional
+  private GetPartitionsProjectionSpec projectionSpec; // required
+  private GetPartitionsFilterSpec filterSpec; // 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 {
+    CAT_NAME((short)1, "catName"),
+    DB_NAME((short)2, "dbName"),
+    TBL_NAME((short)3, "tblName"),
+    WITH_AUTH((short)4, "withAuth"),
+    USER((short)5, "user"),
+    GROUP_NAMES((short)6, "groupNames"),
+    PROJECTION_SPEC((short)7, "projectionSpec"),
+    FILTER_SPEC((short)8, "filterSpec");
+
+    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: // CAT_NAME
+          return CAT_NAME;
+        case 2: // DB_NAME
+          return DB_NAME;
+        case 3: // TBL_NAME
+          return TBL_NAME;
+        case 4: // WITH_AUTH
+          return WITH_AUTH;
+        case 5: // USER
+          return USER;
+        case 6: // GROUP_NAMES
+          return GROUP_NAMES;
+        case 7: // PROJECTION_SPEC
+          return PROJECTION_SPEC;
+        case 8: // FILTER_SPEC
+          return FILTER_SPEC;
+        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 __WITHAUTH_ISSET_ID = 0;
+  private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.CAT_NAME,_Fields.WITH_AUTH,_Fields.USER,_Fields.GROUP_NAMES};
+  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.CAT_NAME, new org.apache.thrift.meta_data.FieldMetaData("catName", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        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.TBL_NAME, new org.apache.thrift.meta_data.FieldMetaData("tblName", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.WITH_AUTH, new org.apache.thrift.meta_data.FieldMetaData("withAuth", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+    tmpMap.put(_Fields.USER, new org.apache.thrift.meta_data.FieldMetaData("user", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.GROUP_NAMES, new org.apache.thrift.meta_data.FieldMetaData("groupNames", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+    tmpMap.put(_Fields.PROJECTION_SPEC, new org.apache.thrift.meta_data.FieldMetaData("projectionSpec", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, GetPartitionsProjectionSpec.class)));
+    tmpMap.put(_Fields.FILTER_SPEC, new org.apache.thrift.meta_data.FieldMetaData("filterSpec", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, GetPartitionsFilterSpec.class)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetPartitionsRequest.class, metaDataMap);
+  }
+
+  public GetPartitionsRequest() {
+  }
+
+  public GetPartitionsRequest(
+    String dbName,
+    String tblName,
+    GetPartitionsProjectionSpec projectionSpec,
+    GetPartitionsFilterSpec filterSpec)
+  {
+    this();
+    this.dbName = dbName;
+    this.tblName = tblName;
+    this.projectionSpec = projectionSpec;
+    this.filterSpec = filterSpec;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public GetPartitionsRequest(GetPartitionsRequest other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.isSetCatName()) {
+      this.catName = other.catName;
+    }
+    if (other.isSetDbName()) {
+      this.dbName = other.dbName;
+    }
+    if (other.isSetTblName()) {
+      this.tblName = other.tblName;
+    }
+    this.withAuth = other.withAuth;
+    if (other.isSetUser()) {
+      this.user = other.user;
+    }
+    if (other.isSetGroupNames()) {
+      List<String> __this__groupNames = new ArrayList<String>(other.groupNames);
+      this.groupNames = __this__groupNames;
+    }
+    if (other.isSetProjectionSpec()) {
+      this.projectionSpec = new GetPartitionsProjectionSpec(other.projectionSpec);
+    }
+    if (other.isSetFilterSpec()) {
+      this.filterSpec = new GetPartitionsFilterSpec(other.filterSpec);
+    }
+  }
+
+  public GetPartitionsRequest deepCopy() {
+    return new GetPartitionsRequest(this);
+  }
+
+  @Override
+  public void clear() {
+    this.catName = null;
+    this.dbName = null;
+    this.tblName = null;
+    setWithAuthIsSet(false);
+    this.withAuth = false;
+    this.user = null;
+    this.groupNames = null;
+    this.projectionSpec = null;
+    this.filterSpec = null;
+  }
+
+  public String getCatName() {
+    return this.catName;
+  }
+
+  public void setCatName(String catName) {
+    this.catName = catName;
+  }
+
+  public void unsetCatName() {
+    this.catName = null;
+  }
+
+  /** Returns true if field catName is set (has been assigned a value) and false otherwise */
+  public boolean isSetCatName() {
+    return this.catName != null;
+  }
+
+  public void setCatNameIsSet(boolean value) {
+    if (!value) {
+      this.catName = 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 getTblName() {
+    return this.tblName;
+  }
+
+  public void setTblName(String tblName) {
+    this.tblName = tblName;
+  }
+
+  public void unsetTblName() {
+    this.tblName = null;
+  }
+
+  /** Returns true if field tblName is set (has been assigned a value) and false otherwise */
+  public boolean isSetTblName() {
+    return this.tblName != null;
+  }
+
+  public void setTblNameIsSet(boolean value) {
+    if (!value) {
+      this.tblName = null;
+    }
+  }
+
+  public boolean isWithAuth() {
+    return this.withAuth;
+  }
+
+  public void setWithAuth(boolean withAuth) {
+    this.withAuth = withAuth;
+    setWithAuthIsSet(true);
+  }
+
+  public void unsetWithAuth() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __WITHAUTH_ISSET_ID);
+  }
+
+  /** Returns true if field withAuth is set (has been assigned a value) and false otherwise */
+  public boolean isSetWithAuth() {
+    return EncodingUtils.testBit(__isset_bitfield, __WITHAUTH_ISSET_ID);
+  }
+
+  public void setWithAuthIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __WITHAUTH_ISSET_ID, value);
+  }
+
+  public String getUser() {
+    return this.user;
+  }
+
+  public void setUser(String user) {
+    this.user = user;
+  }
+
+  public void unsetUser() {
+    this.user = null;
+  }
+
+  /** Returns true if field user is set (has been assigned a value) and false otherwise */
+  public boolean isSetUser() {
+    return this.user != null;
+  }
+
+  public void setUserIsSet(boolean value) {
+    if (!value) {
+      this.user = null;
+    }
+  }
+
+  public int getGroupNamesSize() {
+    return (this.groupNames == null) ? 0 : this.groupNames.size();
+  }
+
+  public java.util.Iterator<String> getGroupNamesIterator() {
+    return (this.groupNames == null) ? null : this.groupNames.iterator();
+  }
+
+  public void addToGroupNames(String elem) {
+    if (this.groupNames == null) {
+      this.groupNames = new ArrayList<String>();
+    }
+    this.groupNames.add(elem);
+  }
+
+  public List<String> getGroupNames() {
+    return this.groupNames;
+  }
+
+  public void setGroupNames(List<String> groupNames) {
+    this.groupNames = groupNames;
+  }
+
+  public void unsetGroupNames() {
+    this.groupNames = null;
+  }
+
+  /** Returns true if field groupNames is set (has been assigned a value) and false otherwise */
+  public boolean isSetGroupNames() {
+    return this.groupNames != null;
+  }
+
+  public void setGroupNamesIsSet(boolean value) {
+    if (!value) {
+      this.groupNames = null;
+    }
+  }
+
+  public GetPartitionsProjectionSpec getProjectionSpec() {
+    return this.projectionSpec;
+  }
+
+  public void setProjectionSpec(GetPartitionsProjectionSpec projectionSpec) {
+    this.projectionSpec = projectionSpec;
+  }
+
+  public void unsetProjectionSpec() {
+    this.projectionSpec = null;
+  }
+
+  /** Returns true if field projectionSpec is set (has been assigned a value) and false otherwise */
+  public boolean isSetProjectionSpec() {
+    return this.projectionSpec != null;
+  }
+
+  public void setProjectionSpecIsSet(boolean value) {
+    if (!value) {
+      this.projectionSpec = null;
+    }
+  }
+
+  public GetPartitionsFilterSpec getFilterSpec() {
+    return this.filterSpec;
+  }
+
+  public void setFilterSpec(GetPartitionsFilterSpec filterSpec) {
+    this.filterSpec = filterSpec;
+  }
+
+  public void unsetFilterSpec() {
+    this.filterSpec = null;
+  }
+
+  /** Returns true if field filterSpec is set (has been assigned a value) and false otherwise */
+  public boolean isSetFilterSpec() {
+    return this.filterSpec != null;
+  }
+
+  public void setFilterSpecIsSet(boolean value) {
+    if (!value) {
+      this.filterSpec = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case CAT_NAME:
+      if (value == null) {
+        unsetCatName();
+      } else {
+        setCatName((String)value);
+      }
+      break;
+
+    case DB_NAME:
+      if (value == null) {
+        unsetDbName();
+      } else {
+        setDbName((String)value);
+      }
+      break;
+
+    case TBL_NAME:
+      if (value == null) {
+        unsetTblName();
+      } else {
+        setTblName((String)value);
+      }
+      break;
+
+    case WITH_AUTH:
+      if (value == null) {
+        unsetWithAuth();
+      } else {
+        setWithAuth((Boolean)value);
+      }
+      break;
+
+    case USER:
+      if (value == null) {
+        unsetUser();
+      } else {
+        setUser((String)value);
+      }
+      break;
+
+    case GROUP_NAMES:
+      if (value == null) {
+        unsetGroupNames();
+      } else {
+        setGroupNames((List<String>)value);
+      }
+      break;
+
+    case PROJECTION_SPEC:
+      if (value == null) {
+        unsetProjectionSpec();
+      } else {
+        setProjectionSpec((GetPartitionsProjectionSpec)value);
+      }
+      break;
+
+    case FILTER_SPEC:
+      if (value == null) {
+        unsetFilterSpec();
+      } else {
+        setFilterSpec((GetPartitionsFilterSpec)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case CAT_NAME:
+      return getCatName();
+
+    case DB_NAME:
+      return getDbName();
+
+    case TBL_NAME:
+      return getTblName();
+
+    case WITH_AUTH:
+      return isWithAuth();
+
+    case USER:
+      return getUser();
+
+    case GROUP_NAMES:
+      return getGroupNames();
+
+    case PROJECTION_SPEC:
+      return getProjectionSpec();
+
+    case FILTER_SPEC:
+      return getFilterSpec();
+
+    }
+    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 CAT_NAME:
+      return isSetCatName();
+    case DB_NAME:
+      return isSetDbName();
+    case TBL_NAME:
+      return isSetTblName();
+    case WITH_AUTH:
+      return isSetWithAuth();
+    case USER:
+      return isSetUser();
+    case GROUP_NAMES:
+      return isSetGroupNames();
+    case PROJECTION_SPEC:
+      return isSetProjectionSpec();
+    case FILTER_SPEC:
+      return isSetFilterSpec();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof GetPartitionsRequest)
+      return this.equals((GetPartitionsRequest)that);
+    return false;
+  }
+
+  public boolean equals(GetPartitionsRequest that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_catName = true && this.isSetCatName();
+    boolean that_present_catName = true && that.isSetCatName();
+    if (this_present_catName || that_present_catName) {
+      if (!(this_present_catName && that_present_catName))
+        return false;
+      if (!this.catName.equals(that.catName))
+        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_tblName = true && this.isSetTblName();
+    boolean that_present_tblName = true && that.isSetTblName();
+    if (this_present_tblName || that_present_tblName) {
+      if (!(this_present_tblName && that_present_tblName))
+        return false;
+      if (!this.tblName.equals(that.tblName))
+        return false;
+    }
+
+    boolean this_present_withAuth = true && this.isSetWithAuth();
+    boolean that_present_withAuth = true && that.isSetWithAuth();
+    if (this_present_withAuth || that_present_withAuth) {
+      if (!(this_present_withAuth && that_present_withAuth))
+        return false;
+      if (this.withAuth != that.withAuth)
+        return false;
+    }
+
+    boolean this_present_user = true && this.isSetUser();
+    boolean that_present_user = true && that.isSetUser();
+    if (this_present_user || that_present_user) {
+      if (!(this_present_user && that_present_user))
+        return false;
+      if (!this.user.equals(that.user))
+        return false;
+    }
+
+    boolean this_present_groupNames = true && this.isSetGroupNames();
+    boolean that_present_groupNames = true && that.isSetGroupNames();
+    if (this_present_groupNames || that_present_groupNames) {
+      if (!(this_present_groupNames && that_present_groupNames))
+        return false;
+      if (!this.groupNames.equals(that.groupNames))
+        return false;
+    }
+
+    boolean this_present_projectionSpec = true && this.isSetProjectionSpec();
+    boolean that_present_projectionSpec = true && that.isSetProjectionSpec();
+    if (this_present_projectionSpec || that_present_projectionSpec) {
+      if (!(this_present_projectionSpec && that_present_projectionSpec))
+        return false;
+      if (!this.projectionSpec.equals(that.projectionSpec))
+        return false;
+    }
+
+    boolean this_present_filterSpec = true && this.isSetFilterSpec();
+    boolean that_present_filterSpec = true && that.isSetFilterSpec();
+    if (this_present_filterSpec || that_present_filterSpec) {
+      if (!(this_present_filterSpec && that_present_filterSpec))
+        return false;
+      if (!this.filterSpec.equals(that.filterSpec))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_catName = true && (isSetCatName());
+    list.add(present_catName);
+    if (present_catName)
+      list.add(catName);
+
+    boolean present_dbName = true && (isSetDbName());
+    list.add(present_dbName);
+    if (present_dbName)
+      list.add(dbName);
+
+    boolean present_tblName = true && (isSetTblName());
+    list.add(present_tblName);
+    if (present_tblName)
+      list.add(tblName);
+
+    boolean present_withAuth = true && (isSetWithAuth());
+    list.add(present_withAuth);
+    if (present_withAuth)
+      list.add(withAuth);
+
+    boolean present_user = true && (isSetUser());
+    list.add(present_user);
+    if (present_user)
+      list.add(user);
+
+    boolean present_groupNames = true && (isSetGroupNames());
+    list.add(present_groupNames);
+    if (present_groupNames)
+      list.add(groupNames);
+
+    boolean present_projectionSpec = true && (isSetProjectionSpec());
+    list.add(present_projectionSpec);
+    if (present_projectionSpec)
+      list.add(projectionSpec);
+
+    boolean present_filterSpec = true && (isSetFilterSpec());
+    list.add(present_filterSpec);
+    if (present_filterSpec)
+      list.add(filterSpec);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(GetPartitionsRequest other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetCatName()).compareTo(other.isSetCatName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetCatName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catName, other.catName);
+      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(isSetTblName()).compareTo(other.isSetTblName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetTblName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tblName, other.tblName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetWithAuth()).compareTo(other.isSetWithAuth());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetWithAuth()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.withAuth, other.withAuth);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetUser()).compareTo(other.isSetUser());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetUser()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.user, other.user);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetGroupNames()).compareTo(other.isSetGroupNames());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetGroupNames()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.groupNames, other.groupNames);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetProjectionSpec()).compareTo(other.isSetProjectionSpec());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetProjectionSpec()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.projectionSpec, other.projectionSpec);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetFilterSpec()).compareTo(other.isSetFilterSpec());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetFilterSpec()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.filterSpec, other.filterSpec);
+      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("GetPartitionsRequest(");
+    boolean first = true;
+
+    if (isSetCatName()) {
+      sb.append("catName:");
+      if (this.catName == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.catName);
+      }
+      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("tblName:");
+    if (this.tblName == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.tblName);
+    }
+    first = false;
+    if (isSetWithAuth()) {
+      if (!first) sb.append(", ");
+      sb.append("withAuth:");
+      sb.append(this.withAuth);
+      first = false;
+    }
+    if (isSetUser()) {
+      if (!first) sb.append(", ");
+      sb.append("user:");
+      if (this.user == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.user);
+      }
+      first = false;
+    }
+    if (isSetGroupNames()) {
+      if (!first) sb.append(", ");
+      sb.append("groupNames:");
+      if (this.groupNames == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.groupNames);
+      }
+      first = false;
+    }
+    if (!first) sb.append(", ");
+    sb.append("projectionSpec:");
+    if (this.projectionSpec == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.projectionSpec);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("filterSpec:");
+    if (this.filterSpec == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.filterSpec);
+    }
+    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 (projectionSpec != null) {
+      projectionSpec.validate();
+    }
+    if (filterSpec != null) {
+      filterSpec.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 GetPartitionsRequestStandardSchemeFactory implements SchemeFactory {
+    public GetPartitionsRequestStandardScheme getScheme() {
+      return new GetPartitionsRequestStandardScheme();
+    }
+  }
+
+  private static class GetPartitionsRequestStandardScheme extends StandardScheme<GetPartitionsRequest> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, GetPartitionsRequest 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: // CAT_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.catName = iprot.readString();
+              struct.setCatNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // 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 3: // TBL_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.tblName = iprot.readString();
+              struct.setTblNameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // WITH_AUTH
+            if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+              struct.withAuth = iprot.readBool();
+              struct.setWithAuthIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 5: // USER
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.user = iprot.readString();
+              struct.setUserIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 6: // GROUP_NAMES
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list992 = iprot.readListBegin();
+                struct.groupNames = new ArrayList<String>(_list992.size);
+                String _elem993;
+                for (int _i994 = 0; _i994 < _list992.size; ++_i994)
+                {
+                  _elem993 = iprot.readString();
+                  struct.groupNames.add(_elem993);
+                }
+                iprot.readListEnd();
+              }
+              struct.setGroupNamesIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 7: // PROJECTION_SPEC
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.projectionSpec = new GetPartitionsProjectionSpec();
+              struct.projectionSpec.read(iprot);
+              struct.setProjectionSpecIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 8: // FILTER_SPEC
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.filterSpec = new GetPartitionsFilterSpec();
+              struct.filterSpec.read(iprot);
+              struct.setFilterSpecIsSet(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, GetPartitionsRequest struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.catName != null) {
+        if (struct.isSetCatName()) {
+          oprot.writeFieldBegin(CAT_NAME_FIELD_DESC);
+          oprot.writeString(struct.catName);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.dbName != null) {
+        oprot.writeFieldBegin(DB_NAME_FIELD_DESC);
+        oprot.writeString(struct.dbName);
+        oprot.writeFieldEnd();
+      }
+      if (struct.tblName != null) {
+        oprot.writeFieldBegin(TBL_NAME_FIELD_DESC);
+        oprot.writeString(struct.tblName);
+        oprot.writeFieldEnd();
+      }
+      if (struct.isSetWithAuth()) {
+        oprot.writeFieldBegin(WITH_AUTH_FIELD_DESC);
+        oprot.writeBool(struct.withAuth);
+        oprot.writeFieldEnd();
+      }
+      if (struct.user != null) {
+        if (struct.isSetUser()) {
+          oprot.writeFieldBegin(USER_FIELD_DESC);
+          oprot.writeString(struct.user);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.groupNames != null) {
+        if (struct.isSetGroupNames()) {
+          oprot.writeFieldBegin(GROUP_NAMES_FIELD_DESC);
+          {
+            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.groupNames.size()));
+            for (String _iter995 : struct.groupNames)
+            {
+              oprot.writeString(_iter995);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.projectionSpec != null) {
+        oprot.writeFieldBegin(PROJECTION_SPEC_FIELD_DESC);
+        struct.projectionSpec.write(oprot);
+        oprot.writeFieldEnd();
+      }
+      if (struct.filterSpec != null) {
+        oprot.writeFieldBegin(FILTER_SPEC_FIELD_DESC);
+        struct.filterSpec.write(oprot);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class GetPartitionsRequestTupleSchemeFactory implements SchemeFactory {
+    public GetPartitionsRequestTupleScheme getScheme() {
+      return new GetPartitionsRequestTupleScheme();
+    }
+  }
+
+  private static class GetPartitionsRequestTupleScheme extends TupleScheme<GetPartitionsRequest> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, GetPartitionsRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      BitSet optionals = new BitSet();
+      if (struct.isSetCatName()) {
+        optionals.set(0);
+      }
+      if (struct.isSetDbName()) {
+        optionals.set(1);
+      }
+      if (struct.isSetTblName()) {
+        optionals.set(2);
+      }
+      if (struct.isSetWithAuth()) {
+        optionals.set(3);
+      }
+      if (struct.isSetUser()) {
+        optionals.set(4);
+      }
+      if (struct.isSetGroupNames()) {
+        optionals.set(5);
+      }
+      if (struct.isSetProjectionSpec()) {
+        optionals.set(6);
+      }
+      if (struct.isSetFilterSpec()) {
+        optionals.set(7);
+      }
+      oprot.writeBitSet(optionals, 8);
+      if (struct.isSetCatName()) {
+        oprot.writeString(struct.catName);
+      }
+      if (struct.isSetDbName()) {
+        oprot.writeString(struct.dbName);
+      }
+      if (struct.isSetTblName()) {
+        oprot.writeString(struct.tblName);
+      }
+      if (struct.isSetWithAuth()) {
+        oprot.writeBool(struct.withAuth);
+      }
+      if (struct.isSetUser()) {
+        oprot.writeString(struct.user);
+      }
+      if (struct.isSetGroupNames()) {
+        {
+          oprot.writeI32(struct.groupNames.size());
+          for (String _iter996 : struct.groupNames)
+          {
+            oprot.writeString(_iter996);
+          }
+        }
+      }
+      if (struct.isSetProjectionSpec()) {
+        struct.projectionSpec.write(oprot);
+      }
+      if (struct.isSetFilterSpec()) {
+        struct.filterSpec.write(oprot);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, GetPartitionsRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      BitSet incoming = iprot.readBitSet(8);
+      if (incoming.get(0)) {
+        struct.catName = iprot.readString();
+        struct.setCatNameIsSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.dbName = iprot.readString();
+        struct.setDbNameIsSet(true);
+      }
+      if (incoming.get(2)) {
+        struct.tblName = iprot.readString();
+        struct.setTblNameIsSet(true);
+      }
+      if (incoming.get(3)) {
+        struct.withAuth = iprot.readBool();
+        struct.setWithAuthIsSet(true);
+      }
+      if (incoming.get(4)) {
+        struct.user = iprot.readString();
+        struct.setUserIsSet(true);
+      }
+      if (incoming.get(5)) {
+        {
+          org.apache.thrift.protocol.TList _list997 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.groupNames = new ArrayList<String>(_list997.size);
+          String _elem998;
+          for (int _i999 = 0; _i999 < _list997.size; ++_i999)
+          {
+            _elem998 = iprot.readString();
+            struct.groupNames.add(_elem998);
+          }
+        }
+        struct.setGroupNamesIsSet(true);
+      }
+      if (incoming.get(6)) {
+        struct.projectionSpec = new GetPartitionsProjectionSpec();
+        struct.projectionSpec.read(iprot);
+        struct.setProjectionSpecIsSet(true);
+      }
+      if (incoming.get(7)) {
+        struct.filterSpec = new GetPartitionsFilterSpec();
+        struct.filterSpec.read(iprot);
+        struct.setFilterSpecIsSet(true);
+      }
+    }
+  }
+
+}
+