You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by se...@apache.org on 2015/12/18 23:43:06 UTC

[1/7] hive git commit: HIVE-12075 : add analyze command to explictly cache file metadata in HBase metastore (Sergey Shelukhin, reviewed by Alan Gates)

Repository: hive
Updated Branches:
  refs/heads/master 542eaf6bc -> 7df62023f


http://git-wip-us.apache.org/repos/asf/hive/blob/7df62023/metastore/src/test/org/apache/hadoop/hive/metastore/TestObjectStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/TestObjectStore.java b/metastore/src/test/org/apache/hadoop/hive/metastore/TestObjectStore.java
index 9089d1c..1157033 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/TestObjectStore.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/TestObjectStore.java
@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.hive.metastore;
 
-import java.nio.ByteBuffer;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
@@ -25,6 +24,7 @@ import java.util.List;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.FileMetadataExprType;
 import org.apache.hadoop.hive.metastore.api.Function;
 import org.apache.hadoop.hive.metastore.api.InvalidInputException;
 import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
@@ -71,12 +71,17 @@ public class TestObjectStore {
     }
 
     @Override
+    public FileMetadataExprType getMetadataType(String inputFormat) {
+      return null;
+    }
+
+    @Override
     public SearchArgument createSarg(byte[] expr) {
       return null;
     }
 
     @Override
-    public ByteBuffer applySargToFileMetadata(SearchArgument sarg, ByteBuffer byteBuffer) {
+    public FileFormatProxy getFileFormatProxy(FileMetadataExprType type) {
       return null;
     }
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/7df62023/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/MockUtils.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/MockUtils.java b/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/MockUtils.java
index 983129a..784648a 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/MockUtils.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/MockUtils.java
@@ -19,6 +19,8 @@
 package org.apache.hadoop.hive.metastore.hbase;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
@@ -29,7 +31,9 @@ import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.metastore.FileFormatProxy;
 import org.apache.hadoop.hive.metastore.PartitionExpressionProxy;
+import org.apache.hadoop.hive.metastore.api.FileMetadataExprType;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
 import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
@@ -73,10 +77,14 @@ public class MockUtils {
     }
 
     @Override
-    public ByteBuffer applySargToFileMetadata(SearchArgument sarg, ByteBuffer byteBuffer) {
+    public FileMetadataExprType getMetadataType(String inputFormat) {
       return null;
     }
 
+    @Override
+    public FileFormatProxy getFileFormatProxy(FileMetadataExprType type) {
+      return null;
+    }
   }
 
   static HBaseStore init(Configuration conf, HTableInterface htable,

http://git-wip-us.apache.org/repos/asf/hive/blob/7df62023/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
index 290f489..30cae88 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
@@ -104,6 +104,7 @@ import org.apache.hadoop.hive.ql.plan.AlterTableDesc;
 import org.apache.hadoop.hive.ql.plan.AlterTableDesc.AlterTableTypes;
 import org.apache.hadoop.hive.ql.plan.AlterTableExchangePartition;
 import org.apache.hadoop.hive.ql.plan.AlterTableSimpleDesc;
+import org.apache.hadoop.hive.ql.plan.CacheMetadataDesc;
 import org.apache.hadoop.hive.ql.plan.CreateDatabaseDesc;
 import org.apache.hadoop.hive.ql.plan.CreateIndexDesc;
 import org.apache.hadoop.hive.ql.plan.CreateTableDesc;
@@ -518,6 +519,11 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
       if (alterTableExchangePartition != null) {
         return exchangeTablePartition(db, alterTableExchangePartition);
       }
+
+      CacheMetadataDesc cacheMetadataDesc = work.getCacheMetadataDesc();
+      if (cacheMetadataDesc != null) {
+        return cacheMetadata(db, cacheMetadataDesc);
+      }
     } catch (Throwable e) {
       failed(e);
       return 1;
@@ -526,6 +532,12 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
     return 0;
   }
 
+  private int cacheMetadata(Hive db, CacheMetadataDesc desc) throws HiveException {
+    db.cacheFileMetadata(desc.getDbName(), desc.getTableName(),
+        desc.getPartName(), desc.isAllParts());
+    return 0;
+  }
+
   private void failed(Throwable e) {
     while (e.getCause() != null && e.getClass() == RuntimeException.class) {
       e = e.getCause();

http://git-wip-us.apache.org/repos/asf/hive/blob/7df62023/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcFileFormatProxy.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcFileFormatProxy.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcFileFormatProxy.java
new file mode 100644
index 0000000..ef76723
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcFileFormatProxy.java
@@ -0,0 +1,74 @@
+/**
+ * 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.ql.io.orc;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.metastore.FileFormatProxy;
+import org.apache.hadoop.hive.metastore.Metastore.SplitInfo;
+import org.apache.hadoop.hive.metastore.Metastore.SplitInfos;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.orc.OrcProto;
+import org.apache.orc.StripeInformation;
+
+/** File format proxy for ORC. */
+public class OrcFileFormatProxy implements FileFormatProxy {
+
+  @Override
+  public ByteBuffer applySargToMetadata(
+      SearchArgument sarg, ByteBuffer byteBuffer) throws IOException {
+    // TODO: ideally we should store shortened representation of only the necessary fields
+    //       in HBase; it will probably require custom SARG application code.
+    ReaderImpl.FooterInfo fi = ReaderImpl.extractMetaInfoFromFooter(byteBuffer, null);
+    OrcProto.Footer footer = fi.getFooter();
+    int stripeCount = footer.getStripesCount();
+    boolean[] result = OrcInputFormat.pickStripesViaTranslatedSarg(
+        sarg, fi.getFileMetaInfo().getWriterVersion(),
+        footer.getTypesList(), fi.getMetadata(), stripeCount);
+    // For ORC case, send the boundaries of the stripes so we don't have to send the footer.
+    SplitInfos.Builder sb = SplitInfos.newBuilder();
+    List<StripeInformation> stripes = fi.getStripes();
+    boolean isEliminated = true;
+    for (int i = 0; i < result.length; ++i) {
+      if (result != null && !result[i]) continue;
+      isEliminated = false;
+      StripeInformation si = stripes.get(i);
+      sb.addInfos(SplitInfo.newBuilder().setIndex(i)
+          .setOffset(si.getOffset()).setLength(si.getLength()));
+    }
+    return isEliminated ? null : ByteBuffer.wrap(sb.build().toByteArray());
+  }
+
+  public ByteBuffer[] getAddedColumnsToCache() {
+    return null; // Nothing so far.
+  }
+
+  public ByteBuffer[][] getAddedValuesToCache(List<ByteBuffer> metadata) {
+    throw new UnsupportedOperationException(); // Nothing so far (and shouldn't be called).
+  }
+
+  public ByteBuffer getMetadataToCache(
+      FileSystem fs, Path path, ByteBuffer[] addedVals) throws IOException {
+    // For now, there's nothing special to return in addedVals. Just return the footer.
+    return OrcFile.createReader(fs, path).getSerializedFileFooter();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/7df62023/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
index c682df2..29df4f9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
@@ -3401,4 +3401,17 @@ private void constructOneLBLocationMap(FileStatus fSta,
       throw new HiveException(e);
     }
   }
+
+  public void cacheFileMetadata(
+      String dbName, String tableName, String partName, boolean allParts) throws HiveException {
+    try {
+      boolean willCache = getMSC().cacheFileMetadata(dbName, tableName, partName, allParts);
+      if (!willCache) {
+        throw new HiveException(
+            "Caching file metadata is not supported by metastore or for this file format");
+      }
+    } catch (TException e) {
+      throw new HiveException(e);
+    }
+  }
 };

http://git-wip-us.apache.org/repos/asf/hive/blob/7df62023/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartitionExpressionForMetastore.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartitionExpressionForMetastore.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartitionExpressionForMetastore.java
index 7cddcc9..96910e3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartitionExpressionForMetastore.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartitionExpressionForMetastore.java
@@ -18,18 +18,16 @@
 
 package org.apache.hadoop.hive.ql.optimizer.ppr;
 
-import java.io.IOException;
-import java.nio.ByteBuffer;
+import org.apache.hadoop.hive.metastore.api.FileMetadataExprType;
+
 import java.util.List;
 
-import org.apache.hadoop.hive.metastore.Metastore.SplitInfo;
-import org.apache.hadoop.hive.metastore.Metastore.SplitInfos;
+import org.apache.hadoop.hive.metastore.FileFormatProxy;
 import org.apache.hadoop.hive.metastore.PartitionExpressionProxy;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.ql.exec.SerializationUtilities;
+import org.apache.hadoop.hive.ql.io.orc.OrcFileFormatProxy;
 import org.apache.hadoop.hive.ql.io.orc.OrcInputFormat;
-import org.apache.hadoop.hive.ql.io.orc.ReaderImpl;
-import org.apache.orc.StripeInformation;
 import org.apache.hadoop.hive.ql.io.sarg.ConvertAstToSearchArg;
 import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -37,7 +35,6 @@ import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
 import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.orc.OrcProto;
 
 /**
  * The basic implementation of PartitionExpressionProxy that uses ql package classes.
@@ -83,32 +80,29 @@ public class PartitionExpressionForMetastore implements PartitionExpressionProxy
   }
 
   @Override
-  public SearchArgument createSarg(byte[] expr) {
-    return ConvertAstToSearchArg.create(expr);
+  public FileFormatProxy getFileFormatProxy(FileMetadataExprType type) {
+    switch (type) {
+    case ORC_SARG: return new OrcFileFormatProxy();
+    default: throw new RuntimeException("Unsupported format " + type);
+    }
   }
 
   @Override
-  public ByteBuffer applySargToFileMetadata(
-      SearchArgument sarg, ByteBuffer byteBuffer) throws IOException {
-    // TODO: ideally we should store shortened representation of only the necessary fields
-    //       in HBase; it will probably require custom SARG application code.
-    ReaderImpl.FooterInfo fi = ReaderImpl.extractMetaInfoFromFooter(byteBuffer, null);
-    OrcProto.Footer footer = fi.getFooter();
-    int stripeCount = footer.getStripesCount();
-    boolean[] result = OrcInputFormat.pickStripesViaTranslatedSarg(
-        sarg, fi.getFileMetaInfo().getWriterVersion(),
-        footer.getTypesList(), fi.getMetadata(), stripeCount);
-    // For ORC case, send the boundaries of the stripes so we don't have to send the footer.
-    SplitInfos.Builder sb = SplitInfos.newBuilder();
-    List<StripeInformation> stripes = fi.getStripes();
-    boolean isEliminated = true;
-    for (int i = 0; i < result.length; ++i) {
-      if (result != null && !result[i]) continue;
-      isEliminated = false;
-      StripeInformation si = stripes.get(i);
-      sb.addInfos(SplitInfo.newBuilder().setIndex(i)
-          .setOffset(si.getOffset()).setLength(si.getLength()));
+  public FileMetadataExprType getMetadataType(String inputFormat) {
+    try {
+      Class<?> ifClass = Class.forName(inputFormat);
+      if (OrcInputFormat.class.isAssignableFrom(ifClass)) {
+        return FileMetadataExprType.ORC_SARG;
+      }
+      return null;
+    } catch (Throwable t) {
+      LOG.warn("Can't create the class for input format " + inputFormat, t);
+      return null;
     }
-    return isEliminated ? null : ByteBuffer.wrap(sb.build().toByteArray());
+  }
+
+  @Override
+  public SearchArgument createSarg(byte[] expr) {
+    return ConvertAstToSearchArg.create(expr);
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/7df62023/ql/src/java/org/apache/hadoop/hive/ql/parse/AnalyzeCommandUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/AnalyzeCommandUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/AnalyzeCommandUtils.java
new file mode 100644
index 0000000..ac1383c
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/AnalyzeCommandUtils.java
@@ -0,0 +1,57 @@
+/**
+ * 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.ql.parse;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.exec.Utilities;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.session.SessionState;
+
+public class AnalyzeCommandUtils {
+  public static boolean isPartitionLevelStats(ASTNode tree) {
+    boolean isPartitioned = false;
+    ASTNode child = (ASTNode) tree.getChild(0);
+    if (child.getChildCount() > 1) {
+      child = (ASTNode) child.getChild(1);
+      if (child.getToken().getType() == HiveParser.TOK_PARTSPEC) {
+        isPartitioned = true;
+      }
+    }
+    return isPartitioned;
+  }
+
+  public static Table getTable(ASTNode tree, BaseSemanticAnalyzer sa) throws SemanticException {
+    String tableName = ColumnStatsSemanticAnalyzer.getUnescapedName((ASTNode) tree.getChild(0).getChild(0));
+    String currentDb = SessionState.get().getCurrentDatabase();
+    String [] names = Utilities.getDbTableName(currentDb, tableName);
+    return sa.getTable(names[0], names[1], true);
+  }
+
+  public static Map<String,String> getPartKeyValuePairsFromAST(Table tbl, ASTNode tree,
+      HiveConf hiveConf) throws SemanticException {
+    ASTNode child = ((ASTNode) tree.getChild(0).getChild(1));
+    Map<String,String> partSpec = new HashMap<String, String>();
+    if (child != null) {
+      partSpec = DDLSemanticAnalyzer.getValidatedPartSpec(tbl, child, hiveConf, false);
+    } //otherwise, it is the case of analyze table T compute statistics for columns;
+    return partSpec;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7df62023/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java
index 832a5bc..1f30cbd 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hive.ql.parse;
 
 import java.io.IOException;
-import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
@@ -83,35 +82,6 @@ public class ColumnStatsSemanticAnalyzer extends SemanticAnalyzer {
     return rwt;
   }
 
-  private boolean isPartitionLevelStats(ASTNode tree) {
-    boolean isPartitioned = false;
-    ASTNode child = (ASTNode) tree.getChild(0);
-    if (child.getChildCount() > 1) {
-      child = (ASTNode) child.getChild(1);
-      if (child.getToken().getType() == HiveParser.TOK_PARTSPEC) {
-        isPartitioned = true;
-      }
-    }
-    return isPartitioned;
-  }
-
-  private Table getTable(ASTNode tree) throws SemanticException {
-    String tableName = getUnescapedName((ASTNode) tree.getChild(0).getChild(0));
-    String currentDb = SessionState.get().getCurrentDatabase();
-    String [] names = Utilities.getDbTableName(currentDb, tableName);
-    return getTable(names[0], names[1], true);
-  }
-
-  private Map<String,String> getPartKeyValuePairsFromAST(Table tbl, ASTNode tree,
-      HiveConf hiveConf) throws SemanticException {
-    ASTNode child = ((ASTNode) tree.getChild(0).getChild(1));
-    Map<String,String> partSpec = new HashMap<String, String>();
-    if (child != null) {
-      partSpec = DDLSemanticAnalyzer.getValidatedPartSpec(tbl, child, hiveConf, false);
-    } //otherwise, it is the case of analyze table T compute statistics for columns;
-    return partSpec;
-  }
-
   private List<String> getColumnName(ASTNode tree) throws SemanticException{
 
     switch (tree.getChildCount()) {
@@ -405,11 +375,11 @@ public class ColumnStatsSemanticAnalyzer extends SemanticAnalyzer {
      * an aggregation.
      */
     if (shouldRewrite(ast)) {
-      tbl = getTable(ast);
+      tbl = AnalyzeCommandUtils.getTable(ast, this);
       colNames = getColumnName(ast);
       // Save away the original AST
       originalTree = ast;
-      boolean isPartitionStats = isPartitionLevelStats(ast);
+      boolean isPartitionStats = AnalyzeCommandUtils.isPartitionLevelStats(ast);
       Map<String,String> partSpec = null;
       checkForPartitionColumns(
           colNames, Utilities.getColumnNamesFromFieldSchema(tbl.getPartitionKeys()));
@@ -420,7 +390,7 @@ public class ColumnStatsSemanticAnalyzer extends SemanticAnalyzer {
 
       if (isPartitionStats) {
         isTableLevel = false;
-        partSpec = getPartKeyValuePairsFromAST(tbl, ast, conf);
+        partSpec = AnalyzeCommandUtils.getPartKeyValuePairsFromAST(tbl, ast, conf);
         handlePartialPartitionSpec(partSpec);
       } else {
         isTableLevel = true;

http://git-wip-us.apache.org/repos/asf/hive/blob/7df62023/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
index c407aae..5e6b606 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
@@ -26,6 +26,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.common.FileUtils;
 import org.apache.hadoop.hive.common.JavaUtils;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
@@ -79,10 +80,12 @@ import org.apache.hadoop.hive.ql.plan.AlterTableDesc;
 import org.apache.hadoop.hive.ql.plan.AlterTableDesc.AlterTableTypes;
 import org.apache.hadoop.hive.ql.plan.AlterTableExchangePartition;
 import org.apache.hadoop.hive.ql.plan.AlterTableSimpleDesc;
+import org.apache.hadoop.hive.ql.plan.CacheMetadataDesc;
 import org.apache.hadoop.hive.ql.plan.ColumnStatsDesc;
 import org.apache.hadoop.hive.ql.plan.ColumnStatsUpdateWork;
 import org.apache.hadoop.hive.ql.plan.CreateDatabaseDesc;
 import org.apache.hadoop.hive.ql.plan.CreateIndexDesc;
+import org.apache.hadoop.hive.ql.plan.DDLDesc;
 import org.apache.hadoop.hive.ql.plan.DDLWork;
 import org.apache.hadoop.hive.ql.plan.DescDatabaseDesc;
 import org.apache.hadoop.hive.ql.plan.DescFunctionDesc;
@@ -492,7 +495,10 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
    case HiveParser.TOK_SHOW_SET_ROLE:
      analyzeSetShowRole(ast);
      break;
-    default:
+   case HiveParser.TOK_CACHE_METADATA:
+     analyzeCacheMetadata(ast);
+     break;
+   default:
       throw new SemanticException("Unsupported command.");
     }
     if (fetchTask != null && !rootTasks.isEmpty()) {
@@ -500,6 +506,24 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     }
   }
 
+  private void analyzeCacheMetadata(ASTNode ast) throws SemanticException {
+    Table tbl = AnalyzeCommandUtils.getTable(ast, this);
+    Map<String,String> partSpec = null;
+    CacheMetadataDesc desc;
+    // In 2 cases out of 3, we could pass the path and type directly to metastore...
+    if (AnalyzeCommandUtils.isPartitionLevelStats(ast)) {
+      partSpec = AnalyzeCommandUtils.getPartKeyValuePairsFromAST(tbl, ast, conf);
+      Partition part = getPartition(tbl, partSpec, true);
+      desc = new CacheMetadataDesc(tbl.getDbName(), tbl.getTableName(), part.getName());
+      inputs.add(new ReadEntity(part));
+    } else {
+      // Should we get all partitions for a partitioned table?
+      desc = new CacheMetadataDesc(tbl.getDbName(), tbl.getTableName(), tbl.isPartitioned());
+      inputs.add(new ReadEntity(tbl));
+    }
+    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), desc), conf));
+  }
+
   private void analyzeAlterTableUpdateStats(ASTNode ast, String tblName, Map<String, String> partSpec)
       throws SemanticException {
     String colName = getUnescapedName((ASTNode) ast.getChild(0));
@@ -511,7 +535,6 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
       try {
         partName = Warehouse.makePartName(partSpec, false);
       } catch (MetaException e) {
-        // TODO Auto-generated catch block
         throw new SemanticException("partition " + partSpec.toString()
             + " not found");
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/7df62023/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
index 1c72b1c..4c4470b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
@@ -312,6 +312,7 @@ KW_ISOLATION: 'ISOLATION';
 KW_LEVEL: 'LEVEL';
 KW_SNAPSHOT: 'SNAPSHOT';
 KW_AUTOCOMMIT: 'AUTOCOMMIT';
+KW_CACHE: 'CACHE';
 
 // Operators
 // NOTE: if you add a new function/operator, add it to sysFuncNames so that describe function _FUNC_ will work.

http://git-wip-us.apache.org/repos/asf/hive/blob/7df62023/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
index d5051ce..5f14c6b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
@@ -364,6 +364,7 @@ TOK_TXN_READ_WRITE;
 TOK_COMMIT;
 TOK_ROLLBACK;
 TOK_SET_AUTOCOMMIT;
+TOK_CACHE_METADATA;
 }
 
 
@@ -1370,9 +1371,14 @@ descStatement
 analyzeStatement
 @init { pushMsg("analyze statement", state); }
 @after { popMsg(state); }
-    : KW_ANALYZE KW_TABLE (parttype=tableOrPartition) KW_COMPUTE KW_STATISTICS ((noscan=KW_NOSCAN) | (partialscan=KW_PARTIALSCAN) 
+    : KW_ANALYZE KW_TABLE (parttype=tableOrPartition)
+      (
+      (KW_COMPUTE) => KW_COMPUTE KW_STATISTICS ((noscan=KW_NOSCAN) | (partialscan=KW_PARTIALSCAN)
                                                       | (KW_FOR KW_COLUMNS (statsColumnName=columnNameList)?))?
       -> ^(TOK_ANALYZE $parttype $noscan? $partialscan? KW_COLUMNS? $statsColumnName?)
+      |
+      (KW_CACHE) => KW_CACHE KW_METADATA -> ^(TOK_CACHE_METADATA $parttype)
+      )
     ;
 
 showStatement

http://git-wip-us.apache.org/repos/asf/hive/blob/7df62023/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java
index 0affe84..98860c6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java
@@ -110,6 +110,7 @@ public final class SemanticAnalyzerFactory {
     commandType.put(HiveParser.TOK_DESCDATABASE, HiveOperation.DESCDATABASE);
     commandType.put(HiveParser.TOK_ALTERTABLE_SKEWED, HiveOperation.ALTERTABLE_SKEWED);
     commandType.put(HiveParser.TOK_ANALYZE, HiveOperation.ANALYZE_TABLE);
+    commandType.put(HiveParser.TOK_CACHE_METADATA, HiveOperation.CACHE_METADATA);
     commandType.put(HiveParser.TOK_ALTERTABLE_PARTCOLTYPE, HiveOperation.ALTERTABLE_PARTCOLTYPE);
     commandType.put(HiveParser.TOK_SHOW_COMPACTIONS, HiveOperation.SHOW_COMPACTIONS);
     commandType.put(HiveParser.TOK_SHOW_TRANSACTIONS, HiveOperation.SHOW_TRANSACTIONS);
@@ -259,6 +260,7 @@ public final class SemanticAnalyzerFactory {
       case HiveParser.TOK_ALTERDATABASE_OWNER:
       case HiveParser.TOK_TRUNCATETABLE:
       case HiveParser.TOK_SHOW_SET_ROLE:
+      case HiveParser.TOK_CACHE_METADATA:
         return new DDLSemanticAnalyzer(conf);
 
       case HiveParser.TOK_CREATEFUNCTION:

http://git-wip-us.apache.org/repos/asf/hive/blob/7df62023/ql/src/java/org/apache/hadoop/hive/ql/plan/CacheMetadataDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/CacheMetadataDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/CacheMetadataDesc.java
new file mode 100644
index 0000000..1649b40
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/CacheMetadataDesc.java
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     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.ql.plan;
+
+
+@SuppressWarnings("serial")
+public class CacheMetadataDesc extends DDLDesc {
+  private final String dbName, tableName, partName;
+  private final boolean isAllParts;
+
+  public CacheMetadataDesc(String dbName, String tableName, String partName) {
+    this(dbName, tableName, partName, false);
+  }
+
+  public CacheMetadataDesc(String dbName, String tableName, boolean isAllParts) {
+    this(dbName, tableName, null, isAllParts);
+  }
+
+  private CacheMetadataDesc(String dbName, String tableName, String partName, boolean isAllParts) {
+    super();
+    this.dbName = dbName;
+    this.tableName = tableName;
+    this.partName = partName;
+    this.isAllParts = isAllParts;
+  }
+
+  public boolean isAllParts() {
+    return isAllParts;
+  }
+
+  public String getPartName() {
+    return partName;
+  }
+
+  public String getDbName() {
+    return dbName;
+  }
+
+  public String getTableName() {
+    return tableName;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/7df62023/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java
index a4c3db1..7bb818c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java
@@ -91,6 +91,7 @@ public class DDLWork implements Serializable {
    */
   protected HashSet<WriteEntity> outputs;
   private AlterTablePartMergeFilesDesc mergeFilesDesc;
+  private CacheMetadataDesc cacheMetadataDesc;
 
   public DDLWork() {
   }
@@ -510,6 +511,12 @@ public class DDLWork implements Serializable {
     this.alterTableExchangePartition = alterTableExchangePartition;
   }
 
+  public DDLWork(HashSet<ReadEntity> inputs, HashSet<WriteEntity> outputs,
+      CacheMetadataDesc cacheMetadataDesc) {
+    this(inputs, outputs);
+    this.cacheMetadataDesc = cacheMetadataDesc;
+  }
+
     /**
    * @return Create Database descriptor
    */
@@ -1140,6 +1147,13 @@ public class DDLWork implements Serializable {
   }
 
   /**
+   * @return information about the metadata to be cached
+   */
+  public CacheMetadataDesc getCacheMetadataDesc() {
+    return this.cacheMetadataDesc;
+  }
+
+  /**
    * @param alterTableExchangePartition
    *          set the value of the table partition to be exchanged
    */

http://git-wip-us.apache.org/repos/asf/hive/blob/7df62023/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java
index af7e43e..07134b3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java
@@ -55,6 +55,7 @@ public enum HiveOperation {
   ALTERTABLE_CLUSTER_SORT("ALTERTABLE_CLUSTER_SORT",
       new Privilege[]{Privilege.ALTER_METADATA}, null),
   ANALYZE_TABLE("ANALYZE_TABLE", null, null),
+  CACHE_METADATA("CACHE_METADATA", new Privilege[]{Privilege.SELECT}, null),
   ALTERTABLE_BUCKETNUM("ALTERTABLE_BUCKETNUM",
       new Privilege[]{Privilege.ALTER_METADATA}, null),
   ALTERPARTITION_BUCKETNUM("ALTERPARTITION_BUCKETNUM",

http://git-wip-us.apache.org/repos/asf/hive/blob/7df62023/ql/src/test/queries/clientpositive/stats_filemetadata.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/stats_filemetadata.q b/ql/src/test/queries/clientpositive/stats_filemetadata.q
new file mode 100644
index 0000000..dc9f242
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/stats_filemetadata.q
@@ -0,0 +1,17 @@
+set hive.mapred.mode=nonstrict;
+
+CREATE TABLE many_files(key string, value string)
+partitioned by (ds string)
+clustered by (key) into 4 buckets
+stored as orc;
+
+insert overwrite table many_files partition (ds='1') select * from src;
+insert overwrite table many_files partition (ds='2') select * from src;
+
+dfs -ls -R ${hiveconf:hive.metastore.warehouse.dir}/many_files/;
+
+analyze table many_files cache metadata;
+
+set hive.fetch.task.conversion=none;
+
+select sum(hash(*)) from many_files;

http://git-wip-us.apache.org/repos/asf/hive/blob/7df62023/ql/src/test/results/clientpositive/tez/stats_filemetadata.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/stats_filemetadata.q.out b/ql/src/test/results/clientpositive/tez/stats_filemetadata.q.out
new file mode 100644
index 0000000..a86e1c1
--- /dev/null
+++ b/ql/src/test/results/clientpositive/tez/stats_filemetadata.q.out
@@ -0,0 +1,54 @@
+PREHOOK: query: CREATE TABLE many_files(key string, value string)
+partitioned by (ds string)
+clustered by (key) into 4 buckets
+stored as orc
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@many_files
+POSTHOOK: query: CREATE TABLE many_files(key string, value string)
+partitioned by (ds string)
+clustered by (key) into 4 buckets
+stored as orc
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@many_files
+PREHOOK: query: insert overwrite table many_files partition (ds='1') select * from src
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: default@many_files@ds=1
+POSTHOOK: query: insert overwrite table many_files partition (ds='1') select * from src
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@many_files@ds=1
+POSTHOOK: Lineage: many_files PARTITION(ds=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: many_files PARTITION(ds=1).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+PREHOOK: query: insert overwrite table many_files partition (ds='2') select * from src
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: default@many_files@ds=2
+POSTHOOK: query: insert overwrite table many_files partition (ds='2') select * from src
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@many_files@ds=2
+POSTHOOK: Lineage: many_files PARTITION(ds=2).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: many_files PARTITION(ds=2).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+#### A masked pattern was here ####
+PREHOOK: query: analyze table many_files cache metadata
+PREHOOK: type: CACHE_METADATA
+PREHOOK: Input: default@many_files
+POSTHOOK: query: analyze table many_files cache metadata
+POSTHOOK: type: CACHE_METADATA
+POSTHOOK: Input: default@many_files
+PREHOOK: query: select sum(hash(*)) from many_files
+PREHOOK: type: QUERY
+PREHOOK: Input: default@many_files
+PREHOOK: Input: default@many_files@ds=1
+PREHOOK: Input: default@many_files@ds=2
+#### A masked pattern was here ####
+POSTHOOK: query: select sum(hash(*)) from many_files
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@many_files
+POSTHOOK: Input: default@many_files@ds=1
+POSTHOOK: Input: default@many_files@ds=2
+#### A masked pattern was here ####
+73724366848

http://git-wip-us.apache.org/repos/asf/hive/blob/7df62023/service/src/gen/thrift/gen-py/hive_service/ThriftHive-remote
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-py/hive_service/ThriftHive-remote b/service/src/gen/thrift/gen-py/hive_service/ThriftHive-remote
index e167d5b..4bd7a5a 100755
--- a/service/src/gen/thrift/gen-py/hive_service/ThriftHive-remote
+++ b/service/src/gen/thrift/gen-py/hive_service/ThriftHive-remote
@@ -54,6 +54,7 @@ if len(sys.argv) <= 1 or sys.argv[1] == '--help':
   print('  void drop_table(string dbname, string name, bool deleteData)')
   print('  void drop_table_with_environment_context(string dbname, string name, bool deleteData, EnvironmentContext environment_context)')
   print('   get_tables(string db_name, string pattern)')
+  print('   get_table_meta(string db_patterns, string tbl_patterns,  tbl_types)')
   print('   get_all_tables(string db_name)')
   print('  Table get_table(string dbname, string tbl_name)')
   print('   get_table_objects_by_name(string dbname,  tbl_names)')
@@ -77,6 +78,7 @@ if len(sys.argv) <= 1 or sys.argv[1] == '--help':
   print('  DropPartitionsResult drop_partitions_req(DropPartitionsRequest req)')
   print('  Partition get_partition(string db_name, string tbl_name,  part_vals)')
   print('  Partition exchange_partition( partitionSpecs, string source_db, string source_table_name, string dest_db, string dest_table_name)')
+  print('   exchange_partitions( partitionSpecs, string source_db, string source_table_name, string dest_db, string dest_table_name)')
   print('  Partition get_partition_with_auth(string db_name, string tbl_name,  part_vals, string user_name,  group_names)')
   print('  Partition get_partition_by_name(string db_name, string tbl_name, string part_name)')
   print('   get_partitions(string db_name, string tbl_name, i16 max_parts)')
@@ -162,6 +164,7 @@ if len(sys.argv) <= 1 or sys.argv[1] == '--help':
   print('  GetFileMetadataResult get_file_metadata(GetFileMetadataRequest req)')
   print('  PutFileMetadataResult put_file_metadata(PutFileMetadataRequest req)')
   print('  ClearFileMetadataResult clear_file_metadata(ClearFileMetadataRequest req)')
+  print('  CacheFileMetadataResult cache_file_metadata(CacheFileMetadataRequest req)')
   print('  string getName()')
   print('  string getVersion()')
   print('  fb_status getStatus()')
@@ -411,6 +414,12 @@ elif cmd == 'get_tables':
     sys.exit(1)
   pp.pprint(client.get_tables(args[0],args[1],))
 
+elif cmd == 'get_table_meta':
+  if len(args) != 3:
+    print('get_table_meta requires 3 args')
+    sys.exit(1)
+  pp.pprint(client.get_table_meta(args[0],args[1],eval(args[2]),))
+
 elif cmd == 'get_all_tables':
   if len(args) != 1:
     print('get_all_tables requires 1 args')
@@ -549,6 +558,12 @@ elif cmd == 'exchange_partition':
     sys.exit(1)
   pp.pprint(client.exchange_partition(eval(args[0]),args[1],args[2],args[3],args[4],))
 
+elif cmd == 'exchange_partitions':
+  if len(args) != 5:
+    print('exchange_partitions requires 5 args')
+    sys.exit(1)
+  pp.pprint(client.exchange_partitions(eval(args[0]),args[1],args[2],args[3],args[4],))
+
 elif cmd == 'get_partition_with_auth':
   if len(args) != 5:
     print('get_partition_with_auth requires 5 args')
@@ -1059,6 +1074,12 @@ elif cmd == 'clear_file_metadata':
     sys.exit(1)
   pp.pprint(client.clear_file_metadata(eval(args[0]),))
 
+elif cmd == 'cache_file_metadata':
+  if len(args) != 1:
+    print('cache_file_metadata requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.cache_file_metadata(eval(args[0]),))
+
 elif cmd == 'getName':
   if len(args) != 0:
     print('getName requires 0 args')

http://git-wip-us.apache.org/repos/asf/hive/blob/7df62023/shims/common/src/main/java/org/apache/hadoop/hive/io/HdfsUtils.java
----------------------------------------------------------------------
diff --git a/shims/common/src/main/java/org/apache/hadoop/hive/io/HdfsUtils.java b/shims/common/src/main/java/org/apache/hadoop/hive/io/HdfsUtils.java
new file mode 100644
index 0000000..c90b34c
--- /dev/null
+++ b/shims/common/src/main/java/org/apache/hadoop/hive/io/HdfsUtils.java
@@ -0,0 +1,61 @@
+/**
+ * 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.io;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hive.shims.HadoopShims;
+import org.apache.hadoop.hive.shims.ShimLoader;
+
+public class HdfsUtils {
+  private static final HadoopShims SHIMS = ShimLoader.getHadoopShims();
+  private static final Log LOG = LogFactory.getLog(HdfsUtils.class);
+
+  public static long getFileId(FileSystem fileSystem, Path path) throws IOException {
+    String pathStr = path.toUri().getPath();
+    if (fileSystem instanceof DistributedFileSystem) {
+      return SHIMS.getFileId(fileSystem, pathStr);
+    }
+    // If we are not on DFS, we just hash the file name + size and hope for the best.
+    // TODO: we assume it only happens in tests. Fix?
+    int nameHash = pathStr.hashCode();
+    long fileSize = fileSystem.getFileStatus(path).getLen();
+    long id = ((fileSize ^ (fileSize >>> 32)) << 32) | ((long)nameHash & 0xffffffffL);
+    LOG.warn("Cannot get unique file ID from "
+        + fileSystem.getClass().getSimpleName() + "; using " + id + "(" + pathStr
+        + "," + nameHash + "," + fileSize + ")");
+    return id;
+  }
+
+  // TODO: this relies on HDFS not changing the format; we assume if we could get inode ID, this
+  //       is still going to work. Otherwise, file IDs can be turned off. Later, we should use
+  //       as public utility method in HDFS to obtain the inode-based path.
+  private static String HDFS_ID_PATH_PREFIX = "/.reserved/.inodes/";
+
+  public static Path getFileIdPath(
+      FileSystem fileSystem, Path path, long fileId) {
+    return (fileSystem instanceof DistributedFileSystem)
+        ? new Path(HDFS_ID_PATH_PREFIX + fileId) : path;
+  }
+}


[6/7] hive git commit: HIVE-12075 : add analyze command to explictly cache file metadata in HBase metastore (Sergey Shelukhin, reviewed by Alan Gates)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/7df62023/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
index 0443f80..22bea87 100644
--- a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
+++ b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
@@ -1240,14 +1240,14 @@ uint32_t ThriftHiveMetastore_get_databases_result::read(::apache::thrift::protoc
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size716;
-            ::apache::thrift::protocol::TType _etype719;
-            xfer += iprot->readListBegin(_etype719, _size716);
-            this->success.resize(_size716);
-            uint32_t _i720;
-            for (_i720 = 0; _i720 < _size716; ++_i720)
+            uint32_t _size721;
+            ::apache::thrift::protocol::TType _etype724;
+            xfer += iprot->readListBegin(_etype724, _size721);
+            this->success.resize(_size721);
+            uint32_t _i725;
+            for (_i725 = 0; _i725 < _size721; ++_i725)
             {
-              xfer += iprot->readString(this->success[_i720]);
+              xfer += iprot->readString(this->success[_i725]);
             }
             xfer += iprot->readListEnd();
           }
@@ -1286,10 +1286,10 @@ uint32_t ThriftHiveMetastore_get_databases_result::write(::apache::thrift::proto
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter721;
-      for (_iter721 = this->success.begin(); _iter721 != this->success.end(); ++_iter721)
+      std::vector<std::string> ::const_iterator _iter726;
+      for (_iter726 = this->success.begin(); _iter726 != this->success.end(); ++_iter726)
       {
-        xfer += oprot->writeString((*_iter721));
+        xfer += oprot->writeString((*_iter726));
       }
       xfer += oprot->writeListEnd();
     }
@@ -1334,14 +1334,14 @@ uint32_t ThriftHiveMetastore_get_databases_presult::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size722;
-            ::apache::thrift::protocol::TType _etype725;
-            xfer += iprot->readListBegin(_etype725, _size722);
-            (*(this->success)).resize(_size722);
-            uint32_t _i726;
-            for (_i726 = 0; _i726 < _size722; ++_i726)
+            uint32_t _size727;
+            ::apache::thrift::protocol::TType _etype730;
+            xfer += iprot->readListBegin(_etype730, _size727);
+            (*(this->success)).resize(_size727);
+            uint32_t _i731;
+            for (_i731 = 0; _i731 < _size727; ++_i731)
             {
-              xfer += iprot->readString((*(this->success))[_i726]);
+              xfer += iprot->readString((*(this->success))[_i731]);
             }
             xfer += iprot->readListEnd();
           }
@@ -1458,14 +1458,14 @@ uint32_t ThriftHiveMetastore_get_all_databases_result::read(::apache::thrift::pr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size727;
-            ::apache::thrift::protocol::TType _etype730;
-            xfer += iprot->readListBegin(_etype730, _size727);
-            this->success.resize(_size727);
-            uint32_t _i731;
-            for (_i731 = 0; _i731 < _size727; ++_i731)
+            uint32_t _size732;
+            ::apache::thrift::protocol::TType _etype735;
+            xfer += iprot->readListBegin(_etype735, _size732);
+            this->success.resize(_size732);
+            uint32_t _i736;
+            for (_i736 = 0; _i736 < _size732; ++_i736)
             {
-              xfer += iprot->readString(this->success[_i731]);
+              xfer += iprot->readString(this->success[_i736]);
             }
             xfer += iprot->readListEnd();
           }
@@ -1504,10 +1504,10 @@ uint32_t ThriftHiveMetastore_get_all_databases_result::write(::apache::thrift::p
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter732;
-      for (_iter732 = this->success.begin(); _iter732 != this->success.end(); ++_iter732)
+      std::vector<std::string> ::const_iterator _iter737;
+      for (_iter737 = this->success.begin(); _iter737 != this->success.end(); ++_iter737)
       {
-        xfer += oprot->writeString((*_iter732));
+        xfer += oprot->writeString((*_iter737));
       }
       xfer += oprot->writeListEnd();
     }
@@ -1552,14 +1552,14 @@ uint32_t ThriftHiveMetastore_get_all_databases_presult::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size733;
-            ::apache::thrift::protocol::TType _etype736;
-            xfer += iprot->readListBegin(_etype736, _size733);
-            (*(this->success)).resize(_size733);
-            uint32_t _i737;
-            for (_i737 = 0; _i737 < _size733; ++_i737)
+            uint32_t _size738;
+            ::apache::thrift::protocol::TType _etype741;
+            xfer += iprot->readListBegin(_etype741, _size738);
+            (*(this->success)).resize(_size738);
+            uint32_t _i742;
+            for (_i742 = 0; _i742 < _size738; ++_i742)
             {
-              xfer += iprot->readString((*(this->success))[_i737]);
+              xfer += iprot->readString((*(this->success))[_i742]);
             }
             xfer += iprot->readListEnd();
           }
@@ -2621,17 +2621,17 @@ uint32_t ThriftHiveMetastore_get_type_all_result::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->success.clear();
-            uint32_t _size738;
-            ::apache::thrift::protocol::TType _ktype739;
-            ::apache::thrift::protocol::TType _vtype740;
-            xfer += iprot->readMapBegin(_ktype739, _vtype740, _size738);
-            uint32_t _i742;
-            for (_i742 = 0; _i742 < _size738; ++_i742)
+            uint32_t _size743;
+            ::apache::thrift::protocol::TType _ktype744;
+            ::apache::thrift::protocol::TType _vtype745;
+            xfer += iprot->readMapBegin(_ktype744, _vtype745, _size743);
+            uint32_t _i747;
+            for (_i747 = 0; _i747 < _size743; ++_i747)
             {
-              std::string _key743;
-              xfer += iprot->readString(_key743);
-              Type& _val744 = this->success[_key743];
-              xfer += _val744.read(iprot);
+              std::string _key748;
+              xfer += iprot->readString(_key748);
+              Type& _val749 = this->success[_key748];
+              xfer += _val749.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -2670,11 +2670,11 @@ uint32_t ThriftHiveMetastore_get_type_all_result::write(::apache::thrift::protoc
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_MAP, 0);
     {
       xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::map<std::string, Type> ::const_iterator _iter745;
-      for (_iter745 = this->success.begin(); _iter745 != this->success.end(); ++_iter745)
+      std::map<std::string, Type> ::const_iterator _iter750;
+      for (_iter750 = this->success.begin(); _iter750 != this->success.end(); ++_iter750)
       {
-        xfer += oprot->writeString(_iter745->first);
-        xfer += _iter745->second.write(oprot);
+        xfer += oprot->writeString(_iter750->first);
+        xfer += _iter750->second.write(oprot);
       }
       xfer += oprot->writeMapEnd();
     }
@@ -2719,17 +2719,17 @@ uint32_t ThriftHiveMetastore_get_type_all_presult::read(::apache::thrift::protoc
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             (*(this->success)).clear();
-            uint32_t _size746;
-            ::apache::thrift::protocol::TType _ktype747;
-            ::apache::thrift::protocol::TType _vtype748;
-            xfer += iprot->readMapBegin(_ktype747, _vtype748, _size746);
-            uint32_t _i750;
-            for (_i750 = 0; _i750 < _size746; ++_i750)
+            uint32_t _size751;
+            ::apache::thrift::protocol::TType _ktype752;
+            ::apache::thrift::protocol::TType _vtype753;
+            xfer += iprot->readMapBegin(_ktype752, _vtype753, _size751);
+            uint32_t _i755;
+            for (_i755 = 0; _i755 < _size751; ++_i755)
             {
-              std::string _key751;
-              xfer += iprot->readString(_key751);
-              Type& _val752 = (*(this->success))[_key751];
-              xfer += _val752.read(iprot);
+              std::string _key756;
+              xfer += iprot->readString(_key756);
+              Type& _val757 = (*(this->success))[_key756];
+              xfer += _val757.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -2883,14 +2883,14 @@ uint32_t ThriftHiveMetastore_get_fields_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size753;
-            ::apache::thrift::protocol::TType _etype756;
-            xfer += iprot->readListBegin(_etype756, _size753);
-            this->success.resize(_size753);
-            uint32_t _i757;
-            for (_i757 = 0; _i757 < _size753; ++_i757)
+            uint32_t _size758;
+            ::apache::thrift::protocol::TType _etype761;
+            xfer += iprot->readListBegin(_etype761, _size758);
+            this->success.resize(_size758);
+            uint32_t _i762;
+            for (_i762 = 0; _i762 < _size758; ++_i762)
             {
-              xfer += this->success[_i757].read(iprot);
+              xfer += this->success[_i762].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -2945,10 +2945,10 @@ uint32_t ThriftHiveMetastore_get_fields_result::write(::apache::thrift::protocol
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter758;
-      for (_iter758 = this->success.begin(); _iter758 != this->success.end(); ++_iter758)
+      std::vector<FieldSchema> ::const_iterator _iter763;
+      for (_iter763 = this->success.begin(); _iter763 != this->success.end(); ++_iter763)
       {
-        xfer += (*_iter758).write(oprot);
+        xfer += (*_iter763).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -3001,14 +3001,14 @@ uint32_t ThriftHiveMetastore_get_fields_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size759;
-            ::apache::thrift::protocol::TType _etype762;
-            xfer += iprot->readListBegin(_etype762, _size759);
-            (*(this->success)).resize(_size759);
-            uint32_t _i763;
-            for (_i763 = 0; _i763 < _size759; ++_i763)
+            uint32_t _size764;
+            ::apache::thrift::protocol::TType _etype767;
+            xfer += iprot->readListBegin(_etype767, _size764);
+            (*(this->success)).resize(_size764);
+            uint32_t _i768;
+            for (_i768 = 0; _i768 < _size764; ++_i768)
             {
-              xfer += (*(this->success))[_i763].read(iprot);
+              xfer += (*(this->success))[_i768].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3194,14 +3194,14 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_result::read(::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size764;
-            ::apache::thrift::protocol::TType _etype767;
-            xfer += iprot->readListBegin(_etype767, _size764);
-            this->success.resize(_size764);
-            uint32_t _i768;
-            for (_i768 = 0; _i768 < _size764; ++_i768)
+            uint32_t _size769;
+            ::apache::thrift::protocol::TType _etype772;
+            xfer += iprot->readListBegin(_etype772, _size769);
+            this->success.resize(_size769);
+            uint32_t _i773;
+            for (_i773 = 0; _i773 < _size769; ++_i773)
             {
-              xfer += this->success[_i768].read(iprot);
+              xfer += this->success[_i773].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3256,10 +3256,10 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_result::write(:
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter769;
-      for (_iter769 = this->success.begin(); _iter769 != this->success.end(); ++_iter769)
+      std::vector<FieldSchema> ::const_iterator _iter774;
+      for (_iter774 = this->success.begin(); _iter774 != this->success.end(); ++_iter774)
       {
-        xfer += (*_iter769).write(oprot);
+        xfer += (*_iter774).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -3312,14 +3312,14 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_presult::read(:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size770;
-            ::apache::thrift::protocol::TType _etype773;
-            xfer += iprot->readListBegin(_etype773, _size770);
-            (*(this->success)).resize(_size770);
-            uint32_t _i774;
-            for (_i774 = 0; _i774 < _size770; ++_i774)
+            uint32_t _size775;
+            ::apache::thrift::protocol::TType _etype778;
+            xfer += iprot->readListBegin(_etype778, _size775);
+            (*(this->success)).resize(_size775);
+            uint32_t _i779;
+            for (_i779 = 0; _i779 < _size775; ++_i779)
             {
-              xfer += (*(this->success))[_i774].read(iprot);
+              xfer += (*(this->success))[_i779].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3489,14 +3489,14 @@ uint32_t ThriftHiveMetastore_get_schema_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size775;
-            ::apache::thrift::protocol::TType _etype778;
-            xfer += iprot->readListBegin(_etype778, _size775);
-            this->success.resize(_size775);
-            uint32_t _i779;
-            for (_i779 = 0; _i779 < _size775; ++_i779)
+            uint32_t _size780;
+            ::apache::thrift::protocol::TType _etype783;
+            xfer += iprot->readListBegin(_etype783, _size780);
+            this->success.resize(_size780);
+            uint32_t _i784;
+            for (_i784 = 0; _i784 < _size780; ++_i784)
             {
-              xfer += this->success[_i779].read(iprot);
+              xfer += this->success[_i784].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3551,10 +3551,10 @@ uint32_t ThriftHiveMetastore_get_schema_result::write(::apache::thrift::protocol
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter780;
-      for (_iter780 = this->success.begin(); _iter780 != this->success.end(); ++_iter780)
+      std::vector<FieldSchema> ::const_iterator _iter785;
+      for (_iter785 = this->success.begin(); _iter785 != this->success.end(); ++_iter785)
       {
-        xfer += (*_iter780).write(oprot);
+        xfer += (*_iter785).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -3607,14 +3607,14 @@ uint32_t ThriftHiveMetastore_get_schema_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size781;
-            ::apache::thrift::protocol::TType _etype784;
-            xfer += iprot->readListBegin(_etype784, _size781);
-            (*(this->success)).resize(_size781);
-            uint32_t _i785;
-            for (_i785 = 0; _i785 < _size781; ++_i785)
+            uint32_t _size786;
+            ::apache::thrift::protocol::TType _etype789;
+            xfer += iprot->readListBegin(_etype789, _size786);
+            (*(this->success)).resize(_size786);
+            uint32_t _i790;
+            for (_i790 = 0; _i790 < _size786; ++_i790)
             {
-              xfer += (*(this->success))[_i785].read(iprot);
+              xfer += (*(this->success))[_i790].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3800,14 +3800,14 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_result::read(::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size786;
-            ::apache::thrift::protocol::TType _etype789;
-            xfer += iprot->readListBegin(_etype789, _size786);
-            this->success.resize(_size786);
-            uint32_t _i790;
-            for (_i790 = 0; _i790 < _size786; ++_i790)
+            uint32_t _size791;
+            ::apache::thrift::protocol::TType _etype794;
+            xfer += iprot->readListBegin(_etype794, _size791);
+            this->success.resize(_size791);
+            uint32_t _i795;
+            for (_i795 = 0; _i795 < _size791; ++_i795)
             {
-              xfer += this->success[_i790].read(iprot);
+              xfer += this->success[_i795].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3862,10 +3862,10 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_result::write(:
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter791;
-      for (_iter791 = this->success.begin(); _iter791 != this->success.end(); ++_iter791)
+      std::vector<FieldSchema> ::const_iterator _iter796;
+      for (_iter796 = this->success.begin(); _iter796 != this->success.end(); ++_iter796)
       {
-        xfer += (*_iter791).write(oprot);
+        xfer += (*_iter796).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -3918,14 +3918,14 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_presult::read(:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size792;
-            ::apache::thrift::protocol::TType _etype795;
-            xfer += iprot->readListBegin(_etype795, _size792);
-            (*(this->success)).resize(_size792);
-            uint32_t _i796;
-            for (_i796 = 0; _i796 < _size792; ++_i796)
+            uint32_t _size797;
+            ::apache::thrift::protocol::TType _etype800;
+            xfer += iprot->readListBegin(_etype800, _size797);
+            (*(this->success)).resize(_size797);
+            uint32_t _i801;
+            for (_i801 = 0; _i801 < _size797; ++_i801)
             {
-              xfer += (*(this->success))[_i796].read(iprot);
+              xfer += (*(this->success))[_i801].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5099,14 +5099,14 @@ uint32_t ThriftHiveMetastore_get_tables_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size797;
-            ::apache::thrift::protocol::TType _etype800;
-            xfer += iprot->readListBegin(_etype800, _size797);
-            this->success.resize(_size797);
-            uint32_t _i801;
-            for (_i801 = 0; _i801 < _size797; ++_i801)
+            uint32_t _size802;
+            ::apache::thrift::protocol::TType _etype805;
+            xfer += iprot->readListBegin(_etype805, _size802);
+            this->success.resize(_size802);
+            uint32_t _i806;
+            for (_i806 = 0; _i806 < _size802; ++_i806)
             {
-              xfer += iprot->readString(this->success[_i801]);
+              xfer += iprot->readString(this->success[_i806]);
             }
             xfer += iprot->readListEnd();
           }
@@ -5145,10 +5145,10 @@ uint32_t ThriftHiveMetastore_get_tables_result::write(::apache::thrift::protocol
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter802;
-      for (_iter802 = this->success.begin(); _iter802 != this->success.end(); ++_iter802)
+      std::vector<std::string> ::const_iterator _iter807;
+      for (_iter807 = this->success.begin(); _iter807 != this->success.end(); ++_iter807)
       {
-        xfer += oprot->writeString((*_iter802));
+        xfer += oprot->writeString((*_iter807));
       }
       xfer += oprot->writeListEnd();
     }
@@ -5193,14 +5193,14 @@ uint32_t ThriftHiveMetastore_get_tables_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size803;
-            ::apache::thrift::protocol::TType _etype806;
-            xfer += iprot->readListBegin(_etype806, _size803);
-            (*(this->success)).resize(_size803);
-            uint32_t _i807;
-            for (_i807 = 0; _i807 < _size803; ++_i807)
+            uint32_t _size808;
+            ::apache::thrift::protocol::TType _etype811;
+            xfer += iprot->readListBegin(_etype811, _size808);
+            (*(this->success)).resize(_size808);
+            uint32_t _i812;
+            for (_i812 = 0; _i812 < _size808; ++_i812)
             {
-              xfer += iprot->readString((*(this->success))[_i807]);
+              xfer += iprot->readString((*(this->success))[_i812]);
             }
             xfer += iprot->readListEnd();
           }
@@ -5275,14 +5275,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->tbl_types.clear();
-            uint32_t _size808;
-            ::apache::thrift::protocol::TType _etype811;
-            xfer += iprot->readListBegin(_etype811, _size808);
-            this->tbl_types.resize(_size808);
-            uint32_t _i812;
-            for (_i812 = 0; _i812 < _size808; ++_i812)
+            uint32_t _size813;
+            ::apache::thrift::protocol::TType _etype816;
+            xfer += iprot->readListBegin(_etype816, _size813);
+            this->tbl_types.resize(_size813);
+            uint32_t _i817;
+            for (_i817 = 0; _i817 < _size813; ++_i817)
             {
-              xfer += iprot->readString(this->tbl_types[_i812]);
+              xfer += iprot->readString(this->tbl_types[_i817]);
             }
             xfer += iprot->readListEnd();
           }
@@ -5319,10 +5319,10 @@ uint32_t ThriftHiveMetastore_get_table_meta_args::write(::apache::thrift::protoc
   xfer += oprot->writeFieldBegin("tbl_types", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->tbl_types.size()));
-    std::vector<std::string> ::const_iterator _iter813;
-    for (_iter813 = this->tbl_types.begin(); _iter813 != this->tbl_types.end(); ++_iter813)
+    std::vector<std::string> ::const_iterator _iter818;
+    for (_iter818 = this->tbl_types.begin(); _iter818 != this->tbl_types.end(); ++_iter818)
     {
-      xfer += oprot->writeString((*_iter813));
+      xfer += oprot->writeString((*_iter818));
     }
     xfer += oprot->writeListEnd();
   }
@@ -5354,10 +5354,10 @@ uint32_t ThriftHiveMetastore_get_table_meta_pargs::write(::apache::thrift::proto
   xfer += oprot->writeFieldBegin("tbl_types", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->tbl_types)).size()));
-    std::vector<std::string> ::const_iterator _iter814;
-    for (_iter814 = (*(this->tbl_types)).begin(); _iter814 != (*(this->tbl_types)).end(); ++_iter814)
+    std::vector<std::string> ::const_iterator _iter819;
+    for (_iter819 = (*(this->tbl_types)).begin(); _iter819 != (*(this->tbl_types)).end(); ++_iter819)
     {
-      xfer += oprot->writeString((*_iter814));
+      xfer += oprot->writeString((*_iter819));
     }
     xfer += oprot->writeListEnd();
   }
@@ -5398,14 +5398,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size815;
-            ::apache::thrift::protocol::TType _etype818;
-            xfer += iprot->readListBegin(_etype818, _size815);
-            this->success.resize(_size815);
-            uint32_t _i819;
-            for (_i819 = 0; _i819 < _size815; ++_i819)
+            uint32_t _size820;
+            ::apache::thrift::protocol::TType _etype823;
+            xfer += iprot->readListBegin(_etype823, _size820);
+            this->success.resize(_size820);
+            uint32_t _i824;
+            for (_i824 = 0; _i824 < _size820; ++_i824)
             {
-              xfer += this->success[_i819].read(iprot);
+              xfer += this->success[_i824].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5444,10 +5444,10 @@ uint32_t ThriftHiveMetastore_get_table_meta_result::write(::apache::thrift::prot
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<TableMeta> ::const_iterator _iter820;
-      for (_iter820 = this->success.begin(); _iter820 != this->success.end(); ++_iter820)
+      std::vector<TableMeta> ::const_iterator _iter825;
+      for (_iter825 = this->success.begin(); _iter825 != this->success.end(); ++_iter825)
       {
-        xfer += (*_iter820).write(oprot);
+        xfer += (*_iter825).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -5492,14 +5492,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size821;
-            ::apache::thrift::protocol::TType _etype824;
-            xfer += iprot->readListBegin(_etype824, _size821);
-            (*(this->success)).resize(_size821);
-            uint32_t _i825;
-            for (_i825 = 0; _i825 < _size821; ++_i825)
+            uint32_t _size826;
+            ::apache::thrift::protocol::TType _etype829;
+            xfer += iprot->readListBegin(_etype829, _size826);
+            (*(this->success)).resize(_size826);
+            uint32_t _i830;
+            for (_i830 = 0; _i830 < _size826; ++_i830)
             {
-              xfer += (*(this->success))[_i825].read(iprot);
+              xfer += (*(this->success))[_i830].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5637,14 +5637,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size826;
-            ::apache::thrift::protocol::TType _etype829;
-            xfer += iprot->readListBegin(_etype829, _size826);
-            this->success.resize(_size826);
-            uint32_t _i830;
-            for (_i830 = 0; _i830 < _size826; ++_i830)
+            uint32_t _size831;
+            ::apache::thrift::protocol::TType _etype834;
+            xfer += iprot->readListBegin(_etype834, _size831);
+            this->success.resize(_size831);
+            uint32_t _i835;
+            for (_i835 = 0; _i835 < _size831; ++_i835)
             {
-              xfer += iprot->readString(this->success[_i830]);
+              xfer += iprot->readString(this->success[_i835]);
             }
             xfer += iprot->readListEnd();
           }
@@ -5683,10 +5683,10 @@ uint32_t ThriftHiveMetastore_get_all_tables_result::write(::apache::thrift::prot
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter831;
-      for (_iter831 = this->success.begin(); _iter831 != this->success.end(); ++_iter831)
+      std::vector<std::string> ::const_iterator _iter836;
+      for (_iter836 = this->success.begin(); _iter836 != this->success.end(); ++_iter836)
       {
-        xfer += oprot->writeString((*_iter831));
+        xfer += oprot->writeString((*_iter836));
       }
       xfer += oprot->writeListEnd();
     }
@@ -5731,14 +5731,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size832;
-            ::apache::thrift::protocol::TType _etype835;
-            xfer += iprot->readListBegin(_etype835, _size832);
-            (*(this->success)).resize(_size832);
-            uint32_t _i836;
-            for (_i836 = 0; _i836 < _size832; ++_i836)
+            uint32_t _size837;
+            ::apache::thrift::protocol::TType _etype840;
+            xfer += iprot->readListBegin(_etype840, _size837);
+            (*(this->success)).resize(_size837);
+            uint32_t _i841;
+            for (_i841 = 0; _i841 < _size837; ++_i841)
             {
-              xfer += iprot->readString((*(this->success))[_i836]);
+              xfer += iprot->readString((*(this->success))[_i841]);
             }
             xfer += iprot->readListEnd();
           }
@@ -6048,14 +6048,14 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_args::read(::apache::thri
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->tbl_names.clear();
-            uint32_t _size837;
-            ::apache::thrift::protocol::TType _etype840;
-            xfer += iprot->readListBegin(_etype840, _size837);
-            this->tbl_names.resize(_size837);
-            uint32_t _i841;
-            for (_i841 = 0; _i841 < _size837; ++_i841)
+            uint32_t _size842;
+            ::apache::thrift::protocol::TType _etype845;
+            xfer += iprot->readListBegin(_etype845, _size842);
+            this->tbl_names.resize(_size842);
+            uint32_t _i846;
+            for (_i846 = 0; _i846 < _size842; ++_i846)
             {
-              xfer += iprot->readString(this->tbl_names[_i841]);
+              xfer += iprot->readString(this->tbl_names[_i846]);
             }
             xfer += iprot->readListEnd();
           }
@@ -6088,10 +6088,10 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_args::write(::apache::thr
   xfer += oprot->writeFieldBegin("tbl_names", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->tbl_names.size()));
-    std::vector<std::string> ::const_iterator _iter842;
-    for (_iter842 = this->tbl_names.begin(); _iter842 != this->tbl_names.end(); ++_iter842)
+    std::vector<std::string> ::const_iterator _iter847;
+    for (_iter847 = this->tbl_names.begin(); _iter847 != this->tbl_names.end(); ++_iter847)
     {
-      xfer += oprot->writeString((*_iter842));
+      xfer += oprot->writeString((*_iter847));
     }
     xfer += oprot->writeListEnd();
   }
@@ -6119,10 +6119,10 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_pargs::write(::apache::th
   xfer += oprot->writeFieldBegin("tbl_names", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->tbl_names)).size()));
-    std::vector<std::string> ::const_iterator _iter843;
-    for (_iter843 = (*(this->tbl_names)).begin(); _iter843 != (*(this->tbl_names)).end(); ++_iter843)
+    std::vector<std::string> ::const_iterator _iter848;
+    for (_iter848 = (*(this->tbl_names)).begin(); _iter848 != (*(this->tbl_names)).end(); ++_iter848)
     {
-      xfer += oprot->writeString((*_iter843));
+      xfer += oprot->writeString((*_iter848));
     }
     xfer += oprot->writeListEnd();
   }
@@ -6163,14 +6163,14 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_result::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size844;
-            ::apache::thrift::protocol::TType _etype847;
-            xfer += iprot->readListBegin(_etype847, _size844);
-            this->success.resize(_size844);
-            uint32_t _i848;
-            for (_i848 = 0; _i848 < _size844; ++_i848)
+            uint32_t _size849;
+            ::apache::thrift::protocol::TType _etype852;
+            xfer += iprot->readListBegin(_etype852, _size849);
+            this->success.resize(_size849);
+            uint32_t _i853;
+            for (_i853 = 0; _i853 < _size849; ++_i853)
             {
-              xfer += this->success[_i848].read(iprot);
+              xfer += this->success[_i853].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -6225,10 +6225,10 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_result::write(::apache::t
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Table> ::const_iterator _iter849;
-      for (_iter849 = this->success.begin(); _iter849 != this->success.end(); ++_iter849)
+      std::vector<Table> ::const_iterator _iter854;
+      for (_iter854 = this->success.begin(); _iter854 != this->success.end(); ++_iter854)
       {
-        xfer += (*_iter849).write(oprot);
+        xfer += (*_iter854).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -6281,14 +6281,14 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_presult::read(::apache::t
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size850;
-            ::apache::thrift::protocol::TType _etype853;
-            xfer += iprot->readListBegin(_etype853, _size850);
-            (*(this->success)).resize(_size850);
-            uint32_t _i854;
-            for (_i854 = 0; _i854 < _size850; ++_i854)
+            uint32_t _size855;
+            ::apache::thrift::protocol::TType _etype858;
+            xfer += iprot->readListBegin(_etype858, _size855);
+            (*(this->success)).resize(_size855);
+            uint32_t _i859;
+            for (_i859 = 0; _i859 < _size855; ++_i859)
             {
-              xfer += (*(this->success))[_i854].read(iprot);
+              xfer += (*(this->success))[_i859].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -6474,14 +6474,14 @@ uint32_t ThriftHiveMetastore_get_table_names_by_filter_result::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size855;
-            ::apache::thrift::protocol::TType _etype858;
-            xfer += iprot->readListBegin(_etype858, _size855);
-            this->success.resize(_size855);
-            uint32_t _i859;
-            for (_i859 = 0; _i859 < _size855; ++_i859)
+            uint32_t _size860;
+            ::apache::thrift::protocol::TType _etype863;
+            xfer += iprot->readListBegin(_etype863, _size860);
+            this->success.resize(_size860);
+            uint32_t _i864;
+            for (_i864 = 0; _i864 < _size860; ++_i864)
             {
-              xfer += iprot->readString(this->success[_i859]);
+              xfer += iprot->readString(this->success[_i864]);
             }
             xfer += iprot->readListEnd();
           }
@@ -6536,10 +6536,10 @@ uint32_t ThriftHiveMetastore_get_table_names_by_filter_result::write(::apache::t
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter860;
-      for (_iter860 = this->success.begin(); _iter860 != this->success.end(); ++_iter860)
+      std::vector<std::string> ::const_iterator _iter865;
+      for (_iter865 = this->success.begin(); _iter865 != this->success.end(); ++_iter865)
       {
-        xfer += oprot->writeString((*_iter860));
+        xfer += oprot->writeString((*_iter865));
       }
       xfer += oprot->writeListEnd();
     }
@@ -6592,14 +6592,14 @@ uint32_t ThriftHiveMetastore_get_table_names_by_filter_presult::read(::apache::t
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size861;
-            ::apache::thrift::protocol::TType _etype864;
-            xfer += iprot->readListBegin(_etype864, _size861);
-            (*(this->success)).resize(_size861);
-            uint32_t _i865;
-            for (_i865 = 0; _i865 < _size861; ++_i865)
+            uint32_t _size866;
+            ::apache::thrift::protocol::TType _etype869;
+            xfer += iprot->readListBegin(_etype869, _size866);
+            (*(this->success)).resize(_size866);
+            uint32_t _i870;
+            for (_i870 = 0; _i870 < _size866; ++_i870)
             {
-              xfer += iprot->readString((*(this->success))[_i865]);
+              xfer += iprot->readString((*(this->success))[_i870]);
             }
             xfer += iprot->readListEnd();
           }
@@ -7933,14 +7933,14 @@ uint32_t ThriftHiveMetastore_add_partitions_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->new_parts.clear();
-            uint32_t _size866;
-            ::apache::thrift::protocol::TType _etype869;
-            xfer += iprot->readListBegin(_etype869, _size866);
-            this->new_parts.resize(_size866);
-            uint32_t _i870;
-            for (_i870 = 0; _i870 < _size866; ++_i870)
+            uint32_t _size871;
+            ::apache::thrift::protocol::TType _etype874;
+            xfer += iprot->readListBegin(_etype874, _size871);
+            this->new_parts.resize(_size871);
+            uint32_t _i875;
+            for (_i875 = 0; _i875 < _size871; ++_i875)
             {
-              xfer += this->new_parts[_i870].read(iprot);
+              xfer += this->new_parts[_i875].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -7969,10 +7969,10 @@ uint32_t ThriftHiveMetastore_add_partitions_args::write(::apache::thrift::protoc
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->new_parts.size()));
-    std::vector<Partition> ::const_iterator _iter871;
-    for (_iter871 = this->new_parts.begin(); _iter871 != this->new_parts.end(); ++_iter871)
+    std::vector<Partition> ::const_iterator _iter876;
+    for (_iter876 = this->new_parts.begin(); _iter876 != this->new_parts.end(); ++_iter876)
     {
-      xfer += (*_iter871).write(oprot);
+      xfer += (*_iter876).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -7996,10 +7996,10 @@ uint32_t ThriftHiveMetastore_add_partitions_pargs::write(::apache::thrift::proto
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->new_parts)).size()));
-    std::vector<Partition> ::const_iterator _iter872;
-    for (_iter872 = (*(this->new_parts)).begin(); _iter872 != (*(this->new_parts)).end(); ++_iter872)
+    std::vector<Partition> ::const_iterator _iter877;
+    for (_iter877 = (*(this->new_parts)).begin(); _iter877 != (*(this->new_parts)).end(); ++_iter877)
     {
-      xfer += (*_iter872).write(oprot);
+      xfer += (*_iter877).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -8208,14 +8208,14 @@ uint32_t ThriftHiveMetastore_add_partitions_pspec_args::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->new_parts.clear();
-            uint32_t _size873;
-            ::apache::thrift::protocol::TType _etype876;
-            xfer += iprot->readListBegin(_etype876, _size873);
-            this->new_parts.resize(_size873);
-            uint32_t _i877;
-            for (_i877 = 0; _i877 < _size873; ++_i877)
+            uint32_t _size878;
+            ::apache::thrift::protocol::TType _etype881;
+            xfer += iprot->readListBegin(_etype881, _size878);
+            this->new_parts.resize(_size878);
+            uint32_t _i882;
+            for (_i882 = 0; _i882 < _size878; ++_i882)
             {
-              xfer += this->new_parts[_i877].read(iprot);
+              xfer += this->new_parts[_i882].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -8244,10 +8244,10 @@ uint32_t ThriftHiveMetastore_add_partitions_pspec_args::write(::apache::thrift::
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->new_parts.size()));
-    std::vector<PartitionSpec> ::const_iterator _iter878;
-    for (_iter878 = this->new_parts.begin(); _iter878 != this->new_parts.end(); ++_iter878)
+    std::vector<PartitionSpec> ::const_iterator _iter883;
+    for (_iter883 = this->new_parts.begin(); _iter883 != this->new_parts.end(); ++_iter883)
     {
-      xfer += (*_iter878).write(oprot);
+      xfer += (*_iter883).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -8271,10 +8271,10 @@ uint32_t ThriftHiveMetastore_add_partitions_pspec_pargs::write(::apache::thrift:
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->new_parts)).size()));
-    std::vector<PartitionSpec> ::const_iterator _iter879;
-    for (_iter879 = (*(this->new_parts)).begin(); _iter879 != (*(this->new_parts)).end(); ++_iter879)
+    std::vector<PartitionSpec> ::const_iterator _iter884;
+    for (_iter884 = (*(this->new_parts)).begin(); _iter884 != (*(this->new_parts)).end(); ++_iter884)
     {
-      xfer += (*_iter879).write(oprot);
+      xfer += (*_iter884).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -8499,14 +8499,14 @@ uint32_t ThriftHiveMetastore_append_partition_args::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size880;
-            ::apache::thrift::protocol::TType _etype883;
-            xfer += iprot->readListBegin(_etype883, _size880);
-            this->part_vals.resize(_size880);
-            uint32_t _i884;
-            for (_i884 = 0; _i884 < _size880; ++_i884)
+            uint32_t _size885;
+            ::apache::thrift::protocol::TType _etype888;
+            xfer += iprot->readListBegin(_etype888, _size885);
+            this->part_vals.resize(_size885);
+            uint32_t _i889;
+            for (_i889 = 0; _i889 < _size885; ++_i889)
             {
-              xfer += iprot->readString(this->part_vals[_i884]);
+              xfer += iprot->readString(this->part_vals[_i889]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8543,10 +8543,10 @@ uint32_t ThriftHiveMetastore_append_partition_args::write(::apache::thrift::prot
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter885;
-    for (_iter885 = this->part_vals.begin(); _iter885 != this->part_vals.end(); ++_iter885)
+    std::vector<std::string> ::const_iterator _iter890;
+    for (_iter890 = this->part_vals.begin(); _iter890 != this->part_vals.end(); ++_iter890)
     {
-      xfer += oprot->writeString((*_iter885));
+      xfer += oprot->writeString((*_iter890));
     }
     xfer += oprot->writeListEnd();
   }
@@ -8578,10 +8578,10 @@ uint32_t ThriftHiveMetastore_append_partition_pargs::write(::apache::thrift::pro
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter886;
-    for (_iter886 = (*(this->part_vals)).begin(); _iter886 != (*(this->part_vals)).end(); ++_iter886)
+    std::vector<std::string> ::const_iterator _iter891;
+    for (_iter891 = (*(this->part_vals)).begin(); _iter891 != (*(this->part_vals)).end(); ++_iter891)
     {
-      xfer += oprot->writeString((*_iter886));
+      xfer += oprot->writeString((*_iter891));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9053,14 +9053,14 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_args::rea
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size887;
-            ::apache::thrift::protocol::TType _etype890;
-            xfer += iprot->readListBegin(_etype890, _size887);
-            this->part_vals.resize(_size887);
-            uint32_t _i891;
-            for (_i891 = 0; _i891 < _size887; ++_i891)
+            uint32_t _size892;
+            ::apache::thrift::protocol::TType _etype895;
+            xfer += iprot->readListBegin(_etype895, _size892);
+            this->part_vals.resize(_size892);
+            uint32_t _i896;
+            for (_i896 = 0; _i896 < _size892; ++_i896)
             {
-              xfer += iprot->readString(this->part_vals[_i891]);
+              xfer += iprot->readString(this->part_vals[_i896]);
             }
             xfer += iprot->readListEnd();
           }
@@ -9105,10 +9105,10 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_args::wri
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter892;
-    for (_iter892 = this->part_vals.begin(); _iter892 != this->part_vals.end(); ++_iter892)
+    std::vector<std::string> ::const_iterator _iter897;
+    for (_iter897 = this->part_vals.begin(); _iter897 != this->part_vals.end(); ++_iter897)
     {
-      xfer += oprot->writeString((*_iter892));
+      xfer += oprot->writeString((*_iter897));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9144,10 +9144,10 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_pargs::wr
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter893;
-    for (_iter893 = (*(this->part_vals)).begin(); _iter893 != (*(this->part_vals)).end(); ++_iter893)
+    std::vector<std::string> ::const_iterator _iter898;
+    for (_iter898 = (*(this->part_vals)).begin(); _iter898 != (*(this->part_vals)).end(); ++_iter898)
     {
-      xfer += oprot->writeString((*_iter893));
+      xfer += oprot->writeString((*_iter898));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9950,14 +9950,14 @@ uint32_t ThriftHiveMetastore_drop_partition_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size894;
-            ::apache::thrift::protocol::TType _etype897;
-            xfer += iprot->readListBegin(_etype897, _size894);
-            this->part_vals.resize(_size894);
-            uint32_t _i898;
-            for (_i898 = 0; _i898 < _size894; ++_i898)
+            uint32_t _size899;
+            ::apache::thrift::protocol::TType _etype902;
+            xfer += iprot->readListBegin(_etype902, _size899);
+            this->part_vals.resize(_size899);
+            uint32_t _i903;
+            for (_i903 = 0; _i903 < _size899; ++_i903)
             {
-              xfer += iprot->readString(this->part_vals[_i898]);
+              xfer += iprot->readString(this->part_vals[_i903]);
             }
             xfer += iprot->readListEnd();
           }
@@ -10002,10 +10002,10 @@ uint32_t ThriftHiveMetastore_drop_partition_args::write(::apache::thrift::protoc
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter899;
-    for (_iter899 = this->part_vals.begin(); _iter899 != this->part_vals.end(); ++_iter899)
+    std::vector<std::string> ::const_iterator _iter904;
+    for (_iter904 = this->part_vals.begin(); _iter904 != this->part_vals.end(); ++_iter904)
     {
-      xfer += oprot->writeString((*_iter899));
+      xfer += oprot->writeString((*_iter904));
     }
     xfer += oprot->writeListEnd();
   }
@@ -10041,10 +10041,10 @@ uint32_t ThriftHiveMetastore_drop_partition_pargs::write(::apache::thrift::proto
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter900;
-    for (_iter900 = (*(this->part_vals)).begin(); _iter900 != (*(this->part_vals)).end(); ++_iter900)
+    std::vector<std::string> ::const_iterator _iter905;
+    for (_iter905 = (*(this->part_vals)).begin(); _iter905 != (*(this->part_vals)).end(); ++_iter905)
     {
-      xfer += oprot->writeString((*_iter900));
+      xfer += oprot->writeString((*_iter905));
     }
     xfer += oprot->writeListEnd();
   }
@@ -10253,14 +10253,14 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_args::read(
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size901;
-            ::apache::thrift::protocol::TType _etype904;
-            xfer += iprot->readListBegin(_etype904, _size901);
-            this->part_vals.resize(_size901);
-            uint32_t _i905;
-            for (_i905 = 0; _i905 < _size901; ++_i905)
+            uint32_t _size906;
+            ::apache::thrift::protocol::TType _etype909;
+            xfer += iprot->readListBegin(_etype909, _size906);
+            this->part_vals.resize(_size906);
+            uint32_t _i910;
+            for (_i910 = 0; _i910 < _size906; ++_i910)
             {
-              xfer += iprot->readString(this->part_vals[_i905]);
+              xfer += iprot->readString(this->part_vals[_i910]);
             }
             xfer += iprot->readListEnd();
           }
@@ -10313,10 +10313,10 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_args::write
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter906;
-    for (_iter906 = this->part_vals.begin(); _iter906 != this->part_vals.end(); ++_iter906)
+    std::vector<std::string> ::const_iterator _iter911;
+    for (_iter911 = this->part_vals.begin(); _iter911 != this->part_vals.end(); ++_iter911)
     {
-      xfer += oprot->writeString((*_iter906));
+      xfer += oprot->writeString((*_iter911));
     }
     xfer += oprot->writeListEnd();
   }
@@ -10356,10 +10356,10 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_pargs::writ
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter907;
-    for (_iter907 = (*(this->part_vals)).begin(); _iter907 != (*(this->part_vals)).end(); ++_iter907)
+    std::vector<std::string> ::const_iterator _iter912;
+    for (_iter912 = (*(this->part_vals)).begin(); _iter912 != (*(this->part_vals)).end(); ++_iter912)
     {
-      xfer += oprot->writeString((*_iter907));
+      xfer += oprot->writeString((*_iter912));
     }
     xfer += oprot->writeListEnd();
   }
@@ -11365,14 +11365,14 @@ uint32_t ThriftHiveMetastore_get_partition_args::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size908;
-            ::apache::thrift::protocol::TType _etype911;
-            xfer += iprot->readListBegin(_etype911, _size908);
-            this->part_vals.resize(_size908);
-            uint32_t _i912;
-            for (_i912 = 0; _i912 < _size908; ++_i912)
+            uint32_t _size913;
+            ::apache::thrift::protocol::TType _etype916;
+            xfer += iprot->readListBegin(_etype916, _size913);
+            this->part_vals.resize(_size913);
+            uint32_t _i917;
+            for (_i917 = 0; _i917 < _size913; ++_i917)
             {
-              xfer += iprot->readString(this->part_vals[_i912]);
+              xfer += iprot->readString(this->part_vals[_i917]);
             }
             xfer += iprot->readListEnd();
           }
@@ -11409,10 +11409,10 @@ uint32_t ThriftHiveMetastore_get_partition_args::write(::apache::thrift::protoco
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter913;
-    for (_iter913 = this->part_vals.begin(); _iter913 != this->part_vals.end(); ++_iter913)
+    std::vector<std::string> ::const_iterator _iter918;
+    for (_iter918 = this->part_vals.begin(); _iter918 != this->part_vals.end(); ++_iter918)
     {
-      xfer += oprot->writeString((*_iter913));
+      xfer += oprot->writeString((*_iter918));
     }
     xfer += oprot->writeListEnd();
   }
@@ -11444,10 +11444,10 @@ uint32_t ThriftHiveMetastore_get_partition_pargs::write(::apache::thrift::protoc
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter914;
-    for (_iter914 = (*(this->part_vals)).begin(); _iter914 != (*(this->part_vals)).end(); ++_iter914)
+    std::vector<std::string> ::const_iterator _iter919;
+    for (_iter919 = (*(this->part_vals)).begin(); _iter919 != (*(this->part_vals)).end(); ++_iter919)
     {
-      xfer += oprot->writeString((*_iter914));
+      xfer += oprot->writeString((*_iter919));
     }
     xfer += oprot->writeListEnd();
   }
@@ -11636,17 +11636,17 @@ uint32_t ThriftHiveMetastore_exchange_partition_args::read(::apache::thrift::pro
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->partitionSpecs.clear();
-            uint32_t _size915;
-            ::apache::thrift::protocol::TType _ktype916;
-            ::apache::thrift::protocol::TType _vtype917;
-            xfer += iprot->readMapBegin(_ktype916, _vtype917, _size915);
-            uint32_t _i919;
-            for (_i919 = 0; _i919 < _size915; ++_i919)
+            uint32_t _size920;
+            ::apache::thrift::protocol::TType _ktype921;
+            ::apache::thrift::protocol::TType _vtype922;
+            xfer += iprot->readMapBegin(_ktype921, _vtype922, _size920);
+            uint32_t _i924;
+            for (_i924 = 0; _i924 < _size920; ++_i924)
             {
-              std::string _key920;
-              xfer += iprot->readString(_key920);
-              std::string& _val921 = this->partitionSpecs[_key920];
-              xfer += iprot->readString(_val921);
+              std::string _key925;
+              xfer += iprot->readString(_key925);
+              std::string& _val926 = this->partitionSpecs[_key925];
+              xfer += iprot->readString(_val926);
             }
             xfer += iprot->readMapEnd();
           }
@@ -11707,11 +11707,11 @@ uint32_t ThriftHiveMetastore_exchange_partition_args::write(::apache::thrift::pr
   xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partitionSpecs.size()));
-    std::map<std::string, std::string> ::const_iterator _iter922;
-    for (_iter922 = this->partitionSpecs.begin(); _iter922 != this->partitionSpecs.end(); ++_iter922)
+    std::map<std::string, std::string> ::const_iterator _iter927;
+    for (_iter927 = this->partitionSpecs.begin(); _iter927 != this->partitionSpecs.end(); ++_iter927)
     {
-      xfer += oprot->writeString(_iter922->first);
-      xfer += oprot->writeString(_iter922->second);
+      xfer += oprot->writeString(_iter927->first);
+      xfer += oprot->writeString(_iter927->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -11751,11 +11751,11 @@ uint32_t ThriftHiveMetastore_exchange_partition_pargs::write(::apache::thrift::p
   xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->partitionSpecs)).size()));
-    std::map<std::string, std::string> ::const_iterator _iter923;
-    for (_iter923 = (*(this->partitionSpecs)).begin(); _iter923 != (*(this->partitionSpecs)).end(); ++_iter923)
+    std::map<std::string, std::string> ::const_iterator _iter928;
+    for (_iter928 = (*(this->partitionSpecs)).begin(); _iter928 != (*(this->partitionSpecs)).end(); ++_iter928)
     {
-      xfer += oprot->writeString(_iter923->first);
-      xfer += oprot->writeString(_iter923->second);
+      xfer += oprot->writeString(_iter928->first);
+      xfer += oprot->writeString(_iter928->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -12000,17 +12000,17 @@ uint32_t ThriftHiveMetastore_exchange_partitions_args::read(::apache::thrift::pr
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->partitionSpecs.clear();
-            uint32_t _size924;
-            ::apache::thrift::protocol::TType _ktype925;
-            ::apache::thrift::protocol::TType _vtype926;
-            xfer += iprot->readMapBegin(_ktype925, _vtype926, _size924);
-            uint32_t _i928;
-            for (_i928 = 0; _i928 < _size924; ++_i928)
+            uint32_t _size929;
+            ::apache::thrift::protocol::TType _ktype930;
+            ::apache::thrift::protocol::TType _vtype931;
+            xfer += iprot->readMapBegin(_ktype930, _vtype931, _size929);
+            uint32_t _i933;
+            for (_i933 = 0; _i933 < _size929; ++_i933)
             {
-              std::string _key929;
-              xfer += iprot->readString(_key929);
-              std::string& _val930 = this->partitionSpecs[_key929];
-              xfer += iprot->readString(_val930);
+              std::string _key934;
+              xfer += iprot->readString(_key934);
+              std::string& _val935 = this->partitionSpecs[_key934];
+              xfer += iprot->readString(_val935);
             }
             xfer += iprot->readMapEnd();
           }
@@ -12071,11 +12071,11 @@ uint32_t ThriftHiveMetastore_exchange_partitions_args::write(::apache::thrift::p
   xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partitionSpecs.size()));
-    std::map<std::string, std::string> ::const_iterator _iter931;
-    for (_iter931 = this->partitionSpecs.begin(); _iter931 != this->partitionSpecs.end(); ++_iter931)
+    std::map<std::string, std::string> ::const_iterator _iter936;
+    for (_iter936 = this->partitionSpecs.begin(); _iter936 != this->partitionSpecs.end(); ++_iter936)
     {
-      xfer += oprot->writeString(_iter931->first);
-      xfer += oprot->writeString(_iter931->second);
+      xfer += oprot->writeString(_iter936->first);
+      xfer += oprot->writeString(_iter936->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -12115,11 +12115,11 @@ uint32_t ThriftHiveMetastore_exchange_partitions_pargs::write(::apache::thrift::
   xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->partitionSpecs)).size()));
-    std::map<std::string, std::string> ::const_iterator _iter932;
-    for (_iter932 = (*(this->partitionSpecs)).begin(); _iter932 != (*(this->partitionSpecs)).end(); ++_iter932)
+    std::map<std::string, std::string> ::const_iterator _iter937;
+    for (_iter937 = (*(this->partitionSpecs)).begin(); _iter937 != (*(this->partitionSpecs)).end(); ++_iter937)
     {
-      xfer += oprot->writeString(_iter932->first);
-      xfer += oprot->writeString(_iter932->second);
+      xfer += oprot->writeString(_iter937->first);
+      xfer += oprot->writeString(_iter937->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -12176,14 +12176,14 @@ uint32_t ThriftHiveMetastore_exchange_partitions_result::read(::apache::thrift::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size933;
-            ::apache::thrift::protocol::TType _etype936;
-            xfer += iprot->readListBegin(_etype936, _size933);
-            this->success.resize(_size933);
-            uint32_t _i937;
-            for (_i937 = 0; _i937 < _size933; ++_i937)
+            uint32_t _size938;
+            ::apache::thrift::protocol::TType _etype941;
+            xfer += iprot->readListBegin(_etype941, _size938);
+            this->success.resize(_size938);
+            uint32_t _i942;
+            for (_i942 = 0; _i942 < _size938; ++_i942)
             {
-              xfer += this->success[_i937].read(iprot);
+              xfer += this->success[_i942].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -12246,10 +12246,10 @@ uint32_t ThriftHiveMetastore_exchange_partitions_result::write(::apache::thrift:
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Partition> ::const_iterator _iter938;
-      for (_iter938 = this->success.begin(); _iter938 != this->success.end(); ++_iter938)
+      std::vector<Partition> ::const_iterator _iter943;
+      for (_iter943 = this->success.begin(); _iter943 != this->success.end(); ++_iter943)
       {
-        xfer += (*_iter938).write(oprot);
+        xfer += (*_iter943).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -12306,14 +12306,14 @@ uint32_t ThriftHiveMetastore_exchange_partitions_presult::read(::apache::thrift:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size939;
-            ::apache::thrift::protocol::TType _etype942;
-            xfer += iprot->readListBegin(_etype942, _size939);
-            (*(this->success)).resize(_size939);
-            uint32_t _i943;
-            for (_i943 = 0; _i943 < _size939; ++_i943)
+            uint32_t _size944;
+            ::apache::thrift::protocol::TType _etype947;
+            xfer += iprot->readListBegin(_etype947, _size944);
+            (*(this->success)).resize(_size944);
+            uint32_t _i948;
+            for (_i948 = 0; _i948 < _size944; ++_i948)
             {
-              xfer += (*(this->success))[_i943].read(iprot);
+              xfer += (*(this->success))[_i948].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -12412,14 +12412,14 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::read(::apache::thrift
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size944;
-            ::apache::thrift::protocol::TType _etype947;
-            xfer += iprot->readListBegin(_etype947, _size944);
-            this->part_vals.resize(_size944);
-            uint32_t _i948;
-            for (_i948 = 0; _i948 < _size944; ++_i948)
+            uint32_t _size949;
+            ::apache::thrift::protocol::TType _etype952;
+            xfer += iprot->readListBegin(_etype952, _size949);
+            this->part_vals.resize(_size949);
+            uint32_t _i953;
+            for (_i953 = 0; _i953 < _size949; ++_i953)
             {
-              xfer += iprot->readString(this->part_vals[_i948]);
+              xfer += iprot->readString(this->part_vals[_i953]);
             }
             xfer += iprot->readListEnd();
           }
@@ -12440,14 +12440,14 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::read(::apache::thrift
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->group_names.clear();
-            uint32_t _size949;
-            ::apache::thrift::protocol::TType _etype952;
-            xfer += iprot->readListBegin(_etype952, _size949);
-            this->group_names.resize(_size949);
-            uint32_t _i953;
-            for (_i953 = 0; _i953 < _size949; ++_i953)
+            uint32_t _size954;
+            ::apache::thrift::protocol::TType _etype957;
+            xfer += iprot->readListBegin(_etype957, _size954);
+            this->group_names.resize(_size954);
+            uint32_t _i958;
+            for (_i958 = 0; _i958 < _size954; ++_i958)
             {
-              xfer += iprot->readString(this->group_names[_i953]);
+              xfer += iprot->readString(this->group_names[_i958]);
             }
             xfer += iprot->readListEnd();
           }
@@ -12484,10 +12484,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::write(::apache::thrif
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter954;
-    for (_iter954 = this->part_vals.begin(); _iter954 != this->part_vals.end(); ++_iter954)
+    std::vector<std::string> ::const_iterator _iter959;
+    for (_iter959 = this->part_vals.begin(); _iter959 != this->part_vals.end(); ++_iter959)
     {
-      xfer += oprot->writeString((*_iter954));
+      xfer += oprot->writeString((*_iter959));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12500,10 +12500,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::write(::apache::thrif
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->group_names.size()));
-    std::vector<std::string> ::const_iterator _iter955;
-    for (_iter955 = this->group_names.begin(); _iter955 != this->group_names.end(); ++_iter955)
+    std::vector<std::string> ::const_iterator _iter960;
+    for (_iter960 = this->group_names.begin(); _iter960 != this->group_names.end(); ++_iter960)
     {
-      xfer += oprot->writeString((*_iter955));
+      xfer += oprot->writeString((*_iter960));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12535,10 +12535,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_pargs::write(::apache::thri
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter956;
-    for (_iter956 = (*(this->part_vals)).begin(); _iter956 != (*(this->part_vals)).end(); ++_iter956)
+    std::vector<std::string> ::const_iterator _iter961;
+    for (_iter961 = (*(this->part_vals)).begin(); _iter961 != (*(this->part_vals)).end(); ++_iter961)
     {
-      xfer += oprot->writeString((*_iter956));
+      xfer += oprot->writeString((*_iter961));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12551,10 +12551,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_pargs::write(::apache::thri
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->group_names)).size()));
-    std::vector<std::string> ::const_iterator _iter957;
-    for (_iter957 = (*(this->group_names)).begin(); _iter957 != (*(this->group_names)).end(); ++_iter957)
+    std::vector<std::string> ::const_iterator _iter962;
+    for (_iter962 = (*(this->group_names)).begin(); _iter962 != (*(this->group_names)).end(); ++_iter962)
     {
-      xfer += oprot->writeString((*_iter957));
+      xfer += oprot->writeString((*_iter962));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13113,14 +13113,14 @@ uint32_t ThriftHiveMetastore_get_partitions_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size958;
-            ::apache::thrift::protocol::TType _etype961;
-            xfer += iprot->readListBegin(_etype961, _size958);
-            this->success.resize(_size958);
-            uint32_t _i962;
-            for (_i962 = 0; _i962 < _size958; ++_i962)
+            uint32_t _size963;
+            ::apache::thrift::protocol::TType _etype966;
+            xfer += iprot->readListBegin(_etype966, _size963);
+            this->success.resize(_size963);
+            uint32_t _i967;
+            for (_i967 = 0; _i967 < _size963; ++_i967)
             {
-              xfer += this->success[_i962].read(iprot);
+              xfer += this->success[_i967].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13167,10 +13167,10 @@ uint32_t ThriftHiveMetastore_get_partitions_result::write(::apache::thrift::prot
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Partition> ::const_iterator _iter963;
-      for (_iter963 = this->success.begin(); _iter963 != this->success.end(); ++_iter963)
+      std::vector<Partition> ::const_iterator _iter968;
+      for (_iter968 = this->success.begin(); _iter968 != this->success.end(); ++_iter968)
       {
-        xfer += (*_iter963).write(oprot);
+        xfer += (*_iter968).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -13219,14 +13219,14 @@ uint32_t ThriftHiveMetastore_get_partitions_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size964;
-            ::apache::thrift::protocol::TType _etype967;
-            xfer += iprot->readListBegin(_etype967, _size964);
-            (*(this->success)).resize(_size964);
-            uint32_t _i968;
-            for (_i968 = 0; _i968 < _size964; ++_i968)
+            uint32_t _size969;
+            ::apache::thrift::protocol::TType _etype972;
+            xfer += iprot->readListBegin(_etype972, _size969);
+            (*(this->success)).resize(_size969);
+            uint32_t _i973;
+            for (_i973 = 0; _i973 < _size969; ++_i973)
             {
-              xfer += (*(this->success))[_i968].read(iprot);
+              xfer += (*(this->success))[_i973].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13325,14 +13325,14 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_args::read(::apache::thrif
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->group_names.clear();
-            uint32_t _size969;
-            ::apache::thrift::protocol::TType _etype972;
-            xfer += iprot->readListBegin(_etype972, _size969);
-            this->group_names.resize(_size969);
-            uint32_t _i973;
-            for (_i973 = 0; _i973 < _size969; ++_i973)
+            uint32_t _size974;
+            ::apache::thrift::protocol::TType _etype977;
+            xfer += iprot->readListBegin(_etype977, _size974);
+            this->group_names.resize(_size974);
+            uint32_t _i978;
+            for (_i978 = 0; _i978 < _size974; ++_i978)
             {
-              xfer += iprot->readString(this->group_names[_i973]);
+              xfer += iprot->readString(this->group_names[_i978]);
             }
             xfer += iprot->readListEnd();
           }
@@ -13377,10 +13377,10 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_args::write(::apache::thri
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->group_names.size()));
-    std::vector<std::string> ::const_iterator _iter974;
-    for (_iter974 = this->group_names.begin(); _iter974 != this->group_names.end(); ++_iter974)
+    std::vector<std::string> ::const_iterator _iter979;
+    for (_iter979 = this->group_names.begin(); _iter979 != this->group_names.end(); ++_iter979)
     {
-      xfer += oprot->writeString((*_iter974));
+      xfer += oprot->writeString((*_iter979));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13420,10 +13420,10 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_pargs::write(::apache::thr
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->group_names)).size()));
-    std::vector<std::string> ::const_iterator _iter975;
-    for (_iter975 = (*(this->group_names)).begin(); _iter975 != (*(this->group_names)).end(); ++_iter975)
+    std::vector<std::string> ::const_iterator _iter980;
+    for (_iter980 = (*(this->group_names)).begin(); _iter980 != (*(this->group_names)).end(); ++_iter980)
     {
-      xfer += oprot->writeString((*_iter975));
+      xfer += oprot->writeString((*_iter980));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13464,14 +13464,14 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_result::read(::apache::thr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size976;
-            ::apache::thrift::protocol::TType _etype979;
-            xfer += iprot->readListBegin(_etype979, _size976);
-            this->success.resize(_size976);
-            uint32_t _i980;
-            for (_i980 = 0; _i980 < _size976; ++_i980)
+            uint32_t _size981;
+            ::apache::thrift::protocol::TType _etype984;
+            xfer += iprot->readListBegin(_etype984, _size981);
+            this->success.resize(_size981);
+            uint32_t _i985;
+            for (_i985 = 0; _i985 < _size981; ++_i985)
             {
-              xfer += this->success[_i980].read(iprot);
+              xfer += this->success[_i985].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13518,10 +13518,10 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_result::write(::apache::th
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Partition> ::const_iterator _iter981;
-      for (_iter981 = this->success.begin(); _iter981 != this->success.end(); ++_iter981)
+      std::vector<Partition> ::const_iterator _iter986;
+      for (_iter986 = this->success.begin(); _iter986 != this->success.end(); ++_iter986)
       {
-        xfer += (*_iter981).write(oprot);
+        xfer += (*_iter986).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -13570,14 +13570,14 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_presult::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size982;
-            ::apache::thrift::protocol::TType _etype985;
-            xfer += iprot->readListBegin(_etype985, _size982);
-            (*(this->success)).resize(_size982);
-            uint32_t _i986;
-            for (_i986 = 0; _i986 < _size982; ++_i986)
+            uint32_t _size987;
+            ::apache::thrift::protocol::TType _etype990;
+            xfer += iprot->readListBegin(_etype990, _size987);
+            (*(this->success)).resize(_size987);
+            uint32_t _i991;
+            for (_i991 = 0; _i991 < _size987; ++_i991)
             {
-              xfer += (*(this->success))[_i986].read(iprot);
+              xfer += (*(this->success))[_i991].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13755,14 +13755,14 @@ uint32_t ThriftHiveMetastore_get_partitions_pspec_result::read(::apache::thrift:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size987;
-            ::apache::thrift::protocol::TType _etype990;
-            xfer += iprot->readListBegin(_etype990, _size987);
-            this->success.resize(_size987);
-            uint32_t _i991;
-            for (_i991 = 0; _i991 < _size987; ++_i991)
+            uint32_t _size992;
+            ::apache::thrift::protocol::TType _etype995;
+            xfer += iprot->readListBegin(_etype995, _size992);
+            this->success.resize(_size992);
+            uint32_t _i996;
+            for (_i996 = 0; _i996 < _size992; ++_i996)
             {
-              xfer += this->success[_i991].read(iprot);
+              xfer += this->success[_i996].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13809,10 +13809,10 @@ uint32_t ThriftHiveMetastore_get_partitions_pspec_result::write(::apache::thrift
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<PartitionSpec> ::const_iterator _iter992;
-      for (_iter992 = this->success.begin(); _iter992 != this->success.end(); ++_iter992)
+      std::vector<PartitionSpec> ::const_iterator _iter997;
+      for (_iter997 = this->success.begin(); _iter997 != this->success.end(); ++_iter997)
       {
-        xfer += (*_iter992).write(oprot);
+        xfer += (*_iter997).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -13861,14 +13861,14 @@ uint32_t ThriftHiveMetastore_get_partitions_pspec_presult::read(::apache::thrift
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size993;
-            ::apache::thrift::protocol::TType _etype996;
-            xfer += iprot->readListBegin(_etype996, _size993);
-            (*(this->success)).resize(_size993);
-            uint32_t _i997;
-            for (_i997 = 0; _i997 < _size993; ++_i997)
+            uint32_t _size998;
+            ::apache::thrift::protocol::TType _etype1001;
+            xfer += iprot->readListBegin(_etype1001, _size998);
+            (*(this->success)).resize(_size998);
+            uint32_t _i1002;
+            for (_i1002 = 0; _i1002 < _size998; ++_i1002)
             {
-              xfer += (*(this->success))[_i997].read(iprot);
+              xfer += (*(this->success))[_i1002].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -14046,14 +14046,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_result::read(::apache::thrift::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size998;
-            ::apache::thrift::protocol::TType _etype1001;
-            xfer += iprot->readListBegin(_etype1001, _size998);
-            this->success.resize(_size998);
-            uint32_t _i1002;
-            for (_i1002 = 0; _i1002 < _size998; ++_i1002)
+            uint32_t _size1003;
+            ::apache::thrift::protocol::TType _etype1006;
+            xfer += iprot->readListBegin(_etype1006, _size1003);
+            this->success.resize(_size1003);
+            uint32_t _i1007;
+            for (_i1007 = 0; _i1007 < _size1003; ++_i1007)
             {
-              xfer += iprot->readString(this->success[_i1002]);
+              xfer += iprot->readString(this->success[_i1007]);
             }
             xfer += iprot->readListEnd();
           }
@@ -14092,10 +14092,10 @@ uint32_t ThriftHiveMetastore_get_partition_names_result::write(::apache::thrift:
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter1003;
-      for (_iter1003 = this->success.begin(); _iter1003 != this->success.end(); ++_iter1003)
+      std::vector<std::string> ::const_iterator _iter1008;
+      for (_iter1008 = this->success.begin(); _iter1008 != this->success.end(); ++_iter1008)
       {
-        xfer += oprot->writeString((*_iter1003));
+        xfer += oprot->writeString((*_iter1008));
       }
       xfer += oprot->writeListEnd();
     }
@@ -14140,14 +14140,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_presult::read(::apache::thrift:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1004;
-            ::apache::thrift::protocol::TType _etype1007;
-            xfer += iprot->readListBegin(_etype1007, _size1004);
-            (*(this->success)).resize(_size1004);
-            uint32_t _i1008;
-            for (_i1008 = 0; _i1008 < _size1004; ++_i1008)
+            uint32_t _size1009;
+            ::apache::thrift::protocol::TType _etype1012;
+            xfer += iprot->readListBegin(_etype1012, _size1009);
+            (*(this->success)).resize(_size1009);
+            uint32_t _i1013;
+            for (_i1013 = 0; _i1013 < _size1009; ++_i1013)
             {
-              xfer += iprot->readString((*(this->success))[_i1008]);
+              xfer += iprot->readString((*(this->success))[_i1013]);
             }
             xfer += iprot->readListEnd();
           }
@@ -14222,14 +14222,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_args::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1009;
-            ::apache::thrift::protocol::TType _etype1012;
-            xfer += iprot->readListBegin(_etype1012, _size1009);
-            this->part_vals.resize(_size1009);
-            uint32_t _i1013;
-            for (_i1013 = 0; _i1013 < _size1009; ++_i1013)
+            uint32_t _size1014;
+            ::apache::thrift::protocol::TType _etype1017;
+            xfer += iprot->readListBegin(_etype1017, _size1014);
+            this->part_vals.resize(_size1014);
+            uint32_t _i1018;
+            for (_i1018 = 0; _i1018 < _size1014; ++_i1018)
             {
-              xfer += iprot->readString(this->part_vals[_i1013]);
+              xfer += iprot->readString(this->part_vals[_i1018]);
             }
             xfer += iprot->readListEnd();
           }
@@ -14274,10 +14274,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_args::write(::apache::thrift::pro
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter1014;
-    for (_iter1014 = this->part_vals.begin(); _iter1014 != this->part_vals.end(); ++_iter1014)
+    std::vector<std::string> ::const_iterator _iter1019;
+    for (_iter1019 = this->part_vals.begin(); _iter1019 != this->part_vals.end(); ++_iter1019)
     {
-      xfer += oprot->writeString((*_iter1014));
+      xfer += oprot->writeString((*_iter1019));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14313,10 +14313,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_pargs::write(::apache::thrift::pr
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter1015;
-    for (_iter1015 = (*(this->part_vals)).begin(); _iter1015 != (*(this->part_vals)).end(); ++_iter1015)
+    std::vector<std::string> ::const_iterator _iter1020;
+    for (_iter1020 = (*(this->part_vals)).begin(); _iter1020 != (*(this->part_vals)).end(); ++_iter1020)
     {
-      xfer += oprot->writeString((*_iter1015));
+      xfer += oprot->writeString((*_iter1020));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14361,14 +14361,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_result::read(::apache::thrift::pr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1016;
-            ::apache::thrift::protocol::TType _etype1019;
-            xfer += iprot->readListBegin(_etype1019, _size1016);
-            this->success.resize(_size1016);
-            uint32_t _i1020;
-            for (_i1020 = 0; _i1020 < _size1016; ++_i1020)
+            uint32_t _size1021;
+            ::apache::thrift::protocol::TType _etype1024;
+            xfer += iprot->readListBegin(_etype1024, _size1021);
+            this->success.resize(_size1021);
+            uint32_t _i1025;
+            for (_i1025 = 0; _i1025 < _size1021; ++_i1025)
             {
-              xfer += this->success[_i1020].read(iprot);
+              xfer += this->success[_i1025].read(iprot);
             }
             xfer += iprot->readListEnd();
       

<TRUNCATED>

[3/7] hive git commit: HIVE-12075 : add analyze command to explictly cache file metadata in HBase metastore (Sergey Shelukhin, reviewed by Alan Gates)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/7df62023/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
index 4690093..47b7afa 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
@@ -304,6 +304,8 @@ public class ThriftHiveMetastore {
 
     public ClearFileMetadataResult clear_file_metadata(ClearFileMetadataRequest req) throws org.apache.thrift.TException;
 
+    public CacheFileMetadataResult cache_file_metadata(CacheFileMetadataRequest req) throws org.apache.thrift.TException;
+
   }
 
   public interface AsyncIface extends com.facebook.fb303.FacebookService .AsyncIface {
@@ -570,6 +572,8 @@ public class ThriftHiveMetastore {
 
     public void clear_file_metadata(ClearFileMetadataRequest req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
+    public void cache_file_metadata(CacheFileMetadataRequest req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
   }
 
   public static class Client extends com.facebook.fb303.FacebookService.Client implements Iface {
@@ -4425,6 +4429,29 @@ public class ThriftHiveMetastore {
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "clear_file_metadata failed: unknown result");
     }
 
+    public CacheFileMetadataResult cache_file_metadata(CacheFileMetadataRequest req) throws org.apache.thrift.TException
+    {
+      send_cache_file_metadata(req);
+      return recv_cache_file_metadata();
+    }
+
+    public void send_cache_file_metadata(CacheFileMetadataRequest req) throws org.apache.thrift.TException
+    {
+      cache_file_metadata_args args = new cache_file_metadata_args();
+      args.setReq(req);
+      sendBase("cache_file_metadata", args);
+    }
+
+    public CacheFileMetadataResult recv_cache_file_metadata() throws org.apache.thrift.TException
+    {
+      cache_file_metadata_result result = new cache_file_metadata_result();
+      receiveBase(result, "cache_file_metadata");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "cache_file_metadata failed: unknown result");
+    }
+
   }
   public static class AsyncClient extends com.facebook.fb303.FacebookService.AsyncClient implements AsyncIface {
     public static class Factory implements org.apache.thrift.async.TAsyncClientFactory<AsyncClient> {
@@ -9097,6 +9124,38 @@ public class ThriftHiveMetastore {
       }
     }
 
+    public void cache_file_metadata(CacheFileMetadataRequest req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      cache_file_metadata_call method_call = new cache_file_metadata_call(req, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class cache_file_metadata_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private CacheFileMetadataRequest req;
+      public cache_file_metadata_call(CacheFileMetadataRequest req, 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.req = req;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("cache_file_metadata", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        cache_file_metadata_args args = new cache_file_metadata_args();
+        args.setReq(req);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public CacheFileMetadataResult getResult() throws 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_cache_file_metadata();
+      }
+    }
+
   }
 
   public static class Processor<I extends Iface> extends com.facebook.fb303.FacebookService.Processor<I> implements org.apache.thrift.TProcessor {
@@ -9241,6 +9300,7 @@ public class ThriftHiveMetastore {
       processMap.put("get_file_metadata", new get_file_metadata());
       processMap.put("put_file_metadata", new put_file_metadata());
       processMap.put("clear_file_metadata", new clear_file_metadata());
+      processMap.put("cache_file_metadata", new cache_file_metadata());
       return processMap;
     }
 
@@ -12619,6 +12679,26 @@ public class ThriftHiveMetastore {
       }
     }
 
+    public static class cache_file_metadata<I extends Iface> extends org.apache.thrift.ProcessFunction<I, cache_file_metadata_args> {
+      public cache_file_metadata() {
+        super("cache_file_metadata");
+      }
+
+      public cache_file_metadata_args getEmptyArgsInstance() {
+        return new cache_file_metadata_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public cache_file_metadata_result getResult(I iface, cache_file_metadata_args args) throws org.apache.thrift.TException {
+        cache_file_metadata_result result = new cache_file_metadata_result();
+        result.success = iface.cache_file_metadata(args.req);
+        return result;
+      }
+    }
+
   }
 
   public static class AsyncProcessor<I extends AsyncIface> extends com.facebook.fb303.FacebookService.AsyncProcessor<I> {
@@ -12763,6 +12843,7 @@ public class ThriftHiveMetastore {
       processMap.put("get_file_metadata", new get_file_metadata());
       processMap.put("put_file_metadata", new put_file_metadata());
       processMap.put("clear_file_metadata", new clear_file_metadata());
+      processMap.put("cache_file_metadata", new cache_file_metadata());
       return processMap;
     }
 
@@ -20808,6 +20889,57 @@ public class ThriftHiveMetastore {
       }
     }
 
+    public static class cache_file_metadata<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, cache_file_metadata_args, CacheFileMetadataResult> {
+      public cache_file_metadata() {
+        super("cache_file_metadata");
+      }
+
+      public cache_file_metadata_args getEmptyArgsInstance() {
+        return new cache_file_metadata_args();
+      }
+
+      public AsyncMethodCallback<CacheFileMetadataResult> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<CacheFileMetadataResult>() {
+          public void onComplete(CacheFileMetadataResult o) {
+            cache_file_metadata_result result = new cache_file_metadata_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;
+            cache_file_metadata_result result = new cache_file_metadata_result();
+            {
+              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, cache_file_metadata_args args, org.apache.thrift.async.AsyncMethodCallback<CacheFileMetadataResult> resultHandler) throws TException {
+        iface.cache_file_metadata(args.req,resultHandler);
+      }
+    }
+
   }
 
   public static class getMetaConf_args implements org.apache.thrift.TBase<getMetaConf_args, getMetaConf_args._Fields>, java.io.Serializable, Cloneable, Comparable<getMetaConf_args>   {
@@ -154642,33 +154774,695 @@ public class ThriftHiveMetastore {
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
       metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(flushCache_result.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(flushCache_result.class, metaDataMap);
+    }
+
+    public flushCache_result() {
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public flushCache_result(flushCache_result other) {
+    }
+
+    public flushCache_result deepCopy() {
+      return new flushCache_result(this);
+    }
+
+    @Override
+    public void clear() {
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      }
+      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) {
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof flushCache_result)
+        return this.equals((flushCache_result)that);
+      return false;
+    }
+
+    public boolean equals(flushCache_result that) {
+      if (that == null)
+        return false;
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      List<Object> list = new ArrayList<Object>();
+
+      return list.hashCode();
+    }
+
+    @Override
+    public int compareTo(flushCache_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      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("flushCache_result(");
+      boolean first = true;
+
+      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 flushCache_resultStandardSchemeFactory implements SchemeFactory {
+      public flushCache_resultStandardScheme getScheme() {
+        return new flushCache_resultStandardScheme();
+      }
+    }
+
+    private static class flushCache_resultStandardScheme extends StandardScheme<flushCache_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, flushCache_result 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) {
+            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, flushCache_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class flushCache_resultTupleSchemeFactory implements SchemeFactory {
+      public flushCache_resultTupleScheme getScheme() {
+        return new flushCache_resultTupleScheme();
+      }
+    }
+
+    private static class flushCache_resultTupleScheme extends TupleScheme<flushCache_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, flushCache_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, flushCache_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+      }
+    }
+
+  }
+
+  public static class get_file_metadata_by_expr_args implements org.apache.thrift.TBase<get_file_metadata_by_expr_args, get_file_metadata_by_expr_args._Fields>, java.io.Serializable, Cloneable, Comparable<get_file_metadata_by_expr_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_file_metadata_by_expr_args");
+
+    private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new get_file_metadata_by_expr_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new get_file_metadata_by_expr_argsTupleSchemeFactory());
+    }
+
+    private GetFileMetadataByExprRequest req; // 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 {
+      REQ((short)1, "req");
+
+      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: // REQ
+            return REQ;
+          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.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT,
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, GetFileMetadataByExprRequest.class)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_file_metadata_by_expr_args.class, metaDataMap);
+    }
+
+    public get_file_metadata_by_expr_args() {
+    }
+
+    public get_file_metadata_by_expr_args(
+      GetFileMetadataByExprRequest req)
+    {
+      this();
+      this.req = req;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public get_file_metadata_by_expr_args(get_file_metadata_by_expr_args other) {
+      if (other.isSetReq()) {
+        this.req = new GetFileMetadataByExprRequest(other.req);
+      }
+    }
+
+    public get_file_metadata_by_expr_args deepCopy() {
+      return new get_file_metadata_by_expr_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.req = null;
+    }
+
+    public GetFileMetadataByExprRequest getReq() {
+      return this.req;
+    }
+
+    public void setReq(GetFileMetadataByExprRequest req) {
+      this.req = req;
+    }
+
+    public void unsetReq() {
+      this.req = null;
+    }
+
+    /** Returns true if field req is set (has been assigned a value) and false otherwise */
+    public boolean isSetReq() {
+      return this.req != null;
+    }
+
+    public void setReqIsSet(boolean value) {
+      if (!value) {
+        this.req = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case REQ:
+        if (value == null) {
+          unsetReq();
+        } else {
+          setReq((GetFileMetadataByExprRequest)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case REQ:
+        return getReq();
+
+      }
+      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 REQ:
+        return isSetReq();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof get_file_metadata_by_expr_args)
+        return this.equals((get_file_metadata_by_expr_args)that);
+      return false;
+    }
+
+    public boolean equals(get_file_metadata_by_expr_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_req = true && this.isSetReq();
+      boolean that_present_req = true && that.isSetReq();
+      if (this_present_req || that_present_req) {
+        if (!(this_present_req && that_present_req))
+          return false;
+        if (!this.req.equals(that.req))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      List<Object> list = new ArrayList<Object>();
+
+      boolean present_req = true && (isSetReq());
+      list.add(present_req);
+      if (present_req)
+        list.add(req);
+
+      return list.hashCode();
+    }
+
+    @Override
+    public int compareTo(get_file_metadata_by_expr_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = Boolean.valueOf(isSetReq()).compareTo(other.isSetReq());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetReq()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, other.req);
+        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("get_file_metadata_by_expr_args(");
+      boolean first = true;
+
+      sb.append("req:");
+      if (this.req == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.req);
+      }
+      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 (req != null) {
+        req.validate();
+      }
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class get_file_metadata_by_expr_argsStandardSchemeFactory implements SchemeFactory {
+      public get_file_metadata_by_expr_argsStandardScheme getScheme() {
+        return new get_file_metadata_by_expr_argsStandardScheme();
+      }
+    }
+
+    private static class get_file_metadata_by_expr_argsStandardScheme extends StandardScheme<get_file_metadata_by_expr_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, get_file_metadata_by_expr_args 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: // REQ
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.req = new GetFileMetadataByExprRequest();
+                struct.req.read(iprot);
+                struct.setReqIsSet(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, get_file_metadata_by_expr_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.req != null) {
+          oprot.writeFieldBegin(REQ_FIELD_DESC);
+          struct.req.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class get_file_metadata_by_expr_argsTupleSchemeFactory implements SchemeFactory {
+      public get_file_metadata_by_expr_argsTupleScheme getScheme() {
+        return new get_file_metadata_by_expr_argsTupleScheme();
+      }
+    }
+
+    private static class get_file_metadata_by_expr_argsTupleScheme extends TupleScheme<get_file_metadata_by_expr_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, get_file_metadata_by_expr_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.isSetReq()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetReq()) {
+          struct.req.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, get_file_metadata_by_expr_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.req = new GetFileMetadataByExprRequest();
+          struct.req.read(iprot);
+          struct.setReqIsSet(true);
+        }
+      }
+    }
+
+  }
+
+  public static class get_file_metadata_by_expr_result implements org.apache.thrift.TBase<get_file_metadata_by_expr_result, get_file_metadata_by_expr_result._Fields>, java.io.Serializable, Cloneable, Comparable<get_file_metadata_by_expr_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_file_metadata_by_expr_result");
+
+    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new get_file_metadata_by_expr_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new get_file_metadata_by_expr_resultTupleSchemeFactory());
+    }
+
+    private GetFileMetadataByExprResult success; // 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 {
+      SUCCESS((short)0, "success");
+
+      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 0: // SUCCESS
+            return SUCCESS;
+          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.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT,
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, GetFileMetadataByExprResult.class)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_file_metadata_by_expr_result.class, metaDataMap);
     }
 
-    public flushCache_result() {
+    public get_file_metadata_by_expr_result() {
+    }
+
+    public get_file_metadata_by_expr_result(
+      GetFileMetadataByExprResult success)
+    {
+      this();
+      this.success = success;
     }
 
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public flushCache_result(flushCache_result other) {
+    public get_file_metadata_by_expr_result(get_file_metadata_by_expr_result other) {
+      if (other.isSetSuccess()) {
+        this.success = new GetFileMetadataByExprResult(other.success);
+      }
     }
 
-    public flushCache_result deepCopy() {
-      return new flushCache_result(this);
+    public get_file_metadata_by_expr_result deepCopy() {
+      return new get_file_metadata_by_expr_result(this);
     }
 
     @Override
     public void clear() {
+      this.success = null;
+    }
+
+    public GetFileMetadataByExprResult getSuccess() {
+      return this.success;
+    }
+
+    public void setSuccess(GetFileMetadataByExprResult success) {
+      this.success = success;
+    }
+
+    public void unsetSuccess() {
+      this.success = null;
+    }
+
+    /** Returns true if field success is set (has been assigned a value) and false otherwise */
+    public boolean isSetSuccess() {
+      return this.success != null;
+    }
+
+    public void setSuccessIsSet(boolean value) {
+      if (!value) {
+        this.success = null;
+      }
     }
 
     public void setFieldValue(_Fields field, Object value) {
       switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unsetSuccess();
+        } else {
+          setSuccess((GetFileMetadataByExprResult)value);
+        }
+        break;
+
       }
     }
 
     public Object getFieldValue(_Fields field) {
       switch (field) {
+      case SUCCESS:
+        return getSuccess();
+
       }
       throw new IllegalStateException();
     }
@@ -154680,6 +155474,8 @@ public class ThriftHiveMetastore {
       }
 
       switch (field) {
+      case SUCCESS:
+        return isSetSuccess();
       }
       throw new IllegalStateException();
     }
@@ -154688,15 +155484,24 @@ public class ThriftHiveMetastore {
     public boolean equals(Object that) {
       if (that == null)
         return false;
-      if (that instanceof flushCache_result)
-        return this.equals((flushCache_result)that);
+      if (that instanceof get_file_metadata_by_expr_result)
+        return this.equals((get_file_metadata_by_expr_result)that);
       return false;
     }
 
-    public boolean equals(flushCache_result that) {
+    public boolean equals(get_file_metadata_by_expr_result that) {
       if (that == null)
         return false;
 
+      boolean this_present_success = true && this.isSetSuccess();
+      boolean that_present_success = true && that.isSetSuccess();
+      if (this_present_success || that_present_success) {
+        if (!(this_present_success && that_present_success))
+          return false;
+        if (!this.success.equals(that.success))
+          return false;
+      }
+
       return true;
     }
 
@@ -154704,17 +155509,32 @@ public class ThriftHiveMetastore {
     public int hashCode() {
       List<Object> list = new ArrayList<Object>();
 
+      boolean present_success = true && (isSetSuccess());
+      list.add(present_success);
+      if (present_success)
+        list.add(success);
+
       return list.hashCode();
     }
 
     @Override
-    public int compareTo(flushCache_result other) {
+    public int compareTo(get_file_metadata_by_expr_result other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
 
       int lastComparison = 0;
 
+      lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetSuccess()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
       return 0;
     }
 
@@ -154732,9 +155552,16 @@ public class ThriftHiveMetastore {
 
     @Override
     public String toString() {
-      StringBuilder sb = new StringBuilder("flushCache_result(");
+      StringBuilder sb = new StringBuilder("get_file_metadata_by_expr_result(");
       boolean first = true;
 
+      sb.append("success:");
+      if (this.success == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.success);
+      }
+      first = false;
       sb.append(")");
       return sb.toString();
     }
@@ -154742,6 +155569,9 @@ public class ThriftHiveMetastore {
     public void validate() throws org.apache.thrift.TException {
       // check for required fields
       // check for sub-struct validity
+      if (success != null) {
+        success.validate();
+      }
     }
 
     private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
@@ -154760,15 +155590,15 @@ public class ThriftHiveMetastore {
       }
     }
 
-    private static class flushCache_resultStandardSchemeFactory implements SchemeFactory {
-      public flushCache_resultStandardScheme getScheme() {
-        return new flushCache_resultStandardScheme();
+    private static class get_file_metadata_by_expr_resultStandardSchemeFactory implements SchemeFactory {
+      public get_file_metadata_by_expr_resultStandardScheme getScheme() {
+        return new get_file_metadata_by_expr_resultStandardScheme();
       }
     }
 
-    private static class flushCache_resultStandardScheme extends StandardScheme<flushCache_result> {
+    private static class get_file_metadata_by_expr_resultStandardScheme extends StandardScheme<get_file_metadata_by_expr_result> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, flushCache_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, get_file_metadata_by_expr_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -154778,6 +155608,15 @@ public class ThriftHiveMetastore {
             break;
           }
           switch (schemeField.id) {
+            case 0: // SUCCESS
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.success = new GetFileMetadataByExprResult();
+                struct.success.read(iprot);
+                struct.setSuccessIsSet(true);
+              } else {
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
             default:
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
           }
@@ -154787,49 +155626,68 @@ public class ThriftHiveMetastore {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, flushCache_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, get_file_metadata_by_expr_result struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.success != null) {
+          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+          struct.success.write(oprot);
+          oprot.writeFieldEnd();
+        }
         oprot.writeFieldStop();
         oprot.writeStructEnd();
       }
 
     }
 
-    private static class flushCache_resultTupleSchemeFactory implements SchemeFactory {
-      public flushCache_resultTupleScheme getScheme() {
-        return new flushCache_resultTupleScheme();
+    private static class get_file_metadata_by_expr_resultTupleSchemeFactory implements SchemeFactory {
+      public get_file_metadata_by_expr_resultTupleScheme getScheme() {
+        return new get_file_metadata_by_expr_resultTupleScheme();
       }
     }
 
-    private static class flushCache_resultTupleScheme extends TupleScheme<flushCache_result> {
+    private static class get_file_metadata_by_expr_resultTupleScheme extends TupleScheme<get_file_metadata_by_expr_result> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, flushCache_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, get_file_metadata_by_expr_result struct) throws org.apache.thrift.TException {
         TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.isSetSuccess()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetSuccess()) {
+          struct.success.write(oprot);
+        }
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, flushCache_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, get_file_metadata_by_expr_result struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.success = new GetFileMetadataByExprResult();
+          struct.success.read(iprot);
+          struct.setSuccessIsSet(true);
+        }
       }
     }
 
   }
 
-  public static class get_file_metadata_by_expr_args implements org.apache.thrift.TBase<get_file_metadata_by_expr_args, get_file_metadata_by_expr_args._Fields>, java.io.Serializable, Cloneable, Comparable<get_file_metadata_by_expr_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_file_metadata_by_expr_args");
+  public static class get_file_metadata_args implements org.apache.thrift.TBase<get_file_metadata_args, get_file_metadata_args._Fields>, java.io.Serializable, Cloneable, Comparable<get_file_metadata_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_file_metadata_args");
 
     private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
-      schemes.put(StandardScheme.class, new get_file_metadata_by_expr_argsStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new get_file_metadata_by_expr_argsTupleSchemeFactory());
+      schemes.put(StandardScheme.class, new get_file_metadata_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new get_file_metadata_argsTupleSchemeFactory());
     }
 
-    private GetFileMetadataByExprRequest req; // required
+    private GetFileMetadataRequest req; // 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 {
@@ -154894,16 +155752,16 @@ public class ThriftHiveMetastore {
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
       tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, GetFileMetadataByExprRequest.class)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, GetFileMetadataRequest.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_file_metadata_by_expr_args.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_file_metadata_args.class, metaDataMap);
     }
 
-    public get_file_metadata_by_expr_args() {
+    public get_file_metadata_args() {
     }
 
-    public get_file_metadata_by_expr_args(
-      GetFileMetadataByExprRequest req)
+    public get_file_metadata_args(
+      GetFileMetadataRequest req)
     {
       this();
       this.req = req;
@@ -154912,14 +155770,14 @@ public class ThriftHiveMetastore {
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public get_file_metadata_by_expr_args(get_file_metadata_by_expr_args other) {
+    public get_file_metadata_args(get_file_metadata_args other) {
       if (other.isSetReq()) {
-        this.req = new GetFileMetadataByExprRequest(other.req);
+        this.req = new GetFileMetadataRequest(other.req);
       }
     }
 
-    public get_file_metadata_by_expr_args deepCopy() {
-      return new get_file_metadata_by_expr_args(this);
+    public get_file_metadata_args deepCopy() {
+      return new get_file_metadata_args(this);
     }
 
     @Override
@@ -154927,11 +155785,11 @@ public class ThriftHiveMetastore {
       this.req = null;
     }
 
-    public GetFileMetadataByExprRequest getReq() {
+    public GetFileMetadataRequest getReq() {
       return this.req;
     }
 
-    public void setReq(GetFileMetadataByExprRequest req) {
+    public void setReq(GetFileMetadataRequest req) {
       this.req = req;
     }
 
@@ -154956,7 +155814,7 @@ public class ThriftHiveMetastore {
         if (value == null) {
           unsetReq();
         } else {
-          setReq((GetFileMetadataByExprRequest)value);
+          setReq((GetFileMetadataRequest)value);
         }
         break;
 
@@ -154989,12 +155847,12 @@ public class ThriftHiveMetastore {
     public boolean equals(Object that) {
       if (that == null)
         return false;
-      if (that instanceof get_file_metadata_by_expr_args)
-        return this.equals((get_file_metadata_by_expr_args)that);
+      if (that instanceof get_file_metadata_args)
+        return this.equals((get_file_metadata_args)that);
       return false;
     }
 
-    public boolean equals(get_file_metadata_by_expr_args that) {
+    public boolean equals(get_file_metadata_args that) {
       if (that == null)
         return false;
 
@@ -155023,7 +155881,7 @@ public class ThriftHiveMetastore {
     }
 
     @Override
-    public int compareTo(get_file_metadata_by_expr_args other) {
+    public int compareTo(get_file_metadata_args other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -155057,7 +155915,7 @@ public class ThriftHiveMetastore {
 
     @Override
     public String toString() {
-      StringBuilder sb = new StringBuilder("get_file_metadata_by_expr_args(");
+      StringBuilder sb = new StringBuilder("get_file_metadata_args(");
       boolean first = true;
 
       sb.append("req:");
@@ -155095,15 +155953,15 @@ public class ThriftHiveMetastore {
       }
     }
 
-    private static class get_file_metadata_by_expr_argsStandardSchemeFactory implements SchemeFactory {
-      public get_file_metadata_by_expr_argsStandardScheme getScheme() {
-        return new get_file_metadata_by_expr_argsStandardScheme();
+    private static class get_file_metadata_argsStandardSchemeFactory implements SchemeFactory {
+      public get_file_metadata_argsStandardScheme getScheme() {
+        return new get_file_metadata_argsStandardScheme();
       }
     }
 
-    private static class get_file_metadata_by_expr_argsStandardScheme extends StandardScheme<get_file_metadata_by_expr_args> {
+    private static class get_file_metadata_argsStandardScheme extends StandardScheme<get_file_metadata_args> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, get_file_metadata_by_expr_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, get_file_metadata_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -155115,7 +155973,7 @@ public class ThriftHiveMetastore {
           switch (schemeField.id) {
             case 1: // REQ
               if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.req = new GetFileMetadataByExprRequest();
+                struct.req = new GetFileMetadataRequest();
                 struct.req.read(iprot);
                 struct.setReqIsSet(true);
               } else { 
@@ -155131,7 +155989,7 @@ public class ThriftHiveMetastore {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, get_file_metadata_by_expr_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, get_file_metadata_args struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -155146,16 +156004,16 @@ public class ThriftHiveMetastore {
 
     }
 
-    private static class get_file_metadata_by_expr_argsTupleSchemeFactory implements SchemeFactory {
-      public get_file_metadata_by_expr_argsTupleScheme getScheme() {
-        return new get_file_metadata_by_expr_argsTupleScheme();
+    private static class get_file_metadata_argsTupleSchemeFactory implements SchemeFactory {
+      public get_file_metadata_argsTupleScheme getScheme() {
+        return new get_file_metadata_argsTupleScheme();
       }
     }
 
-    private static class get_file_metadata_by_expr_argsTupleScheme extends TupleScheme<get_file_metadata_by_expr_args> {
+    private static class get_file_metadata_argsTupleScheme extends TupleScheme<get_file_metadata_args> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, get_file_metadata_by_expr_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, get_file_metadata_args struct) throws org.apache.thrift.TException {
         TTupleProtocol oprot = (TTupleProtocol) prot;
         BitSet optionals = new BitSet();
         if (struct.isSetReq()) {
@@ -155168,11 +156026,11 @@ public class ThriftHiveMetastore {
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, get_file_metadata_by_expr_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, get_file_metadata_args struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
-          struct.req = new GetFileMetadataByExprRequest();
+          struct.req = new GetFileMetadataRequest();
           struct.req.read(iprot);
           struct.setReqIsSet(true);
         }
@@ -155181,18 +156039,18 @@ public class ThriftHiveMetastore {
 
   }
 
-  public static class get_file_metadata_by_expr_result implements org.apache.thrift.TBase<get_file_metadata_by_expr_result, get_file_metadata_by_expr_result._Fields>, java.io.Serializable, Cloneable, Comparable<get_file_metadata_by_expr_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_file_metadata_by_expr_result");
+  public static class get_file_metadata_result implements org.apache.thrift.TBase<get_file_metadata_result, get_file_metadata_result._Fields>, java.io.Serializable, Cloneable, Comparable<get_file_metadata_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_file_metadata_result");
 
     private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0);
 
     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
-      schemes.put(StandardScheme.class, new get_file_metadata_by_expr_resultStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new get_file_metadata_by_expr_resultTupleSchemeFactory());
+      schemes.put(StandardScheme.class, new get_file_metadata_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new get_file_metadata_resultTupleSchemeFactory());
     }
 
-    private GetFileMetadataByExprResult success; // required
+    private GetFileMetadataResult success; // 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 {
@@ -155257,16 +156115,16 @@ public class ThriftHiveMetastore {
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
       tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, GetFileMetadataByExprResult.class)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, GetFileMetadataResult.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_file_metadata_by_expr_result.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_file_metadata_result.class, metaDataMap);
     }
 
-    public get_file_metadata_by_expr_result() {
+    public get_file_metadata_result() {
     }
 
-    public get_file_metadata_by_expr_result(
-      GetFileMetadataByExprResult success)
+    public get_file_metadata_result(
+      GetFileMetadataResult success)
     {
       this();
       this.success = success;
@@ -155275,14 +156133,14 @@ public class ThriftHiveMetastore {
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public get_file_metadata_by_expr_result(get_file_metadata_by_expr_result other) {
+    public get_file_metadata_result(get_file_metadata_result other) {
       if (other.isSetSuccess()) {
-        this.success = new GetFileMetadataByExprResult(other.success);
+        this.success = new GetFileMetadataResult(other.success);
       }
     }
 
-    public get_file_metadata_by_expr_result deepCopy() {
-      return new get_file_metadata_by_expr_result(this);
+    public get_file_metadata_result deepCopy() {
+      return new get_file_metadata_result(this);
     }
 
     @Override
@@ -155290,11 +156148,11 @@ public class ThriftHiveMetastore {
       this.success = null;
     }
 
-    public GetFileMetadataByExprResult getSuccess() {
+    public GetFileMetadataResult getSuccess() {
       return this.success;
     }
 
-    public void setSuccess(GetFileMetadataByExprResult success) {
+    public void setSuccess(GetFileMetadataResult success) {
       this.success = success;
     }
 
@@ -155319,7 +156177,7 @@ public class ThriftHiveMetastore {
         if (value == null) {
           unsetSuccess();
         } else {
-          setSuccess((GetFileMetadataByExprResult)value);
+          setSuccess((GetFileMetadataResult)value);
         }
         break;
 
@@ -155352,12 +156210,12 @@ public class ThriftHiveMetastore {
     public boolean equals(Object that) {
       if (that == null)
         return false;
-      if (that instanceof get_file_metadata_by_expr_result)
-        return this.equals((get_file_metadata_by_expr_result)that);
+      if (that instanceof get_file_metadata_result)
+        return this.equals((get_file_metadata_result)that);
       return false;
     }
 
-    public boolean equals(get_file_metadata_by_expr_result that) {
+    public boolean equals(get_file_metadata_result that) {
       if (that == null)
         return false;
 
@@ -155386,7 +156244,7 @@ public class ThriftHiveMetastore {
     }
 
     @Override
-    public int compareTo(get_file_metadata_by_expr_result other) {
+    public int compareTo(get_file_metadata_result other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -155420,7 +156278,7 @@ public class ThriftHiveMetastore {
 
     @Override
     public String toString() {
-      StringBuilder sb = new StringBuilder("get_file_metadata_by_expr_result(");
+      StringBuilder sb = new StringBuilder("get_file_metadata_result(");
       boolean first = true;
 
       sb.append("success:");
@@ -155458,15 +156316,15 @@ public class ThriftHiveMetastore {
       }
     }
 
-    private static class get_file_metadata_by_expr_resultStandardSchemeFactory implements SchemeFactory {
-      public get_file_metadata_by_expr_resultStandardScheme getScheme() {
-        return new get_file_metadata_by_expr_resultStandardScheme();
+    private static class get_file_metadata_resultStandardSchemeFactory implements SchemeFactory {
+      public get_file_metadata_resultStandardScheme getScheme() {
+        return new get_file_metadata_resultStandardScheme();
       }
     }
 
-    private static class get_file_metadata_by_expr_resultStandardScheme extends StandardScheme<get_file_metadata_by_expr_result> {
+    private static class get_file_metadata_resultStandardScheme extends StandardScheme<get_file_metadata_result> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, get_file_metadata_by_expr_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, get_file_metadata_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -155478,7 +156336,7 @@ public class ThriftHiveMetastore {
           switch (schemeField.id) {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.success = new GetFileMetadataByExprResult();
+                struct.success = new GetFileMetadataResult();
                 struct.success.read(iprot);
                 struct.setSuccessIsSet(true);
               } else { 
@@ -155494,7 +156352,7 @@ public class ThriftHiveMetastore {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, get_file_metadata_by_expr_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, get_file_metadata_result struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -155509,16 +156367,16 @@ public class ThriftHiveMetastore {
 
     }
 
-    private static class get_file_metadata_by_expr_resultTupleSchemeFactory implements SchemeFactory {
-      public get_file_metadata_by_expr_resultTupleScheme getScheme() {
-        return new get_file_metadata_by_expr_resultTupleScheme();
+    private static class get_file_metadata_resultTupleSchemeFactory implements SchemeFactory {
+      public get_file_metadata_resultTupleScheme getScheme() {
+        return new get_file_metadata_resultTupleScheme();
       }
     }
 
-    private static class get_file_metadata_by_expr_resultTupleScheme extends TupleScheme<get_file_metadata_by_expr_result> {
+    private static class get_file_metadata_resultTupleScheme extends TupleScheme<get_file_metadata_result> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, get_file_metadata_by_expr_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, get_file_metadata_result struct) throws org.apache.thrift.TException {
         TTupleProtocol oprot = (TTupleProtocol) prot;
         BitSet optionals = new BitSet();
         if (struct.isSetSuccess()) {
@@ -155531,11 +156389,11 @@ public class ThriftHiveMetastore {
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, get_file_metadata_by_expr_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, get_file_metadata_result struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
-          struct.success = new GetFileMetadataByExprResult();
+          struct.success = new GetFileMetadataResult();
           struct.success.read(iprot);
           struct.setSuccessIsSet(true);
         }
@@ -155544,18 +156402,18 @@ public class ThriftHiveMetastore {
 
   }
 
-  public static class get_file_metadata_args implements org.apache.thrift.TBase<get_file_metadata_args, get_file_metadata_args._Fields>, java.io.Serializable, Cloneable, Comparable<get_file_metadata_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_file_metadata_args");
+  public static class put_file_metadata_args implements org.apache.thrift.TBase<put_file_metadata_args, put_file_metadata_args._Fields>, java.io.Serializable, Cloneable, Comparable<put_file_metadata_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("put_file_metadata_args");
 
     private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
-      schemes.put(StandardScheme.class, new get_file_metadata_argsStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new get_file_metadata_argsTupleSchemeFactory());
+      schemes.put(StandardScheme.class, new put_file_metadata_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new put_file_metadata_argsTupleSchemeFactory());
     }
 
-    private GetFileMetadataRequest req; // required
+    private PutFileMetadataRequest req; // 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 {
@@ -155620,16 +156478,16 @@ public class ThriftHiveMetastore {
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
       tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, GetFileMetadataRequest.class)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, PutFileMetadataRequest.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_file_metadata_args.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(put_file_metadata_args.class, metaDataMap);
     }
 
-    public get_file_metadata_args() {
+    public put_file_metadata_args() {
     }
 
-    public get_file_metadata_args(
-      GetFileMetadataRequest req)
+    public put_file_metadata_args(
+      PutFileMetadataRequest req)
     {
       this();
       this.req = req;
@@ -155638,14 +156496,14 @@ public class ThriftHiveMetastore {
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public get_file_metadata_args(get_file_metadata_args other) {
+    public put_file_metadata_args(put_file_metadata_args other) {
       if (other.isSetReq()) {
-        this.req = new GetFileMetadataRequest(other.req);
+        this.req = new PutFileMetadataRequest(other.req);
       }
     }
 
-    public get_file_metadata_args deepCopy() {
-      return new get_file_metadata_args(this);
+    public put_file_metadata_args deepCopy() {
+      return new put_file_metadata_args(this);
     }
 
     @Override
@@ -155653,11 +156511,11 @@ public class ThriftHiveMetastore {
       this.req = null;
     }
 
-    public GetFileMetadataRequest getReq() {
+    public PutFileMetadataRequest getReq() {
       return this.req;
     }
 
-    public void setReq(GetFileMetadataRequest req) {
+    public void setReq(PutFileMetadataRequest req) {
       this.req = req;
     }
 
@@ -155682,7 +156540,7 @@ public class ThriftHiveMetastore {
         if (value == null) {
           unsetReq();
         } else {
-          setReq((GetFileMetadataRequest)value);
+          setReq((PutFileMetadataRequest)value);
         }
         break;
 
@@ -155715,12 +156573,12 @@ public class ThriftHiveMetastore {
     public boolean equals(Object that) {
       if (that == null)
         return false;
-      if (that instanceof get_file_metadata_args)
-        return this.equals((get_file_metadata_args)that);
+      if (that instanceof put_file_metadata_args)
+        return this.equals((put_file_metadata_args)that);
       return false;
     }
 
-    public boolean equals(get_file_metadata_args that) {
+    public boolean equals(put_file_metadata_args that) {
       if (that == null)
         return false;
 
@@ -155749,7 +156607,7 @@ public class ThriftHiveMetastore {
     }
 
     @Override
-    public int compareTo(get_file_metadata_args other) {
+    public int compareTo(put_file_metadata_args other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -155783,7 +156641,7 @@ public class ThriftHiveMetastore {
 
     @Override
     public String toString() {
-      StringBuilder sb = new StringBuilder("get_file_metadata_args(");
+      StringBuilder sb = new StringBuilder("put_file_metadata_args(");
       boolean first = true;
 
       sb.append("req:");
@@ -155821,15 +156679,15 @@ public class ThriftHiveMetastore {
       }
     }
 
-    private static class get_file_metadata_argsStandardSchemeFactory implements SchemeFactory {
-      public get_file_metadata_argsStandardScheme getScheme() {
-        return new get_file_metadata_argsStandardScheme();
+    private static class put_file_metadata_argsStandardSchemeFactory implements SchemeFactory {
+      public put_file_metadata_argsStandardScheme getScheme() {
+        return new put_file_metadata_argsStandardScheme();
       }
     }
 
-    private static class get_file_metadata_argsStandardScheme extends StandardScheme<get_file_metadata_args> {
+    private static class put_file_metadata_argsStandardScheme extends StandardScheme<put_file_metadata_args> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, get_file_metadata_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, put_file_metadata_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -155841,7 +156699,7 @@ public class ThriftHiveMetastore {
           switch (schemeField.id) {
             case 1: // REQ
               if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.req = new GetFileMetadataRequest();
+                struct.req = new PutFileMetadataRequest();
                 struct.req.read(iprot);
                 struct.setReqIsSet(true);
               } else { 
@@ -155857,7 +156715,7 @@ public class ThriftHiveMetastore {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, get_file_metadata_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, put_file_metadata_args struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -155872,16 +156730,16 @@ public class ThriftHiveMetastore {
 
     }
 
-    private static class get_file_metadata_argsTupleSchemeFactory implements SchemeFactory {
-      public get_file_metadata_argsTupleScheme getScheme() {
-        return new get_file_metadata_argsTupleScheme();
+    private static class put_file_metadata_argsTupleSchemeFactory implements SchemeFactory {
+      public put_file_metadata_argsTupleScheme getScheme() {
+        return new put_file_metadata_argsTupleScheme();
       }
     }
 
-    private static class get_file_metadata_argsTupleScheme extends TupleScheme<get_file_metadata_args> {
+    private static class put_file_metadata_argsTupleScheme extends TupleScheme<put_file_metadata_args> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, get_file_metadata_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, put_file_metadata_args struct) throws org.apache.thrift.TException {
         TTupleProtocol oprot = (TTupleProtocol) prot;
         BitSet optionals = new BitSet();
         if (struct.isSetReq()) {
@@ -155894,11 +156752,11 @@ public class ThriftHiveMetastore {
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, get_file_metadata_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, put_file_metadata_args struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
-          struct.req = new GetFileMetadataRequest();
+          struct.req = new PutFileMetadataRequest();
           struct.req.read(iprot);
           struct.setReqIsSet(true);
         }
@@ -155907,18 +156765,18 @@ public class ThriftHiveMetastore {
 
   }
 
-  public static class get_file_metadata_result implements org.apache.thrift.TBase<get_file_metadata_result, get_file_metadata_result._Fields>, java.io.Serializable, Cloneable, Comparable<get_file_metadata_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_file_metadata_result");
+  public static class put_file_metadata_result implements org.apache.thrift.TBase<put_file_metadata_result, put_file_metadata_result._Fields>, java.io.Serializable, Cloneable, Comparable<put_file_metadata_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("put_file_metadata_result");
 
     private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0);
 
     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
-      schemes.put(StandardScheme.class, new get_file_metadata_resultStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new get_file_metadata_resultTupleSchemeFactory());
+      schemes.put(StandardScheme.class, new put_file_metadata_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new put_file_metadata_resultTupleSchemeFactory());
     }
 
-    private GetFileMetadataResult success; // required
+    private PutFileMetadataResult success; // 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 {
@@ -155983,16 +156841,16 @@ public class ThriftHiveMetastore {
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
       tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, GetFileMetadataResult.class)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, PutFileMetadataResult.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_file_metadata_result.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(put_file_metadata_result.class, metaDataMap);
     }
 
-    public get_file_metadata_result() {
+    public put_file_metadata_result() {
     }
 
-    public get_file_metadata_result(
-      GetFileMetadataResult success)
+    public put_file_metadata_result(
+      PutFileMetadataResult success)
     {
       this();
       this.success = success;
@@ -156001,14 +156859,14 @@ public class ThriftHiveMetastore {
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public get_file_metadata_result(get_file_metadata_result other) {
+    public put_file_metadata_result(put_file_metadata_result other) {
       if (other.isSetSuccess()) {
-        this.success = new GetFileMetadataResult(other.success);
+        this.success = new PutFileMetadataResult(other.success);
       }
     }
 
-    public get_file_metadata_result deepCopy() {
-      return new get_file_metadata_result(this);
+    public put_file_metadata_result deepCopy() {
+      return new put_file_metadata_result(this);
     }
 
     @Override
@@ -156016,11 +156874,11 @@ public class ThriftHiveMetastore {
       this.success = null;
     }
 
-    public GetFileMetadataResult getSuccess() {
+    public PutFileMetadataResult getSuccess() {
       return this.success;
     }
 
-    public void setSuccess(GetFileMetadataResult success) {
+    public void setSuccess(PutFileMetadataResult success) {
       this.success = success;
     }
 
@@ -156045,7 +156903,7 @@ public class ThriftHiveMetastore {
         if (value == null) {
           unsetSuccess();
         } else {
-          setSuccess((GetFileMetadataResult)value);
+          setSuccess((PutFileMetadataResult)value);
         }
         break;
 
@@ -156078,12 +156936,12 @@ public class ThriftHiveMetastore {
     public boolean equals(Object that) {
       if (that == null)
         return false;
-      if (that instanceof get_file_metadata_result)
-        return this.equals((get_file_metadata_result)that);
+      if (that instanceof put_file_metadata_result)
+        return this.equals((put_file_metadata_result)that);
       return false;
     }
 
-    public boolean equals(get_file_metadata_result that) {
+    public boolean equals(put_file_metadata_result that) {
       if (that == null)
         return false;
 
@@ -156112,7 +156970,7 @@ public class ThriftHiveMetastore {
     }
 
     @Override
-    public int compareTo(get_file_metadata_result other) {
+    public int compareTo(put_file_metadata_result other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -156146,7 +157004,7 @@ public class ThriftHiveMetastore {
 
     @Override
     public String toString() {
-      StringBuilder sb = new StringBuilder("get_file_metadata_result(");
+      StringBuilder sb = new StringBuilder("put_file_metadata_result(");
       boolean first = true;
 
       sb.append("success:");
@@ -156184,15 +157042,15 @@ public class ThriftHiveMetastore {
       }
     }
 
-    private static class get_file_metadata_resultStandardSchemeFactory implements SchemeFactory {
-      public get_file_metadata_resultStandardScheme getScheme() {
-        return new get_file_metadata_resultStandardScheme();
+    private static class put_file_metadata_resultStandardSchemeFactory implements SchemeFactory {
+      public put_file_metadata_resultStandardScheme getScheme() {
+        return new put_file_metadata_resultStandardScheme();
       }
     }
 
-    private static class get_file_metadata_resultStandardScheme extends StandardScheme<get_file_metadata_result> {
+    private static class put_file_metadata_resultStandardScheme extends StandardScheme<put_file_metadata_result> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, get_file_metadata_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, put_file_metadata_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -156204,7 +157062,7 @@ public class ThriftHiveMetastore {
           switch (schemeField.id) {
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.success = new GetFileMetadataResult();
+                struct.success = new PutFileMetadataResult();
                 struct.success.read(iprot);
                 struct.setSuccessIsSet(true);
               } else { 
@@ -156220,7 +157078,7 @@ public class ThriftHiveMetastore {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, get_file_metadata_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, put_file_metadata_result struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -156235,16 +157093,16 @@ public class ThriftHiveMetastore {
 
     }
 
-    private static class get_file_metadata_resultTupleSchemeFactory implements SchemeFactory {
-      public get_file_metadata_resultTupleScheme getScheme() {
-        return new get_file_metadata_resultTupleScheme();
+    private static class put_file_metadata_resultTupleSchemeFactory implements SchemeFactory {
+      public put_file_metadata_resultTupleScheme getScheme() {
+        return new put_file_metadata_resultTupleScheme();
       }
     }
 
-    private static class get_file_metadata_resultTupleScheme extends TupleScheme<get_file_metadata_result> {
+    private static class put_file_metadata_resultTupleScheme extends TupleScheme<put_file_metadata_result> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, get_file_metadata_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, put_file_metadata_result struct) throws org.apache.thrift.TException {
         TTupleProtocol oprot = (TTupleProtocol) prot;
         BitSet optionals = new BitSet();
         if (struct.isSetSuccess()) {
@@ -156257,11 +157115,11 @@ public class ThriftHiveMetastore {
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, get_file_metadata_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, put_file_metadata_result struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
-          struct.success = new GetFileMetadataResult();
+          struct.success = new PutFileMetadataResult();
           struct.success.read(iprot);
           struct.setSuccessIsSet(true);
         }
@@ -156270,18 +157128,18 @@ public class ThriftHiveMetastore {
 
   }
 
-  public static class put_file_metadata_args implements org.apache.thrift.TBase<put_file_metadata_args, put_file_metadata_args._Fields>, java.io.Serializable, Cloneable, Comparable<put_file_metadata_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("put_file_metadata_args");
+  public static class clear_file_metadata_args implements org.apache.thrift.TBase<clear_file_metadata_args, clear_file_metadata_args._Fields>, java.io.Serializable, Cloneable, Comparable<clear_file_metadata_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("clear_file_metadata_args");
 
     private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
-      schemes.put(StandardScheme.class, new put_file_metadata_argsStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new put_file_metadata_argsTupleSchemeFactory());
+      schemes.put(StandardScheme.class, new clear_file_metadata_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new clear_file_metadata_argsTupleSchemeFactory());
     }
 
-    private PutFileMetadataRequest req; // required
+    private ClearFileMetadataRequest req; // 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 {
@@ -156346,16 +157204,16 @@ public class ThriftHiveMetastore {
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
       tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, PutFileMetadataRequest.class)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ClearFileMetadataRequest.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(put_file_metadata_args.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(clear_file_metadata_args.class, metaDataMap);
     }
 
-    public put_file_metadata_args() {
+    public clear_file_metadata_args() {
     }
 
-    public put_file_metadata_args(
-      PutFileMetadataRequest req)
+    public clear_file_metadata_args(
+      ClearFileMetadataRequest req)
     {
       this();
       this.req = req;
@@ -156364,14 +157222,14 @@ public class ThriftHiveMetastore {
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public put_file_metadata_args(put_file_metadata_args other) {
+    public clear_file_metadata_args(clear_file_metadata_args other) {
       if (other.isSetReq()) {
-        this.req = new PutFileMetadataRequest(other.req);
+        this.req = new ClearFileMetadataRequest(other.req);
       }
     }
 
-    public put_file_metadata_args deepCopy() {
-      return new put_file_metadata_args(this);
+    public clear_file_metadata_args deepCopy() {
+      return new clear_file_metadata_args(this);
     }
 
     @Override
@@ -156379,11 +157237,11 @@ public class ThriftHiveMetastore {
       this.req = null;
     }
 
-    public PutFileMetadataRequest getReq() {
+    public ClearFileMetadataRequest getReq() {
       return this.req;
     }
 
-    public void setReq(PutFileMetadataRequest req) {
+    public void setReq(ClearFileMetadataRequest req) {
       this.req = req;
     }
 
@@ -156408,7 +157266,7 @@ public class ThriftHiveMetastore {
         if (value == null) {
           unsetReq();
         } else {
-          setReq((PutFileMetadataRequest)value);
+          setReq((ClearFileMetadataRequest)value);
         }
         break;
 
@@ -156441,12 +157299,12 @@ public class ThriftHiveMetastore {
     public boolean equals(Object that) {
       if (that == null)
         return false;
-      if (that instanceof put_file_metadata_args)
-        return this.equals((put_file_metadata_args)that);
+      if (that instanceof clear_file_metadata_args)
+        return this.equals((clear_file_metadata_args)that);
       return false;
     }
 
-    public boolean equals(put_file_metadata_args that) {
+    public boolean equals(clear_file_metadata_args that) {
       if (that == null)
         return false;
 
@@ -156475,7 +157333,7 @@ public class ThriftHiveMetastore {
     }
 
     @Override
-    public int compareTo(put_file_metadata_args other) {
+    public int compareTo(clear_file_metadata_args other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -156509,7 +157367,7 @@ public class ThriftHiveMetastore {
 
     @Override
     public String toString() {
-      StringBuilder sb = new StringBuilder("put_file_metadata_args(");
+      StringBuilder sb = new StringBuilder("clear_file_metadata_args(");
       boolean first = true;
 
       sb.append("req:");
@@ -156547,15 +157405,15 @@ public class ThriftHiveMetastore {
       }
     }
 
-    private static class put_file_metadata_argsStandardSchemeFactory implements SchemeFactory {
-      public put_file_metadata_argsStandardScheme getScheme() {
-        return new put_file_metadata_argsStandardScheme();
+    private static class clear_file_metadata_argsStandardSchemeFactory implements SchemeFactory {
+      public clear_file_metadata_argsStandardScheme getScheme() {
+        return new clear_file_metadata_argsStandardScheme();
       }
     }
 
-    private static class put_file_metadata_argsStandardScheme extends StandardScheme<put_file_metadata_args> {
+    private static class clear_file_metadata_argsStandardScheme extends StandardScheme<clear_file_metadata_args> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, put_file_metadata_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, clear_file_metadata_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -156567,7 +157425,7 @@ public class ThriftHiveMetastore {
           switch (schemeField.id) {
             case 1: // REQ
               if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.req = new PutFileMetadataRequest();
+                struct.req = new ClearFileMetadataRequest();
                 struct.req.read(iprot);
                 struct.setReqIsSet(true);
               } else { 
@@ -156583,7 +157441,7 @@ public class ThriftHiveMetastore {
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, put_file_metadata_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, clear_file_metadata_args struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -156598,16 +157456,16 @@ public class ThriftHiveMetastore {
 
     }
 
-    private static class put_file_metadata_argsTupleSchemeFactory implements SchemeFactory {
-      public put_file_metadata_argsTupleScheme getScheme() {
-        return new put_file_metadata_argsTupleScheme();
+    private static class clear_file_metadata_argsTupleSchemeFactory implements SchemeFactory {
+      public clear_file_metadata_argsTupleScheme getScheme() {
+        return new clear_file_metadata_argsTupleScheme();
       }
     }
 
-    private static class put_file_metadata_argsTupleScheme extends TupleScheme<put_file_metadata_args> {
+    private static class clear_file_metadata_argsTupleScheme extends TupleScheme<clear_file_metadata_args> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, put_file_metadata_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, clear_file_metadata_args struct) throws org.apache.thrift.TException {
         TTupleProtocol oprot = (TTupleProtocol) prot;
         BitSet optionals = new BitSet();
         if (struct.isSetReq()) {
@@ -156620,11 +157478,11 @@ public class ThriftHiveMetastore {
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, put_file_metadata_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, clear_file_metadata_args struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
-          struct.req = new PutFileMetadataRequest();
+          struct.req = new ClearFileMetadataRequest();
           struct.req.read(iprot);
           struct.setReqIsSet(true);
         }
@@ -156633,18 +157491,18 @@ public class ThriftHiveMetastore {
 
   }
 
-  public static class put_file_metadata_result implements org.apache.thrift.TBase<put_file_metadata_result, put_file_metadata_result._Fields>, java.io.Serializable, Cloneable, Comparable<put_file_metadata_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("put_file_metadata_result");
+  public static class clear_file_metadata_result implements org.apache.thrift.TBase<clear_file_metadata_result, clear_file_metadata_result._Fields>, java.io.Serializable, Cloneable, Comparable<clear_file_metadata_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("clear_file_metadata_result");
 
     private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0);
 
     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
-      schemes.put(StandardScheme.class, new put_

<TRUNCATED>

[7/7] hive git commit: HIVE-12075 : add analyze command to explictly cache file metadata in HBase metastore (Sergey Shelukhin, reviewed by Alan Gates)

Posted by se...@apache.org.
HIVE-12075 : add analyze command to explictly cache file metadata in HBase metastore (Sergey Shelukhin, reviewed by Alan Gates)


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

Branch: refs/heads/master
Commit: 7df62023f8a328046055486de46121fd16b7458a
Parents: 542eaf6
Author: Sergey Shelukhin <se...@apache.org>
Authored: Fri Dec 18 14:41:29 2015 -0800
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Fri Dec 18 14:41:29 2015 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/conf/HiveConf.java   |    5 +-
 .../test/resources/testconfiguration.properties |    1 +
 metastore/if/hive_metastore.thrift              |   17 +-
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.cpp  | 2257 ++++++++++--------
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.h    |  126 +
 .../ThriftHiveMetastore_server.skeleton.cpp     |    5 +
 .../gen/thrift/gen-cpp/hive_metastore_types.cpp |  575 +++--
 .../gen/thrift/gen-cpp/hive_metastore_types.h   |  125 +-
 .../metastore/api/CacheFileMetadataRequest.java |  702 ++++++
 .../metastore/api/CacheFileMetadataResult.java  |  386 +++
 .../metastore/api/FileMetadataExprType.java     |    4 -
 .../metastore/api/PutFileMetadataRequest.java   |  124 +-
 .../hive/metastore/api/ThriftHiveMetastore.java | 1478 +++++++++---
 .../gen-php/metastore/ThriftHiveMetastore.php   |  215 ++
 .../src/gen/thrift/gen-php/metastore/Types.php  |  242 ++
 .../hive_metastore/ThriftHiveMetastore-remote   |    7 +
 .../hive_metastore/ThriftHiveMetastore.py       |  189 ++
 .../gen/thrift/gen-py/hive_metastore/ttypes.py  |  190 +-
 .../gen/thrift/gen-rb/hive_metastore_types.rb   |   48 +-
 .../gen/thrift/gen-rb/thrift_hive_metastore.rb  |   54 +
 .../hadoop/hive/metastore/FileFormatProxy.java  |   64 +
 .../hive/metastore/FileMetadataHandler.java     |   84 +-
 .../hive/metastore/FileMetadataManager.java     |  129 +
 .../hadoop/hive/metastore/HiveMetaStore.java    |  140 +-
 .../hive/metastore/HiveMetaStoreClient.java     |   17 +
 .../hadoop/hive/metastore/IMetaStoreClient.java |    3 +
 .../hadoop/hive/metastore/MetaStoreUtils.java   |    6 +-
 .../hadoop/hive/metastore/ObjectStore.java      |   17 +-
 .../metastore/PartitionExpressionProxy.java     |   36 +-
 .../apache/hadoop/hive/metastore/RawStore.java  |    7 +-
 .../filemeta/OrcFileMetadataHandler.java        |   24 +-
 .../hive/metastore/hbase/HBaseReadWrite.java    |   60 +-
 .../hadoop/hive/metastore/hbase/HBaseStore.java |   50 +-
 .../hive/metastore/hbase/MetadataStore.java     |   52 +
 .../DummyRawStoreControlledCommit.java          |    8 +-
 .../DummyRawStoreForJdoConnection.java          |    8 +-
 .../MockPartitionExpressionForMetastore.java    |    9 +-
 .../hadoop/hive/metastore/TestObjectStore.java  |    9 +-
 .../hadoop/hive/metastore/hbase/MockUtils.java  |   10 +-
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |   12 +
 .../hive/ql/io/orc/OrcFileFormatProxy.java      |   74 +
 .../apache/hadoop/hive/ql/metadata/Hive.java    |   13 +
 .../ppr/PartitionExpressionForMetastore.java    |   54 +-
 .../hive/ql/parse/AnalyzeCommandUtils.java      |   57 +
 .../ql/parse/ColumnStatsSemanticAnalyzer.java   |   36 +-
 .../hive/ql/parse/DDLSemanticAnalyzer.java      |   27 +-
 .../org/apache/hadoop/hive/ql/parse/HiveLexer.g |    1 +
 .../apache/hadoop/hive/ql/parse/HiveParser.g    |    8 +-
 .../hive/ql/parse/SemanticAnalyzerFactory.java  |    2 +
 .../hadoop/hive/ql/plan/CacheMetadataDesc.java  |   58 +
 .../org/apache/hadoop/hive/ql/plan/DDLWork.java |   14 +
 .../hadoop/hive/ql/plan/HiveOperation.java      |    1 +
 .../queries/clientpositive/stats_filemetadata.q |   17 +
 .../clientpositive/tez/stats_filemetadata.q.out |   54 +
 .../gen-py/hive_service/ThriftHive-remote       |   21 +
 .../org/apache/hadoop/hive/io/HdfsUtils.java    |   61 +
 56 files changed, 6440 insertions(+), 1553 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/7df62023/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index 67c4213..96a3fb5 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -200,7 +200,8 @@ public class HiveConf extends Configuration {
       HiveConf.ConfVars.METASTORE_HBASE_AGGR_STATS_CACHE_ENTRIES,
       HiveConf.ConfVars.METASTORE_HBASE_AGGR_STATS_MEMORY_TTL,
       HiveConf.ConfVars.METASTORE_HBASE_AGGR_STATS_INVALIDATOR_FREQUENCY,
-      HiveConf.ConfVars.METASTORE_HBASE_AGGR_STATS_HBASE_TTL
+      HiveConf.ConfVars.METASTORE_HBASE_AGGR_STATS_HBASE_TTL,
+      HiveConf.ConfVars.METASTORE_HBASE_FILE_METADATA_THREADS
       };
 
   /**
@@ -443,6 +444,8 @@ public class HiveConf extends Configuration {
         new TimeValidator(TimeUnit.SECONDS),
         "Number of seconds stats entries live in HBase cache after they are created.  They may be" +
             " invalided by updates or partition drops before this.  Default is one week."),
+    METASTORE_HBASE_FILE_METADATA_THREADS("hive.metastore.hbase.file.metadata.threads", 1,
+        "Number of threads to use to read file metadata in background to cache it."),
 
     METASTORETHRIFTCONNECTIONRETRIES("hive.metastore.connect.retries", 3,
         "Number of retries while opening a connection to metastore"),

http://git-wip-us.apache.org/repos/asf/hive/blob/7df62023/itests/src/test/resources/testconfiguration.properties
----------------------------------------------------------------------
diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties
index 1e7dce3..2d230ef 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -434,6 +434,7 @@ minillap.query.files=bucket_map_join_tez1.q,\
   mapjoin_decimal.q,\
   lvj_mapjoin.q,\
   llapdecider.q,\
+  stats_filemetadata.q,\
   mrr.q,\
   orc_ppd_basic.q,\
   tez_bmj_schema_evolution.q,\

http://git-wip-us.apache.org/repos/asf/hive/blob/7df62023/metastore/if/hive_metastore.thrift
----------------------------------------------------------------------
diff --git a/metastore/if/hive_metastore.thrift b/metastore/if/hive_metastore.thrift
index bb754f1..e6403ed 100755
--- a/metastore/if/hive_metastore.thrift
+++ b/metastore/if/hive_metastore.thrift
@@ -756,7 +756,8 @@ struct PutFileMetadataResult {
 // Request type for put_file_metadata
 struct PutFileMetadataRequest {
   1: required list<i64> fileIds,
-  2: required list<binary> metadata
+  2: required list<binary> metadata,
+  3: optional FileMetadataExprType type
 }
 
 // Return type for clear_file_metadata
@@ -768,6 +769,19 @@ struct ClearFileMetadataRequest {
   1: required list<i64> fileIds
 }
 
+// Return type for cache_file_metadata
+struct CacheFileMetadataResult {
+  1: required bool isSupported
+}
+
+// Request type for cache_file_metadata
+struct CacheFileMetadataRequest {
+  1: required string dbName,
+  2: required string tblName,
+  3: optional string partName,
+  4: optional bool isAllParts
+}
+
 struct GetAllFunctionsResponse {
   1: optional list<Function> functions
 }
@@ -1267,6 +1281,7 @@ service ThriftHiveMetastore extends fb303.FacebookService
   GetFileMetadataResult get_file_metadata(1:GetFileMetadataRequest req)
   PutFileMetadataResult put_file_metadata(1:PutFileMetadataRequest req)
   ClearFileMetadataResult clear_file_metadata(1:ClearFileMetadataRequest req)
+  CacheFileMetadataResult cache_file_metadata(1:CacheFileMetadataRequest req)
 
 }
 


[2/7] hive git commit: HIVE-12075 : add analyze command to explictly cache file metadata in HBase metastore (Sergey Shelukhin, reviewed by Alan Gates)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/7df62023/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php b/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
index 34c2205..75f4a47 100644
--- a/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
+++ b/metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
@@ -1051,6 +1051,11 @@ interface ThriftHiveMetastoreIf extends \FacebookServiceIf {
    * @return \metastore\ClearFileMetadataResult
    */
   public function clear_file_metadata(\metastore\ClearFileMetadataRequest $req);
+  /**
+   * @param \metastore\CacheFileMetadataRequest $req
+   * @return \metastore\CacheFileMetadataResult
+   */
+  public function cache_file_metadata(\metastore\CacheFileMetadataRequest $req);
 }
 
 class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metastore\ThriftHiveMetastoreIf {
@@ -8559,6 +8564,57 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas
     throw new \Exception("clear_file_metadata failed: unknown result");
   }
 
+  public function cache_file_metadata(\metastore\CacheFileMetadataRequest $req)
+  {
+    $this->send_cache_file_metadata($req);
+    return $this->recv_cache_file_metadata();
+  }
+
+  public function send_cache_file_metadata(\metastore\CacheFileMetadataRequest $req)
+  {
+    $args = new \metastore\ThriftHiveMetastore_cache_file_metadata_args();
+    $args->req = $req;
+    $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
+    if ($bin_accel)
+    {
+      thrift_protocol_write_binary($this->output_, 'cache_file_metadata', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
+    }
+    else
+    {
+      $this->output_->writeMessageBegin('cache_file_metadata', TMessageType::CALL, $this->seqid_);
+      $args->write($this->output_);
+      $this->output_->writeMessageEnd();
+      $this->output_->getTransport()->flush();
+    }
+  }
+
+  public function recv_cache_file_metadata()
+  {
+    $bin_accel = ($this->input_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_read_binary');
+    if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_cache_file_metadata_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_cache_file_metadata_result();
+      $result->read($this->input_);
+      $this->input_->readMessageEnd();
+    }
+    if ($result->success !== null) {
+      return $result->success;
+    }
+    throw new \Exception("cache_file_metadata failed: unknown result");
+  }
+
 }
 
 // HELPER FUNCTIONS AND STRUCTURES
@@ -39493,4 +39549,163 @@ class ThriftHiveMetastore_clear_file_metadata_result {
 
 }
 
+class ThriftHiveMetastore_cache_file_metadata_args {
+  static $_TSPEC;
+
+  /**
+   * @var \metastore\CacheFileMetadataRequest
+   */
+  public $req = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'req',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\CacheFileMetadataRequest',
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['req'])) {
+        $this->req = $vals['req'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_cache_file_metadata_args';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRUCT) {
+            $this->req = new \metastore\CacheFileMetadataRequest();
+            $xfer += $this->req->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_cache_file_metadata_args');
+    if ($this->req !== null) {
+      if (!is_object($this->req)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('req', TType::STRUCT, 1);
+      $xfer += $this->req->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class ThriftHiveMetastore_cache_file_metadata_result {
+  static $_TSPEC;
+
+  /**
+   * @var \metastore\CacheFileMetadataResult
+   */
+  public $success = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        0 => array(
+          'var' => 'success',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\CacheFileMetadataResult',
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['success'])) {
+        $this->success = $vals['success'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ThriftHiveMetastore_cache_file_metadata_result';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 0:
+          if ($ftype == TType::STRUCT) {
+            $this->success = new \metastore\CacheFileMetadataResult();
+            $xfer += $this->success->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('ThriftHiveMetastore_cache_file_metadata_result');
+    if ($this->success !== null) {
+      if (!is_object($this->success)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('success', TType::STRUCT, 0);
+      $xfer += $this->success->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
 

http://git-wip-us.apache.org/repos/asf/hive/blob/7df62023/metastore/src/gen/thrift/gen-php/metastore/Types.php
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-php/metastore/Types.php b/metastore/src/gen/thrift/gen-php/metastore/Types.php
index e43a13d..fe4c966 100644
--- a/metastore/src/gen/thrift/gen-php/metastore/Types.php
+++ b/metastore/src/gen/thrift/gen-php/metastore/Types.php
@@ -14564,6 +14564,10 @@ class PutFileMetadataRequest {
    * @var string[]
    */
   public $metadata = null;
+  /**
+   * @var int
+   */
+  public $type = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
@@ -14584,6 +14588,10 @@ class PutFileMetadataRequest {
             'type' => TType::STRING,
             ),
           ),
+        3 => array(
+          'var' => 'type',
+          'type' => TType::I32,
+          ),
         );
     }
     if (is_array($vals)) {
@@ -14593,6 +14601,9 @@ class PutFileMetadataRequest {
       if (isset($vals['metadata'])) {
         $this->metadata = $vals['metadata'];
       }
+      if (isset($vals['type'])) {
+        $this->type = $vals['type'];
+      }
     }
   }
 
@@ -14649,6 +14660,13 @@ class PutFileMetadataRequest {
             $xfer += $input->skip($ftype);
           }
           break;
+        case 3:
+          if ($ftype == TType::I32) {
+            $xfer += $input->readI32($this->type);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
         default:
           $xfer += $input->skip($ftype);
           break;
@@ -14696,6 +14714,11 @@ class PutFileMetadataRequest {
       }
       $xfer += $output->writeFieldEnd();
     }
+    if ($this->type !== null) {
+      $xfer += $output->writeFieldBegin('type', TType::I32, 3);
+      $xfer += $output->writeI32($this->type);
+      $xfer += $output->writeFieldEnd();
+    }
     $xfer += $output->writeFieldStop();
     $xfer += $output->writeStructEnd();
     return $xfer;
@@ -14854,6 +14877,225 @@ class ClearFileMetadataRequest {
 
 }
 
+class CacheFileMetadataResult {
+  static $_TSPEC;
+
+  /**
+   * @var bool
+   */
+  public $isSupported = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'isSupported',
+          'type' => TType::BOOL,
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['isSupported'])) {
+        $this->isSupported = $vals['isSupported'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'CacheFileMetadataResult';
+  }
+
+  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::BOOL) {
+            $xfer += $input->readBool($this->isSupported);
+          } 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('CacheFileMetadataResult');
+    if ($this->isSupported !== null) {
+      $xfer += $output->writeFieldBegin('isSupported', TType::BOOL, 1);
+      $xfer += $output->writeBool($this->isSupported);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class CacheFileMetadataRequest {
+  static $_TSPEC;
+
+  /**
+   * @var string
+   */
+  public $dbName = null;
+  /**
+   * @var string
+   */
+  public $tblName = null;
+  /**
+   * @var string
+   */
+  public $partName = null;
+  /**
+   * @var bool
+   */
+  public $isAllParts = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'dbName',
+          'type' => TType::STRING,
+          ),
+        2 => array(
+          'var' => 'tblName',
+          'type' => TType::STRING,
+          ),
+        3 => array(
+          'var' => 'partName',
+          'type' => TType::STRING,
+          ),
+        4 => array(
+          'var' => 'isAllParts',
+          'type' => TType::BOOL,
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['dbName'])) {
+        $this->dbName = $vals['dbName'];
+      }
+      if (isset($vals['tblName'])) {
+        $this->tblName = $vals['tblName'];
+      }
+      if (isset($vals['partName'])) {
+        $this->partName = $vals['partName'];
+      }
+      if (isset($vals['isAllParts'])) {
+        $this->isAllParts = $vals['isAllParts'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'CacheFileMetadataRequest';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->dbName);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->tblName);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 3:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->partName);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 4:
+          if ($ftype == TType::BOOL) {
+            $xfer += $input->readBool($this->isAllParts);
+          } 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('CacheFileMetadataRequest');
+    if ($this->dbName !== null) {
+      $xfer += $output->writeFieldBegin('dbName', TType::STRING, 1);
+      $xfer += $output->writeString($this->dbName);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->tblName !== null) {
+      $xfer += $output->writeFieldBegin('tblName', TType::STRING, 2);
+      $xfer += $output->writeString($this->tblName);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->partName !== null) {
+      $xfer += $output->writeFieldBegin('partName', TType::STRING, 3);
+      $xfer += $output->writeString($this->partName);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->isAllParts !== null) {
+      $xfer += $output->writeFieldBegin('isAllParts', TType::BOOL, 4);
+      $xfer += $output->writeBool($this->isAllParts);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
 class GetAllFunctionsResponse {
   static $_TSPEC;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/7df62023/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote b/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
index 22d794f..9564ded 100755
--- a/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
+++ b/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
@@ -155,6 +155,7 @@ if len(sys.argv) <= 1 or sys.argv[1] == '--help':
   print('  GetFileMetadataResult get_file_metadata(GetFileMetadataRequest req)')
   print('  PutFileMetadataResult put_file_metadata(PutFileMetadataRequest req)')
   print('  ClearFileMetadataResult clear_file_metadata(ClearFileMetadataRequest req)')
+  print('  CacheFileMetadataResult cache_file_metadata(CacheFileMetadataRequest req)')
   print('  string getName()')
   print('  string getVersion()')
   print('  fb_status getStatus()')
@@ -1010,6 +1011,12 @@ elif cmd == 'clear_file_metadata':
     sys.exit(1)
   pp.pprint(client.clear_file_metadata(eval(args[0]),))
 
+elif cmd == 'cache_file_metadata':
+  if len(args) != 1:
+    print('cache_file_metadata requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.cache_file_metadata(eval(args[0]),))
+
 elif cmd == 'getName':
   if len(args) != 0:
     print('getName requires 0 args')

http://git-wip-us.apache.org/repos/asf/hive/blob/7df62023/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py b/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
index 5c72a27..97acb5d 100644
--- a/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
+++ b/metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
@@ -1073,6 +1073,13 @@ class Iface(fb303.FacebookService.Iface):
     """
     pass
 
+  def cache_file_metadata(self, req):
+    """
+    Parameters:
+     - req
+    """
+    pass
+
 
 class Client(fb303.FacebookService.Client, Iface):
   """
@@ -5873,6 +5880,37 @@ class Client(fb303.FacebookService.Client, Iface):
       return result.success
     raise TApplicationException(TApplicationException.MISSING_RESULT, "clear_file_metadata failed: unknown result")
 
+  def cache_file_metadata(self, req):
+    """
+    Parameters:
+     - req
+    """
+    self.send_cache_file_metadata(req)
+    return self.recv_cache_file_metadata()
+
+  def send_cache_file_metadata(self, req):
+    self._oprot.writeMessageBegin('cache_file_metadata', TMessageType.CALL, self._seqid)
+    args = cache_file_metadata_args()
+    args.req = req
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_cache_file_metadata(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = cache_file_metadata_result()
+    result.read(iprot)
+    iprot.readMessageEnd()
+    if result.success is not None:
+      return result.success
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "cache_file_metadata failed: unknown result")
+
 
 class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
   def __init__(self, handler):
@@ -6008,6 +6046,7 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
     self._processMap["get_file_metadata"] = Processor.process_get_file_metadata
     self._processMap["put_file_metadata"] = Processor.process_put_file_metadata
     self._processMap["clear_file_metadata"] = Processor.process_clear_file_metadata
+    self._processMap["cache_file_metadata"] = Processor.process_cache_file_metadata
 
   def process(self, iprot, oprot):
     (name, type, seqid) = iprot.readMessageBegin()
@@ -9263,6 +9302,25 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
     oprot.writeMessageEnd()
     oprot.trans.flush()
 
+  def process_cache_file_metadata(self, seqid, iprot, oprot):
+    args = cache_file_metadata_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = cache_file_metadata_result()
+    try:
+      result.success = self._handler.cache_file_metadata(args.req)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("cache_file_metadata", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
 
 # HELPER FUNCTIONS AND STRUCTURES
 
@@ -31884,3 +31942,134 @@ class clear_file_metadata_result:
 
   def __ne__(self, other):
     return not (self == other)
+
+class cache_file_metadata_args:
+  """
+  Attributes:
+   - req
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRUCT, 'req', (CacheFileMetadataRequest, CacheFileMetadataRequest.thrift_spec), None, ), # 1
+  )
+
+  def __init__(self, req=None,):
+    self.req = req
+
+  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.req = CacheFileMetadataRequest()
+          self.req.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('cache_file_metadata_args')
+    if self.req is not None:
+      oprot.writeFieldBegin('req', TType.STRUCT, 1)
+      self.req.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.req)
+    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 cache_file_metadata_result:
+  """
+  Attributes:
+   - success
+  """
+
+  thrift_spec = (
+    (0, TType.STRUCT, 'success', (CacheFileMetadataResult, CacheFileMetadataResult.thrift_spec), None, ), # 0
+  )
+
+  def __init__(self, success=None,):
+    self.success = success
+
+  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 = CacheFileMetadataResult()
+          self.success.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('cache_file_metadata_result')
+    if self.success is not None:
+      oprot.writeFieldBegin('success', TType.STRUCT, 0)
+      self.success.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.success)
+    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)

http://git-wip-us.apache.org/repos/asf/hive/blob/7df62023/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py b/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
index ba525ed..fbeab5e 100644
--- a/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
+++ b/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
@@ -10266,17 +10266,20 @@ class PutFileMetadataRequest:
   Attributes:
    - fileIds
    - metadata
+   - type
   """
 
   thrift_spec = (
     None, # 0
     (1, TType.LIST, 'fileIds', (TType.I64,None), None, ), # 1
     (2, TType.LIST, 'metadata', (TType.STRING,None), None, ), # 2
+    (3, TType.I32, 'type', None, None, ), # 3
   )
 
-  def __init__(self, fileIds=None, metadata=None,):
+  def __init__(self, fileIds=None, metadata=None, type=None,):
     self.fileIds = fileIds
     self.metadata = metadata
+    self.type = type
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -10307,6 +10310,11 @@ class PutFileMetadataRequest:
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.I32:
+          self.type = iprot.readI32()
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -10331,6 +10339,10 @@ class PutFileMetadataRequest:
         oprot.writeString(iter507)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
+    if self.type is not None:
+      oprot.writeFieldBegin('type', TType.I32, 3)
+      oprot.writeI32(self.type)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -10346,6 +10358,7 @@ class PutFileMetadataRequest:
     value = 17
     value = (value * 31) ^ hash(self.fileIds)
     value = (value * 31) ^ hash(self.metadata)
+    value = (value * 31) ^ hash(self.type)
     return value
 
   def __repr__(self):
@@ -10480,6 +10493,181 @@ class ClearFileMetadataRequest:
   def __ne__(self, other):
     return not (self == other)
 
+class CacheFileMetadataResult:
+  """
+  Attributes:
+   - isSupported
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.BOOL, 'isSupported', None, None, ), # 1
+  )
+
+  def __init__(self, isSupported=None,):
+    self.isSupported = isSupported
+
+  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.BOOL:
+          self.isSupported = iprot.readBool()
+        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('CacheFileMetadataResult')
+    if self.isSupported is not None:
+      oprot.writeFieldBegin('isSupported', TType.BOOL, 1)
+      oprot.writeBool(self.isSupported)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.isSupported is None:
+      raise TProtocol.TProtocolException(message='Required field isSupported is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.isSupported)
+    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 CacheFileMetadataRequest:
+  """
+  Attributes:
+   - dbName
+   - tblName
+   - partName
+   - isAllParts
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'dbName', None, None, ), # 1
+    (2, TType.STRING, 'tblName', None, None, ), # 2
+    (3, TType.STRING, 'partName', None, None, ), # 3
+    (4, TType.BOOL, 'isAllParts', None, None, ), # 4
+  )
+
+  def __init__(self, dbName=None, tblName=None, partName=None, isAllParts=None,):
+    self.dbName = dbName
+    self.tblName = tblName
+    self.partName = partName
+    self.isAllParts = isAllParts
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.dbName = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRING:
+          self.tblName = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.STRING:
+          self.partName = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.BOOL:
+          self.isAllParts = iprot.readBool()
+        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('CacheFileMetadataRequest')
+    if self.dbName is not None:
+      oprot.writeFieldBegin('dbName', TType.STRING, 1)
+      oprot.writeString(self.dbName)
+      oprot.writeFieldEnd()
+    if self.tblName is not None:
+      oprot.writeFieldBegin('tblName', TType.STRING, 2)
+      oprot.writeString(self.tblName)
+      oprot.writeFieldEnd()
+    if self.partName is not None:
+      oprot.writeFieldBegin('partName', TType.STRING, 3)
+      oprot.writeString(self.partName)
+      oprot.writeFieldEnd()
+    if self.isAllParts is not None:
+      oprot.writeFieldBegin('isAllParts', TType.BOOL, 4)
+      oprot.writeBool(self.isAllParts)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.dbName is None:
+      raise TProtocol.TProtocolException(message='Required field dbName is unset!')
+    if self.tblName is None:
+      raise TProtocol.TProtocolException(message='Required field tblName is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.dbName)
+    value = (value * 31) ^ hash(self.tblName)
+    value = (value * 31) ^ hash(self.partName)
+    value = (value * 31) ^ hash(self.isAllParts)
+    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 GetAllFunctionsResponse:
   """
   Attributes:

http://git-wip-us.apache.org/repos/asf/hive/blob/7df62023/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb b/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
index f943f2d..a7b1e86 100644
--- a/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
+++ b/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
@@ -2358,10 +2358,12 @@ class PutFileMetadataRequest
   include ::Thrift::Struct, ::Thrift::Struct_Union
   FILEIDS = 1
   METADATA = 2
+  TYPE = 3
 
   FIELDS = {
     FILEIDS => {:type => ::Thrift::Types::LIST, :name => 'fileIds', :element => {:type => ::Thrift::Types::I64}},
-    METADATA => {:type => ::Thrift::Types::LIST, :name => 'metadata', :element => {:type => ::Thrift::Types::STRING, :binary => true}}
+    METADATA => {:type => ::Thrift::Types::LIST, :name => 'metadata', :element => {:type => ::Thrift::Types::STRING, :binary => true}},
+    TYPE => {:type => ::Thrift::Types::I32, :name => 'type', :optional => true, :enum_class => ::FileMetadataExprType}
   }
 
   def struct_fields; FIELDS; end
@@ -2369,6 +2371,9 @@ class PutFileMetadataRequest
   def validate
     raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field fileIds is unset!') unless @fileIds
     raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field metadata is unset!') unless @metadata
+    unless @type.nil? || ::FileMetadataExprType::VALID_VALUES.include?(@type)
+      raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Invalid value of field type!')
+    end
   end
 
   ::Thrift::Struct.generate_accessors self
@@ -2406,6 +2411,47 @@ class ClearFileMetadataRequest
   ::Thrift::Struct.generate_accessors self
 end
 
+class CacheFileMetadataResult
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  ISSUPPORTED = 1
+
+  FIELDS = {
+    ISSUPPORTED => {:type => ::Thrift::Types::BOOL, :name => 'isSupported'}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field isSupported is unset!') if @isSupported.nil?
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
+class CacheFileMetadataRequest
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  DBNAME = 1
+  TBLNAME = 2
+  PARTNAME = 3
+  ISALLPARTS = 4
+
+  FIELDS = {
+    DBNAME => {:type => ::Thrift::Types::STRING, :name => 'dbName'},
+    TBLNAME => {:type => ::Thrift::Types::STRING, :name => 'tblName'},
+    PARTNAME => {:type => ::Thrift::Types::STRING, :name => 'partName', :optional => true},
+    ISALLPARTS => {:type => ::Thrift::Types::BOOL, :name => 'isAllParts', :optional => true}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field dbName is unset!') unless @dbName
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field tblName is unset!') unless @tblName
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
 class GetAllFunctionsResponse
   include ::Thrift::Struct, ::Thrift::Struct_Union
   FUNCTIONS = 1

http://git-wip-us.apache.org/repos/asf/hive/blob/7df62023/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb b/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
index 5fe54b5..a90a180 100644
--- a/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
+++ b/metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
@@ -2199,6 +2199,21 @@ module ThriftHiveMetastore
       raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'clear_file_metadata failed: unknown result')
     end
 
+    def cache_file_metadata(req)
+      send_cache_file_metadata(req)
+      return recv_cache_file_metadata()
+    end
+
+    def send_cache_file_metadata(req)
+      send_message('cache_file_metadata', Cache_file_metadata_args, :req => req)
+    end
+
+    def recv_cache_file_metadata()
+      result = receive_message(Cache_file_metadata_result)
+      return result.success unless result.success.nil?
+      raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'cache_file_metadata failed: unknown result')
+    end
+
   end
 
   class Processor < ::FacebookService::Processor 
@@ -3853,6 +3868,13 @@ module ThriftHiveMetastore
       write_result(result, oprot, 'clear_file_metadata', seqid)
     end
 
+    def process_cache_file_metadata(seqid, iprot, oprot)
+      args = read_args(iprot, Cache_file_metadata_args)
+      result = Cache_file_metadata_result.new()
+      result.success = @handler.cache_file_metadata(args.req)
+      write_result(result, oprot, 'cache_file_metadata', seqid)
+    end
+
   end
 
   # HELPER FUNCTIONS AND STRUCTURES
@@ -8831,5 +8853,37 @@ module ThriftHiveMetastore
     ::Thrift::Struct.generate_accessors self
   end
 
+  class Cache_file_metadata_args
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    REQ = 1
+
+    FIELDS = {
+      REQ => {:type => ::Thrift::Types::STRUCT, :name => 'req', :class => ::CacheFileMetadataRequest}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Cache_file_metadata_result
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    SUCCESS = 0
+
+    FIELDS = {
+      SUCCESS => {:type => ::Thrift::Types::STRUCT, :name => 'success', :class => ::CacheFileMetadataResult}
+    }
+
+    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/7df62023/metastore/src/java/org/apache/hadoop/hive/metastore/FileFormatProxy.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/FileFormatProxy.java b/metastore/src/java/org/apache/hadoop/hive/metastore/FileFormatProxy.java
new file mode 100644
index 0000000..ec0be2b
--- /dev/null
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/FileFormatProxy.java
@@ -0,0 +1,64 @@
+/**
+ * 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 java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+
+/**
+ * Same as PartitionExpressionProxy, but for file format specific methods for metadata cache.
+ */
+public interface FileFormatProxy {
+
+  /**
+   * Applies SARG to file metadata, and produces some result for this file.
+   * @param sarg SARG
+   * @param byteBuffer File metadata from metastore cache.
+   * @return The result to return to client for this file, or null if file is eliminated.
+   * @throws IOException
+   */
+  ByteBuffer applySargToMetadata(SearchArgument sarg, ByteBuffer byteBuffer) throws IOException;
+
+  /**
+   * @param fs The filesystem of the file.
+   * @param path The file path.
+   * @param addedVals Output parameter; additional column values for columns returned by
+   *                  getAddedColumnsToCache to cache in MS.
+   * @return The ORC file metadata for a given file.
+   */
+  ByteBuffer getMetadataToCache(
+      FileSystem fs, Path path, ByteBuffer[] addedVals) throws IOException;
+
+  /**
+   * @return Additional column names to cache in MS for this format.
+   */
+  ByteBuffer[] getAddedColumnsToCache();
+
+  /**
+   * @param metadata File metadatas.
+   * @return Additional values for columns returned by getAddedColumnsToCache to cache in MS
+   *         for respective metadatas.
+   */
+  ByteBuffer[][] getAddedValuesToCache(List<ByteBuffer> metadata);
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7df62023/metastore/src/java/org/apache/hadoop/hive/metastore/FileMetadataHandler.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/FileMetadataHandler.java b/metastore/src/java/org/apache/hadoop/hive/metastore/FileMetadataHandler.java
index 7c3525a..bd4e188 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/FileMetadataHandler.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/FileMetadataHandler.java
@@ -22,9 +22,89 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.List;
 
-public interface FileMetadataHandler {
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.metastore.api.FileMetadataExprType;
+import org.apache.hadoop.hive.metastore.hbase.MetadataStore;
 
-  void getFileMetadataByExpr(List<Long> fileIds, byte[] expr,
+/**
+ * The base implementation of a file metadata handler for a specific file type.
+ * There are currently two classes for each file type (of 1), this one, which is very simple due
+ * to the fact that it just calls the proxy class for most calls; and the proxy class, that
+ * contains the actual implementation that depends on some stuff in QL (for ORC).
+ */
+public abstract class FileMetadataHandler {
+  static final Log LOG = LogFactory.getLog(FileMetadataHandler.class);
+
+  private Configuration conf;
+  private PartitionExpressionProxy expressionProxy;
+  private FileFormatProxy fileFormatProxy;
+  private MetadataStore store;
+
+  /**
+   * Same as RawStore.getFileMetadataByExpr.
+   */
+  public abstract void getFileMetadataByExpr(List<Long> fileIds, byte[] expr,
       ByteBuffer[] metadatas, ByteBuffer[] results, boolean[] eliminated) throws IOException;
 
+  protected abstract FileMetadataExprType getType();
+
+  protected PartitionExpressionProxy getExpressionProxy() {
+    return expressionProxy;
+  }
+
+  protected FileFormatProxy getFileFormatProxy() {
+    return fileFormatProxy;
+  }
+
+  protected MetadataStore getStore() {
+    return store;
+  }
+
+  /**
+   * Configures the handler. Called once before use.
+   * @param conf Config.
+   * @param expressionProxy Expression proxy to access ql stuff.
+   * @param store Storage interface to manipulate the metadata.
+   */
+  public void configure(
+      Configuration conf, PartitionExpressionProxy expressionProxy, MetadataStore store) {
+    this.conf = conf;
+    this.expressionProxy = expressionProxy;
+    this.store = store;
+    this.fileFormatProxy = expressionProxy.getFileFormatProxy(getType());
+  }
+
+  /**
+   * Caches the file metadata for a particular file.
+   * @param fileId File id.
+   * @param fs The filesystem of the file.
+   * @param path Path to the file.
+   */
+  public void cacheFileMetadata(long fileId, FileSystem fs, Path path)
+      throws IOException, InterruptedException {
+    // ORC is in ql, so we cannot do anything here. For now, all the logic is in the proxy.
+    ByteBuffer[] cols = fileFormatProxy.getAddedColumnsToCache();
+    ByteBuffer[] vals = (cols == null) ? null : new ByteBuffer[cols.length];
+    ByteBuffer metadata = fileFormatProxy.getMetadataToCache(fs, path, vals);
+    LOG.info("Caching file metadata for " + path + ", size " + metadata.remaining());
+    store.storeFileMetadata(fileId, metadata, cols, vals);
+  }
+
+  /**
+   * @return the added column names to be cached in metastore with the metadata for this type.
+   */
+  public ByteBuffer[] createAddedCols() {
+    return fileFormatProxy.getAddedColumnsToCache();
+  }
+
+  /**
+   * @return the values for the added columns returned by createAddedCols for respective metadatas.
+   */
+  public ByteBuffer[][] createAddedColVals(List<ByteBuffer> metadata) {
+    return fileFormatProxy.getAddedValuesToCache(metadata);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/7df62023/metastore/src/java/org/apache/hadoop/hive/metastore/FileMetadataManager.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/FileMetadataManager.java b/metastore/src/java/org/apache/hadoop/hive/metastore/FileMetadataManager.java
new file mode 100644
index 0000000..9b43328
--- /dev/null
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/FileMetadataManager.java
@@ -0,0 +1,129 @@
+/**
+ * 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.hadoop.fs.LocatedFileStatus;
+
+import org.apache.hadoop.fs.RemoteIterator;
+
+import java.util.LinkedList;
+
+import java.util.Queue;
+
+import java.util.ArrayList;
+
+import java.util.List;
+
+import com.google.common.collect.Lists;
+
+import java.io.IOException;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.io.HdfsUtils;
+import org.apache.hadoop.hive.metastore.HiveMetaStore.ThreadLocalRawStore;
+import org.apache.hadoop.hive.metastore.api.FileMetadataExprType;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.shims.HadoopShims;
+import org.apache.hadoop.hive.shims.ShimLoader;
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+public class FileMetadataManager {
+  private static final Log LOG = LogFactory.getLog(FileMetadataManager.class);
+  private static final HadoopShims SHIMS = ShimLoader.getHadoopShims();
+
+  private final ThreadLocalRawStore tlms;
+  private final ExecutorService threadPool;
+  private final HiveConf conf;
+
+  private final class CacheUpdateRequest implements Callable<Void> {
+    FileMetadataExprType type;
+    String location;
+
+    public CacheUpdateRequest(FileMetadataExprType type, String location) {
+      this.type = type;
+      this.location = location;
+    }
+
+    @Override
+    public Void call() throws Exception {
+      try {
+        cacheMetadata(type, location);
+      } catch (InterruptedException ex) {
+        Thread.currentThread().interrupt();
+      } catch (Exception ex) {
+        // Nobody can see this exception on the threadpool; just log it.
+        LOG.error("Failed to cache file metadata in background for " + type + ", " + location, ex);
+      }
+      return null;
+    }
+  }
+
+  public FileMetadataManager(ThreadLocalRawStore tlms, HiveConf conf) {
+    this.tlms = tlms;
+    this.conf = conf;
+    int numThreads = HiveConf.getIntVar(conf, ConfVars.METASTORE_HBASE_FILE_METADATA_THREADS);
+    this.threadPool = Executors.newFixedThreadPool(numThreads,
+        new ThreadFactoryBuilder().setNameFormat("File-Metadata-%d").setDaemon(true).build());
+  }
+
+  public void queueCacheMetadata(String location, FileMetadataExprType type) {
+    threadPool.submit(new CacheUpdateRequest(type, location));
+  }
+
+  private void cacheMetadata(FileMetadataExprType type, String location)
+      throws MetaException, IOException, InterruptedException {
+    Path path = new Path(location);
+    FileSystem fs = path.getFileSystem(conf);
+    List<Path> files;
+    if (!fs.isDirectory(path)) {
+      files = Lists.newArrayList(path);
+    } else {
+      files = new ArrayList<>();
+      RemoteIterator<LocatedFileStatus> iter = fs.listFiles(path, true);
+      while (iter.hasNext()) {
+        // TODO: use fileId right from the list after HDFS-7878; or get dfs client and do it
+        LocatedFileStatus lfs = iter.next();
+        if (lfs.isDirectory()) continue;
+        files.add(lfs.getPath());
+      }
+    }
+    for (Path file : files) {
+      long fileId;
+      try {
+        fileId = SHIMS.getFileId(fs, Path.getPathWithoutSchemeAndAuthority(file).toString());
+      } catch (UnsupportedOperationException ex) {
+        LOG.error("Cannot cache file metadata for " + location + "; "
+            + fs.getClass().getCanonicalName() + " does not support fileId");
+        return;
+      }
+      LOG.info("Caching file metadata for " + file + " (file ID " + fileId + ")");
+      file = HdfsUtils.getFileIdPath(fs, file, fileId);
+      tlms.getMS().getFileMetadataHandler(type).cacheFileMetadata(fileId, fs, file);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/7df62023/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
index 0940fd7..81d7128 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@ -78,6 +78,7 @@ import org.apache.hadoop.hive.metastore.events.PreEventContext;
 import org.apache.hadoop.hive.metastore.events.PreLoadPartitionDoneEvent;
 import org.apache.hadoop.hive.metastore.events.PreReadDatabaseEvent;
 import org.apache.hadoop.hive.metastore.events.PreReadTableEvent;
+import org.apache.hadoop.hive.metastore.filemeta.OrcFileMetadataHandler;
 import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
 import org.apache.hadoop.hive.metastore.txn.TxnHandler;
 import org.apache.hadoop.hive.serde2.Deserializer;
@@ -196,13 +197,25 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     }
   }
 
-  public static class HMSHandler extends FacebookBase implements IHMSHandler {
+  /**
+   * An ugly interface because everything about this file is ugly. RawStore is threadlocal so this
+   * thread-local disease propagates everywhere, and FileMetadataManager cannot just get a RawStore
+   * or handlers to use; it will need to have this method to make thread-local handlers and a
+   * thread-local RawStore.
+   */
+  public interface ThreadLocalRawStore {
+    RawStore getMS() throws MetaException;
+  }
+
+  public static class HMSHandler extends FacebookBase implements IHMSHandler, ThreadLocalRawStore {
     public static final Logger LOG = HiveMetaStore.LOG;
     private String rawStoreClassName;
     private final HiveConf hiveConf; // stores datastore (jpox) properties,
                                      // right now they come from jpox.properties
 
     private static String currentUrl;
+    private FileMetadataManager fileMetadataManager;
+    private PartitionExpressionProxy expressionProxy;
 
     //For Metrics
     private int initDatabaseCount, initTableCount, initPartCount;
@@ -444,6 +457,9 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         Timer cleaner = new Timer("Metastore Events Cleaner Thread", true);
         cleaner.schedule(new EventCleanerTask(this), cleanFreq, cleanFreq);
       }
+
+      expressionProxy = PartFilterExprUtil.createExpressionProxy(hiveConf);
+      fileMetadataManager = new FileMetadataManager((ThreadLocalRawStore)this, hiveConf);
     }
 
     private String addPrefix(String s) {
@@ -510,6 +526,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
      */
     @InterfaceAudience.LimitedPrivate({"HCATALOG"})
     @InterfaceStability.Evolving
+    @Override
     public RawStore getMS() throws MetaException {
       RawStore ms = threadLocalMS.get();
       if (ms == null) {
@@ -1527,9 +1544,9 @@ public class HiveMetaStore extends ThriftHiveMetastore {
 
       boolean trashEnabled = false;
       try {
-	trashEnabled = 0 < hiveConf.getFloat("fs.trash.interval", -1);
+  trashEnabled = 0 < hiveConf.getFloat("fs.trash.interval", -1);
       } catch(NumberFormatException ex) {
-	// nothing to do
+  // nothing to do
       }
 
       if (trashEnabled) {
@@ -5763,7 +5780,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     public PutFileMetadataResult put_file_metadata(PutFileMetadataRequest req) throws TException {
       RawStore ms = getMS();
       if (ms.isFileMetadataSupported()) {
-        ms.putFileMetadata(req.getFileIds(), req.getMetadata());
+        ms.putFileMetadata(req.getFileIds(), req.getMetadata(), req.getType());
       }
       return new PutFileMetadataResult();
     }
@@ -5771,10 +5788,109 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     @Override
     public ClearFileMetadataResult clear_file_metadata(ClearFileMetadataRequest req)
         throws TException {
-      getMS().putFileMetadata(req.getFileIds(), null);
+      getMS().putFileMetadata(req.getFileIds(), null, null);
       return new ClearFileMetadataResult();
     }
 
+    @Override
+    public CacheFileMetadataResult cache_file_metadata(
+        CacheFileMetadataRequest req) throws TException {
+      RawStore ms = getMS();
+      if (!ms.isFileMetadataSupported()) {
+        return new CacheFileMetadataResult(false);
+      }
+      String dbName = req.getDbName(), tblName = req.getTblName(),
+          partName = req.isSetPartName() ? req.getPartName() : null;
+      boolean isAllPart = req.isSetIsAllParts() && req.isIsAllParts();
+      ms.openTransaction();
+      boolean success = false;
+      try {
+        Table tbl = ms.getTable(dbName, tblName);
+        if (tbl == null) {
+          throw new NoSuchObjectException(dbName + "." + tblName + " not found");
+        }
+        boolean isPartitioned = tbl.isSetPartitionKeys() && tbl.getPartitionKeysSize() > 0;
+        String tableInputFormat = tbl.isSetSd() ? tbl.getSd().getInputFormat() : null;
+        if (!isPartitioned) {
+          if (partName != null || isAllPart) {
+            throw new MetaException("Table is not partitioned");
+          }
+          if (!tbl.isSetSd() || !tbl.getSd().isSetLocation()) {
+            throw new MetaException(
+                "Table does not have storage location; this operation is not supported on views");
+          }
+          FileMetadataExprType type = expressionProxy.getMetadataType(tableInputFormat);
+          if (type == null) {
+            throw new MetaException("The operation is not supported for " + tableInputFormat);
+          }
+          fileMetadataManager.queueCacheMetadata(tbl.getSd().getLocation(), type);
+          success = true;
+        } else {
+          List<String> partNames = null;
+          if (partName != null) {
+            partNames = Lists.newArrayList(partName);
+          } else if (isAllPart) {
+            partNames = ms.listPartitionNames(dbName, tblName, (short)-1);
+          } else {
+            throw new MetaException("Table is partitioned");
+          }
+          int batchSize = HiveConf.getIntVar(
+              hiveConf, ConfVars.METASTORE_BATCH_RETRIEVE_OBJECTS_MAX);
+          int index = 0;
+          int successCount = 0, failCount = 0;
+          HashSet<String> failFormats = null;
+          while (index < partNames.size()) {
+            int currentBatchSize = Math.min(batchSize, partNames.size() - index);
+            List<String> nameBatch = partNames.subList(index, index + currentBatchSize);
+            index += currentBatchSize;
+            List<Partition> parts = ms.getPartitionsByNames(dbName, tblName, nameBatch);
+            for (Partition part : parts) {
+              if (!part.isSetSd() || !part.getSd().isSetLocation()) {
+                throw new MetaException("Partition does not have storage location;" +
+                    " this operation is not supported on views");
+              }
+              String inputFormat = part.getSd().isSetInputFormat()
+                  ? part.getSd().getInputFormat() : tableInputFormat;
+              FileMetadataExprType type = expressionProxy.getMetadataType(inputFormat);
+              if (type == null) {
+                ++failCount;
+                if (failFormats == null) {
+                  failFormats = new HashSet<>();
+                }
+                failFormats.add(inputFormat);
+              } else {
+                ++successCount;
+                fileMetadataManager.queueCacheMetadata(part.getSd().getLocation(), type);
+              }
+            }
+          }
+          success = true; // Regardless of the following exception
+          if (failCount > 0) {
+            String errorMsg = "The operation failed for " + failCount + " partitions and "
+                + "succeeded for " + successCount + " partitions; unsupported formats: ";
+            boolean isFirst = true;
+            for (String s : failFormats) {
+              if (!isFirst) {
+                errorMsg += ", ";
+              }
+              isFirst = false;
+              errorMsg += s;
+            }
+            throw new MetaException(errorMsg);
+          }
+        }
+      } finally {
+        if (success) {
+          if (!ms.commitTransaction()) {
+            throw new MetaException("Failed to commit");
+          }
+        } else {
+          ms.rollbackTransaction();
+        }
+      }
+      return new CacheFileMetadataResult(true);
+    }
+
     @VisibleForTesting
     public void updateMetrics() throws MetaException {
       initTableCount = getMS().getTableCount();
@@ -6284,4 +6400,18 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         ".  Root Cause: ", ex);
     }
   }
+
+  public static Map<FileMetadataExprType, FileMetadataHandler> createHandlerMap() {
+    Map<FileMetadataExprType, FileMetadataHandler> fmHandlers = new HashMap<>();
+    for (FileMetadataExprType v : FileMetadataExprType.values()) {
+      switch (v) {
+      case ORC_SARG:
+        fmHandlers.put(v, new OrcFileMetadataHandler());
+        break;
+      default:
+        throw new AssertionError("Unsupported type " + v);
+      }
+    }
+    return fmHandlers;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/7df62023/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
index 178796d..a17c6d8 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
@@ -63,6 +63,8 @@ import org.apache.hadoop.hive.metastore.api.AddPartitionsRequest;
 import org.apache.hadoop.hive.metastore.api.AddPartitionsResult;
 import org.apache.hadoop.hive.metastore.api.AggrStats;
 import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
+import org.apache.hadoop.hive.metastore.api.CacheFileMetadataRequest;
+import org.apache.hadoop.hive.metastore.api.CacheFileMetadataResult;
 import org.apache.hadoop.hive.metastore.api.CheckLockRequest;
 import org.apache.hadoop.hive.metastore.api.ClearFileMetadataRequest;
 import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
@@ -2261,4 +2263,19 @@ public class HiveMetaStoreClient implements IMetaStoreClient {
   public boolean isSameConfObj(HiveConf c) {
     return conf == c;
   }
+
+  @Override
+  public boolean cacheFileMetadata(
+      String dbName, String tableName, String partName, boolean allParts) throws TException {
+    CacheFileMetadataRequest req = new CacheFileMetadataRequest();
+    req.setDbName(dbName);
+    req.setTblName(tableName);
+    if (partName != null) {
+      req.setPartName(partName);
+    } else {
+      req.setIsAllParts(allParts);
+    }
+    CacheFileMetadataResult result = client.cache_file_metadata(req);
+    return result.isIsSupported();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/7df62023/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java b/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
index aa96f77..25e0d38 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
@@ -1509,4 +1509,7 @@ public interface IMetaStoreClient {
   void putFileMetadata(List<Long> fileIds, List<ByteBuffer> metadata) throws TException;
 
   boolean isSameConfObj(HiveConf c);
+
+  boolean cacheFileMetadata(String dbName, String tableName, String partName,
+      boolean allParts) throws TException;
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/7df62023/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java b/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java
index 23068f8..432f7d0 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java
@@ -1516,10 +1516,12 @@ public class MetaStoreUtils {
     return listeners;
   }
 
-  public static Class<?> getClass(String rawStoreClassName)
+  @SuppressWarnings("unchecked")
+  public static Class<? extends RawStore> getClass(String rawStoreClassName)
       throws MetaException {
     try {
-      return Class.forName(rawStoreClassName, true, JavaUtils.getClassLoader());
+      return (Class<? extends RawStore>)
+          Class.forName(rawStoreClassName, true, JavaUtils.getClassLoader());
     } catch (ClassNotFoundException e) {
       throw new MetaException(rawStoreClassName + " class not found");
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/7df62023/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
index abfe2b8..05d5b57 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
@@ -7756,18 +7756,19 @@ public class ObjectStore implements RawStore, Configurable {
   }
 
   @Override
-  public ByteBuffer[] getFileMetadata(List<Long> fileIds) {
-    throw new UnsupportedOperationException();
+  public boolean isFileMetadataSupported() {
+    return false;
   }
 
   @Override
-  public void putFileMetadata(List<Long> fileIds, List<ByteBuffer> metadata) {
+  public ByteBuffer[] getFileMetadata(List<Long> fileIds) {
     throw new UnsupportedOperationException();
   }
 
   @Override
-  public boolean isFileMetadataSupported() {
-    return false;
+  public void putFileMetadata(
+      List<Long> fileIds, List<ByteBuffer> metadata, FileMetadataExprType type) {
+    throw new UnsupportedOperationException();
   }
 
   @Override
@@ -7776,6 +7777,11 @@ public class ObjectStore implements RawStore, Configurable {
     throw new UnsupportedOperationException();
   }
 
+  @Override
+  public FileMetadataHandler getFileMetadataHandler(FileMetadataExprType type) {
+    throw new UnsupportedOperationException();
+  }
+
   /**
    * Removed cached classloaders from DataNucleus
    * DataNucleus caches classloaders in NucleusContext.
@@ -7803,5 +7809,4 @@ public class ObjectStore implements RawStore, Configurable {
       }
     }
   }
-
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/7df62023/metastore/src/java/org/apache/hadoop/hive/metastore/PartitionExpressionProxy.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/PartitionExpressionProxy.java b/metastore/src/java/org/apache/hadoop/hive/metastore/PartitionExpressionProxy.java
index ed59829..8d6dc12 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/PartitionExpressionProxy.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/PartitionExpressionProxy.java
@@ -18,17 +18,17 @@
 
 package org.apache.hadoop.hive.metastore;
 
-import java.io.IOException;
-import java.nio.ByteBuffer;
 import java.util.List;
 
+import org.apache.hadoop.hive.metastore.api.FileMetadataExprType;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
 import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
 
 /**
- * The proxy interface that metastore uses to manipulate and apply
- * serialized filter expressions coming from client.
+ * The proxy interface that metastore uses for variety of QL operations (metastore can't depend
+ * on QL because QL depends on metastore; creating metastore-client module would be a proper way
+ * to solve this problem).
  */
 public interface PartitionExpressionProxy {
 
@@ -48,24 +48,28 @@ public interface PartitionExpressionProxy {
    * @param partitionNames Partition names; the list is modified in place.
    * @return Whether there were any unknown partitions preserved in the name list.
    */
-  public boolean filterPartitionsByExpr(List<String> partColumnNames,
+  boolean filterPartitionsByExpr(List<String> partColumnNames,
       List<PrimitiveTypeInfo> partColumnTypeInfos, byte[] expr,
       String defaultPartitionName, List<String> partitionNames) throws MetaException;
 
   /**
-   * Creates SARG from serialized representation.
-   * @param expr SARG, serialized as Kryo.
-   * @return SARG.
+   * Determines the file metadata type from input format of the source table or partition.
+   * @param inputFormat Input format name.
+   * @return The file metadata type.
    */
-  public SearchArgument createSarg(byte[] expr);
+  FileMetadataExprType getMetadataType(String inputFormat);
 
   /**
-   * Applies SARG to file metadata, and produces some result for this file.
-   * @param sarg SARG
-   * @param byteBuffer File metadata from metastore cache.
-   * @return The result to return to client for this file, or null if file is eliminated.
-   * @throws IOException
+   * Gets a separate proxy that can be used to call file-format-specific methods.
+   * @param type The file metadata type.
+   * @return The proxy.
+   */
+  FileFormatProxy getFileFormatProxy(FileMetadataExprType type);
+
+  /**
+   * Creates SARG from serialized representation.
+   * @param expr SARG, serialized as Kryo.
+   * @return SARG.
    */
-  public ByteBuffer applySargToFileMetadata(SearchArgument sarg, ByteBuffer byteBuffer)
-      throws IOException;
+  SearchArgument createSarg(byte[] expr);
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/7df62023/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java
index e118a3b..d228f24 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java
@@ -610,8 +610,10 @@ public interface RawStore extends Configurable {
   /**
    * @param fileIds List of file IDs from the filesystem.
    * @param metadata Metadata buffers corresponding to fileIds in the list.
+   * @param type The type; determines the class that can do additiona processing for metadata.
    */
-  void putFileMetadata(List<Long> fileIds, List<ByteBuffer> metadata) throws MetaException;
+  void putFileMetadata(List<Long> fileIds, List<ByteBuffer> metadata,
+      FileMetadataExprType type) throws MetaException;
 
   /**
    * @return Whether file metadata cache is supported by this implementation.
@@ -635,6 +637,9 @@ public interface RawStore extends Configurable {
       ByteBuffer[] metadatas, ByteBuffer[] exprResults, boolean[] eliminated)
           throws MetaException;
 
+  /** Gets file metadata handler for the corresponding type. */
+  FileMetadataHandler getFileMetadataHandler(FileMetadataExprType type);
+
   /**
    * Gets total number of tables.
    */

http://git-wip-us.apache.org/repos/asf/hive/blob/7df62023/metastore/src/java/org/apache/hadoop/hive/metastore/filemeta/OrcFileMetadataHandler.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/filemeta/OrcFileMetadataHandler.java b/metastore/src/java/org/apache/hadoop/hive/metastore/filemeta/OrcFileMetadataHandler.java
index 14189da..1b388aa 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/filemeta/OrcFileMetadataHandler.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/filemeta/OrcFileMetadataHandler.java
@@ -22,42 +22,34 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.List;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.metastore.FileMetadataHandler;
-import org.apache.hadoop.hive.metastore.PartitionExpressionProxy;
-import org.apache.hadoop.hive.metastore.hbase.HBaseReadWrite;
+import org.apache.hadoop.hive.metastore.api.FileMetadataExprType;
 import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
 
-public class OrcFileMetadataHandler implements FileMetadataHandler {
-  private final Configuration conf;
-  private final PartitionExpressionProxy expressionProxy;
-  private final HBaseReadWrite hbase;
+public class OrcFileMetadataHandler extends FileMetadataHandler {
 
-  public OrcFileMetadataHandler(Configuration conf,
-      PartitionExpressionProxy expressionProxy, HBaseReadWrite hbase) {
-    this.conf = conf;
-    this.expressionProxy = expressionProxy;
-    this.hbase = hbase;
+  @Override
+  protected FileMetadataExprType getType() {
+    return FileMetadataExprType.ORC_SARG;
   }
 
   @Override
   public void getFileMetadataByExpr(List<Long> fileIds, byte[] expr,
       ByteBuffer[] metadatas, ByteBuffer[] results, boolean[] eliminated) throws IOException {
-    SearchArgument sarg = expressionProxy.createSarg(expr);
+    SearchArgument sarg = getExpressionProxy().createSarg(expr);
     // For now, don't push anything into HBase, nor store anything special in HBase
     if (metadatas == null) {
       // null means don't return metadata; we'd need the array anyway for now.
       metadatas = new ByteBuffer[results.length];
     }
-    hbase.getFileMetadata(fileIds, metadatas);
+    getStore().getFileMetadata(fileIds, metadatas);
     for (int i = 0; i < metadatas.length;  ++i) {
       if (metadatas[i] == null) continue;
-      ByteBuffer result = expressionProxy.applySargToFileMetadata(sarg, metadatas[i]);
+      ByteBuffer result = getFileFormatProxy().applySargToMetadata(sarg, metadatas[i]);
       eliminated[i] = (result == null);
       if (!eliminated[i]) {
         results[i] = result;
       }
     }
   }
-
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/7df62023/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseReadWrite.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseReadWrite.java b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseReadWrite.java
index 287394e..81f1324 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseReadWrite.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseReadWrite.java
@@ -86,7 +86,7 @@ import java.util.Set;
 /**
  * Class to manage storing object in and reading them from HBase.
  */
-public class HBaseReadWrite {
+public class HBaseReadWrite implements MetadataStore {
 
   final static String AGGR_STATS_TABLE = "HBMS_AGGR_STATS";
   final static String DB_TABLE = "HBMS_DBS";
@@ -2154,6 +2154,7 @@ public class HBaseReadWrite {
    * @param fileIds file ID list.
    * @return Serialized file metadata.
    */
+  @Override
   public void getFileMetadata(List<Long> fileIds, ByteBuffer[] result) throws IOException {
     byte[][] keys = new byte[fileIds.size()][];
     for (int i = 0; i < fileIds.size(); ++i) {
@@ -2164,15 +2165,64 @@ public class HBaseReadWrite {
 
   /**
    * @param fileIds file ID list.
-   * @param metadata Serialized file metadata.
-   */
-  void storeFileMetadata(List<Long> fileIds, List<ByteBuffer> metadata)
+   * @param metadataBuffers Serialized file metadatas, one per file ID.
+   * @param addedCols The column names for additional columns created by file-format-specific
+   *                  metadata handler, to be stored in the cache.
+   * @param addedVals The values for addedCols; one value per file ID per added column.
+   */
+  @Override
+  public void storeFileMetadata(List<Long> fileIds, List<ByteBuffer> metadataBuffers,
+      ByteBuffer[] addedCols, ByteBuffer[][] addedVals)
       throws IOException, InterruptedException {
     byte[][] keys = new byte[fileIds.size()][];
     for (int i = 0; i < fileIds.size(); ++i) {
       keys[i] = HBaseUtils.makeLongKey(fileIds.get(i));
     }
-    multiModify(FILE_METADATA_TABLE, keys, CATALOG_CF, CATALOG_COL, metadata);
+    // HBase APIs are weird. To supply bytebuffer value, you have to also have bytebuffer
+    // column name, but not column family. So there. Perhaps we should add these to constants too.
+    ByteBuffer colNameBuf = ByteBuffer.wrap(CATALOG_COL);
+    @SuppressWarnings("deprecation")
+    HTableInterface htab = conn.getHBaseTable(FILE_METADATA_TABLE);
+    List<Row> actions = new ArrayList<>(keys.length);
+    for (int keyIx = 0; keyIx < keys.length; ++keyIx) {
+      ByteBuffer value = (metadataBuffers != null) ? metadataBuffers.get(keyIx) : null;
+      ByteBuffer[] av = addedVals == null ? null : addedVals[keyIx];
+      if (value == null) {
+        actions.add(new Delete(keys[keyIx]));
+        assert av == null;
+      } else {
+        Put p = new Put(keys[keyIx]);
+        p.addColumn(CATALOG_CF, colNameBuf, HConstants.LATEST_TIMESTAMP, value);
+        if (av != null) {
+          assert av.length == addedCols.length;
+          for (int colIx = 0; colIx < addedCols.length; ++colIx) {
+            p.addColumn(STATS_CF, addedCols[colIx], HConstants.LATEST_TIMESTAMP, av[colIx]);
+          }
+        }
+        actions.add(p);
+      }
+    }
+    Object[] results = new Object[keys.length];
+    htab.batch(actions, results);
+    // TODO: should we check results array? we don't care about partial results
+    conn.flush(htab);
+  }
+
+  @Override
+  public void storeFileMetadata(long fileId, ByteBuffer metadata,
+      ByteBuffer[] addedCols, ByteBuffer[] addedVals) throws IOException, InterruptedException {
+    @SuppressWarnings("deprecation")
+    HTableInterface htab = conn.getHBaseTable(FILE_METADATA_TABLE);
+    Put p = new Put(HBaseUtils.makeLongKey(fileId));
+    p.addColumn(CATALOG_CF, ByteBuffer.wrap(CATALOG_COL), HConstants.LATEST_TIMESTAMP, metadata);
+    assert (addedCols == null && addedVals == null) || (addedCols.length == addedVals.length);
+    if (addedCols != null) {
+      for (int i = 0; i < addedCols.length; ++i) {
+        p.addColumn(STATS_CF, addedCols[i], HConstants.LATEST_TIMESTAMP, addedVals[i]);
+      }
+    }
+    htab.put(p);
+    conn.flush(htab);
   }
 
   /**********************************************************************************************

http://git-wip-us.apache.org/repos/asf/hive/blob/7df62023/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
index b9509ab..fcf983f 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
@@ -67,7 +67,6 @@ 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.filemeta.OrcFileMetadataHandler;
 import org.apache.hadoop.hive.metastore.hbase.HBaseFilterPlanUtil.PlanResult;
 import org.apache.hadoop.hive.metastore.hbase.HBaseFilterPlanUtil.ScanPlan;
 import org.apache.hadoop.hive.metastore.parser.ExpressionTree;
@@ -80,6 +79,7 @@ import org.apache.thrift.TException;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.LinkedList;
@@ -99,7 +99,7 @@ public class HBaseStore implements RawStore {
   private Configuration conf;
   private int txnNestLevel = 0;
   private PartitionExpressionProxy expressionProxy = null;
-  private Map<FileMetadataExprType, FileMetadataHandler> fmHandlers = new HashMap<>();
+  private Map<FileMetadataExprType, FileMetadataHandler> fmHandlers;
 
   public HBaseStore() {
   }
@@ -2308,26 +2308,32 @@ public class HBaseStore implements RawStore {
     // setConf is being called with new configuration object (though that
     // is not expected to happen, doing it just for safety)
     // TODO: why not re-intialize HBaseReadWrite?
-    if (expressionProxy == null || conf != configuration) {
-      expressionProxy = PartFilterExprUtil.createExpressionProxy(configuration);
-    }
+    Configuration oldConf = conf;
     conf = configuration;
-    createFileMetadataHandlers();
+    if (expressionProxy != null && conf != oldConf) {
+      LOG.warn("Unexpected setConf when we were already configured");
+    }
+    if (expressionProxy == null || conf != oldConf) {
+      expressionProxy = PartFilterExprUtil.createExpressionProxy(conf);
+    }
+    if (conf != oldConf) {
+      fmHandlers = HiveMetaStore.createHandlerMap();
+      configureFileMetadataHandlers(fmHandlers.values());
+    }
   }
 
-  private void createFileMetadataHandlers() {
-    for (FileMetadataExprType v : FileMetadataExprType.values()) {
-      switch (v) {
-      case ORC_SARG:
-        fmHandlers.put(v, new OrcFileMetadataHandler(conf, expressionProxy, getHBase()));
-        break;
-      default:
-        throw new AssertionError("Unsupported type " + v);
-      }
+  private void configureFileMetadataHandlers(Collection<FileMetadataHandler> fmHandlers) {
+    for (FileMetadataHandler fmh : fmHandlers) {
+      fmh.configure(conf, expressionProxy, getHBase());
     }
   }
 
   @Override
+  public FileMetadataHandler getFileMetadataHandler(FileMetadataExprType type) {
+    return fmHandlers.get(type);
+  }
+
+  @Override
   public Configuration getConf() {
     return conf;
 
@@ -2476,11 +2482,21 @@ public class HBaseStore implements RawStore {
   }
 
   @Override
-  public void putFileMetadata(List<Long> fileIds, List<ByteBuffer> metadata) throws MetaException {
+  public void putFileMetadata(List<Long> fileIds, List<ByteBuffer> metadata,
+      FileMetadataExprType type) throws MetaException {
     openTransaction();
     boolean commit = false;
     try {
-      getHBase().storeFileMetadata(fileIds, metadata);
+      ByteBuffer[][] addedVals = null;
+      ByteBuffer[] addedCols = null;
+      if (type != null) {
+        FileMetadataHandler fmh = fmHandlers.get(type);
+        addedCols = fmh.createAddedCols();
+        if (addedCols != null) {
+          addedVals = fmh.createAddedColVals(metadata);
+        }
+      }
+      getHBase().storeFileMetadata(fileIds, metadata, addedCols, addedVals);
       commit = true;
     } catch (IOException | InterruptedException e) {
       LOG.error("Unable to store file metadata", e);

http://git-wip-us.apache.org/repos/asf/hive/blob/7df62023/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/MetadataStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/MetadataStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/MetadataStore.java
new file mode 100644
index 0000000..0382e8a
--- /dev/null
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/MetadataStore.java
@@ -0,0 +1,52 @@
+/**
+ * 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.hbase;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+
+public interface MetadataStore {
+  /**
+   * @param fileIds file ID list.
+   * @param result The ref parameter, used to return the serialized file metadata.
+   */
+  void getFileMetadata(List<Long> fileIds, ByteBuffer[] result) throws IOException;
+
+  /**
+   * @param fileIds file ID list.
+   * @param metadataBuffers Serialized file metadata, one per file ID.
+   * @param addedCols The column names for additional columns created by file-format-specific
+   *                  metadata handler, to be stored in the cache.
+   * @param addedVals The values for addedCols; one value per file ID per added column.
+   */
+  void storeFileMetadata(List<Long> fileIds, List<ByteBuffer> metadataBuffers,
+      ByteBuffer[] addedCols, ByteBuffer[][] addedVals) throws IOException, InterruptedException;
+
+  /**
+   * @param fileId The file ID.
+   * @param metadataBuffers Serialized file metadata.
+   * @param addedCols The column names for additional columns created by file-format-specific
+   *                  metadata handler, to be stored in the cache.
+   * @param addedVals The values for addedCols; one value per added column.
+   */
+  void storeFileMetadata(long fileId, ByteBuffer metadata, ByteBuffer[] addedCols,
+      ByteBuffer[] addedVals) throws IOException, InterruptedException;
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7df62023/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java b/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
index c1156b3..477a3be 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
@@ -775,7 +775,8 @@ public class DummyRawStoreControlledCommit implements RawStore, Configurable {
   }
 
   @Override
-  public void putFileMetadata(List<Long> fileIds, List<ByteBuffer> metadata) {
+  public void putFileMetadata(
+      List<Long> fileIds, List<ByteBuffer> metadata, FileMetadataExprType type) {
   }
 
   @Override
@@ -803,4 +804,9 @@ public class DummyRawStoreControlledCommit implements RawStore, Configurable {
   public int getDatabaseCount() throws MetaException {
     return objectStore.getDatabaseCount();
   }
+
+  @Override
+  public FileMetadataHandler getFileMetadataHandler(FileMetadataExprType type) {
+    return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/7df62023/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java b/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
index bf20e99..ecabd5d 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
@@ -792,7 +792,8 @@ public class DummyRawStoreForJdoConnection implements RawStore {
   }
 
   @Override
-  public void putFileMetadata(List<Long> fileIds, List<ByteBuffer> metadata) {
+  public void putFileMetadata(
+      List<Long> fileIds, List<ByteBuffer> metadata, FileMetadataExprType type) {
   }
 
   @Override
@@ -819,6 +820,11 @@ public class DummyRawStoreForJdoConnection implements RawStore {
   public int getDatabaseCount() throws MetaException {
     return 0;
   }
+
+  @Override
+  public FileMetadataHandler getFileMetadataHandler(FileMetadataExprType type) {
+    return null;
+  }
 }
 
 

http://git-wip-us.apache.org/repos/asf/hive/blob/7df62023/metastore/src/test/org/apache/hadoop/hive/metastore/MockPartitionExpressionForMetastore.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/MockPartitionExpressionForMetastore.java b/metastore/src/test/org/apache/hadoop/hive/metastore/MockPartitionExpressionForMetastore.java
index d72bf76..032af69 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/MockPartitionExpressionForMetastore.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/MockPartitionExpressionForMetastore.java
@@ -18,11 +18,11 @@
 
 package org.apache.hadoop.hive.metastore;
 
+import org.apache.hadoop.hive.metastore.api.FileMetadataExprType;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
 import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
 
-import java.nio.ByteBuffer;
 import java.util.List;
 
 /**
@@ -42,12 +42,17 @@ public class MockPartitionExpressionForMetastore implements PartitionExpressionP
   }
 
   @Override
+  public FileMetadataExprType getMetadataType(String inputFormat) {
+    return null;
+  }
+
+  @Override
   public SearchArgument createSarg(byte[] expr) {
     return null;
   }
 
   @Override
-  public ByteBuffer applySargToFileMetadata(SearchArgument sarg, ByteBuffer byteBuffer) {
+  public FileFormatProxy getFileFormatProxy(FileMetadataExprType type) {
     return null;
   }
 }


[5/7] hive git commit: HIVE-12075 : add analyze command to explictly cache file metadata in HBase metastore (Sergey Shelukhin, reviewed by Alan Gates)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/7df62023/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
index cea9000..62a2007 100644
--- a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
+++ b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
@@ -153,6 +153,7 @@ class ThriftHiveMetastoreIf : virtual public  ::facebook::fb303::FacebookService
   virtual void get_file_metadata(GetFileMetadataResult& _return, const GetFileMetadataRequest& req) = 0;
   virtual void put_file_metadata(PutFileMetadataResult& _return, const PutFileMetadataRequest& req) = 0;
   virtual void clear_file_metadata(ClearFileMetadataResult& _return, const ClearFileMetadataRequest& req) = 0;
+  virtual void cache_file_metadata(CacheFileMetadataResult& _return, const CacheFileMetadataRequest& req) = 0;
 };
 
 class ThriftHiveMetastoreIfFactory : virtual public  ::facebook::fb303::FacebookServiceIfFactory {
@@ -598,6 +599,9 @@ class ThriftHiveMetastoreNull : virtual public ThriftHiveMetastoreIf , virtual p
   void clear_file_metadata(ClearFileMetadataResult& /* _return */, const ClearFileMetadataRequest& /* req */) {
     return;
   }
+  void cache_file_metadata(CacheFileMetadataResult& /* _return */, const CacheFileMetadataRequest& /* req */) {
+    return;
+  }
 };
 
 typedef struct _ThriftHiveMetastore_getMetaConf_args__isset {
@@ -17023,6 +17027,110 @@ class ThriftHiveMetastore_clear_file_metadata_presult {
 
 };
 
+typedef struct _ThriftHiveMetastore_cache_file_metadata_args__isset {
+  _ThriftHiveMetastore_cache_file_metadata_args__isset() : req(false) {}
+  bool req :1;
+} _ThriftHiveMetastore_cache_file_metadata_args__isset;
+
+class ThriftHiveMetastore_cache_file_metadata_args {
+ public:
+
+  ThriftHiveMetastore_cache_file_metadata_args(const ThriftHiveMetastore_cache_file_metadata_args&);
+  ThriftHiveMetastore_cache_file_metadata_args& operator=(const ThriftHiveMetastore_cache_file_metadata_args&);
+  ThriftHiveMetastore_cache_file_metadata_args() {
+  }
+
+  virtual ~ThriftHiveMetastore_cache_file_metadata_args() throw();
+  CacheFileMetadataRequest req;
+
+  _ThriftHiveMetastore_cache_file_metadata_args__isset __isset;
+
+  void __set_req(const CacheFileMetadataRequest& val);
+
+  bool operator == (const ThriftHiveMetastore_cache_file_metadata_args & rhs) const
+  {
+    if (!(req == rhs.req))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_cache_file_metadata_args &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_cache_file_metadata_args & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+
+class ThriftHiveMetastore_cache_file_metadata_pargs {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_cache_file_metadata_pargs() throw();
+  const CacheFileMetadataRequest* req;
+
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_cache_file_metadata_result__isset {
+  _ThriftHiveMetastore_cache_file_metadata_result__isset() : success(false) {}
+  bool success :1;
+} _ThriftHiveMetastore_cache_file_metadata_result__isset;
+
+class ThriftHiveMetastore_cache_file_metadata_result {
+ public:
+
+  ThriftHiveMetastore_cache_file_metadata_result(const ThriftHiveMetastore_cache_file_metadata_result&);
+  ThriftHiveMetastore_cache_file_metadata_result& operator=(const ThriftHiveMetastore_cache_file_metadata_result&);
+  ThriftHiveMetastore_cache_file_metadata_result() {
+  }
+
+  virtual ~ThriftHiveMetastore_cache_file_metadata_result() throw();
+  CacheFileMetadataResult success;
+
+  _ThriftHiveMetastore_cache_file_metadata_result__isset __isset;
+
+  void __set_success(const CacheFileMetadataResult& val);
+
+  bool operator == (const ThriftHiveMetastore_cache_file_metadata_result & rhs) const
+  {
+    if (!(success == rhs.success))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_cache_file_metadata_result &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_cache_file_metadata_result & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_cache_file_metadata_presult__isset {
+  _ThriftHiveMetastore_cache_file_metadata_presult__isset() : success(false) {}
+  bool success :1;
+} _ThriftHiveMetastore_cache_file_metadata_presult__isset;
+
+class ThriftHiveMetastore_cache_file_metadata_presult {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_cache_file_metadata_presult() throw();
+  CacheFileMetadataResult* success;
+
+  _ThriftHiveMetastore_cache_file_metadata_presult__isset __isset;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+
+};
+
 class ThriftHiveMetastoreClient : virtual public ThriftHiveMetastoreIf, public  ::facebook::fb303::FacebookServiceClient {
  public:
   ThriftHiveMetastoreClient(boost::shared_ptr< ::apache::thrift::protocol::TProtocol> prot) :
@@ -17427,6 +17535,9 @@ class ThriftHiveMetastoreClient : virtual public ThriftHiveMetastoreIf, public
   void clear_file_metadata(ClearFileMetadataResult& _return, const ClearFileMetadataRequest& req);
   void send_clear_file_metadata(const ClearFileMetadataRequest& req);
   void recv_clear_file_metadata(ClearFileMetadataResult& _return);
+  void cache_file_metadata(CacheFileMetadataResult& _return, const CacheFileMetadataRequest& req);
+  void send_cache_file_metadata(const CacheFileMetadataRequest& req);
+  void recv_cache_file_metadata(CacheFileMetadataResult& _return);
 };
 
 class ThriftHiveMetastoreProcessor : public  ::facebook::fb303::FacebookServiceProcessor {
@@ -17568,6 +17679,7 @@ class ThriftHiveMetastoreProcessor : public  ::facebook::fb303::FacebookServiceP
   void process_get_file_metadata(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_put_file_metadata(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_clear_file_metadata(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
+  void process_cache_file_metadata(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
  public:
   ThriftHiveMetastoreProcessor(boost::shared_ptr<ThriftHiveMetastoreIf> iface) :
      ::facebook::fb303::FacebookServiceProcessor(iface),
@@ -17703,6 +17815,7 @@ class ThriftHiveMetastoreProcessor : public  ::facebook::fb303::FacebookServiceP
     processMap_["get_file_metadata"] = &ThriftHiveMetastoreProcessor::process_get_file_metadata;
     processMap_["put_file_metadata"] = &ThriftHiveMetastoreProcessor::process_put_file_metadata;
     processMap_["clear_file_metadata"] = &ThriftHiveMetastoreProcessor::process_clear_file_metadata;
+    processMap_["cache_file_metadata"] = &ThriftHiveMetastoreProcessor::process_cache_file_metadata;
   }
 
   virtual ~ThriftHiveMetastoreProcessor() {}
@@ -18995,6 +19108,16 @@ class ThriftHiveMetastoreMultiface : virtual public ThriftHiveMetastoreIf, publi
     return;
   }
 
+  void cache_file_metadata(CacheFileMetadataResult& _return, const CacheFileMetadataRequest& req) {
+    size_t sz = ifaces_.size();
+    size_t i = 0;
+    for (; i < (sz - 1); ++i) {
+      ifaces_[i]->cache_file_metadata(_return, req);
+    }
+    ifaces_[i]->cache_file_metadata(_return, req);
+    return;
+  }
+
 };
 
 // The 'concurrent' client is a thread safe client that correctly handles
@@ -19404,6 +19527,9 @@ class ThriftHiveMetastoreConcurrentClient : virtual public ThriftHiveMetastoreIf
   void clear_file_metadata(ClearFileMetadataResult& _return, const ClearFileMetadataRequest& req);
   int32_t send_clear_file_metadata(const ClearFileMetadataRequest& req);
   void recv_clear_file_metadata(ClearFileMetadataResult& _return, const int32_t seqid);
+  void cache_file_metadata(CacheFileMetadataResult& _return, const CacheFileMetadataRequest& req);
+  int32_t send_cache_file_metadata(const CacheFileMetadataRequest& req);
+  void recv_cache_file_metadata(CacheFileMetadataResult& _return, const int32_t seqid);
 };
 
 #ifdef _WIN32

http://git-wip-us.apache.org/repos/asf/hive/blob/7df62023/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
index c0d9401..383a0d2 100644
--- a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
+++ b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
@@ -677,6 +677,11 @@ class ThriftHiveMetastoreHandler : virtual public ThriftHiveMetastoreIf {
     printf("clear_file_metadata\n");
   }
 
+  void cache_file_metadata(CacheFileMetadataResult& _return, const CacheFileMetadataRequest& req) {
+    // Your implementation goes here
+    printf("cache_file_metadata\n");
+  }
+
 };
 
 int main(int argc, char **argv) {

http://git-wip-us.apache.org/repos/asf/hive/blob/7df62023/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
index ee28d0d..d997e33 100644
--- a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
+++ b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
@@ -14744,6 +14744,11 @@ void PutFileMetadataRequest::__set_metadata(const std::vector<std::string> & val
   this->metadata = val;
 }
 
+void PutFileMetadataRequest::__set_type(const FileMetadataExprType::type val) {
+  this->type = val;
+__isset.type = true;
+}
+
 uint32_t PutFileMetadataRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
@@ -14807,6 +14812,16 @@ uint32_t PutFileMetadataRequest::read(::apache::thrift::protocol::TProtocol* ipr
           xfer += iprot->skip(ftype);
         }
         break;
+      case 3:
+        if (ftype == ::apache::thrift::protocol::T_I32) {
+          int32_t ecast660;
+          xfer += iprot->readI32(ecast660);
+          this->type = (FileMetadataExprType::type)ecast660;
+          this->__isset.type = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -14831,10 +14846,10 @@ uint32_t PutFileMetadataRequest::write(::apache::thrift::protocol::TProtocol* op
   xfer += oprot->writeFieldBegin("fileIds", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I64, static_cast<uint32_t>(this->fileIds.size()));
-    std::vector<int64_t> ::const_iterator _iter660;
-    for (_iter660 = this->fileIds.begin(); _iter660 != this->fileIds.end(); ++_iter660)
+    std::vector<int64_t> ::const_iterator _iter661;
+    for (_iter661 = this->fileIds.begin(); _iter661 != this->fileIds.end(); ++_iter661)
     {
-      xfer += oprot->writeI64((*_iter660));
+      xfer += oprot->writeI64((*_iter661));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14843,15 +14858,20 @@ uint32_t PutFileMetadataRequest::write(::apache::thrift::protocol::TProtocol* op
   xfer += oprot->writeFieldBegin("metadata", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->metadata.size()));
-    std::vector<std::string> ::const_iterator _iter661;
-    for (_iter661 = this->metadata.begin(); _iter661 != this->metadata.end(); ++_iter661)
+    std::vector<std::string> ::const_iterator _iter662;
+    for (_iter662 = this->metadata.begin(); _iter662 != this->metadata.end(); ++_iter662)
     {
-      xfer += oprot->writeBinary((*_iter661));
+      xfer += oprot->writeBinary((*_iter662));
     }
     xfer += oprot->writeListEnd();
   }
   xfer += oprot->writeFieldEnd();
 
+  if (this->__isset.type) {
+    xfer += oprot->writeFieldBegin("type", ::apache::thrift::protocol::T_I32, 3);
+    xfer += oprot->writeI32((int32_t)this->type);
+    xfer += oprot->writeFieldEnd();
+  }
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
   return xfer;
@@ -14861,15 +14881,21 @@ void swap(PutFileMetadataRequest &a, PutFileMetadataRequest &b) {
   using ::std::swap;
   swap(a.fileIds, b.fileIds);
   swap(a.metadata, b.metadata);
+  swap(a.type, b.type);
+  swap(a.__isset, b.__isset);
 }
 
-PutFileMetadataRequest::PutFileMetadataRequest(const PutFileMetadataRequest& other662) {
-  fileIds = other662.fileIds;
-  metadata = other662.metadata;
-}
-PutFileMetadataRequest& PutFileMetadataRequest::operator=(const PutFileMetadataRequest& other663) {
+PutFileMetadataRequest::PutFileMetadataRequest(const PutFileMetadataRequest& other663) {
   fileIds = other663.fileIds;
   metadata = other663.metadata;
+  type = other663.type;
+  __isset = other663.__isset;
+}
+PutFileMetadataRequest& PutFileMetadataRequest::operator=(const PutFileMetadataRequest& other664) {
+  fileIds = other664.fileIds;
+  metadata = other664.metadata;
+  type = other664.type;
+  __isset = other664.__isset;
   return *this;
 }
 void PutFileMetadataRequest::printTo(std::ostream& out) const {
@@ -14877,6 +14903,7 @@ void PutFileMetadataRequest::printTo(std::ostream& out) const {
   out << "PutFileMetadataRequest(";
   out << "fileIds=" << to_string(fileIds);
   out << ", " << "metadata=" << to_string(metadata);
+  out << ", " << "type="; (__isset.type ? (out << to_string(type)) : (out << "<null>"));
   out << ")";
 }
 
@@ -14929,11 +14956,11 @@ void swap(ClearFileMetadataResult &a, ClearFileMetadataResult &b) {
   (void) b;
 }
 
-ClearFileMetadataResult::ClearFileMetadataResult(const ClearFileMetadataResult& other664) {
-  (void) other664;
-}
-ClearFileMetadataResult& ClearFileMetadataResult::operator=(const ClearFileMetadataResult& other665) {
+ClearFileMetadataResult::ClearFileMetadataResult(const ClearFileMetadataResult& other665) {
   (void) other665;
+}
+ClearFileMetadataResult& ClearFileMetadataResult::operator=(const ClearFileMetadataResult& other666) {
+  (void) other666;
   return *this;
 }
 void ClearFileMetadataResult::printTo(std::ostream& out) const {
@@ -14977,14 +15004,14 @@ uint32_t ClearFileMetadataRequest::read(::apache::thrift::protocol::TProtocol* i
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->fileIds.clear();
-            uint32_t _size666;
-            ::apache::thrift::protocol::TType _etype669;
-            xfer += iprot->readListBegin(_etype669, _size666);
-            this->fileIds.resize(_size666);
-            uint32_t _i670;
-            for (_i670 = 0; _i670 < _size666; ++_i670)
+            uint32_t _size667;
+            ::apache::thrift::protocol::TType _etype670;
+            xfer += iprot->readListBegin(_etype670, _size667);
+            this->fileIds.resize(_size667);
+            uint32_t _i671;
+            for (_i671 = 0; _i671 < _size667; ++_i671)
             {
-              xfer += iprot->readI64(this->fileIds[_i670]);
+              xfer += iprot->readI64(this->fileIds[_i671]);
             }
             xfer += iprot->readListEnd();
           }
@@ -15015,10 +15042,10 @@ uint32_t ClearFileMetadataRequest::write(::apache::thrift::protocol::TProtocol*
   xfer += oprot->writeFieldBegin("fileIds", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I64, static_cast<uint32_t>(this->fileIds.size()));
-    std::vector<int64_t> ::const_iterator _iter671;
-    for (_iter671 = this->fileIds.begin(); _iter671 != this->fileIds.end(); ++_iter671)
+    std::vector<int64_t> ::const_iterator _iter672;
+    for (_iter672 = this->fileIds.begin(); _iter672 != this->fileIds.end(); ++_iter672)
     {
-      xfer += oprot->writeI64((*_iter671));
+      xfer += oprot->writeI64((*_iter672));
     }
     xfer += oprot->writeListEnd();
   }
@@ -15034,11 +15061,11 @@ void swap(ClearFileMetadataRequest &a, ClearFileMetadataRequest &b) {
   swap(a.fileIds, b.fileIds);
 }
 
-ClearFileMetadataRequest::ClearFileMetadataRequest(const ClearFileMetadataRequest& other672) {
-  fileIds = other672.fileIds;
-}
-ClearFileMetadataRequest& ClearFileMetadataRequest::operator=(const ClearFileMetadataRequest& other673) {
+ClearFileMetadataRequest::ClearFileMetadataRequest(const ClearFileMetadataRequest& other673) {
   fileIds = other673.fileIds;
+}
+ClearFileMetadataRequest& ClearFileMetadataRequest::operator=(const ClearFileMetadataRequest& other674) {
+  fileIds = other674.fileIds;
   return *this;
 }
 void ClearFileMetadataRequest::printTo(std::ostream& out) const {
@@ -15049,6 +15076,248 @@ void ClearFileMetadataRequest::printTo(std::ostream& out) const {
 }
 
 
+CacheFileMetadataResult::~CacheFileMetadataResult() throw() {
+}
+
+
+void CacheFileMetadataResult::__set_isSupported(const bool val) {
+  this->isSupported = val;
+}
+
+uint32_t CacheFileMetadataResult::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
+  uint32_t xfer = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TType ftype;
+  int16_t fid;
+
+  xfer += iprot->readStructBegin(fname);
+
+  using ::apache::thrift::protocol::TProtocolException;
+
+  bool isset_isSupported = false;
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    switch (fid)
+    {
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_BOOL) {
+          xfer += iprot->readBool(this->isSupported);
+          isset_isSupported = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  if (!isset_isSupported)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  return xfer;
+}
+
+uint32_t CacheFileMetadataResult::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+  xfer += oprot->writeStructBegin("CacheFileMetadataResult");
+
+  xfer += oprot->writeFieldBegin("isSupported", ::apache::thrift::protocol::T_BOOL, 1);
+  xfer += oprot->writeBool(this->isSupported);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+void swap(CacheFileMetadataResult &a, CacheFileMetadataResult &b) {
+  using ::std::swap;
+  swap(a.isSupported, b.isSupported);
+}
+
+CacheFileMetadataResult::CacheFileMetadataResult(const CacheFileMetadataResult& other675) {
+  isSupported = other675.isSupported;
+}
+CacheFileMetadataResult& CacheFileMetadataResult::operator=(const CacheFileMetadataResult& other676) {
+  isSupported = other676.isSupported;
+  return *this;
+}
+void CacheFileMetadataResult::printTo(std::ostream& out) const {
+  using ::apache::thrift::to_string;
+  out << "CacheFileMetadataResult(";
+  out << "isSupported=" << to_string(isSupported);
+  out << ")";
+}
+
+
+CacheFileMetadataRequest::~CacheFileMetadataRequest() throw() {
+}
+
+
+void CacheFileMetadataRequest::__set_dbName(const std::string& val) {
+  this->dbName = val;
+}
+
+void CacheFileMetadataRequest::__set_tblName(const std::string& val) {
+  this->tblName = val;
+}
+
+void CacheFileMetadataRequest::__set_partName(const std::string& val) {
+  this->partName = val;
+__isset.partName = true;
+}
+
+void CacheFileMetadataRequest::__set_isAllParts(const bool val) {
+  this->isAllParts = val;
+__isset.isAllParts = true;
+}
+
+uint32_t CacheFileMetadataRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
+  uint32_t xfer = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TType ftype;
+  int16_t fid;
+
+  xfer += iprot->readStructBegin(fname);
+
+  using ::apache::thrift::protocol::TProtocolException;
+
+  bool isset_dbName = false;
+  bool isset_tblName = false;
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    switch (fid)
+    {
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->dbName);
+          isset_dbName = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 2:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->tblName);
+          isset_tblName = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 3:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->partName);
+          this->__isset.partName = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 4:
+        if (ftype == ::apache::thrift::protocol::T_BOOL) {
+          xfer += iprot->readBool(this->isAllParts);
+          this->__isset.isAllParts = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  if (!isset_dbName)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  if (!isset_tblName)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  return xfer;
+}
+
+uint32_t CacheFileMetadataRequest::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+  xfer += oprot->writeStructBegin("CacheFileMetadataRequest");
+
+  xfer += oprot->writeFieldBegin("dbName", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeString(this->dbName);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("tblName", ::apache::thrift::protocol::T_STRING, 2);
+  xfer += oprot->writeString(this->tblName);
+  xfer += oprot->writeFieldEnd();
+
+  if (this->__isset.partName) {
+    xfer += oprot->writeFieldBegin("partName", ::apache::thrift::protocol::T_STRING, 3);
+    xfer += oprot->writeString(this->partName);
+    xfer += oprot->writeFieldEnd();
+  }
+  if (this->__isset.isAllParts) {
+    xfer += oprot->writeFieldBegin("isAllParts", ::apache::thrift::protocol::T_BOOL, 4);
+    xfer += oprot->writeBool(this->isAllParts);
+    xfer += oprot->writeFieldEnd();
+  }
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+void swap(CacheFileMetadataRequest &a, CacheFileMetadataRequest &b) {
+  using ::std::swap;
+  swap(a.dbName, b.dbName);
+  swap(a.tblName, b.tblName);
+  swap(a.partName, b.partName);
+  swap(a.isAllParts, b.isAllParts);
+  swap(a.__isset, b.__isset);
+}
+
+CacheFileMetadataRequest::CacheFileMetadataRequest(const CacheFileMetadataRequest& other677) {
+  dbName = other677.dbName;
+  tblName = other677.tblName;
+  partName = other677.partName;
+  isAllParts = other677.isAllParts;
+  __isset = other677.__isset;
+}
+CacheFileMetadataRequest& CacheFileMetadataRequest::operator=(const CacheFileMetadataRequest& other678) {
+  dbName = other678.dbName;
+  tblName = other678.tblName;
+  partName = other678.partName;
+  isAllParts = other678.isAllParts;
+  __isset = other678.__isset;
+  return *this;
+}
+void CacheFileMetadataRequest::printTo(std::ostream& out) const {
+  using ::apache::thrift::to_string;
+  out << "CacheFileMetadataRequest(";
+  out << "dbName=" << to_string(dbName);
+  out << ", " << "tblName=" << to_string(tblName);
+  out << ", " << "partName="; (__isset.partName ? (out << to_string(partName)) : (out << "<null>"));
+  out << ", " << "isAllParts="; (__isset.isAllParts ? (out << to_string(isAllParts)) : (out << "<null>"));
+  out << ")";
+}
+
+
 GetAllFunctionsResponse::~GetAllFunctionsResponse() throw() {
 }
 
@@ -15083,14 +15352,14 @@ uint32_t GetAllFunctionsResponse::read(::apache::thrift::protocol::TProtocol* ip
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->functions.clear();
-            uint32_t _size674;
-            ::apache::thrift::protocol::TType _etype677;
-            xfer += iprot->readListBegin(_etype677, _size674);
-            this->functions.resize(_size674);
-            uint32_t _i678;
-            for (_i678 = 0; _i678 < _size674; ++_i678)
+            uint32_t _size679;
+            ::apache::thrift::protocol::TType _etype682;
+            xfer += iprot->readListBegin(_etype682, _size679);
+            this->functions.resize(_size679);
+            uint32_t _i683;
+            for (_i683 = 0; _i683 < _size679; ++_i683)
             {
-              xfer += this->functions[_i678].read(iprot);
+              xfer += this->functions[_i683].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -15120,10 +15389,10 @@ uint32_t GetAllFunctionsResponse::write(::apache::thrift::protocol::TProtocol* o
     xfer += oprot->writeFieldBegin("functions", ::apache::thrift::protocol::T_LIST, 1);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->functions.size()));
-      std::vector<Function> ::const_iterator _iter679;
-      for (_iter679 = this->functions.begin(); _iter679 != this->functions.end(); ++_iter679)
+      std::vector<Function> ::const_iterator _iter684;
+      for (_iter684 = this->functions.begin(); _iter684 != this->functions.end(); ++_iter684)
       {
-        xfer += (*_iter679).write(oprot);
+        xfer += (*_iter684).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -15140,13 +15409,13 @@ void swap(GetAllFunctionsResponse &a, GetAllFunctionsResponse &b) {
   swap(a.__isset, b.__isset);
 }
 
-GetAllFunctionsResponse::GetAllFunctionsResponse(const GetAllFunctionsResponse& other680) {
-  functions = other680.functions;
-  __isset = other680.__isset;
+GetAllFunctionsResponse::GetAllFunctionsResponse(const GetAllFunctionsResponse& other685) {
+  functions = other685.functions;
+  __isset = other685.__isset;
 }
-GetAllFunctionsResponse& GetAllFunctionsResponse::operator=(const GetAllFunctionsResponse& other681) {
-  functions = other681.functions;
-  __isset = other681.__isset;
+GetAllFunctionsResponse& GetAllFunctionsResponse::operator=(const GetAllFunctionsResponse& other686) {
+  functions = other686.functions;
+  __isset = other686.__isset;
   return *this;
 }
 void GetAllFunctionsResponse::printTo(std::ostream& out) const {
@@ -15288,19 +15557,19 @@ void swap(TableMeta &a, TableMeta &b) {
   swap(a.__isset, b.__isset);
 }
 
-TableMeta::TableMeta(const TableMeta& other682) {
-  dbName = other682.dbName;
-  tableName = other682.tableName;
-  tableType = other682.tableType;
-  comments = other682.comments;
-  __isset = other682.__isset;
+TableMeta::TableMeta(const TableMeta& other687) {
+  dbName = other687.dbName;
+  tableName = other687.tableName;
+  tableType = other687.tableType;
+  comments = other687.comments;
+  __isset = other687.__isset;
 }
-TableMeta& TableMeta::operator=(const TableMeta& other683) {
-  dbName = other683.dbName;
-  tableName = other683.tableName;
-  tableType = other683.tableType;
-  comments = other683.comments;
-  __isset = other683.__isset;
+TableMeta& TableMeta::operator=(const TableMeta& other688) {
+  dbName = other688.dbName;
+  tableName = other688.tableName;
+  tableType = other688.tableType;
+  comments = other688.comments;
+  __isset = other688.__isset;
   return *this;
 }
 void TableMeta::printTo(std::ostream& out) const {
@@ -15383,13 +15652,13 @@ void swap(MetaException &a, MetaException &b) {
   swap(a.__isset, b.__isset);
 }
 
-MetaException::MetaException(const MetaException& other684) : TException() {
-  message = other684.message;
-  __isset = other684.__isset;
+MetaException::MetaException(const MetaException& other689) : TException() {
+  message = other689.message;
+  __isset = other689.__isset;
 }
-MetaException& MetaException::operator=(const MetaException& other685) {
-  message = other685.message;
-  __isset = other685.__isset;
+MetaException& MetaException::operator=(const MetaException& other690) {
+  message = other690.message;
+  __isset = other690.__isset;
   return *this;
 }
 void MetaException::printTo(std::ostream& out) const {
@@ -15480,13 +15749,13 @@ void swap(UnknownTableException &a, UnknownTableException &b) {
   swap(a.__isset, b.__isset);
 }
 
-UnknownTableException::UnknownTableException(const UnknownTableException& other686) : TException() {
-  message = other686.message;
-  __isset = other686.__isset;
+UnknownTableException::UnknownTableException(const UnknownTableException& other691) : TException() {
+  message = other691.message;
+  __isset = other691.__isset;
 }
-UnknownTableException& UnknownTableException::operator=(const UnknownTableException& other687) {
-  message = other687.message;
-  __isset = other687.__isset;
+UnknownTableException& UnknownTableException::operator=(const UnknownTableException& other692) {
+  message = other692.message;
+  __isset = other692.__isset;
   return *this;
 }
 void UnknownTableException::printTo(std::ostream& out) const {
@@ -15577,13 +15846,13 @@ void swap(UnknownDBException &a, UnknownDBException &b) {
   swap(a.__isset, b.__isset);
 }
 
-UnknownDBException::UnknownDBException(const UnknownDBException& other688) : TException() {
-  message = other688.message;
-  __isset = other688.__isset;
+UnknownDBException::UnknownDBException(const UnknownDBException& other693) : TException() {
+  message = other693.message;
+  __isset = other693.__isset;
 }
-UnknownDBException& UnknownDBException::operator=(const UnknownDBException& other689) {
-  message = other689.message;
-  __isset = other689.__isset;
+UnknownDBException& UnknownDBException::operator=(const UnknownDBException& other694) {
+  message = other694.message;
+  __isset = other694.__isset;
   return *this;
 }
 void UnknownDBException::printTo(std::ostream& out) const {
@@ -15674,13 +15943,13 @@ void swap(AlreadyExistsException &a, AlreadyExistsException &b) {
   swap(a.__isset, b.__isset);
 }
 
-AlreadyExistsException::AlreadyExistsException(const AlreadyExistsException& other690) : TException() {
-  message = other690.message;
-  __isset = other690.__isset;
+AlreadyExistsException::AlreadyExistsException(const AlreadyExistsException& other695) : TException() {
+  message = other695.message;
+  __isset = other695.__isset;
 }
-AlreadyExistsException& AlreadyExistsException::operator=(const AlreadyExistsException& other691) {
-  message = other691.message;
-  __isset = other691.__isset;
+AlreadyExistsException& AlreadyExistsException::operator=(const AlreadyExistsException& other696) {
+  message = other696.message;
+  __isset = other696.__isset;
   return *this;
 }
 void AlreadyExistsException::printTo(std::ostream& out) const {
@@ -15771,13 +16040,13 @@ void swap(InvalidPartitionException &a, InvalidPartitionException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidPartitionException::InvalidPartitionException(const InvalidPartitionException& other692) : TException() {
-  message = other692.message;
-  __isset = other692.__isset;
+InvalidPartitionException::InvalidPartitionException(const InvalidPartitionException& other697) : TException() {
+  message = other697.message;
+  __isset = other697.__isset;
 }
-InvalidPartitionException& InvalidPartitionException::operator=(const InvalidPartitionException& other693) {
-  message = other693.message;
-  __isset = other693.__isset;
+InvalidPartitionException& InvalidPartitionException::operator=(const InvalidPartitionException& other698) {
+  message = other698.message;
+  __isset = other698.__isset;
   return *this;
 }
 void InvalidPartitionException::printTo(std::ostream& out) const {
@@ -15868,13 +16137,13 @@ void swap(UnknownPartitionException &a, UnknownPartitionException &b) {
   swap(a.__isset, b.__isset);
 }
 
-UnknownPartitionException::UnknownPartitionException(const UnknownPartitionException& other694) : TException() {
-  message = other694.message;
-  __isset = other694.__isset;
+UnknownPartitionException::UnknownPartitionException(const UnknownPartitionException& other699) : TException() {
+  message = other699.message;
+  __isset = other699.__isset;
 }
-UnknownPartitionException& UnknownPartitionException::operator=(const UnknownPartitionException& other695) {
-  message = other695.message;
-  __isset = other695.__isset;
+UnknownPartitionException& UnknownPartitionException::operator=(const UnknownPartitionException& other700) {
+  message = other700.message;
+  __isset = other700.__isset;
   return *this;
 }
 void UnknownPartitionException::printTo(std::ostream& out) const {
@@ -15965,13 +16234,13 @@ void swap(InvalidObjectException &a, InvalidObjectException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidObjectException::InvalidObjectException(const InvalidObjectException& other696) : TException() {
-  message = other696.message;
-  __isset = other696.__isset;
+InvalidObjectException::InvalidObjectException(const InvalidObjectException& other701) : TException() {
+  message = other701.message;
+  __isset = other701.__isset;
 }
-InvalidObjectException& InvalidObjectException::operator=(const InvalidObjectException& other697) {
-  message = other697.message;
-  __isset = other697.__isset;
+InvalidObjectException& InvalidObjectException::operator=(const InvalidObjectException& other702) {
+  message = other702.message;
+  __isset = other702.__isset;
   return *this;
 }
 void InvalidObjectException::printTo(std::ostream& out) const {
@@ -16062,13 +16331,13 @@ void swap(NoSuchObjectException &a, NoSuchObjectException &b) {
   swap(a.__isset, b.__isset);
 }
 
-NoSuchObjectException::NoSuchObjectException(const NoSuchObjectException& other698) : TException() {
-  message = other698.message;
-  __isset = other698.__isset;
+NoSuchObjectException::NoSuchObjectException(const NoSuchObjectException& other703) : TException() {
+  message = other703.message;
+  __isset = other703.__isset;
 }
-NoSuchObjectException& NoSuchObjectException::operator=(const NoSuchObjectException& other699) {
-  message = other699.message;
-  __isset = other699.__isset;
+NoSuchObjectException& NoSuchObjectException::operator=(const NoSuchObjectException& other704) {
+  message = other704.message;
+  __isset = other704.__isset;
   return *this;
 }
 void NoSuchObjectException::printTo(std::ostream& out) const {
@@ -16159,13 +16428,13 @@ void swap(IndexAlreadyExistsException &a, IndexAlreadyExistsException &b) {
   swap(a.__isset, b.__isset);
 }
 
-IndexAlreadyExistsException::IndexAlreadyExistsException(const IndexAlreadyExistsException& other700) : TException() {
-  message = other700.message;
-  __isset = other700.__isset;
+IndexAlreadyExistsException::IndexAlreadyExistsException(const IndexAlreadyExistsException& other705) : TException() {
+  message = other705.message;
+  __isset = other705.__isset;
 }
-IndexAlreadyExistsException& IndexAlreadyExistsException::operator=(const IndexAlreadyExistsException& other701) {
-  message = other701.message;
-  __isset = other701.__isset;
+IndexAlreadyExistsException& IndexAlreadyExistsException::operator=(const IndexAlreadyExistsException& other706) {
+  message = other706.message;
+  __isset = other706.__isset;
   return *this;
 }
 void IndexAlreadyExistsException::printTo(std::ostream& out) const {
@@ -16256,13 +16525,13 @@ void swap(InvalidOperationException &a, InvalidOperationException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidOperationException::InvalidOperationException(const InvalidOperationException& other702) : TException() {
-  message = other702.message;
-  __isset = other702.__isset;
+InvalidOperationException::InvalidOperationException(const InvalidOperationException& other707) : TException() {
+  message = other707.message;
+  __isset = other707.__isset;
 }
-InvalidOperationException& InvalidOperationException::operator=(const InvalidOperationException& other703) {
-  message = other703.message;
-  __isset = other703.__isset;
+InvalidOperationException& InvalidOperationException::operator=(const InvalidOperationException& other708) {
+  message = other708.message;
+  __isset = other708.__isset;
   return *this;
 }
 void InvalidOperationException::printTo(std::ostream& out) const {
@@ -16353,13 +16622,13 @@ void swap(ConfigValSecurityException &a, ConfigValSecurityException &b) {
   swap(a.__isset, b.__isset);
 }
 
-ConfigValSecurityException::ConfigValSecurityException(const ConfigValSecurityException& other704) : TException() {
-  message = other704.message;
-  __isset = other704.__isset;
+ConfigValSecurityException::ConfigValSecurityException(const ConfigValSecurityException& other709) : TException() {
+  message = other709.message;
+  __isset = other709.__isset;
 }
-ConfigValSecurityException& ConfigValSecurityException::operator=(const ConfigValSecurityException& other705) {
-  message = other705.message;
-  __isset = other705.__isset;
+ConfigValSecurityException& ConfigValSecurityException::operator=(const ConfigValSecurityException& other710) {
+  message = other710.message;
+  __isset = other710.__isset;
   return *this;
 }
 void ConfigValSecurityException::printTo(std::ostream& out) const {
@@ -16450,13 +16719,13 @@ void swap(InvalidInputException &a, InvalidInputException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidInputException::InvalidInputException(const InvalidInputException& other706) : TException() {
-  message = other706.message;
-  __isset = other706.__isset;
+InvalidInputException::InvalidInputException(const InvalidInputException& other711) : TException() {
+  message = other711.message;
+  __isset = other711.__isset;
 }
-InvalidInputException& InvalidInputException::operator=(const InvalidInputException& other707) {
-  message = other707.message;
-  __isset = other707.__isset;
+InvalidInputException& InvalidInputException::operator=(const InvalidInputException& other712) {
+  message = other712.message;
+  __isset = other712.__isset;
   return *this;
 }
 void InvalidInputException::printTo(std::ostream& out) const {
@@ -16547,13 +16816,13 @@ void swap(NoSuchTxnException &a, NoSuchTxnException &b) {
   swap(a.__isset, b.__isset);
 }
 
-NoSuchTxnException::NoSuchTxnException(const NoSuchTxnException& other708) : TException() {
-  message = other708.message;
-  __isset = other708.__isset;
+NoSuchTxnException::NoSuchTxnException(const NoSuchTxnException& other713) : TException() {
+  message = other713.message;
+  __isset = other713.__isset;
 }
-NoSuchTxnException& NoSuchTxnException::operator=(const NoSuchTxnException& other709) {
-  message = other709.message;
-  __isset = other709.__isset;
+NoSuchTxnException& NoSuchTxnException::operator=(const NoSuchTxnException& other714) {
+  message = other714.message;
+  __isset = other714.__isset;
   return *this;
 }
 void NoSuchTxnException::printTo(std::ostream& out) const {
@@ -16644,13 +16913,13 @@ void swap(TxnAbortedException &a, TxnAbortedException &b) {
   swap(a.__isset, b.__isset);
 }
 
-TxnAbortedException::TxnAbortedException(const TxnAbortedException& other710) : TException() {
-  message = other710.message;
-  __isset = other710.__isset;
+TxnAbortedException::TxnAbortedException(const TxnAbortedException& other715) : TException() {
+  message = other715.message;
+  __isset = other715.__isset;
 }
-TxnAbortedException& TxnAbortedException::operator=(const TxnAbortedException& other711) {
-  message = other711.message;
-  __isset = other711.__isset;
+TxnAbortedException& TxnAbortedException::operator=(const TxnAbortedException& other716) {
+  message = other716.message;
+  __isset = other716.__isset;
   return *this;
 }
 void TxnAbortedException::printTo(std::ostream& out) const {
@@ -16741,13 +17010,13 @@ void swap(TxnOpenException &a, TxnOpenException &b) {
   swap(a.__isset, b.__isset);
 }
 
-TxnOpenException::TxnOpenException(const TxnOpenException& other712) : TException() {
-  message = other712.message;
-  __isset = other712.__isset;
+TxnOpenException::TxnOpenException(const TxnOpenException& other717) : TException() {
+  message = other717.message;
+  __isset = other717.__isset;
 }
-TxnOpenException& TxnOpenException::operator=(const TxnOpenException& other713) {
-  message = other713.message;
-  __isset = other713.__isset;
+TxnOpenException& TxnOpenException::operator=(const TxnOpenException& other718) {
+  message = other718.message;
+  __isset = other718.__isset;
   return *this;
 }
 void TxnOpenException::printTo(std::ostream& out) const {
@@ -16838,13 +17107,13 @@ void swap(NoSuchLockException &a, NoSuchLockException &b) {
   swap(a.__isset, b.__isset);
 }
 
-NoSuchLockException::NoSuchLockException(const NoSuchLockException& other714) : TException() {
-  message = other714.message;
-  __isset = other714.__isset;
+NoSuchLockException::NoSuchLockException(const NoSuchLockException& other719) : TException() {
+  message = other719.message;
+  __isset = other719.__isset;
 }
-NoSuchLockException& NoSuchLockException::operator=(const NoSuchLockException& other715) {
-  message = other715.message;
-  __isset = other715.__isset;
+NoSuchLockException& NoSuchLockException::operator=(const NoSuchLockException& other720) {
+  message = other720.message;
+  __isset = other720.__isset;
   return *this;
 }
 void NoSuchLockException::printTo(std::ostream& out) const {

http://git-wip-us.apache.org/repos/asf/hive/blob/7df62023/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
index 05c288c..97d3362 100644
--- a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
+++ b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
@@ -347,6 +347,10 @@ class ClearFileMetadataResult;
 
 class ClearFileMetadataRequest;
 
+class CacheFileMetadataResult;
+
+class CacheFileMetadataRequest;
+
 class GetAllFunctionsResponse;
 
 class TableMeta;
@@ -5992,29 +5996,42 @@ inline std::ostream& operator<<(std::ostream& out, const PutFileMetadataResult&
   return out;
 }
 
+typedef struct _PutFileMetadataRequest__isset {
+  _PutFileMetadataRequest__isset() : type(false) {}
+  bool type :1;
+} _PutFileMetadataRequest__isset;
 
 class PutFileMetadataRequest {
  public:
 
   PutFileMetadataRequest(const PutFileMetadataRequest&);
   PutFileMetadataRequest& operator=(const PutFileMetadataRequest&);
-  PutFileMetadataRequest() {
+  PutFileMetadataRequest() : type((FileMetadataExprType::type)0) {
   }
 
   virtual ~PutFileMetadataRequest() throw();
   std::vector<int64_t>  fileIds;
   std::vector<std::string>  metadata;
+  FileMetadataExprType::type type;
+
+  _PutFileMetadataRequest__isset __isset;
 
   void __set_fileIds(const std::vector<int64_t> & val);
 
   void __set_metadata(const std::vector<std::string> & val);
 
+  void __set_type(const FileMetadataExprType::type val);
+
   bool operator == (const PutFileMetadataRequest & rhs) const
   {
     if (!(fileIds == rhs.fileIds))
       return false;
     if (!(metadata == rhs.metadata))
       return false;
+    if (__isset.type != rhs.__isset.type)
+      return false;
+    else if (__isset.type && !(type == rhs.type))
+      return false;
     return true;
   }
   bool operator != (const PutFileMetadataRequest &rhs) const {
@@ -6112,6 +6129,112 @@ inline std::ostream& operator<<(std::ostream& out, const ClearFileMetadataReques
   return out;
 }
 
+
+class CacheFileMetadataResult {
+ public:
+
+  CacheFileMetadataResult(const CacheFileMetadataResult&);
+  CacheFileMetadataResult& operator=(const CacheFileMetadataResult&);
+  CacheFileMetadataResult() : isSupported(0) {
+  }
+
+  virtual ~CacheFileMetadataResult() throw();
+  bool isSupported;
+
+  void __set_isSupported(const bool val);
+
+  bool operator == (const CacheFileMetadataResult & rhs) const
+  {
+    if (!(isSupported == rhs.isSupported))
+      return false;
+    return true;
+  }
+  bool operator != (const CacheFileMetadataResult &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const CacheFileMetadataResult & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  virtual void printTo(std::ostream& out) const;
+};
+
+void swap(CacheFileMetadataResult &a, CacheFileMetadataResult &b);
+
+inline std::ostream& operator<<(std::ostream& out, const CacheFileMetadataResult& obj)
+{
+  obj.printTo(out);
+  return out;
+}
+
+typedef struct _CacheFileMetadataRequest__isset {
+  _CacheFileMetadataRequest__isset() : partName(false), isAllParts(false) {}
+  bool partName :1;
+  bool isAllParts :1;
+} _CacheFileMetadataRequest__isset;
+
+class CacheFileMetadataRequest {
+ public:
+
+  CacheFileMetadataRequest(const CacheFileMetadataRequest&);
+  CacheFileMetadataRequest& operator=(const CacheFileMetadataRequest&);
+  CacheFileMetadataRequest() : dbName(), tblName(), partName(), isAllParts(0) {
+  }
+
+  virtual ~CacheFileMetadataRequest() throw();
+  std::string dbName;
+  std::string tblName;
+  std::string partName;
+  bool isAllParts;
+
+  _CacheFileMetadataRequest__isset __isset;
+
+  void __set_dbName(const std::string& val);
+
+  void __set_tblName(const std::string& val);
+
+  void __set_partName(const std::string& val);
+
+  void __set_isAllParts(const bool val);
+
+  bool operator == (const CacheFileMetadataRequest & rhs) const
+  {
+    if (!(dbName == rhs.dbName))
+      return false;
+    if (!(tblName == rhs.tblName))
+      return false;
+    if (__isset.partName != rhs.__isset.partName)
+      return false;
+    else if (__isset.partName && !(partName == rhs.partName))
+      return false;
+    if (__isset.isAllParts != rhs.__isset.isAllParts)
+      return false;
+    else if (__isset.isAllParts && !(isAllParts == rhs.isAllParts))
+      return false;
+    return true;
+  }
+  bool operator != (const CacheFileMetadataRequest &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const CacheFileMetadataRequest & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  virtual void printTo(std::ostream& out) const;
+};
+
+void swap(CacheFileMetadataRequest &a, CacheFileMetadataRequest &b);
+
+inline std::ostream& operator<<(std::ostream& out, const CacheFileMetadataRequest& obj)
+{
+  obj.printTo(out);
+  return out;
+}
+
 typedef struct _GetAllFunctionsResponse__isset {
   _GetAllFunctionsResponse__isset() : functions(false) {}
   bool functions :1;

http://git-wip-us.apache.org/repos/asf/hive/blob/7df62023/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CacheFileMetadataRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CacheFileMetadataRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CacheFileMetadataRequest.java
new file mode 100644
index 0000000..6b8ab11
--- /dev/null
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CacheFileMetadataRequest.java
@@ -0,0 +1,702 @@
+/**
+ * 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)")
+public class CacheFileMetadataRequest implements org.apache.thrift.TBase<CacheFileMetadataRequest, CacheFileMetadataRequest._Fields>, java.io.Serializable, Cloneable, Comparable<CacheFileMetadataRequest> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CacheFileMetadataRequest");
+
+  private static final org.apache.thrift.protocol.TField DB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("dbName", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField TBL_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tblName", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField PART_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("partName", org.apache.thrift.protocol.TType.STRING, (short)3);
+  private static final org.apache.thrift.protocol.TField IS_ALL_PARTS_FIELD_DESC = new org.apache.thrift.protocol.TField("isAllParts", org.apache.thrift.protocol.TType.BOOL, (short)4);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new CacheFileMetadataRequestStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new CacheFileMetadataRequestTupleSchemeFactory());
+  }
+
+  private String dbName; // required
+  private String tblName; // required
+  private String partName; // optional
+  private boolean isAllParts; // 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 {
+    DB_NAME((short)1, "dbName"),
+    TBL_NAME((short)2, "tblName"),
+    PART_NAME((short)3, "partName"),
+    IS_ALL_PARTS((short)4, "isAllParts");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // DB_NAME
+          return DB_NAME;
+        case 2: // TBL_NAME
+          return TBL_NAME;
+        case 3: // PART_NAME
+          return PART_NAME;
+        case 4: // IS_ALL_PARTS
+          return IS_ALL_PARTS;
+        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 __ISALLPARTS_ISSET_ID = 0;
+  private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.PART_NAME,_Fields.IS_ALL_PARTS};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.DB_NAME, new org.apache.thrift.meta_data.FieldMetaData("dbName", org.apache.thrift.TFieldRequirementType.REQUIRED,
+        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.REQUIRED,
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.PART_NAME, new org.apache.thrift.meta_data.FieldMetaData("partName", org.apache.thrift.TFieldRequirementType.OPTIONAL,
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.IS_ALL_PARTS, new org.apache.thrift.meta_data.FieldMetaData("isAllParts", org.apache.thrift.TFieldRequirementType.OPTIONAL,
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(CacheFileMetadataRequest.class, metaDataMap);
+  }
+
+  public CacheFileMetadataRequest() {
+  }
+
+  public CacheFileMetadataRequest(
+    String dbName,
+    String tblName)
+  {
+    this();
+    this.dbName = dbName;
+    this.tblName = tblName;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public CacheFileMetadataRequest(CacheFileMetadataRequest other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.isSetDbName()) {
+      this.dbName = other.dbName;
+    }
+    if (other.isSetTblName()) {
+      this.tblName = other.tblName;
+    }
+    if (other.isSetPartName()) {
+      this.partName = other.partName;
+    }
+    this.isAllParts = other.isAllParts;
+  }
+
+  public CacheFileMetadataRequest deepCopy() {
+    return new CacheFileMetadataRequest(this);
+  }
+
+  @Override
+  public void clear() {
+    this.dbName = null;
+    this.tblName = null;
+    this.partName = null;
+    setIsAllPartsIsSet(false);
+    this.isAllParts = false;
+  }
+
+  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 String getPartName() {
+    return this.partName;
+  }
+
+  public void setPartName(String partName) {
+    this.partName = partName;
+  }
+
+  public void unsetPartName() {
+    this.partName = null;
+  }
+
+  /** Returns true if field partName is set (has been assigned a value) and false otherwise */
+  public boolean isSetPartName() {
+    return this.partName != null;
+  }
+
+  public void setPartNameIsSet(boolean value) {
+    if (!value) {
+      this.partName = null;
+    }
+  }
+
+  public boolean isIsAllParts() {
+    return this.isAllParts;
+  }
+
+  public void setIsAllParts(boolean isAllParts) {
+    this.isAllParts = isAllParts;
+    setIsAllPartsIsSet(true);
+  }
+
+  public void unsetIsAllParts() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ISALLPARTS_ISSET_ID);
+  }
+
+  /** Returns true if field isAllParts is set (has been assigned a value) and false otherwise */
+  public boolean isSetIsAllParts() {
+    return EncodingUtils.testBit(__isset_bitfield, __ISALLPARTS_ISSET_ID);
+  }
+
+  public void setIsAllPartsIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ISALLPARTS_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    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 PART_NAME:
+      if (value == null) {
+        unsetPartName();
+      } else {
+        setPartName((String)value);
+      }
+      break;
+
+    case IS_ALL_PARTS:
+      if (value == null) {
+        unsetIsAllParts();
+      } else {
+        setIsAllParts((Boolean)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case DB_NAME:
+      return getDbName();
+
+    case TBL_NAME:
+      return getTblName();
+
+    case PART_NAME:
+      return getPartName();
+
+    case IS_ALL_PARTS:
+      return isIsAllParts();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case DB_NAME:
+      return isSetDbName();
+    case TBL_NAME:
+      return isSetTblName();
+    case PART_NAME:
+      return isSetPartName();
+    case IS_ALL_PARTS:
+      return isSetIsAllParts();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof CacheFileMetadataRequest)
+      return this.equals((CacheFileMetadataRequest)that);
+    return false;
+  }
+
+  public boolean equals(CacheFileMetadataRequest that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_dbName = true && this.isSetDbName();
+    boolean that_present_dbName = true && that.isSetDbName();
+    if (this_present_dbName || that_present_dbName) {
+      if (!(this_present_dbName && that_present_dbName))
+        return false;
+      if (!this.dbName.equals(that.dbName))
+        return false;
+    }
+
+    boolean this_present_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_partName = true && this.isSetPartName();
+    boolean that_present_partName = true && that.isSetPartName();
+    if (this_present_partName || that_present_partName) {
+      if (!(this_present_partName && that_present_partName))
+        return false;
+      if (!this.partName.equals(that.partName))
+        return false;
+    }
+
+    boolean this_present_isAllParts = true && this.isSetIsAllParts();
+    boolean that_present_isAllParts = true && that.isSetIsAllParts();
+    if (this_present_isAllParts || that_present_isAllParts) {
+      if (!(this_present_isAllParts && that_present_isAllParts))
+        return false;
+      if (this.isAllParts != that.isAllParts)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_dbName = true && (isSetDbName());
+    list.add(present_dbName);
+    if (present_dbName)
+      list.add(dbName);
+
+    boolean present_tblName = true && (isSetTblName());
+    list.add(present_tblName);
+    if (present_tblName)
+      list.add(tblName);
+
+    boolean present_partName = true && (isSetPartName());
+    list.add(present_partName);
+    if (present_partName)
+      list.add(partName);
+
+    boolean present_isAllParts = true && (isSetIsAllParts());
+    list.add(present_isAllParts);
+    if (present_isAllParts)
+      list.add(isAllParts);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(CacheFileMetadataRequest other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetDbName()).compareTo(other.isSetDbName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetDbName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.dbName, other.dbName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(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(isSetPartName()).compareTo(other.isSetPartName());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetPartName()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.partName, other.partName);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetIsAllParts()).compareTo(other.isSetIsAllParts());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetIsAllParts()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.isAllParts, other.isAllParts);
+      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("CacheFileMetadataRequest(");
+    boolean first = true;
+
+    sb.append("dbName:");
+    if (this.dbName == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.dbName);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("tblName:");
+    if (this.tblName == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.tblName);
+    }
+    first = false;
+    if (isSetPartName()) {
+      if (!first) sb.append(", ");
+      sb.append("partName:");
+      if (this.partName == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.partName);
+      }
+      first = false;
+    }
+    if (isSetIsAllParts()) {
+      if (!first) sb.append(", ");
+      sb.append("isAllParts:");
+      sb.append(this.isAllParts);
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!isSetDbName()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'dbName' is unset! Struct:" + toString());
+    }
+
+    if (!isSetTblName()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'tblName' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class CacheFileMetadataRequestStandardSchemeFactory implements SchemeFactory {
+    public CacheFileMetadataRequestStandardScheme getScheme() {
+      return new CacheFileMetadataRequestStandardScheme();
+    }
+  }
+
+  private static class CacheFileMetadataRequestStandardScheme extends StandardScheme<CacheFileMetadataRequest> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, CacheFileMetadataRequest struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) {
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // DB_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.dbName = iprot.readString();
+              struct.setDbNameIsSet(true);
+            } else {
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // 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 3: // PART_NAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.partName = iprot.readString();
+              struct.setPartNameIsSet(true);
+            } else {
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // IS_ALL_PARTS
+            if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+              struct.isAllParts = iprot.readBool();
+              struct.setIsAllPartsIsSet(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, CacheFileMetadataRequest struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.dbName != null) {
+        oprot.writeFieldBegin(DB_NAME_FIELD_DESC);
+        oprot.writeString(struct.dbName);
+        oprot.writeFieldEnd();
+      }
+      if (struct.tblName != null) {
+        oprot.writeFieldBegin(TBL_NAME_FIELD_DESC);
+        oprot.writeString(struct.tblName);
+        oprot.writeFieldEnd();
+      }
+      if (struct.partName != null) {
+        if (struct.isSetPartName()) {
+          oprot.writeFieldBegin(PART_NAME_FIELD_DESC);
+          oprot.writeString(struct.partName);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.isSetIsAllParts()) {
+        oprot.writeFieldBegin(IS_ALL_PARTS_FIELD_DESC);
+        oprot.writeBool(struct.isAllParts);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class CacheFileMetadataRequestTupleSchemeFactory implements SchemeFactory {
+    public CacheFileMetadataRequestTupleScheme getScheme() {
+      return new CacheFileMetadataRequestTupleScheme();
+    }
+  }
+
+  private static class CacheFileMetadataRequestTupleScheme extends TupleScheme<CacheFileMetadataRequest> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, CacheFileMetadataRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.dbName);
+      oprot.writeString(struct.tblName);
+      BitSet optionals = new BitSet();
+      if (struct.isSetPartName()) {
+        optionals.set(0);
+      }
+      if (struct.isSetIsAllParts()) {
+        optionals.set(1);
+      }
+      oprot.writeBitSet(optionals, 2);
+      if (struct.isSetPartName()) {
+        oprot.writeString(struct.partName);
+      }
+      if (struct.isSetIsAllParts()) {
+        oprot.writeBool(struct.isAllParts);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, CacheFileMetadataRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.dbName = iprot.readString();
+      struct.setDbNameIsSet(true);
+      struct.tblName = iprot.readString();
+      struct.setTblNameIsSet(true);
+      BitSet incoming = iprot.readBitSet(2);
+      if (incoming.get(0)) {
+        struct.partName = iprot.readString();
+        struct.setPartNameIsSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.isAllParts = iprot.readBool();
+        struct.setIsAllPartsIsSet(true);
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/7df62023/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CacheFileMetadataResult.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CacheFileMetadataResult.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CacheFileMetadataResult.java
new file mode 100644
index 0000000..4336f96
--- /dev/null
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CacheFileMetadataResult.java
@@ -0,0 +1,386 @@
+/**
+ * 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)")
+public class CacheFileMetadataResult implements org.apache.thrift.TBase<CacheFileMetadataResult, CacheFileMetadataResult._Fields>, java.io.Serializable, Cloneable, Comparable<CacheFileMetadataResult> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CacheFileMetadataResult");
+
+  private static final org.apache.thrift.protocol.TField IS_SUPPORTED_FIELD_DESC = new org.apache.thrift.protocol.TField("isSupported", org.apache.thrift.protocol.TType.BOOL, (short)1);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new CacheFileMetadataResultStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new CacheFileMetadataResultTupleSchemeFactory());
+  }
+
+  private boolean isSupported; // 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 {
+    IS_SUPPORTED((short)1, "isSupported");
+
+    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: // IS_SUPPORTED
+          return IS_SUPPORTED;
+        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 __ISSUPPORTED_ISSET_ID = 0;
+  private byte __isset_bitfield = 0;
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.IS_SUPPORTED, new org.apache.thrift.meta_data.FieldMetaData("isSupported", org.apache.thrift.TFieldRequirementType.REQUIRED,
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(CacheFileMetadataResult.class, metaDataMap);
+  }
+
+  public CacheFileMetadataResult() {
+  }
+
+  public CacheFileMetadataResult(
+    boolean isSupported)
+  {
+    this();
+    this.isSupported = isSupported;
+    setIsSupportedIsSet(true);
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public CacheFileMetadataResult(CacheFileMetadataResult other) {
+    __isset_bitfield = other.__isset_bitfield;
+    this.isSupported = other.isSupported;
+  }
+
+  public CacheFileMetadataResult deepCopy() {
+    return new CacheFileMetadataResult(this);
+  }
+
+  @Override
+  public void clear() {
+    setIsSupportedIsSet(false);
+    this.isSupported = false;
+  }
+
+  public boolean isIsSupported() {
+    return this.isSupported;
+  }
+
+  public void setIsSupported(boolean isSupported) {
+    this.isSupported = isSupported;
+    setIsSupportedIsSet(true);
+  }
+
+  public void unsetIsSupported() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ISSUPPORTED_ISSET_ID);
+  }
+
+  /** Returns true if field isSupported is set (has been assigned a value) and false otherwise */
+  public boolean isSetIsSupported() {
+    return EncodingUtils.testBit(__isset_bitfield, __ISSUPPORTED_ISSET_ID);
+  }
+
+  public void setIsSupportedIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ISSUPPORTED_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case IS_SUPPORTED:
+      if (value == null) {
+        unsetIsSupported();
+      } else {
+        setIsSupported((Boolean)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case IS_SUPPORTED:
+      return isIsSupported();
+
+    }
+    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 IS_SUPPORTED:
+      return isSetIsSupported();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof CacheFileMetadataResult)
+      return this.equals((CacheFileMetadataResult)that);
+    return false;
+  }
+
+  public boolean equals(CacheFileMetadataResult that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_isSupported = true;
+    boolean that_present_isSupported = true;
+    if (this_present_isSupported || that_present_isSupported) {
+      if (!(this_present_isSupported && that_present_isSupported))
+        return false;
+      if (this.isSupported != that.isSupported)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_isSupported = true;
+    list.add(present_isSupported);
+    if (present_isSupported)
+      list.add(isSupported);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(CacheFileMetadataResult other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetIsSupported()).compareTo(other.isSetIsSupported());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetIsSupported()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.isSupported, other.isSupported);
+      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("CacheFileMetadataResult(");
+    boolean first = true;
+
+    sb.append("isSupported:");
+    sb.append(this.isSupported);
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!isSetIsSupported()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'isSupported' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class CacheFileMetadataResultStandardSchemeFactory implements SchemeFactory {
+    public CacheFileMetadataResultStandardScheme getScheme() {
+      return new CacheFileMetadataResultStandardScheme();
+    }
+  }
+
+  private static class CacheFileMetadataResultStandardScheme extends StandardScheme<CacheFileMetadataResult> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, CacheFileMetadataResult 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: // IS_SUPPORTED
+            if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+              struct.isSupported = iprot.readBool();
+              struct.setIsSupportedIsSet(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, CacheFileMetadataResult struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      oprot.writeFieldBegin(IS_SUPPORTED_FIELD_DESC);
+      oprot.writeBool(struct.isSupported);
+      oprot.writeFieldEnd();
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class CacheFileMetadataResultTupleSchemeFactory implements SchemeFactory {
+    public CacheFileMetadataResultTupleScheme getScheme() {
+      return new CacheFileMetadataResultTupleScheme();
+    }
+  }
+
+  private static class CacheFileMetadataResultTupleScheme extends TupleScheme<CacheFileMetadataResult> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, CacheFileMetadataResult struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeBool(struct.isSupported);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, CacheFileMetadataResult struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.isSupported = iprot.readBool();
+      struct.setIsSupportedIsSet(true);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/7df62023/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FileMetadataExprType.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FileMetadataExprType.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FileMetadataExprType.java
index 4e393e2..8ffaa89 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FileMetadataExprType.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FileMetadataExprType.java
@@ -7,10 +7,6 @@
 package org.apache.hadoop.hive.metastore.api;
 
 
-import java.util.Map;
-import java.util.HashMap;
-import org.apache.thrift.TEnum;
-
 public enum FileMetadataExprType implements org.apache.thrift.TEnum {
   ORC_SARG(1);
 


[4/7] hive git commit: HIVE-12075 : add analyze command to explictly cache file metadata in HBase metastore (Sergey Shelukhin, reviewed by Alan Gates)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/7df62023/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataRequest.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataRequest.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataRequest.java
index a5fef3d..1b7bbf0 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataRequest.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataRequest.java
@@ -40,6 +40,7 @@ public class PutFileMetadataRequest implements org.apache.thrift.TBase<PutFileMe
 
   private static final org.apache.thrift.protocol.TField FILE_IDS_FIELD_DESC = new org.apache.thrift.protocol.TField("fileIds", org.apache.thrift.protocol.TType.LIST, (short)1);
   private static final org.apache.thrift.protocol.TField METADATA_FIELD_DESC = new org.apache.thrift.protocol.TField("metadata", org.apache.thrift.protocol.TType.LIST, (short)2);
+  private static final org.apache.thrift.protocol.TField TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("type", org.apache.thrift.protocol.TType.I32, (short)3);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -49,11 +50,17 @@ public class PutFileMetadataRequest implements org.apache.thrift.TBase<PutFileMe
 
   private List<Long> fileIds; // required
   private List<ByteBuffer> metadata; // required
+  private FileMetadataExprType type; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
     FILE_IDS((short)1, "fileIds"),
-    METADATA((short)2, "metadata");
+    METADATA((short)2, "metadata"),
+    /**
+     *
+     * @see FileMetadataExprType
+     */
+    TYPE((short)3, "type");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -72,6 +79,8 @@ public class PutFileMetadataRequest implements org.apache.thrift.TBase<PutFileMe
           return FILE_IDS;
         case 2: // METADATA
           return METADATA;
+        case 3: // TYPE
+          return TYPE;
         default:
           return null;
       }
@@ -112,6 +121,7 @@ public class PutFileMetadataRequest implements org.apache.thrift.TBase<PutFileMe
   }
 
   // isset id assignments
+  private static final _Fields optionals[] = {_Fields.TYPE};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -121,6 +131,8 @@ public class PutFileMetadataRequest implements org.apache.thrift.TBase<PutFileMe
     tmpMap.put(_Fields.METADATA, new org.apache.thrift.meta_data.FieldMetaData("metadata", org.apache.thrift.TFieldRequirementType.REQUIRED, 
         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            , true))));
+    tmpMap.put(_Fields.TYPE, new org.apache.thrift.meta_data.FieldMetaData("type", org.apache.thrift.TFieldRequirementType.OPTIONAL,
+        new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, FileMetadataExprType.class)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(PutFileMetadataRequest.class, metaDataMap);
   }
@@ -149,6 +161,9 @@ public class PutFileMetadataRequest implements org.apache.thrift.TBase<PutFileMe
       List<ByteBuffer> __this__metadata = new ArrayList<ByteBuffer>(other.metadata);
       this.metadata = __this__metadata;
     }
+    if (other.isSetType()) {
+      this.type = other.type;
+    }
   }
 
   public PutFileMetadataRequest deepCopy() {
@@ -159,6 +174,7 @@ public class PutFileMetadataRequest implements org.apache.thrift.TBase<PutFileMe
   public void clear() {
     this.fileIds = null;
     this.metadata = null;
+    this.type = null;
   }
 
   public int getFileIdsSize() {
@@ -237,6 +253,37 @@ public class PutFileMetadataRequest implements org.apache.thrift.TBase<PutFileMe
     }
   }
 
+  /**
+   *
+   * @see FileMetadataExprType
+   */
+  public FileMetadataExprType getType() {
+    return this.type;
+  }
+
+  /**
+   *
+   * @see FileMetadataExprType
+   */
+  public void setType(FileMetadataExprType type) {
+    this.type = type;
+  }
+
+  public void unsetType() {
+    this.type = null;
+  }
+
+  /** Returns true if field type is set (has been assigned a value) and false otherwise */
+  public boolean isSetType() {
+    return this.type != null;
+  }
+
+  public void setTypeIsSet(boolean value) {
+    if (!value) {
+      this.type = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case FILE_IDS:
@@ -255,6 +302,14 @@ public class PutFileMetadataRequest implements org.apache.thrift.TBase<PutFileMe
       }
       break;
 
+    case TYPE:
+      if (value == null) {
+        unsetType();
+      } else {
+        setType((FileMetadataExprType)value);
+      }
+      break;
+
     }
   }
 
@@ -266,6 +321,9 @@ public class PutFileMetadataRequest implements org.apache.thrift.TBase<PutFileMe
     case METADATA:
       return getMetadata();
 
+    case TYPE:
+      return getType();
+
     }
     throw new IllegalStateException();
   }
@@ -281,6 +339,8 @@ public class PutFileMetadataRequest implements org.apache.thrift.TBase<PutFileMe
       return isSetFileIds();
     case METADATA:
       return isSetMetadata();
+    case TYPE:
+      return isSetType();
     }
     throw new IllegalStateException();
   }
@@ -316,6 +376,15 @@ public class PutFileMetadataRequest implements org.apache.thrift.TBase<PutFileMe
         return false;
     }
 
+    boolean this_present_type = true && this.isSetType();
+    boolean that_present_type = true && that.isSetType();
+    if (this_present_type || that_present_type) {
+      if (!(this_present_type && that_present_type))
+        return false;
+      if (!this.type.equals(that.type))
+        return false;
+    }
+
     return true;
   }
 
@@ -333,6 +402,11 @@ public class PutFileMetadataRequest implements org.apache.thrift.TBase<PutFileMe
     if (present_metadata)
       list.add(metadata);
 
+    boolean present_type = true && (isSetType());
+    list.add(present_type);
+    if (present_type)
+      list.add(type.getValue());
+
     return list.hashCode();
   }
 
@@ -364,6 +438,16 @@ public class PutFileMetadataRequest implements org.apache.thrift.TBase<PutFileMe
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetType()).compareTo(other.isSetType());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetType()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.type, other.type);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -399,6 +483,16 @@ public class PutFileMetadataRequest implements org.apache.thrift.TBase<PutFileMe
       org.apache.thrift.TBaseHelper.toString(this.metadata, sb);
     }
     first = false;
+    if (isSetType()) {
+      if (!first) sb.append(", ");
+      sb.append("type:");
+      if (this.type == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.type);
+      }
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -486,6 +580,14 @@ public class PutFileMetadataRequest implements org.apache.thrift.TBase<PutFileMe
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 3: // TYPE
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.type = org.apache.hadoop.hive.metastore.api.FileMetadataExprType.findByValue(iprot.readI32());
+              struct.setTypeIsSet(true);
+            } else {
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -523,6 +625,13 @@ public class PutFileMetadataRequest implements org.apache.thrift.TBase<PutFileMe
         }
         oprot.writeFieldEnd();
       }
+      if (struct.type != null) {
+        if (struct.isSetType()) {
+          oprot.writeFieldBegin(TYPE_FIELD_DESC);
+          oprot.writeI32(struct.type.getValue());
+          oprot.writeFieldEnd();
+        }
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -554,6 +663,14 @@ public class PutFileMetadataRequest implements org.apache.thrift.TBase<PutFileMe
           oprot.writeBinary(_iter569);
         }
       }
+      BitSet optionals = new BitSet();
+      if (struct.isSetType()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.isSetType()) {
+        oprot.writeI32(struct.type.getValue());
+      }
     }
 
     @Override
@@ -581,6 +698,11 @@ public class PutFileMetadataRequest implements org.apache.thrift.TBase<PutFileMe
         }
       }
       struct.setMetadataIsSet(true);
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        struct.type = org.apache.hadoop.hive.metastore.api.FileMetadataExprType.findByValue(iprot.readI32());
+        struct.setTypeIsSet(true);
+      }
     }
   }