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

[1/6] hive git commit: Revert "HIVE-19171 : Persist runtime statistics in metastore (Zoltan Haindrich via Ashutosh Chauhan)"

Repository: hive
Updated Branches:
  refs/heads/master 997ad34c0 -> f0199500f


http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
index 184ecb6..125d5a7 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
@@ -55,7 +55,6 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
 import java.util.regex.Pattern;
-import java.util.stream.Collectors;
 
 import javax.jdo.JDOCanRetryException;
 import javax.jdo.JDODataStoreException;
@@ -84,6 +83,7 @@ import org.apache.hadoop.hive.common.StatsSetupConst;
 import org.apache.hadoop.hive.metastore.MetaStoreDirectSql.SqlFilterForPushdown;
 import org.apache.hadoop.hive.metastore.api.AggrStats;
 import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
+import org.apache.hadoop.hive.metastore.api.BasicTxnInfo;
 import org.apache.hadoop.hive.metastore.api.Catalog;
 import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsDesc;
@@ -124,7 +124,6 @@ import org.apache.hadoop.hive.metastore.api.ResourceType;
 import org.apache.hadoop.hive.metastore.api.ResourceUri;
 import org.apache.hadoop.hive.metastore.api.Role;
 import org.apache.hadoop.hive.metastore.api.RolePrincipalGrant;
-import org.apache.hadoop.hive.metastore.api.RuntimeStat;
 import org.apache.hadoop.hive.metastore.api.SQLCheckConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLDefaultConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
@@ -187,7 +186,6 @@ import org.apache.hadoop.hive.metastore.model.MPartitionPrivilege;
 import org.apache.hadoop.hive.metastore.model.MResourceUri;
 import org.apache.hadoop.hive.metastore.model.MRole;
 import org.apache.hadoop.hive.metastore.model.MRoleMap;
-import org.apache.hadoop.hive.metastore.model.MRuntimeStat;
 import org.apache.hadoop.hive.metastore.model.MSchemaVersion;
 import org.apache.hadoop.hive.metastore.model.MSerDeInfo;
 import org.apache.hadoop.hive.metastore.model.MStorageDescriptor;
@@ -212,6 +210,7 @@ import org.apache.hadoop.hive.metastore.utils.FileUtils;
 import org.apache.hadoop.hive.metastore.utils.JavaUtils;
 import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
 import org.apache.hadoop.hive.metastore.utils.ObjectPair;
+import org.apache.thrift.TDeserializer;
 import org.apache.thrift.TException;
 import org.datanucleus.AbstractNucleusContext;
 import org.datanucleus.ClassLoaderResolver;
@@ -810,9 +809,7 @@ public class ObjectStore implements RawStore, Configurable {
       pm.makePersistent(mCat);
       committed = commitTransaction();
     } finally {
-      if (!committed) {
-        rollbackTransaction();
-      }
+      if (!committed) rollbackTransaction();
     }
   }
 
@@ -835,9 +832,7 @@ public class ObjectStore implements RawStore, Configurable {
       pm.makePersistent(mCat);
       committed = commitTransaction();
     } finally {
-      if (!committed) {
-        rollbackTransaction();
-      }
+      if (!committed) rollbackTransaction();
     }
   }
 
@@ -845,9 +840,7 @@ public class ObjectStore implements RawStore, Configurable {
   public Catalog getCatalog(String catalogName) throws NoSuchObjectException, MetaException {
     LOG.debug("Fetching catalog " + catalogName);
     MCatalog mCat = getMCatalog(catalogName);
-    if (mCat == null) {
-      throw new NoSuchObjectException("No catalog " + catalogName);
-    }
+    if (mCat == null) throw new NoSuchObjectException("No catalog " + catalogName);
     return mCatToCat(mCat);
   }
 
@@ -881,15 +874,11 @@ public class ObjectStore implements RawStore, Configurable {
       openTransaction();
       MCatalog mCat = getMCatalog(catalogName);
       pm.retrieve(mCat);
-      if (mCat == null) {
-        throw new NoSuchObjectException("No catalog " + catalogName);
-      }
+      if (mCat == null) throw new NoSuchObjectException("No catalog " + catalogName);
       pm.deletePersistent(mCat);
       committed = commitTransaction();
     } finally {
-      if (!committed) {
-        rollbackTransaction();
-      }
+      if (!committed) rollbackTransaction();
     }
   }
 
@@ -914,18 +903,14 @@ public class ObjectStore implements RawStore, Configurable {
   private MCatalog catToMCat(Catalog cat) {
     MCatalog mCat = new MCatalog();
     mCat.setName(normalizeIdentifier(cat.getName()));
-    if (cat.isSetDescription()) {
-      mCat.setDescription(cat.getDescription());
-    }
+    if (cat.isSetDescription()) mCat.setDescription(cat.getDescription());
     mCat.setLocationUri(cat.getLocationUri());
     return mCat;
   }
 
   private Catalog mCatToCat(MCatalog mCat) {
     Catalog cat = new Catalog(mCat.getName(), mCat.getLocationUri());
-    if (mCat.getDescription() != null) {
-      cat.setDescription(mCat.getDescription());
-    }
+    if (mCat.getDescription() != null) cat.setDescription(mCat.getDescription());
     return cat;
   }
 
@@ -1998,18 +1983,10 @@ public class ObjectStore implements RawStore, Configurable {
     }
     SerDeInfo serde =
         new SerDeInfo(ms.getName(), ms.getSerializationLib(), convertMap(ms.getParameters()));
-    if (ms.getDescription() != null) {
-      serde.setDescription(ms.getDescription());
-    }
-    if (ms.getSerializerClass() != null) {
-      serde.setSerializerClass(ms.getSerializerClass());
-    }
-    if (ms.getDeserializerClass() != null) {
-      serde.setDeserializerClass(ms.getDeserializerClass());
-    }
-    if (ms.getSerdeType() > 0) {
-      serde.setSerdeType(SerdeType.findByValue(ms.getSerdeType()));
-    }
+    if (ms.getDescription() != null) serde.setDescription(ms.getDescription());
+    if (ms.getSerializerClass() != null) serde.setSerializerClass(ms.getSerializerClass());
+    if (ms.getDeserializerClass() != null) serde.setDeserializerClass(ms.getDeserializerClass());
+    if (ms.getSerdeType() > 0) serde.setSerdeType(SerdeType.findByValue(ms.getSerdeType()));
     return serde;
   }
 
@@ -3702,7 +3679,7 @@ public class ObjectStore implements RawStore, Configurable {
       @Override
       protected boolean canUseDirectSql(GetHelper<Integer> ctx) throws MetaException {
         return directSql.generateSqlFilterForPushdown(ctx.getTable(), exprTree, filter);
-      }
+      };
 
       @Override
       protected Integer getSqlResult(GetHelper<Integer> ctx) throws MetaException {
@@ -10021,9 +9998,7 @@ public class ObjectStore implements RawStore, Configurable {
       pm.makePersistent(mSchema);
       committed = commitTransaction();
     } finally {
-      if (!committed) {
-        rollbackTransaction();
-      }
+      if (!committed) rollbackTransaction();
     }
   }
 
@@ -10042,17 +10017,11 @@ public class ObjectStore implements RawStore, Configurable {
       oldMSchema.setCompatibility(newSchema.getCompatibility().getValue());
       oldMSchema.setValidationLevel(newSchema.getValidationLevel().getValue());
       oldMSchema.setCanEvolve(newSchema.isCanEvolve());
-      if (newSchema.isSetSchemaGroup()) {
-        oldMSchema.setSchemaGroup(newSchema.getSchemaGroup());
-      }
-      if (newSchema.isSetDescription()) {
-        oldMSchema.setDescription(newSchema.getDescription());
-      }
+      if (newSchema.isSetSchemaGroup()) oldMSchema.setSchemaGroup(newSchema.getSchemaGroup());
+      if (newSchema.isSetDescription()) oldMSchema.setDescription(newSchema.getDescription());
       committed = commitTransaction();
     } finally {
-      if (!committed) {
-        rollbackTransaction();
-      }
+      if (!committed) rollbackTransaction();
     }
   }
 
@@ -10066,9 +10035,7 @@ public class ObjectStore implements RawStore, Configurable {
       committed = commitTransaction();
       return schema;
     } finally {
-      if (!committed) {
-        rollbackTransaction();
-      }
+      if (!committed) rollbackTransaction();
     }
   }
 
@@ -10087,9 +10054,7 @@ public class ObjectStore implements RawStore, Configurable {
       pm.retrieve(mSchema);
       return mSchema;
     } finally {
-      if (query != null) {
-        query.closeAll();
-      }
+      if (query != null) query.closeAll();
     }
   }
 
@@ -10106,9 +10071,7 @@ public class ObjectStore implements RawStore, Configurable {
       }
       committed = commitTransaction();
     } finally {
-      if (!committed) {
-        rollbackTransaction();
-      }
+      if (!committed) rollbackTransaction();
     }
   }
 
@@ -10133,9 +10096,7 @@ public class ObjectStore implements RawStore, Configurable {
       pm.makePersistent(mSchemaVersion);
       committed = commitTransaction();
     } finally {
-      if (!committed) {
-        rollbackTransaction();
-      }
+      if (!committed) rollbackTransaction();;
     }
   }
 
@@ -10152,17 +10113,11 @@ public class ObjectStore implements RawStore, Configurable {
       }
 
       // We only support changing the SerDe mapping and the state.
-      if (newVersion.isSetSerDe()) {
-        oldMSchemaVersion.setSerDe(convertToMSerDeInfo(newVersion.getSerDe()));
-      }
-      if (newVersion.isSetState()) {
-        oldMSchemaVersion.setState(newVersion.getState().getValue());
-      }
+      if (newVersion.isSetSerDe()) oldMSchemaVersion.setSerDe(convertToMSerDeInfo(newVersion.getSerDe()));
+      if (newVersion.isSetState()) oldMSchemaVersion.setState(newVersion.getState().getValue());
       committed = commitTransaction();
     } finally {
-      if (!committed) {
-        commitTransaction();
-      }
+      if (!committed) commitTransaction();
     }
   }
 
@@ -10177,9 +10132,7 @@ public class ObjectStore implements RawStore, Configurable {
       committed = commitTransaction();
       return schemaVersion;
     } finally {
-      if (!committed) {
-        rollbackTransaction();
-      }
+      if (!committed) rollbackTransaction();;
     }
   }
 
@@ -10199,15 +10152,11 @@ public class ObjectStore implements RawStore, Configurable {
       pm.retrieve(mSchemaVersion);
       if (mSchemaVersion != null) {
         pm.retrieveAll(mSchemaVersion.getCols());
-        if (mSchemaVersion.getSerDe() != null) {
-          pm.retrieve(mSchemaVersion.getSerDe());
-        }
+        if (mSchemaVersion.getSerDe() != null) pm.retrieve(mSchemaVersion.getSerDe());
       }
       return mSchemaVersion;
     } finally {
-      if (query != null) {
-        query.closeAll();
-      }
+      if (query != null) query.closeAll();
     }
   }
 
@@ -10231,9 +10180,7 @@ public class ObjectStore implements RawStore, Configurable {
       pm.retrieve(mSchemaVersion);
       if (mSchemaVersion != null) {
         pm.retrieveAll(mSchemaVersion.getCols());
-        if (mSchemaVersion.getSerDe() != null) {
-          pm.retrieve(mSchemaVersion.getSerDe());
-        }
+        if (mSchemaVersion.getSerDe() != null) pm.retrieve(mSchemaVersion.getSerDe());
       }
       SchemaVersion version = mSchemaVersion == null ? null : convertToSchemaVersion(mSchemaVersion);
       committed = commitTransaction();
@@ -10259,15 +10206,11 @@ public class ObjectStore implements RawStore, Configurable {
       query.setOrdering("version descending");
       List<MSchemaVersion> mSchemaVersions = query.setParameters(name, dbName, catName).executeList();
       pm.retrieveAll(mSchemaVersions);
-      if (mSchemaVersions == null || mSchemaVersions.isEmpty()) {
-        return null;
-      }
+      if (mSchemaVersions == null || mSchemaVersions.isEmpty()) return null;
       List<SchemaVersion> schemaVersions = new ArrayList<>(mSchemaVersions.size());
       for (MSchemaVersion mSchemaVersion : mSchemaVersions) {
         pm.retrieveAll(mSchemaVersion.getCols());
-        if (mSchemaVersion.getSerDe() != null) {
-          pm.retrieve(mSchemaVersion.getSerDe());
-        }
+        if (mSchemaVersion.getSerDe() != null) pm.retrieve(mSchemaVersion.getSerDe());
         schemaVersions.add(convertToSchemaVersion(mSchemaVersion));
       }
       committed = commitTransaction();
@@ -10289,12 +10232,8 @@ public class ObjectStore implements RawStore, Configurable {
     Query query = null;
     try {
       openTransaction();
-      if (colName != null) {
-        colName = normalizeIdentifier(colName);
-      }
-      if (type != null) {
-        type = normalizeIdentifier(type);
-      }
+      if (colName != null) colName = normalizeIdentifier(colName);
+      if (type != null) type = normalizeIdentifier(type);
       Map<String, String> parameters = new HashMap<>(3);
       StringBuilder sql = new StringBuilder("select SCHEMA_VERSION_ID from " +
           "SCHEMA_VERSION, COLUMNS_V2 where SCHEMA_VERSION.CD_ID = COLUMNS_V2.CD_ID ");
@@ -10320,16 +10259,12 @@ public class ObjectStore implements RawStore, Configurable {
       query = pm.newQuery("javax.jdo.query.SQL", sql.toString());
       query.setClass(MSchemaVersion.class);
       List<MSchemaVersion> mSchemaVersions = query.setNamedParameters(parameters).executeList();
-      if (mSchemaVersions == null || mSchemaVersions.isEmpty()) {
-        return Collections.emptyList();
-      }
+      if (mSchemaVersions == null || mSchemaVersions.isEmpty()) return Collections.emptyList();
       pm.retrieveAll(mSchemaVersions);
       List<SchemaVersion> schemaVersions = new ArrayList<>(mSchemaVersions.size());
       for (MSchemaVersion mSchemaVersion : mSchemaVersions) {
         pm.retrieveAll(mSchemaVersion.getCols());
-        if (mSchemaVersion.getSerDe() != null) {
-          pm.retrieve(mSchemaVersion.getSerDe());
-        }
+        if (mSchemaVersion.getSerDe() != null) pm.retrieve(mSchemaVersion.getSerDe());
         schemaVersions.add(convertToSchemaVersion(mSchemaVersion));
       }
       committed = commitTransaction();
@@ -10356,9 +10291,7 @@ public class ObjectStore implements RawStore, Configurable {
       }
       committed = commitTransaction();
     } finally {
-      if (!committed) {
-        rollbackTransaction();
-      }
+      if (!committed) rollbackTransaction();
     }
   }
 
@@ -10375,9 +10308,7 @@ public class ObjectStore implements RawStore, Configurable {
       committed = commitTransaction();
       return serde;
     } finally {
-      if (!committed) {
-        rollbackTransaction();
-      }
+      if (!committed) rollbackTransaction();;
     }
   }
 
@@ -10391,9 +10322,7 @@ public class ObjectStore implements RawStore, Configurable {
       pm.retrieve(mSerDeInfo);
       return mSerDeInfo;
     } finally {
-      if (query != null) {
-        query.closeAll();
-      }
+      if (query != null) query.closeAll();
     }
   }
 
@@ -10409,9 +10338,7 @@ public class ObjectStore implements RawStore, Configurable {
       pm.makePersistent(mSerde);
       committed = commitTransaction();
     } finally {
-      if (!committed) {
-        rollbackTransaction();
-      }
+      if (!committed) rollbackTransaction();
     }
 
   }
@@ -10428,9 +10355,7 @@ public class ObjectStore implements RawStore, Configurable {
   }
 
   private ISchema convertToISchema(MISchema mSchema) {
-    if (mSchema == null) {
-      return null;
-    }
+    if (mSchema == null) return null;
     ISchema schema = new ISchema(SchemaType.findByValue(mSchema.getSchemaType()),
                                  mSchema.getName(),
                                  mSchema.getDb().getCatalogName(),
@@ -10438,12 +10363,8 @@ public class ObjectStore implements RawStore, Configurable {
                                  SchemaCompatibility.findByValue(mSchema.getCompatibility()),
                                  SchemaValidation.findByValue(mSchema.getValidationLevel()),
                                  mSchema.getCanEvolve());
-    if (mSchema.getDescription() != null) {
-      schema.setDescription(mSchema.getDescription());
-    }
-    if (mSchema.getSchemaGroup() != null) {
-      schema.setSchemaGroup(mSchema.getSchemaGroup());
-    }
+    if (mSchema.getDescription() != null) schema.setDescription(mSchema.getDescription());
+    if (mSchema.getSchemaGroup() != null) schema.setSchemaGroup(mSchema.getSchemaGroup());
     return schema;
   }
 
@@ -10464,33 +10385,19 @@ public class ObjectStore implements RawStore, Configurable {
   }
 
   private SchemaVersion convertToSchemaVersion(MSchemaVersion mSchemaVersion) throws MetaException {
-    if (mSchemaVersion == null) {
-      return null;
-    }
+    if (mSchemaVersion == null) return null;
     SchemaVersion schemaVersion = new SchemaVersion(
         new ISchemaName(mSchemaVersion.getiSchema().getDb().getCatalogName(),
             mSchemaVersion.getiSchema().getDb().getName(), mSchemaVersion.getiSchema().getName()),
         mSchemaVersion.getVersion(),
         mSchemaVersion.getCreatedAt(),
         convertToFieldSchemas(mSchemaVersion.getCols().getCols()));
-    if (mSchemaVersion.getState() > 0) {
-      schemaVersion.setState(SchemaVersionState.findByValue(mSchemaVersion.getState()));
-    }
-    if (mSchemaVersion.getDescription() != null) {
-      schemaVersion.setDescription(mSchemaVersion.getDescription());
-    }
-    if (mSchemaVersion.getSchemaText() != null) {
-      schemaVersion.setSchemaText(mSchemaVersion.getSchemaText());
-    }
-    if (mSchemaVersion.getFingerprint() != null) {
-      schemaVersion.setFingerprint(mSchemaVersion.getFingerprint());
-    }
-    if (mSchemaVersion.getName() != null) {
-      schemaVersion.setName(mSchemaVersion.getName());
-    }
-    if (mSchemaVersion.getSerDe() != null) {
-      schemaVersion.setSerDe(convertToSerDeInfo(mSchemaVersion.getSerDe()));
-    }
+    if (mSchemaVersion.getState() > 0) schemaVersion.setState(SchemaVersionState.findByValue(mSchemaVersion.getState()));
+    if (mSchemaVersion.getDescription() != null) schemaVersion.setDescription(mSchemaVersion.getDescription());
+    if (mSchemaVersion.getSchemaText() != null) schemaVersion.setSchemaText(mSchemaVersion.getSchemaText());
+    if (mSchemaVersion.getFingerprint() != null) schemaVersion.setFingerprint(mSchemaVersion.getFingerprint());
+    if (mSchemaVersion.getName() != null) schemaVersion.setName(mSchemaVersion.getName());
+    if (mSchemaVersion.getSerDe() != null) schemaVersion.setSerDe(convertToSerDeInfo(mSchemaVersion.getSerDe()));
     return schemaVersion;
   }
 
@@ -11600,61 +11507,4 @@ public class ObjectStore implements RawStore, Configurable {
       rollbackAndCleanup(commited, (Query)null);
     }
   }
-
-  @Override
-  public void addRuntimeStat(RuntimeStat stat) throws MetaException {
-    LOG.debug("runtimeStat: " + stat);
-    MRuntimeStat mStat = MRuntimeStat.fromThrift(stat);
-    pm.makePersistent(mStat);
-  }
-
-  @Override
-  public int deleteRuntimeStats(int maxRetainedWeight, int maxRetainSecs) throws MetaException {
-    List<MRuntimeStat> all = getMRuntimeStats();
-    int retentionTime = 0;
-    if (maxRetainSecs >= 0) {
-      retentionTime = (int) (System.currentTimeMillis() / 1000) - maxRetainSecs;
-    }
-    if (maxRetainedWeight < 0) {
-      maxRetainedWeight = Integer.MAX_VALUE;
-    }
-
-    Object maxIdToRemove = null;
-    long totalWeight = 0;
-    int deleted = 0;
-    for (MRuntimeStat mRuntimeStat : all) {
-      totalWeight += mRuntimeStat.getWeight();
-      if (totalWeight > maxRetainedWeight || mRuntimeStat.getCreatedTime() < retentionTime) {
-        LOG.debug("removing runtime stat: " + mRuntimeStat);
-        pm.deletePersistent(mRuntimeStat);
-        deleted++;
-      }
-    }
-    return deleted;
-  }
-
-  @Override
-  public List<RuntimeStat> getRuntimeStats() throws MetaException {
-    boolean committed = false;
-    try {
-      openTransaction();
-      List<MRuntimeStat> mStats = getMRuntimeStats();
-      List<RuntimeStat> stats = mStats.stream().map(MRuntimeStat::toThrift).collect(Collectors.toList());
-      committed = commitTransaction();
-      return stats;
-    } finally {
-      if (!committed) {
-        rollbackTransaction();
-      }
-    }
-  }
-
-  private List<MRuntimeStat> getMRuntimeStats() {
-    Query<MRuntimeStat> query = pm.newQuery(MRuntimeStat.class);
-    query.setOrdering("createTime descending");
-    List<MRuntimeStat> res = (List<MRuntimeStat>) query.execute();
-    pm.retrieveAll(res);
-    return res;
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java
index 2c9f2e5..f6c46ee 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hive.metastore;
 
+import org.apache.hadoop.hive.metastore.api.AlterISchemaRequest;
 import org.apache.hadoop.hive.metastore.api.CreationMetadata;
 import org.apache.hadoop.hive.metastore.api.ISchemaName;
 import org.apache.hadoop.hive.metastore.api.SchemaVersionDescriptor;
@@ -37,6 +38,7 @@ import org.apache.hadoop.hive.metastore.api.AggrStats;
 import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
 import org.apache.hadoop.hive.metastore.api.Catalog;
 import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
+import org.apache.hadoop.hive.metastore.api.CreationMetadata;
 import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
@@ -44,6 +46,7 @@ import org.apache.hadoop.hive.metastore.api.FileMetadataExprType;
 import org.apache.hadoop.hive.metastore.api.Function;
 import org.apache.hadoop.hive.metastore.api.HiveObjectPrivilege;
 import org.apache.hadoop.hive.metastore.api.ISchema;
+import org.apache.hadoop.hive.metastore.api.ISchemaName;
 import org.apache.hadoop.hive.metastore.api.InvalidInputException;
 import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
 import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
@@ -63,8 +66,8 @@ import org.apache.hadoop.hive.metastore.api.PrincipalType;
 import org.apache.hadoop.hive.metastore.api.PrivilegeBag;
 import org.apache.hadoop.hive.metastore.api.Role;
 import org.apache.hadoop.hive.metastore.api.RolePrincipalGrant;
-import org.apache.hadoop.hive.metastore.api.RuntimeStat;
 import org.apache.hadoop.hive.metastore.api.SQLCheckConstraint;
+import org.apache.hadoop.hive.metastore.api.SchemaVersionDescriptor;
 import org.apache.hadoop.hive.metastore.api.SQLDefaultConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
 import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
@@ -78,6 +81,7 @@ import org.apache.hadoop.hive.metastore.api.Type;
 import org.apache.hadoop.hive.metastore.api.UnknownDBException;
 import org.apache.hadoop.hive.metastore.api.UnknownPartitionException;
 import org.apache.hadoop.hive.metastore.api.UnknownTableException;
+import org.apache.hadoop.hive.metastore.api.WMFullResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMMapping;
 import org.apache.hadoop.hive.metastore.api.WMNullablePool;
 import org.apache.hadoop.hive.metastore.api.WMNullableResourcePlan;
@@ -1625,13 +1629,4 @@ public interface RawStore extends Configurable {
    * @throws MetaException general database exception
    */
   void addSerde(SerDeInfo serde) throws AlreadyExistsException, MetaException;
-
-  /** Adds a RuntimeStat for persistence. */
-  void addRuntimeStat(RuntimeStat stat) throws MetaException;
-
-  /** Reads runtime statistic entries. */
-  List<RuntimeStat> getRuntimeStats() throws MetaException;
-
-  /** Removes outdated statistics. */
-  int deleteRuntimeStats(int maxRetained, int maxRetainSecs) throws MetaException;
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RuntimeStatsCleanerTask.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RuntimeStatsCleanerTask.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RuntimeStatsCleanerTask.java
deleted file mode 100644
index 202058e..0000000
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RuntimeStatsCleanerTask.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * 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.conf.Configuration;
-import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hive.metastore.RawStore;
-import java.util.concurrent.TimeUnit;
-
-/**
- * Metastore task to handle RuntimeStat related expiration.
- */
-public class RuntimeStatsCleanerTask implements MetastoreTaskThread {
-  private static final Logger LOG = LoggerFactory.getLogger(RuntimeStatsCleanerTask.class);
-
-  private Configuration conf;
-
-  @Override
-  public long runFrequency(TimeUnit unit) {
-    return MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.RUNTIME_STATS_CLEAN_FREQUENCY, unit);
-  }
-
-  @Override
-  public void setConf(Configuration configuration) {
-    conf = configuration;
-  }
-
-  @Override
-  public Configuration getConf() {
-    return conf;
-  }
-
-  @Override
-  public void run() {
-
-    try {
-      RawStore ms = HiveMetaStore.HMSHandler.getMSForConf(conf);
-      int maxRetained = MetastoreConf.getIntVar(conf, MetastoreConf.ConfVars.RUNTIME_STATS_MAX_ENTRIES);
-      int maxRetainSecs=(int) MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.RUNTIME_STATS_MAX_AGE, TimeUnit.SECONDS);
-      int deleteCnt = ms.deleteRuntimeStats(maxRetained, maxRetainSecs);
-
-      if (deleteCnt > 0L){
-        LOG.info("Number of deleted entries: " + deleteCnt);
-      }
-    } catch (Exception e) {
-      LOG.error("Exception while trying to delete: " + e.getMessage(), e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
index 92d000b..ebdcbc2 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
@@ -89,7 +89,6 @@ import org.apache.hadoop.hive.metastore.columnstats.aggr.ColumnStatsAggregator;
 import org.apache.hadoop.hive.metastore.columnstats.aggr.ColumnStatsAggregatorFactory;
 import org.apache.hadoop.hive.metastore.api.Role;
 import org.apache.hadoop.hive.metastore.api.RolePrincipalGrant;
-import org.apache.hadoop.hive.metastore.api.RuntimeStat;
 import org.apache.hadoop.hive.metastore.api.SQLCheckConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLDefaultConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
@@ -221,9 +220,8 @@ public class CachedStore implements RawStore, Configurable {
         LOG.info("Going to cache catalogs: "
             + org.apache.commons.lang.StringUtils.join(catalogsToCache, ", "));
         List<Catalog> catalogs = new ArrayList<>(catalogsToCache.size());
-        for (String catName : catalogsToCache) {
+        for (String catName : catalogsToCache)
           catalogs.add(rawStore.getCatalog(catName));
-        }
         sharedCache.populateCatalogsInCache(catalogs);
       } catch (MetaException | NoSuchObjectException e) {
         LOG.warn("Failed to populate catalogs in cache, going to try again", e);
@@ -2177,7 +2175,6 @@ public class CachedStore implements RawStore, Configurable {
     return rawStore.addNotNullConstraints(nns);
   }
 
-  @Override
   public List<String> addDefaultConstraints(List<SQLDefaultConstraint> nns)
       throws InvalidObjectException, MetaException {
     // TODO constraintCache
@@ -2198,7 +2195,6 @@ public class CachedStore implements RawStore, Configurable {
     rawStore.createISchema(schema);
   }
 
-  @Override
   public List<ColStatsObjWithSourceInfo> getPartitionColStatsForDatabase(String catName, String dbName)
       throws MetaException, NoSuchObjectException {
     return rawStore.getPartitionColStatsForDatabase(catName, dbName);
@@ -2470,19 +2466,4 @@ public class CachedStore implements RawStore, Configurable {
     sharedCache.resetCatalogCache();
     setCachePrewarmedState(false);
   }
-
-  @Override
-  public void addRuntimeStat(RuntimeStat stat) throws MetaException {
-    rawStore.addRuntimeStat(stat);
-  }
-
-  @Override
-  public List<RuntimeStat> getRuntimeStats() throws MetaException {
-    return rawStore.getRuntimeStats();
-  }
-
-  @Override
-  public int deleteRuntimeStats(int maxRetained, int maxRetainSecs) throws MetaException {
-    return rawStore.deleteRuntimeStats(maxRetained, maxRetainSecs);
-  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java
index 552eeca..59749e4 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java
@@ -24,7 +24,6 @@ import org.apache.hadoop.hive.metastore.HiveAlterHandler;
 import org.apache.hadoop.hive.metastore.MaterializationsCacheCleanerTask;
 import org.apache.hadoop.hive.metastore.MaterializationsRebuildLockCleanerTask;
 import org.apache.hadoop.hive.metastore.MetastoreTaskThread;
-import org.apache.hadoop.hive.metastore.RuntimeStatsCleanerTask;
 import org.apache.hadoop.hive.metastore.events.EventCleanerTask;
 import org.apache.hadoop.hive.metastore.security.MetastoreDelegationTokenManager;
 import org.apache.hadoop.hive.metastore.txn.AcidCompactionHistoryService;
@@ -578,15 +577,6 @@ public class MetastoreConf {
          "hive.metastore.materializations.invalidation.max.duration",
          86400, TimeUnit.SECONDS, "Maximum duration for query producing a materialization. After this time, transaction" +
          "entries that are not relevant for materializations can be removed from invalidation cache."),
-
-    RUNTIME_STATS_CLEAN_FREQUENCY("runtime.stats.clean.frequency", "hive.metastore.runtime.stats.clean.frequency", 3600,
-        TimeUnit.SECONDS, "Frequency at which timer task runs to remove outdated runtime stat entries."),
-    RUNTIME_STATS_MAX_AGE("runtime.stats.max.age", "hive.metastore.runtime.stats.max.age", 86400 * 3, TimeUnit.SECONDS,
-        "Stat entries which are older than this are removed."),
-    RUNTIME_STATS_MAX_ENTRIES("runtime.stats.max.entries", "hive.metastore.runtime.stats.max.entries", 100_000,
-        "Maximum number of runtime stats to keep; unit is operator stat infos - a complicated query has ~100 of these."
-            + "See also: hive.query.reexecution.stats.cache.size"),
-
     // Parameters for exporting metadata on table drop (requires the use of the)
     // org.apache.hadoop.hive.ql.parse.MetaDataExportListener preevent listener
     METADATA_EXPORT_LOCATION("metastore.metadata.export.location", "hive.metadata.export.location",
@@ -742,10 +732,10 @@ public class MetastoreConf {
             + "The only supported special character right now is '/'. This flag applies only to quoted table names.\n"
             + "The default value is true."),
     TASK_THREADS_ALWAYS("metastore.task.threads.always", "metastore.task.threads.always",
-        EventCleanerTask.class.getName() + "," + RuntimeStatsCleanerTask.class.getName() + "," +
+        EventCleanerTask.class.getName() + "," +
         "org.apache.hadoop.hive.metastore.repl.DumpDirCleanerTask" + "," +
         MaterializationsCacheCleanerTask.class.getName() + "," +
-            MaterializationsRebuildLockCleanerTask.class.getName() + "," + RuntimeStatsCleanerTask.class.getName(),
+        MaterializationsRebuildLockCleanerTask.class.getName(),
         "Comma separated list of tasks that will be started in separate threads.  These will " +
             "always be started, regardless of whether the metastore is running in embedded mode " +
             "or in server mode.  They must implement " + MetastoreTaskThread.class.getName()),
@@ -1137,22 +1127,16 @@ public class MetastoreConf {
        */
       hiveSiteURL = findConfigFile(classLoader, "hive-site.xml");
     }
-    if (hiveSiteURL != null) {
-      conf.addResource(hiveSiteURL);
-    }
+    if (hiveSiteURL != null) conf.addResource(hiveSiteURL);
 
     // Now add hivemetastore-site.xml.  Again we add this before our own config files so that the
     // newer overrides the older.
     hiveMetastoreSiteURL = findConfigFile(classLoader, "hivemetastore-site.xml");
-    if (hiveMetastoreSiteURL != null) {
-      conf.addResource(hiveMetastoreSiteURL);
-    }
+    if (hiveMetastoreSiteURL != null) conf.addResource(hiveMetastoreSiteURL);
 
     // Add in our conf file
     metastoreSiteURL = findConfigFile(classLoader, "metastore-site.xml");
-    if (metastoreSiteURL !=  null) {
-      conf.addResource(metastoreSiteURL);
-    }
+    if (metastoreSiteURL !=  null) conf.addResource(metastoreSiteURL);
 
     // If a system property that matches one of our conf value names is set then use the value
     // it's set to to set our own conf value.
@@ -1284,12 +1268,8 @@ public class MetastoreConf {
   public static Collection<String> getStringCollection(Configuration conf, ConfVars var) {
     assert var.defaultVal.getClass() == String.class;
     String val = conf.get(var.varname);
-    if (val == null) {
-      val = conf.get(var.hiveName, (String)var.defaultVal);
-    }
-    if (val == null) {
-      return Collections.emptySet();
-    }
+    if (val == null) val = conf.get(var.hiveName, (String)var.defaultVal);
+    if (val == null) return Collections.emptySet();
     return StringUtils.asSet(val.split(","));
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MRuntimeStat.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MRuntimeStat.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MRuntimeStat.java
deleted file mode 100644
index 054ce7c..0000000
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MRuntimeStat.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * 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.model;
-
-import org.apache.hadoop.hive.metastore.api.RuntimeStat;
-
-/**
- * Represents a runtime stat query entry.
- *
- * As a query may contain a large number of operatorstat entries; they are stored together in a single row in the metastore.
- * The number of operator stat entries this entity has; is shown in the weight column.
- */
-public class MRuntimeStat {
-
-  private int createTime;
-  private int weight;
-  private byte[] payload;
-
-  public static MRuntimeStat fromThrift(RuntimeStat stat) {
-    MRuntimeStat ret = new MRuntimeStat();
-    ret.weight = stat.getWeight();
-    ret.payload = stat.getPayload();
-    ret.createTime = (int) (System.currentTimeMillis() / 1000);
-    return ret;
-  }
-
-  public static RuntimeStat toThrift(MRuntimeStat stat) {
-    RuntimeStat ret = new RuntimeStat();
-    ret.setWeight(stat.weight);
-    ret.setCreateTime(stat.createTime);
-    ret.setPayload(stat.payload);
-    return ret;
-  }
-
-  public int getWeight() {
-    return weight;
-  }
-
-  public int getCreatedTime() {
-    return createTime;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/standalone-metastore/src/main/resources/package.jdo
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/resources/package.jdo b/standalone-metastore/src/main/resources/package.jdo
index 221192e..9ddf598 100644
--- a/standalone-metastore/src/main/resources/package.jdo
+++ b/standalone-metastore/src/main/resources/package.jdo
@@ -1339,20 +1339,6 @@
         <column name="SERDE_ID"/>
       </field>
     </class>
-    <class name="MRuntimeStat" identity-type="datastore" table="RUNTIME_STATS" detachable="true">
-      <datastore-identity>
-        <column name="RS_ID"/>
-      </datastore-identity>
-      <field name="createTime">
-        <column name="CREATE_TIME" jdbc-type="integer"/>
-      </field>
-      <field name="weight">
-        <column name="WEIGHT" jdbc-type="integer"/>
-      </field>
-      <field name="payload">
-        <column name="PAYLOAD" jdbc-type="BLOB" allows-null="true"/>
-      </field>
-   </class>
   </package>
 </jdo>
 

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/standalone-metastore/src/main/sql/derby/hive-schema-3.0.0.derby.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/derby/hive-schema-3.0.0.derby.sql b/standalone-metastore/src/main/sql/derby/hive-schema-3.0.0.derby.sql
index 48d28cb..adfa4c5 100644
--- a/standalone-metastore/src/main/sql/derby/hive-schema-3.0.0.derby.sql
+++ b/standalone-metastore/src/main/sql/derby/hive-schema-3.0.0.derby.sql
@@ -676,16 +676,6 @@ CREATE TABLE REPL_TXN_MAP (
   RTM_TARGET_TXN_ID bigint NOT NULL,
   PRIMARY KEY (RTM_REPL_POLICY, RTM_SRC_TXN_ID)
 );
-
-CREATE TABLE "APP"."RUNTIME_STATS" (
-  "RS_ID" bigint primary key,
-  "CREATE_TIME" integer not null,
-  "WEIGHT" integer not null,
-  "PAYLOAD" BLOB
-);
-
-CREATE INDEX IDX_RUNTIME_STATS_CREATE_TIME ON RUNTIME_STATS(CREATE_TIME);
-
 -- -----------------------------------------------------------------
 -- Record schema version. Should be the last step in the init script
 -- -----------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/standalone-metastore/src/main/sql/derby/upgrade-2.3.0-to-3.0.0.derby.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/derby/upgrade-2.3.0-to-3.0.0.derby.sql b/standalone-metastore/src/main/sql/derby/upgrade-2.3.0-to-3.0.0.derby.sql
index ed6c4cd..a75b740 100644
--- a/standalone-metastore/src/main/sql/derby/upgrade-2.3.0-to-3.0.0.derby.sql
+++ b/standalone-metastore/src/main/sql/derby/upgrade-2.3.0-to-3.0.0.derby.sql
@@ -238,16 +238,6 @@ CREATE TABLE MIN_HISTORY_LEVEL (
 
 CREATE INDEX MIN_HISTORY_LEVEL_IDX ON MIN_HISTORY_LEVEL (MHL_MIN_OPEN_TXNID);
 
-
-CREATE TABLE "APP"."RUNTIME_STATS" (
-  "RS_ID" bigint primary key,
-  "CREATE_TIME" integer not null,
-  "WEIGHT" integer not null,
-  "PAYLOAD" BLOB
-);
-
-CREATE INDEX IDX_RUNTIME_STATS_CREATE_TIME ON RUNTIME_STATS(CREATE_TIME);
-
 -- This needs to be the last thing done.  Insert any changes above this line.
 UPDATE "APP".VERSION SET SCHEMA_VERSION='3.0.0', VERSION_COMMENT='Hive release version 3.0.0' where VER_ID=1;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/standalone-metastore/src/main/sql/mssql/hive-schema-3.0.0.mssql.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/mssql/hive-schema-3.0.0.mssql.sql b/standalone-metastore/src/main/sql/mssql/hive-schema-3.0.0.mssql.sql
index 6e31b16..91c581c 100644
--- a/standalone-metastore/src/main/sql/mssql/hive-schema-3.0.0.mssql.sql
+++ b/standalone-metastore/src/main/sql/mssql/hive-schema-3.0.0.mssql.sql
@@ -1230,15 +1230,6 @@ CREATE UNIQUE INDEX PART_TABLE_PK ON SEQUENCE_TABLE (SEQUENCE_NAME);
 
 INSERT INTO SEQUENCE_TABLE (SEQUENCE_NAME, NEXT_VAL) VALUES ('org.apache.hadoop.hive.metastore.model.MNotificationLog', 1);
 
-CREATE TABLE RUNTIME_STATS (
-  RS_ID bigint primary key,
-  CREATE_TIME bigint NOT NULL,
-  WEIGHT bigint NOT NULL,
-  PAYLOAD varbinary(max)
-);
-
-CREATE INDEX IDX_RUNTIME_STATS_CREATE_TIME ON RUNTIME_STATS(CREATE_TIME);
-
 -- -----------------------------------------------------------------
 -- Record schema version. Should be the last step in the init script
 -- -----------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/standalone-metastore/src/main/sql/mssql/upgrade-2.3.0-to-3.0.0.mssql.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/mssql/upgrade-2.3.0-to-3.0.0.mssql.sql b/standalone-metastore/src/main/sql/mssql/upgrade-2.3.0-to-3.0.0.mssql.sql
index c2504d3..87f5884 100644
--- a/standalone-metastore/src/main/sql/mssql/upgrade-2.3.0-to-3.0.0.mssql.sql
+++ b/standalone-metastore/src/main/sql/mssql/upgrade-2.3.0-to-3.0.0.mssql.sql
@@ -306,15 +306,6 @@ PRIMARY KEY CLUSTERED
 
 CREATE INDEX MIN_HISTORY_LEVEL_IDX ON MIN_HISTORY_LEVEL (MHL_MIN_OPEN_TXNID);
 
-CREATE TABLE RUNTIME_STATS (
-  RS_ID bigint primary key,
-  CREATE_TIME bigint NOT NULL,
-  WEIGHT bigint NOT NULL,
-  PAYLOAD varbinary(max)
-);
-
-CREATE INDEX IDX_RUNTIME_STATS_CREATE_TIME ON RUNTIME_STATS(CREATE_TIME);
-
 -- These lines need to be last.  Insert any changes above.
 UPDATE VERSION SET SCHEMA_VERSION='3.0.0', VERSION_COMMENT='Hive release version 3.0.0' where VER_ID=1;
 SELECT 'Finished upgrading MetaStore schema from 2.3.0 to 3.0.0' AS MESSAGE;

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/standalone-metastore/src/main/sql/mysql/hive-schema-3.0.0.mysql.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/mysql/hive-schema-3.0.0.mysql.sql b/standalone-metastore/src/main/sql/mysql/hive-schema-3.0.0.mysql.sql
index 4309911..7e2a57a 100644
--- a/standalone-metastore/src/main/sql/mysql/hive-schema-3.0.0.mysql.sql
+++ b/standalone-metastore/src/main/sql/mysql/hive-schema-3.0.0.mysql.sql
@@ -1154,16 +1154,6 @@ CREATE TABLE REPL_TXN_MAP (
   PRIMARY KEY (RTM_REPL_POLICY, RTM_SRC_TXN_ID)
 ) ENGINE=InnoDB DEFAULT CHARSET=latin1;
 
-
-CREATE TABLE RUNTIME_STATS (
-  RS_ID bigint primary key,
-  CREATE_TIME bigint NOT NULL,
-  WEIGHT bigint NOT NULL,
-  PAYLOAD blob
-) ENGINE=InnoDB DEFAULT CHARSET=latin1;
-
-CREATE INDEX IDX_RUNTIME_STATS_CREATE_TIME ON RUNTIME_STATS(CREATE_TIME);
-
 -- -----------------------------------------------------------------
 -- Record schema version. Should be the last step in the init script
 -- -----------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/standalone-metastore/src/main/sql/mysql/upgrade-2.3.0-to-3.0.0.mysql.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/mysql/upgrade-2.3.0-to-3.0.0.mysql.sql b/standalone-metastore/src/main/sql/mysql/upgrade-2.3.0-to-3.0.0.mysql.sql
index e01b4da..5ba68ca 100644
--- a/standalone-metastore/src/main/sql/mysql/upgrade-2.3.0-to-3.0.0.mysql.sql
+++ b/standalone-metastore/src/main/sql/mysql/upgrade-2.3.0-to-3.0.0.mysql.sql
@@ -279,15 +279,6 @@ CREATE TABLE MIN_HISTORY_LEVEL (
 
 CREATE INDEX MIN_HISTORY_LEVEL_IDX ON MIN_HISTORY_LEVEL (MHL_MIN_OPEN_TXNID);
 
-CREATE TABLE RUNTIME_STATS (
-  RS_ID bigint primary key,
-  CREATE_TIME bigint NOT NULL,
-  WEIGHT bigint NOT NULL,
-  PAYLOAD blob
-) ENGINE=InnoDB DEFAULT CHARSET=latin1;
-
-CREATE INDEX IDX_RUNTIME_STATS_CREATE_TIME ON RUNTIME_STATS(CREATE_TIME);
-
 -- These lines need to be last.  Insert any changes above.
 UPDATE VERSION SET SCHEMA_VERSION='3.0.0', VERSION_COMMENT='Hive release version 3.0.0' where VER_ID=1;
 SELECT 'Finished upgrading MetaStore schema from 2.3.0 to 3.0.0' AS ' ';

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/standalone-metastore/src/main/sql/oracle/hive-schema-3.0.0.oracle.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/oracle/hive-schema-3.0.0.oracle.sql b/standalone-metastore/src/main/sql/oracle/hive-schema-3.0.0.oracle.sql
index a45c7bb..f9e1a19 100644
--- a/standalone-metastore/src/main/sql/oracle/hive-schema-3.0.0.oracle.sql
+++ b/standalone-metastore/src/main/sql/oracle/hive-schema-3.0.0.oracle.sql
@@ -1124,16 +1124,6 @@ CREATE TABLE REPL_TXN_MAP (
   PRIMARY KEY (RTM_REPL_POLICY, RTM_SRC_TXN_ID)
 );
 
-CREATE TABLE RUNTIME_STATS (
-  RS_ID NUMBER primary key,
-  CREATE_TIME NUMBER(10) NOT NULL,
-  WEIGHT NUMBER(10) NOT NULL,
-  PAYLOAD BLOB
-);
-
-CREATE INDEX IDX_RUNTIME_STATS_CREATE_TIME ON RUNTIME_STATS(CREATE_TIME);
-
-
 -- -----------------------------------------------------------------
 -- Record schema version. Should be the last step in the init script
 -- -----------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/standalone-metastore/src/main/sql/oracle/upgrade-2.3.0-to-3.0.0.oracle.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/oracle/upgrade-2.3.0-to-3.0.0.oracle.sql b/standalone-metastore/src/main/sql/oracle/upgrade-2.3.0-to-3.0.0.oracle.sql
index 327800b..a769d24 100644
--- a/standalone-metastore/src/main/sql/oracle/upgrade-2.3.0-to-3.0.0.oracle.sql
+++ b/standalone-metastore/src/main/sql/oracle/upgrade-2.3.0-to-3.0.0.oracle.sql
@@ -297,15 +297,6 @@ CREATE TABLE MIN_HISTORY_LEVEL (
 
 CREATE INDEX MIN_HISTORY_LEVEL_IDX ON MIN_HISTORY_LEVEL (MHL_MIN_OPEN_TXNID);
 
-CREATE TABLE RUNTIME_STATS (
-  RS_ID NUMBER primary key,
-  CREATE_TIME NUMBER(10) NOT NULL,
-  WEIGHT NUMBER(10) NOT NULL,
-  PAYLOAD BLOB
-);
-
-CREATE INDEX IDX_RUNTIME_STATS_CREATE_TIME ON RUNTIME_STATS(CREATE_TIME);
-
 -- These lines need to be last.  Insert any changes above.
 UPDATE VERSION SET SCHEMA_VERSION='3.0.0', VERSION_COMMENT='Hive release version 3.0.0' where VER_ID=1;
 SELECT 'Finished upgrading MetaStore schema from 2.3.0 to 3.0.0' AS Status from dual;

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/standalone-metastore/src/main/sql/postgres/hive-schema-3.0.0.postgres.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/postgres/hive-schema-3.0.0.postgres.sql b/standalone-metastore/src/main/sql/postgres/hive-schema-3.0.0.postgres.sql
index 2484744..6fed072 100644
--- a/standalone-metastore/src/main/sql/postgres/hive-schema-3.0.0.postgres.sql
+++ b/standalone-metastore/src/main/sql/postgres/hive-schema-3.0.0.postgres.sql
@@ -1811,17 +1811,6 @@ CREATE TABLE REPL_TXN_MAP (
   PRIMARY KEY (RTM_REPL_POLICY, RTM_SRC_TXN_ID)
 );
 
-
-CREATE TABLE RUNTIME_STATS (
- RS_ID bigint primary key,
- CREATE_TIME bigint NOT NULL,
- WEIGHT bigint NOT NULL,
- PAYLOAD bytea
-);
-
-CREATE INDEX IDX_RUNTIME_STATS_CREATE_TIME ON RUNTIME_STATS(CREATE_TIME);
-
-
 -- -----------------------------------------------------------------
 -- Record schema version. Should be the last step in the init script
 -- -----------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/standalone-metastore/src/main/sql/postgres/upgrade-2.3.0-to-3.0.0.postgres.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/postgres/upgrade-2.3.0-to-3.0.0.postgres.sql b/standalone-metastore/src/main/sql/postgres/upgrade-2.3.0-to-3.0.0.postgres.sql
index 63932a9..7b6b3b7 100644
--- a/standalone-metastore/src/main/sql/postgres/upgrade-2.3.0-to-3.0.0.postgres.sql
+++ b/standalone-metastore/src/main/sql/postgres/upgrade-2.3.0-to-3.0.0.postgres.sql
@@ -314,15 +314,6 @@ CREATE TABLE MIN_HISTORY_LEVEL (
 
 CREATE INDEX MIN_HISTORY_LEVEL_IDX ON MIN_HISTORY_LEVEL (MHL_MIN_OPEN_TXNID);
 
-CREATE TABLE RUNTIME_STATS (
- RS_ID bigint primary key,
- CREATE_TIME bigint NOT NULL,
- WEIGHT bigint NOT NULL,
- PAYLOAD bytea
-);
-
-CREATE INDEX IDX_RUNTIME_STATS_CREATE_TIME ON RUNTIME_STATS(CREATE_TIME);
-
 -- These lines need to be last.  Insert any changes above.
 UPDATE "VERSION" SET "SCHEMA_VERSION"='3.0.0', "VERSION_COMMENT"='Hive release version 3.0.0' where "VER_ID"=1;
 SELECT 'Finished upgrading MetaStore schema from 2.3.0 to 3.0.0';

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/standalone-metastore/src/main/thrift/hive_metastore.thrift
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/thrift/hive_metastore.thrift b/standalone-metastore/src/main/thrift/hive_metastore.thrift
index c56a4f9..12e4e40 100644
--- a/standalone-metastore/src/main/thrift/hive_metastore.thrift
+++ b/standalone-metastore/src/main/thrift/hive_metastore.thrift
@@ -1517,15 +1517,6 @@ struct GetSerdeRequest {
   1: string serdeName
 }
 
-struct RuntimeStat {
-  1: optional i32 createTime,
-  2: required i32 weight,
-  3: required binary payload
-}
-
-struct GetRuntimeStatsRequest {
-}
-
 // Exceptions.
 
 exception MetaException {
@@ -2180,9 +2171,6 @@ service ThriftHiveMetastore extends fb303.FacebookService
 
   LockResponse get_lock_materialization_rebuild(1: string dbName, 2: string tableName, 3: i64 txnId)
   bool heartbeat_lock_materialization_rebuild(1: string dbName, 2: string tableName, 3: i64 txnId)
-  
-  void add_runtime_stats(1: RuntimeStat stat) throws(1:MetaException o1)
-  list<RuntimeStat> get_runtime_stats(1: GetRuntimeStatsRequest rqst) throws(1:MetaException o1)
 }
 
 // * Note about the DDL_TIME: When creating or altering a table or a partition,

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
index defc68f..304f567 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
@@ -66,7 +66,6 @@ import org.apache.hadoop.hive.metastore.api.WMTrigger;
 import org.apache.hadoop.hive.metastore.api.WMValidateResourcePlanResponse;
 import org.apache.hadoop.hive.metastore.api.Role;
 import org.apache.hadoop.hive.metastore.api.RolePrincipalGrant;
-import org.apache.hadoop.hive.metastore.api.RuntimeStat;
 import org.apache.hadoop.hive.metastore.api.SQLCheckConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLDefaultConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
@@ -281,7 +280,6 @@ public class DummyRawStoreControlledCommit implements RawStore, Configurable {
     objectStore.updateCreationMetadata(catName, dbname, tablename, cm);
   }
 
-  @Override
   public List<String> getTables(String catName, String dbName, String pattern) throws MetaException {
     return objectStore.getTables(catName, dbName, pattern);
   }
@@ -1094,7 +1092,6 @@ public class DummyRawStoreControlledCommit implements RawStore, Configurable {
     return null;
   }
 
-  @Override
   public void createISchema(ISchema schema) throws AlreadyExistsException, MetaException,
       NoSuchObjectException {
     objectStore.createISchema(schema);
@@ -1164,19 +1161,4 @@ public class DummyRawStoreControlledCommit implements RawStore, Configurable {
   public void addSerde(SerDeInfo serde) throws AlreadyExistsException, MetaException {
     objectStore.addSerde(serde);
   }
-
-  @Override
-  public void addRuntimeStat(RuntimeStat stat) throws MetaException {
-    objectStore.addRuntimeStat(stat);
-  }
-
-  @Override
-  public List<RuntimeStat> getRuntimeStats() throws MetaException {
-    return objectStore.getRuntimeStats();
-  }
-
-  @Override
-  public int deleteRuntimeStats(int maxRetained, int maxRetainSecs) throws MetaException {
-    return objectStore.deleteRuntimeStats(maxRetained, maxRetainSecs);
-  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
index 20c5d8a..85c6727 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
@@ -64,7 +64,6 @@ import org.apache.hadoop.hive.metastore.api.WMTrigger;
 import org.apache.hadoop.hive.metastore.api.WMValidateResourcePlanResponse;
 import org.apache.hadoop.hive.metastore.api.Role;
 import org.apache.hadoop.hive.metastore.api.RolePrincipalGrant;
-import org.apache.hadoop.hive.metastore.api.RuntimeStat;
 import org.apache.hadoop.hive.metastore.api.SQLCheckConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLDefaultConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
@@ -277,7 +276,6 @@ public class DummyRawStoreForJdoConnection implements RawStore {
       throws MetaException {
   }
 
-  @Override
   public List<String> getTables(String catName, String dbName, String pattern) throws MetaException {
     return Collections.emptyList();
   }
@@ -1082,7 +1080,6 @@ public class DummyRawStoreForJdoConnection implements RawStore {
     return null;
   }
 
-  @Override
   public void createISchema(ISchema schema) throws AlreadyExistsException, MetaException {
 
   }
@@ -1151,18 +1148,4 @@ public class DummyRawStoreForJdoConnection implements RawStore {
   public void addSerde(SerDeInfo serde) throws AlreadyExistsException, MetaException {
 
   }
-
-  @Override
-  public void addRuntimeStat(RuntimeStat stat) throws MetaException {
-  }
-
-  @Override
-  public List<RuntimeStat> getRuntimeStats() throws MetaException {
-    return Collections.emptyList();
-  }
-
-  @Override
-  public int deleteRuntimeStats(int maxRetained, int maxRetainSecs) throws MetaException {
-    return 0;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java
index bf87cfc..cb51763 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java
@@ -3352,14 +3352,4 @@ public class HiveMetaStoreClientPreCatalog implements IMetaStoreClient, AutoClos
   public boolean heartbeatLockMaterializationRebuild(String dbName, String tableName, long txnId) throws TException {
     throw new UnsupportedOperationException();
   }
-
-  @Override
-  public void addRuntimeStat(RuntimeStat stat) throws TException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public List<RuntimeStat> getRuntimeStats() throws TException {
-    throw new UnsupportedOperationException();
-  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestRuntimeStats.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestRuntimeStats.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestRuntimeStats.java
deleted file mode 100644
index 2db7a8b..0000000
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestRuntimeStats.java
+++ /dev/null
@@ -1,100 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hive.metastore.client;
-
-import org.apache.hadoop.hive.metastore.IMetaStoreClient;
-import org.apache.hadoop.hive.metastore.annotation.MetastoreUnitTest;
-import org.apache.hadoop.hive.metastore.api.RuntimeStat;
-import org.apache.hadoop.hive.metastore.minihms.AbstractMetaStoreService;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import java.util.List;
-
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotEquals;
-import static org.junit.Assert.assertNotNull;
-
-@RunWith(Parameterized.class)
-@Category(MetastoreUnitTest.class)
-public class TestRuntimeStats extends MetaStoreClientTest {
-  private final AbstractMetaStoreService metaStore;
-  private IMetaStoreClient client;
-
-  public TestRuntimeStats(String name, AbstractMetaStoreService metaStore) throws Exception {
-    this.metaStore = metaStore;
-    this.metaStore.start();
-  }
-
-  @Before
-  public void setUp() throws Exception {
-    client = metaStore.getClient();
-
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    client.close();
-    client = null;
-  }
-
-  @Test
-  public void testRuntimeStatHandling() throws Exception {
-    List<RuntimeStat> rs0 = client.getRuntimeStats();
-    assertNotNull(rs0);
-    assertEquals(0, rs0.size());
-
-    RuntimeStat stat = createStat(1);
-    client.addRuntimeStat(stat);
-
-    List<RuntimeStat> rs1 = client.getRuntimeStats();
-    assertNotNull(rs1);
-    assertEquals(1, rs1.size());
-    assertArrayEquals(stat.getPayload(), rs1.get(0).getPayload());
-    assertEquals(stat.getWeight(), rs1.get(0).getWeight());
-    // server sets createtime
-    assertNotEquals(stat.getCreateTime(), rs1.get(0).getCreateTime());
-
-    client.addRuntimeStat(createStat(2));
-    client.addRuntimeStat(createStat(3));
-    client.addRuntimeStat(createStat(4));
-
-    List<RuntimeStat> rs2 = client.getRuntimeStats();
-    assertEquals(4, rs2.size());
-
-  }
-
-  private RuntimeStat createStat(int w) {
-
-    byte[] payload = new byte[w];
-    for (int i = 0; i < payload.length; i++) {
-      payload[i] = 'x';
-    }
-
-    RuntimeStat stat = new RuntimeStat();
-    stat.setWeight(w);
-    stat.setPayload(payload);
-    return stat;
-  }
-
-}


[3/6] hive git commit: Revert "HIVE-19171 : Persist runtime statistics in metastore (Zoltan Haindrich via Ashutosh Chauhan)"

Posted by pr...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
index e2f0e82..a354f27 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
@@ -446,10 +446,6 @@ import org.slf4j.LoggerFactory;
 
     public boolean heartbeat_lock_materialization_rebuild(String dbName, String tableName, long txnId) throws org.apache.thrift.TException;
 
-    public void add_runtime_stats(RuntimeStat stat) throws MetaException, org.apache.thrift.TException;
-
-    public List<RuntimeStat> get_runtime_stats(GetRuntimeStatsRequest rqst) throws MetaException, org.apache.thrift.TException;
-
   }
 
   @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public interface AsyncIface extends com.facebook.fb303.FacebookService .AsyncIface {
@@ -858,10 +854,6 @@ import org.slf4j.LoggerFactory;
 
     public void heartbeat_lock_materialization_rebuild(String dbName, String tableName, long txnId, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
-    public void add_runtime_stats(RuntimeStat stat, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
-
-    public void get_runtime_stats(GetRuntimeStatsRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
-
   }
 
   @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class Client extends com.facebook.fb303.FacebookService.Client implements Iface {
@@ -6700,55 +6692,6 @@ import org.slf4j.LoggerFactory;
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "heartbeat_lock_materialization_rebuild failed: unknown result");
     }
 
-    public void add_runtime_stats(RuntimeStat stat) throws MetaException, org.apache.thrift.TException
-    {
-      send_add_runtime_stats(stat);
-      recv_add_runtime_stats();
-    }
-
-    public void send_add_runtime_stats(RuntimeStat stat) throws org.apache.thrift.TException
-    {
-      add_runtime_stats_args args = new add_runtime_stats_args();
-      args.setStat(stat);
-      sendBase("add_runtime_stats", args);
-    }
-
-    public void recv_add_runtime_stats() throws MetaException, org.apache.thrift.TException
-    {
-      add_runtime_stats_result result = new add_runtime_stats_result();
-      receiveBase(result, "add_runtime_stats");
-      if (result.o1 != null) {
-        throw result.o1;
-      }
-      return;
-    }
-
-    public List<RuntimeStat> get_runtime_stats(GetRuntimeStatsRequest rqst) throws MetaException, org.apache.thrift.TException
-    {
-      send_get_runtime_stats(rqst);
-      return recv_get_runtime_stats();
-    }
-
-    public void send_get_runtime_stats(GetRuntimeStatsRequest rqst) throws org.apache.thrift.TException
-    {
-      get_runtime_stats_args args = new get_runtime_stats_args();
-      args.setRqst(rqst);
-      sendBase("get_runtime_stats", args);
-    }
-
-    public List<RuntimeStat> recv_get_runtime_stats() throws MetaException, org.apache.thrift.TException
-    {
-      get_runtime_stats_result result = new get_runtime_stats_result();
-      receiveBase(result, "get_runtime_stats");
-      if (result.isSetSuccess()) {
-        return result.success;
-      }
-      if (result.o1 != null) {
-        throw result.o1;
-      }
-      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "get_runtime_stats failed: unknown result");
-    }
-
   }
   @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class AsyncClient extends com.facebook.fb303.FacebookService.AsyncClient implements AsyncIface {
     @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class Factory implements org.apache.thrift.async.TAsyncClientFactory<AsyncClient> {
@@ -13717,70 +13660,6 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    public void add_runtime_stats(RuntimeStat stat, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
-      checkReady();
-      add_runtime_stats_call method_call = new add_runtime_stats_call(stat, resultHandler, this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class add_runtime_stats_call extends org.apache.thrift.async.TAsyncMethodCall {
-      private RuntimeStat stat;
-      public add_runtime_stats_call(RuntimeStat stat, 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.stat = stat;
-      }
-
-      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("add_runtime_stats", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        add_runtime_stats_args args = new add_runtime_stats_args();
-        args.setStat(stat);
-        args.write(prot);
-        prot.writeMessageEnd();
-      }
-
-      public void getResult() throws MetaException, org.apache.thrift.TException {
-        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
-          throw new IllegalStateException("Method call not finished!");
-        }
-        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
-        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
-        (new Client(prot)).recv_add_runtime_stats();
-      }
-    }
-
-    public void get_runtime_stats(GetRuntimeStatsRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
-      checkReady();
-      get_runtime_stats_call method_call = new get_runtime_stats_call(rqst, resultHandler, this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_runtime_stats_call extends org.apache.thrift.async.TAsyncMethodCall {
-      private GetRuntimeStatsRequest rqst;
-      public get_runtime_stats_call(GetRuntimeStatsRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
-        super(client, protocolFactory, transport, resultHandler, false);
-        this.rqst = rqst;
-      }
-
-      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("get_runtime_stats", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        get_runtime_stats_args args = new get_runtime_stats_args();
-        args.setRqst(rqst);
-        args.write(prot);
-        prot.writeMessageEnd();
-      }
-
-      public List<RuntimeStat> getResult() throws MetaException, org.apache.thrift.TException {
-        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
-          throw new IllegalStateException("Method call not finished!");
-        }
-        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
-        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
-        return (new Client(prot)).recv_get_runtime_stats();
-      }
-    }
-
   }
 
   @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class Processor<I extends Iface> extends com.facebook.fb303.FacebookService.Processor<I> implements org.apache.thrift.TProcessor {
@@ -13996,8 +13875,6 @@ import org.slf4j.LoggerFactory;
       processMap.put("get_serde", new get_serde());
       processMap.put("get_lock_materialization_rebuild", new get_lock_materialization_rebuild());
       processMap.put("heartbeat_lock_materialization_rebuild", new heartbeat_lock_materialization_rebuild());
-      processMap.put("add_runtime_stats", new add_runtime_stats());
-      processMap.put("get_runtime_stats", new get_runtime_stats());
       return processMap;
     }
 
@@ -19191,54 +19068,6 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class add_runtime_stats<I extends Iface> extends org.apache.thrift.ProcessFunction<I, add_runtime_stats_args> {
-      public add_runtime_stats() {
-        super("add_runtime_stats");
-      }
-
-      public add_runtime_stats_args getEmptyArgsInstance() {
-        return new add_runtime_stats_args();
-      }
-
-      protected boolean isOneway() {
-        return false;
-      }
-
-      public add_runtime_stats_result getResult(I iface, add_runtime_stats_args args) throws org.apache.thrift.TException {
-        add_runtime_stats_result result = new add_runtime_stats_result();
-        try {
-          iface.add_runtime_stats(args.stat);
-        } catch (MetaException o1) {
-          result.o1 = o1;
-        }
-        return result;
-      }
-    }
-
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_runtime_stats<I extends Iface> extends org.apache.thrift.ProcessFunction<I, get_runtime_stats_args> {
-      public get_runtime_stats() {
-        super("get_runtime_stats");
-      }
-
-      public get_runtime_stats_args getEmptyArgsInstance() {
-        return new get_runtime_stats_args();
-      }
-
-      protected boolean isOneway() {
-        return false;
-      }
-
-      public get_runtime_stats_result getResult(I iface, get_runtime_stats_args args) throws org.apache.thrift.TException {
-        get_runtime_stats_result result = new get_runtime_stats_result();
-        try {
-          result.success = iface.get_runtime_stats(args.rqst);
-        } catch (MetaException o1) {
-          result.o1 = o1;
-        }
-        return result;
-      }
-    }
-
   }
 
   @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class AsyncProcessor<I extends AsyncIface> extends com.facebook.fb303.FacebookService.AsyncProcessor<I> {
@@ -19454,8 +19283,6 @@ import org.slf4j.LoggerFactory;
       processMap.put("get_serde", new get_serde());
       processMap.put("get_lock_materialization_rebuild", new get_lock_materialization_rebuild());
       processMap.put("heartbeat_lock_materialization_rebuild", new heartbeat_lock_materialization_rebuild());
-      processMap.put("add_runtime_stats", new add_runtime_stats());
-      processMap.put("get_runtime_stats", new get_runtime_stats());
       return processMap;
     }
 
@@ -31844,137 +31671,385 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class add_runtime_stats<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, add_runtime_stats_args, Void> {
-      public add_runtime_stats() {
-        super("add_runtime_stats");
+  }
+
+  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class getMetaConf_args implements org.apache.thrift.TBase<getMetaConf_args, getMetaConf_args._Fields>, java.io.Serializable, Cloneable, Comparable<getMetaConf_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getMetaConf_args");
+
+    private static final org.apache.thrift.protocol.TField KEY_FIELD_DESC = new org.apache.thrift.protocol.TField("key", org.apache.thrift.protocol.TType.STRING, (short)1);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new getMetaConf_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new getMetaConf_argsTupleSchemeFactory());
+    }
+
+    private String key; // 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 {
+      KEY((short)1, "key");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
       }
 
-      public add_runtime_stats_args getEmptyArgsInstance() {
-        return new add_runtime_stats_args();
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // KEY
+            return KEY;
+          default:
+            return null;
+        }
       }
 
-      public AsyncMethodCallback<Void> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
-        final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new AsyncMethodCallback<Void>() { 
-          public void onComplete(Void o) {
-            add_runtime_stats_result result = new add_runtime_stats_result();
-            try {
-              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
-              return;
-            } catch (Exception e) {
-              LOGGER.error("Exception writing to internal frame buffer", e);
-            }
-            fb.close();
-          }
-          public void onError(Exception e) {
-            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
-            org.apache.thrift.TBase msg;
-            add_runtime_stats_result result = new add_runtime_stats_result();
-            if (e instanceof MetaException) {
-                        result.o1 = (MetaException) e;
-                        result.setO1IsSet(true);
-                        msg = result;
-            }
-             else 
-            {
-              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
-              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
-            }
-            try {
-              fcall.sendResponse(fb,msg,msgType,seqid);
-              return;
-            } catch (Exception ex) {
-              LOGGER.error("Exception writing to internal frame buffer", ex);
-            }
-            fb.close();
-          }
-        };
+      /**
+       * 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;
       }
 
-      protected boolean isOneway() {
+      /**
+       * 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.KEY, new org.apache.thrift.meta_data.FieldMetaData("key", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getMetaConf_args.class, metaDataMap);
+    }
+
+    public getMetaConf_args() {
+    }
+
+    public getMetaConf_args(
+      String key)
+    {
+      this();
+      this.key = key;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public getMetaConf_args(getMetaConf_args other) {
+      if (other.isSetKey()) {
+        this.key = other.key;
+      }
+    }
+
+    public getMetaConf_args deepCopy() {
+      return new getMetaConf_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.key = null;
+    }
+
+    public String getKey() {
+      return this.key;
+    }
+
+    public void setKey(String key) {
+      this.key = key;
+    }
+
+    public void unsetKey() {
+      this.key = null;
+    }
+
+    /** Returns true if field key is set (has been assigned a value) and false otherwise */
+    public boolean isSetKey() {
+      return this.key != null;
+    }
+
+    public void setKeyIsSet(boolean value) {
+      if (!value) {
+        this.key = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case KEY:
+        if (value == null) {
+          unsetKey();
+        } else {
+          setKey((String)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case KEY:
+        return getKey();
+
+      }
+      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 KEY:
+        return isSetKey();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
         return false;
+      if (that instanceof getMetaConf_args)
+        return this.equals((getMetaConf_args)that);
+      return false;
+    }
+
+    public boolean equals(getMetaConf_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_key = true && this.isSetKey();
+      boolean that_present_key = true && that.isSetKey();
+      if (this_present_key || that_present_key) {
+        if (!(this_present_key && that_present_key))
+          return false;
+        if (!this.key.equals(that.key))
+          return false;
       }
 
-      public void start(I iface, add_runtime_stats_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
-        iface.add_runtime_stats(args.stat,resultHandler);
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      List<Object> list = new ArrayList<Object>();
+
+      boolean present_key = true && (isSetKey());
+      list.add(present_key);
+      if (present_key)
+        list.add(key);
+
+      return list.hashCode();
+    }
+
+    @Override
+    public int compareTo(getMetaConf_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
       }
+
+      int lastComparison = 0;
+
+      lastComparison = Boolean.valueOf(isSetKey()).compareTo(other.isSetKey());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetKey()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.key, other.key);
+        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);
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_runtime_stats<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_runtime_stats_args, List<RuntimeStat>> {
-      public get_runtime_stats() {
-        super("get_runtime_stats");
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("getMetaConf_args(");
+      boolean first = true;
+
+      sb.append("key:");
+      if (this.key == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.key);
       }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+    }
 
-      public get_runtime_stats_args getEmptyArgsInstance() {
-        return new get_runtime_stats_args();
+    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);
       }
+    }
 
-      public AsyncMethodCallback<List<RuntimeStat>> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
-        final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new AsyncMethodCallback<List<RuntimeStat>>() { 
-          public void onComplete(List<RuntimeStat> o) {
-            get_runtime_stats_result result = new get_runtime_stats_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();
+    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 getMetaConf_argsStandardSchemeFactory implements SchemeFactory {
+      public getMetaConf_argsStandardScheme getScheme() {
+        return new getMetaConf_argsStandardScheme();
+      }
+    }
+
+    private static class getMetaConf_argsStandardScheme extends StandardScheme<getMetaConf_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, getMetaConf_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;
           }
-          public void onError(Exception e) {
-            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
-            org.apache.thrift.TBase msg;
-            get_runtime_stats_result result = new get_runtime_stats_result();
-            if (e instanceof MetaException) {
-                        result.o1 = (MetaException) e;
-                        result.setO1IsSet(true);
-                        msg = result;
-            }
-             else 
-            {
-              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
-              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
-            }
-            try {
-              fcall.sendResponse(fb,msg,msgType,seqid);
-              return;
-            } catch (Exception ex) {
-              LOGGER.error("Exception writing to internal frame buffer", ex);
-            }
-            fb.close();
+          switch (schemeField.id) {
+            case 1: // KEY
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+                struct.key = iprot.readString();
+                struct.setKeyIsSet(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();
       }
 
-      protected boolean isOneway() {
-        return false;
+      public void write(org.apache.thrift.protocol.TProtocol oprot, getMetaConf_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.key != null) {
+          oprot.writeFieldBegin(KEY_FIELD_DESC);
+          oprot.writeString(struct.key);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class getMetaConf_argsTupleSchemeFactory implements SchemeFactory {
+      public getMetaConf_argsTupleScheme getScheme() {
+        return new getMetaConf_argsTupleScheme();
+      }
+    }
+
+    private static class getMetaConf_argsTupleScheme extends TupleScheme<getMetaConf_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, getMetaConf_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.isSetKey()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetKey()) {
+          oprot.writeString(struct.key);
+        }
       }
 
-      public void start(I iface, get_runtime_stats_args args, org.apache.thrift.async.AsyncMethodCallback<List<RuntimeStat>> resultHandler) throws TException {
-        iface.get_runtime_stats(args.rqst,resultHandler);
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, getMetaConf_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.key = iprot.readString();
+          struct.setKeyIsSet(true);
+        }
       }
     }
 
   }
 
-  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class getMetaConf_args implements org.apache.thrift.TBase<getMetaConf_args, getMetaConf_args._Fields>, java.io.Serializable, Cloneable, Comparable<getMetaConf_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getMetaConf_args");
+  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class getMetaConf_result implements org.apache.thrift.TBase<getMetaConf_result, getMetaConf_result._Fields>, java.io.Serializable, Cloneable, Comparable<getMetaConf_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getMetaConf_result");
 
-    private static final org.apache.thrift.protocol.TField KEY_FIELD_DESC = new org.apache.thrift.protocol.TField("key", org.apache.thrift.protocol.TType.STRING, (short)1);
+    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRING, (short)0);
+    private static final org.apache.thrift.protocol.TField O1_FIELD_DESC = new org.apache.thrift.protocol.TField("o1", 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 getMetaConf_argsStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new getMetaConf_argsTupleSchemeFactory());
+      schemes.put(StandardScheme.class, new getMetaConf_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new getMetaConf_resultTupleSchemeFactory());
     }
 
-    private String key; // required
+    private String success; // required
+    private MetaException o1; // 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 {
-      KEY((short)1, "key");
+      SUCCESS((short)0, "success"),
+      O1((short)1, "o1");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -31989,371 +32064,10 @@ import org.slf4j.LoggerFactory;
        */
       public static _Fields findByThriftId(int fieldId) {
         switch(fieldId) {
-          case 1: // KEY
-            return KEY;
-          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.KEY, new org.apache.thrift.meta_data.FieldMetaData("key", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
-      metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getMetaConf_args.class, metaDataMap);
-    }
-
-    public getMetaConf_args() {
-    }
-
-    public getMetaConf_args(
-      String key)
-    {
-      this();
-      this.key = key;
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public getMetaConf_args(getMetaConf_args other) {
-      if (other.isSetKey()) {
-        this.key = other.key;
-      }
-    }
-
-    public getMetaConf_args deepCopy() {
-      return new getMetaConf_args(this);
-    }
-
-    @Override
-    public void clear() {
-      this.key = null;
-    }
-
-    public String getKey() {
-      return this.key;
-    }
-
-    public void setKey(String key) {
-      this.key = key;
-    }
-
-    public void unsetKey() {
-      this.key = null;
-    }
-
-    /** Returns true if field key is set (has been assigned a value) and false otherwise */
-    public boolean isSetKey() {
-      return this.key != null;
-    }
-
-    public void setKeyIsSet(boolean value) {
-      if (!value) {
-        this.key = null;
-      }
-    }
-
-    public void setFieldValue(_Fields field, Object value) {
-      switch (field) {
-      case KEY:
-        if (value == null) {
-          unsetKey();
-        } else {
-          setKey((String)value);
-        }
-        break;
-
-      }
-    }
-
-    public Object getFieldValue(_Fields field) {
-      switch (field) {
-      case KEY:
-        return getKey();
-
-      }
-      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 KEY:
-        return isSetKey();
-      }
-      throw new IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(Object that) {
-      if (that == null)
-        return false;
-      if (that instanceof getMetaConf_args)
-        return this.equals((getMetaConf_args)that);
-      return false;
-    }
-
-    public boolean equals(getMetaConf_args that) {
-      if (that == null)
-        return false;
-
-      boolean this_present_key = true && this.isSetKey();
-      boolean that_present_key = true && that.isSetKey();
-      if (this_present_key || that_present_key) {
-        if (!(this_present_key && that_present_key))
-          return false;
-        if (!this.key.equals(that.key))
-          return false;
-      }
-
-      return true;
-    }
-
-    @Override
-    public int hashCode() {
-      List<Object> list = new ArrayList<Object>();
-
-      boolean present_key = true && (isSetKey());
-      list.add(present_key);
-      if (present_key)
-        list.add(key);
-
-      return list.hashCode();
-    }
-
-    @Override
-    public int compareTo(getMetaConf_args other) {
-      if (!getClass().equals(other.getClass())) {
-        return getClass().getName().compareTo(other.getClass().getName());
-      }
-
-      int lastComparison = 0;
-
-      lastComparison = Boolean.valueOf(isSetKey()).compareTo(other.isSetKey());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetKey()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.key, other.key);
-        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("getMetaConf_args(");
-      boolean first = true;
-
-      sb.append("key:");
-      if (this.key == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.key);
-      }
-      first = false;
-      sb.append(")");
-      return sb.toString();
-    }
-
-    public void validate() throws org.apache.thrift.TException {
-      // check for required fields
-      // check for sub-struct validity
-    }
-
-    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
-      try {
-        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
-      } catch (org.apache.thrift.TException te) {
-        throw new java.io.IOException(te);
-      }
-    }
-
-    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
-      try {
-        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
-      } catch (org.apache.thrift.TException te) {
-        throw new java.io.IOException(te);
-      }
-    }
-
-    private static class getMetaConf_argsStandardSchemeFactory implements SchemeFactory {
-      public getMetaConf_argsStandardScheme getScheme() {
-        return new getMetaConf_argsStandardScheme();
-      }
-    }
-
-    private static class getMetaConf_argsStandardScheme extends StandardScheme<getMetaConf_args> {
-
-      public void read(org.apache.thrift.protocol.TProtocol iprot, getMetaConf_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: // KEY
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
-                struct.key = iprot.readString();
-                struct.setKeyIsSet(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, getMetaConf_args struct) throws org.apache.thrift.TException {
-        struct.validate();
-
-        oprot.writeStructBegin(STRUCT_DESC);
-        if (struct.key != null) {
-          oprot.writeFieldBegin(KEY_FIELD_DESC);
-          oprot.writeString(struct.key);
-          oprot.writeFieldEnd();
-        }
-        oprot.writeFieldStop();
-        oprot.writeStructEnd();
-      }
-
-    }
-
-    private static class getMetaConf_argsTupleSchemeFactory implements SchemeFactory {
-      public getMetaConf_argsTupleScheme getScheme() {
-        return new getMetaConf_argsTupleScheme();
-      }
-    }
-
-    private static class getMetaConf_argsTupleScheme extends TupleScheme<getMetaConf_args> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, getMetaConf_args struct) throws org.apache.thrift.TException {
-        TTupleProtocol oprot = (TTupleProtocol) prot;
-        BitSet optionals = new BitSet();
-        if (struct.isSetKey()) {
-          optionals.set(0);
-        }
-        oprot.writeBitSet(optionals, 1);
-        if (struct.isSetKey()) {
-          oprot.writeString(struct.key);
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, getMetaConf_args struct) throws org.apache.thrift.TException {
-        TTupleProtocol iprot = (TTupleProtocol) prot;
-        BitSet incoming = iprot.readBitSet(1);
-        if (incoming.get(0)) {
-          struct.key = iprot.readString();
-          struct.setKeyIsSet(true);
-        }
-      }
-    }
-
-  }
-
-  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class getMetaConf_result implements org.apache.thrift.TBase<getMetaConf_result, getMetaConf_result._Fields>, java.io.Serializable, Cloneable, Comparable<getMetaConf_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getMetaConf_result");
-
-    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRING, (short)0);
-    private static final org.apache.thrift.protocol.TField O1_FIELD_DESC = new org.apache.thrift.protocol.TField("o1", 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 getMetaConf_resultStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new getMetaConf_resultTupleSchemeFactory());
-    }
-
-    private String success; // required
-    private MetaException o1; // 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"),
-      O1((short)1, "o1");
-
-      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;
-          case 1: // O1
-            return O1;
+          case 0: // SUCCESS
+            return SUCCESS;
+          case 1: // O1
+            return O1;
           default:
             return null;
         }
@@ -233792,1608 +233506,4 @@ import org.slf4j.LoggerFactory;
 
   }
 
-  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class add_runtime_stats_args implements org.apache.thrift.TBase<add_runtime_stats_args, add_runtime_stats_args._Fields>, java.io.Serializable, Cloneable, Comparable<add_runtime_stats_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("add_runtime_stats_args");
-
-    private static final org.apache.thrift.protocol.TField STAT_FIELD_DESC = new org.apache.thrift.protocol.TField("stat", 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 add_runtime_stats_argsStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new add_runtime_stats_argsTupleSchemeFactory());
-    }
-
-    private RuntimeStat stat; // 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 {
-      STAT((short)1, "stat");
-
-      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: // STAT
-            return STAT;
-          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.STAT, new org.apache.thrift.meta_data.FieldMetaData("stat", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, RuntimeStat.class)));
-      metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(add_runtime_stats_args.class, metaDataMap);
-    }
-
-    public add_runtime_stats_args() {
-    }
-
-    public add_runtime_stats_args(
-      RuntimeStat stat)
-    {
-      this();
-      this.stat = stat;
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public add_runtime_stats_args(add_runtime_stats_args other) {
-      if (other.isSetStat()) {
-        this.stat = new RuntimeStat(other.stat);
-      }
-    }
-
-    public add_runtime_stats_args deepCopy() {
-      return new add_runtime_stats_args(this);
-    }
-
-    @Override
-    public void clear() {
-      this.stat = null;
-    }
-
-    public RuntimeStat getStat() {
-      return this.stat;
-    }
-
-    public void setStat(RuntimeStat stat) {
-      this.stat = stat;
-    }
-
-    public void unsetStat() {
-      this.stat = null;
-    }
-
-    /** Returns true if field stat is set (has been assigned a value) and false otherwise */
-    public boolean isSetStat() {
-      return this.stat != null;
-    }
-
-    public void setStatIsSet(boolean value) {
-      if (!value) {
-        this.stat = null;
-      }
-    }
-
-    public void setFieldValue(_Fields field, Object value) {
-      switch (field) {
-      case STAT:
-        if (value == null) {
-          unsetStat();
-        } else {
-          setStat((RuntimeStat)value);
-        }
-        break;
-
-      }
-    }
-
-    public Object getFieldValue(_Fields field) {
-      switch (field) {
-      case STAT:
-        return getStat();
-
-      }
-      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 STAT:
-        return isSetStat();
-      }
-      throw new IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(Object that) {
-      if (that == null)
-        return false;
-      if (that instanceof add_runtime_stats_args)
-        return this.equals((add_runtime_stats_args)that);
-      return false;
-    }
-
-    public boolean equals(add_runtime_stats_args that) {
-      if (that == null)
-        return false;
-
-      boolean this_present_stat = true && this.isSetStat();
-      boolean that_present_stat = true && that.isSetStat();
-      if (this_present_stat || that_present_stat) {
-        if (!(this_present_stat && that_present_stat))
-          return false;
-        if (!this.stat.equals(that.stat))
-          return false;
-      }
-
-      return true;
-    }
-
-    @Override
-    public int hashCode() {
-      List<Object> list = new ArrayList<Object>();
-
-      boolean present_stat = true && (isSetStat());
-      list.add(present_stat);
-      if (present_stat)
-        list.add(stat);
-
-      return list.hashCode();
-    }
-
-    @Override
-    public int compareTo(add_runtime_stats_args other) {
-      if (!getClass().equals(other.getClass())) {
-        return getClass().getName().compareTo(other.getClass().getName());
-      }
-
-      int lastComparison = 0;
-
-      lastComparison = Boolean.valueOf(isSetStat()).compareTo(other.isSetStat());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetStat()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.stat, other.stat);
-        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("add_runtime_stats_args(");
-      boolean first = true;
-
-      sb.append("stat:");
-      if (this.stat == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.stat);
-      }
-      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 (stat != null) {
-        stat.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 add_runtime_stats_argsStandardSchemeFactory implements SchemeFactory {
-      public add_runtime_stats_argsStandardScheme getScheme() {
-        return new add_runtime_stats_argsStandardScheme();
-      }
-    }
-
-    private static class add_runtime_stats_argsStandardScheme extends StandardScheme<add_runtime_stats_args> {
-
-      public void read(org.apache.thrift.protocol.TProtocol iprot, add_runtime_stats_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: // STAT
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.stat = new RuntimeStat();
-                struct.stat.read(iprot);
-                struct.setStatIsSet(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, add_runtime_stats_args struct) throws org.apache.thrift.TException {
-        struct.validate();
-
-        oprot.writeStructBegin(STRUCT_DESC);
-        if (struct.stat != null) {
-          oprot.writeFieldBegin(STAT_FIELD_DESC);
-          struct.stat.write(oprot);
-          oprot.writeFieldEnd();
-        }
-        oprot.writeFieldStop();
-        oprot.writeStructEnd();
-      }
-
-    }
-
-    private static class add_runtime_stats_argsTupleSchemeFactory implements SchemeFactory {
-      public add_runtime_stats_argsTupleScheme getScheme() {
-        return new add_runtime_stats_argsTupleScheme();
-      }
-    }
-
-    private static class add_runtime_stats_argsTupleScheme extends TupleScheme<add_runtime_stats_args> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, add_runtime_stats_args struct) throws org.apache.thrift.TException {
-        TTupleProtocol oprot = (TTupleProtocol) prot;
-        BitSet optionals = new BitSet();
-        if (struct.isSetStat()) {
-          optionals.set(0);
-        }
-        oprot.writeBitSet(optionals, 1);
-        if (struct.isSetStat()) {
-          struct.stat.write(oprot);
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, add_runtime_stats_args struct) throws org.apache.thrift.TException {
-        TTupleProtocol iprot = (TTupleProtocol) prot;
-        BitSet incoming = iprot.readBitSet(1);
-        if (incoming.get(0)) {
-          struct.stat = new RuntimeStat();
-          struct.stat.read(iprot);
-          struct.setStatIsSet(true);
-        }
-      }
-    }
-
-  }
-
-  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class add_runtime_stats_result implements org.apache.thrift.TBase<add_runtime_stats_result, add_runtime_stats_result._Fields>, java.io.Serializable, Cloneable, Comparable<add_runtime_stats_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("add_runtime_stats_result");
-
-    private static final org.apache.thrift.protocol.TField O1_FIELD_DESC = new org.apache.thrift.protocol.TField("o1", 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 add_runtime_stats_resultStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new add_runtime_stats_resultTupleSchemeFactory());
-    }
-
-    private MetaException o1; // 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 {
-      O1((short)1, "o1");
-
-      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: // O1
-            return O1;
-          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.O1, new org.apache.thrift.meta_data.FieldMetaData("o1", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(add_runtime_stats_result.class, metaDataMap);
-    }
-
-    public add_runtime_stats_result() {
-    }
-
-    public add_runtime_stats_result(
-      MetaException o1)
-    {
-      this();
-      this.o1 = o1;
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public add_runtime_stats_result(add_runtime_stats_result other) {
-      if (other.isSetO1()) {
-        this.o1 = new MetaException(other.o1);
-      }
-    }
-
-    public add_runtime_stats_result deepCopy() {
-      return new add_runtime_stats_result(this);
-    }
-
-    @Override
-    public void clear() {
-      this.o1 = null;
-    }
-
-    public MetaException getO1() {
-      return this.o1;
-    }
-
-    public void setO1(MetaException o1) {
-      this.o1 = o1;
-    }
-
-    public void unsetO1() {
-      this.o1 = null;
-    }
-
-    /** Returns true if field o1 is set (has been assigned a value) and false otherwise */
-    public boolean isSetO1() {
-      return this.o1 != null;
-    }
-
-    public void setO1IsSet(boolean value) {
-      if (!value) {
-        this.o1 = null;
-      }
-    }
-
-    public void setFieldValue(_Fields field, Object value) {
-      switch (field) {
-      case O1:
-        if (value == null) {
-          unsetO1();
-        } else {
-          setO1((MetaException)value);
-        }
-        break;
-
-      }
-    }
-
-    public Object getFieldValue(_Fields field) {
-      switch (field) {
-      case O1:
-        return getO1();
-
-      }
-      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 O1:
-        return isSetO1();
-      }
-      throw new IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(Object that) {
-      if (that == null)
-        return false;
-      if (that instanceof add_runtime_stats_result)
-        return this.equals((add_runtime_stats_result)that);
-      return false;
-    }
-
-    public boolean equals(add_runtime_stats_result that) {
-      if (that == null)
-        return false;
-
-      boolean this_present_o1 = true && this.isSetO1();
-      boolean that_present_o1 = true && that.isSetO1();
-      if (this_present_o1 || that_present_o1) {
-        if (!(this_present_o1 && that_present_o1))
-          return false;
-        if (!this.o1.equals(that.o1))
-          return false;
-      }
-
-      return true;
-    }
-
-    @Override
-    public int hashCode() {
-      List<Object> list = new ArrayList<Object>();
-
-      boolean present_o1 = true && (isSetO1());
-      list.add(present_o1);
-      if (present_o1)
-        list.add(o1);
-
-      return list.hashCode();
-    }
-
-    @Override
-    public int compareTo(add_runtime_stats_result other) {
-      if (!getClass().equals(other.getClass())) {
-        return getClass().getName().compareTo(other.getClass().getName());
-      }
-
-      int lastComparison = 0;
-
-      lastComparison = Boolean.valueOf(isSetO1()).compareTo(other.isSetO1());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetO1()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o1, other.o1);
-        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("add_runtime_stats_result(");
-      boolean first = true;
-
-      sb.append("o1:");
-      if (this.o1 == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.o1);
-      }
-      first = false;
-      sb.append(")");
-      return sb.toString();
-    }
-
-    public void validate() throws org.apache.thrift.TException {
-      // check for required fields
-      // check for sub-struct validity
-    }
-
-    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
-      try {
-        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
-      } catch (org.apache.thrift.TException te) {
-        throw new java.io.IOException(te);
-      }
-    }
-
-    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
-      try {
-        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
-      } catch (org.apache.thrift.TException te) {
-        throw new java.io.IOException(te);
-      }
-    }
-
-    private static class add_runtime_stats_resultStandardSchemeFactory implements SchemeFactory {
-      public add_runtime_stats_resultStandardScheme getScheme() {
-        return new add_runtime_stats_resultStandardScheme();
-      }
-    }
-
-    private static class add_runtime_stats_resultStandardScheme extends StandardScheme<add_runtime_stats_result> {
-
-      public void read(org.apache.thrift.protocol.TProtocol iprot, add_runtime_stats_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) {
-            case 1: // O1
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.o1 = new MetaException();
-                struct.o1.read(iprot);
-                struct.setO1IsSet(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, add_runtime_stats_result struct) throws org.apache.thrift.TException {
-        struct.validate();
-
-        oprot.writeStructBegin(STRUCT_DESC);
-        if (struct.o1 != null) {
-          oprot.writeFieldBegin(O1_FIELD_DESC);
-          struct.o1.write(oprot);
-          oprot.writeFieldEnd();
-        }
-        oprot.writeFieldStop();
-        oprot.writeStructEnd();
-      }
-
-    }
-
-    private static class add_runtime_stats_resultTupleSchemeFactory implements SchemeFactory {
-      public add_runtime_stats_resultTupleScheme getScheme() {
-        return new add_runtime_stats_resultTupleScheme();
-      }
-    }
-
-    private static class add_runtime_stats_resultTupleScheme extends TupleScheme<add_runtime_stats_result> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, add_runtime_stats_result struct) throws org.apache.thrift.TException {
-        TTupleProtocol oprot = (TTupleProtocol) prot;
-        BitSet optionals = new BitSet();
-        if (struct.isSetO1()) {
-          optionals.set(0);
-        }
-        oprot.writeBitSet(optionals, 1);
-        if (struct.isSetO1()) {
-          struct.o1.write(oprot);
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, add_runtime_stats_result struct) throws org.apache.thrift.TException {
-        TTupleProtocol iprot = (TTupleProtocol) prot;
-        BitSet incoming = iprot.readBitSet(1);
-        if (incoming.get(0)) {
-          struct.o1 = new MetaException();
-          struct.o1.read(iprot);
-          struct.setO1IsSet(true);
-        }
-      }
-    }
-
-  }
-
-  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_runtime_stats_args implements org.apache.thrift.TBase<get_runtime_stats_args, get_runtime_stats_args._Fields>, java.io.Serializable, Cloneable, Comparable<get_runtime_stats_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_runtime_stats_args");
-
-    private static final org.apache.thrift.protocol.TField RQST_FIELD_DESC = new org.apache.thrift.protocol.TField("rqst", 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_runtime_stats_argsStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new get_runtime_stats_argsTupleSchemeFactory());
-    }
-
-    private GetRuntimeStatsRequest rqst; // 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 {
-      RQST((short)1, "rqst");
-
-      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: // RQST
-            return RQST;
-          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.RQST, new org.apache.thrift.meta_data.FieldMetaData("rqst", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, GetRuntimeStatsRequest.class)));
-      metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_runtime_stats_args.class, metaDataMap);
-    }
-
-    public get_runtime_stats_args() {
-    }
-
-    public get_runtime_stats_args(
-      GetRuntimeStatsRequest rqst)
-    {
-      this();
-      this.rqst = rqst;
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public get_runtime_stats_args(get_runtime_stats_args other) {
-      if (other.isSetRqst()) {
-        this.rqst = new GetRuntimeStatsRequest(other.rqst);
-      }
-    }
-
-    public get_runtime_stats_args deepCopy() {
-      return new get_runtime_stats_args(this);
-    }
-
-    @Override
-    public void clear() {
-      this.rqst = null;
-    }
-
-    public GetRuntimeStatsRequest getRqst() {
-      return this.rqst;
-    }
-
-    public void setRqst(GetRuntimeStatsRequest rqst) {
-      this.rqst = rqst;
-    }
-
-    public void unsetRqst() {
-      this.rqst = null;
-    }
-
-    /** Returns true if field rqst is set (has been assigned a value) and false otherwise */
-    public boolean isSetRqst() {
-      return this.rqst != null;
-    }
-
-    public void setRqstIsSet(boolean value) {
-      if (!value) {
-        this.rqst = null;
-      }
-    }
-
-    public void setFieldValue(_Fields field, Object value) {
-      switch (field) {
-      case RQST:
-        if (value == null) {
-          unsetRqst();
-        } else {
-          setRqst((GetRuntimeStatsRequest)value);
-        }
-        break;
-
-      }
-    }
-
-    public Object getFieldValue(_Fields field) {
-      switch (field) {
-      case RQST:
-        return getRqst();
-
-      }
-      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 RQST:
-        return isSetRqst();
-      }
-      throw new IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(Object that) {
-      if (that == null)
-        return false;
-      if (that instanceof get_runtime_stats_args)
-        return this.equals((get_runtime_stats_args)that);
-      return false;
-    }
-
-    public boolean equals(get_runtime_stats_args that) {
-      if (that == null)
-        return false;
-
-      boolean this_present_rqst = true && this.isSetRqst();
-      boolean that_present_rqst = true && that.isSetRqst();
-      if (this_present_rqst || that_present_rqst) {
-        if (!(this_present_rqst && that_present_rqst))
-          return false;
-        if (!this.rqst.equals(that.rqst))
-          return false;
-      }
-
-      return true;
-    }
-
-    @Override
-    public int hashCode() {
-      List<Object> list = new ArrayList<Object>();
-
-      boolean present_rqst = true && (isSetRqst());
-      list.add(present_rqst);
-      if (present_rqst)
-        list.add(rqst);
-
-      return list.hashCode();
-    }
-
-    @Override
-    public int compareTo(get_runtime_stats_args other) {
-      if (!getClass().equals(other.getClass())) {
-        return getClass().getName().compareTo(other.getClass().getName());
-      }
-
-      int lastComparison = 0;
-
-      lastComparison = Boolean.valueOf(isSetRqst()).compareTo(other.isSetRqst());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetRqst()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.rqst, other.rqst);
-        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_runtime_stats_args(");
-      boolean first = true;
-
-      sb.append("rqst:");
-      if (this.rqst == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.rqst);
-      }
-      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 (rqst != null) {
-        rqst.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_runtime_stats_argsStandardSchemeFactory implements SchemeFactory {
-      public get_runtime_stats_argsStandardScheme getScheme() {
-        return new get_runtime_stats_argsStandardScheme();
-      }
-    }
-
-    private static class get_runtime_stats_argsStandardScheme extends StandardScheme<get_runtime_stats_args> {
-
-      public void read(org.apache.thrift.protocol.TProtocol iprot, get_runtime_stats_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: // RQST
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.rqst = new GetRuntimeStatsRequest();
-                struct.rqst.read(iprot);
-                struct.setRqstIsSet(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_runtime_stats_args struct) throws org.apache.thrift.TException {
-        struct.validate();
-
-        oprot.writeStructBegin(STRUCT_DESC);
-        if (struct.rqst != null) {
-          oprot.writeFieldBegin(RQST_FIELD_DESC);
-          struct.rqst.write(oprot);
-          oprot.writeFieldEnd();
-        }
-        oprot.writeFieldStop();
-        oprot.writeStructEnd();
-      }
-
-    }
-
-    private static class get_runtime_stats_argsTupleSchemeFactory implements SchemeFactory {
-      public get_runtime_stats_argsTupleScheme getScheme() {
-        return new get_runtime_stats_argsTupleScheme();
-      }
-    }
-
-    private static class get_runtime_stats_argsTupleScheme extends TupleScheme<get_runtime_stats_args> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, get_runtime_stats_args struct) throws org.apache.thrift.TException {
-        TTupleProtocol oprot = (TTupleProtocol) prot;
-        BitSet optionals = new BitSet();
-        if (struct.isSetRqst()) {
-          optionals.set(0);
-        }
-        oprot.writeBitSet(optionals, 1);
-        if (struct.isSetRqst()) {
-          struct.rqst.write(oprot);
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, get_runtime_stats_args struct) throws org.apache.thrift.TException {
-        TTupleProtocol iprot = (TTupleProtocol) prot;
-        BitSet incoming = iprot.readBitSet(1);
-        if (incoming.get(0)) {
-          struct.rqst = new GetRuntimeStatsRequest();
-          struct.rqst.read(iprot);
-          struct.setRqstIsSet(true);
-        }
-      }
-    }
-
-  }
-
-  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_runtime_stats_result implements org.apache.thrift.TBase<get_runtime_stats_result, get_runtime_stats_result._Fields>, java.io.Serializable, Cloneable, Comparable<get_runtime_stats_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_runtime_stats_result");
-
-    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.LIST, (short)0);
-    private static final org.apache.thrift.protocol.TField O1_FIELD_DESC = new org.apache.thrift.protocol.TField("o1", 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_runtime_stats_resultStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new get_runtime_stats_resultTupleSchemeFactory());
-    }
-
-    private List<RuntimeStat> success; // required
-    private MetaException o1; // 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"),
-      O1((short)1, "o1");
-
-      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;
-          case 1: // O1
-            return O1;
-          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.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
-              new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, RuntimeStat.class))));
-      tmpMap.put(_Fields.O1, new org.apache.thrift.meta_data.FieldMetaData("o1", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
-      metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_runtime_stats_result.class, metaDataMap);
-    }
-
-    public get_runtime_stats_result() {
-    }
-
-    public get_runtime_stats_result(
-      List<RuntimeStat> success,
-      MetaException o1)
-    {
-      this();
-      this.success = success;
-      this.o1 = o1;
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public get_runtime_stats_result(get_runtime_stats_result other) {
-      if (other.isSetSuccess()) {
-        List<RuntimeStat> __this__success = new ArrayList<RuntimeStat>(other.success.size());
-        for (RuntimeStat other_element : other.success) {
-          __this__success.add(new RuntimeStat(other_element));
-        }
-        this.success = __this__success;
-      }
-      if (other.isSetO1()) {
-        this.o1 = new MetaException(other.o1);
-      }
-    }
-
-    public get_runtime_stats_result deepCopy() {
-      return new get_runtime_stats_result(this);
-    }
-
-    @Override
-    public void clear() {
-      this.success = null;
-      this.o1 = null;
-    }
-
-    public int getSuccessSize() {
-      return (this.success == null) ? 0 : this.success.size();
-    }
-
-    public java.util.Iterator<RuntimeStat> getSuccessIterator() {
-      return (this.success == null) ? null : this.success.iterator();
-    }
-
-    public void addToSuccess(RuntimeStat elem) {
-      if (this.success == null) {
-        this.success = new ArrayList<RuntimeStat>();
-      }
-      this.success.add(elem);
-    }
-
-    public List<RuntimeStat> getSuccess() {
-      return this.success;
-    }
-
-    public void setSuccess(List<RuntimeStat> 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 MetaException getO1() {
-      return this.o1;
-    }
-
-    public void setO1(MetaException o1) {
-      this.o1 = o1;
-    }
-
-    public void unsetO1() {
-      this.o1 = null;
-    }
-
-    /** Returns true if field o1 is set (has been assigned a value) and false otherwise */
-    public boolean isSetO1() {
-      return this.o1 != null;
-    }
-
-    public void setO1IsSet(boolean value) {
-      if (!value) {
-        this.o1 = null;
-      }
-    }
-
-    public void setFieldValue(_Fields field, Object value) {
-      switch (field) {
-      case SUCCESS:
-        if (value == null) {
-          unsetSuccess();
-        } else {
-          setSuccess((List<RuntimeStat>)value);
-        }
-        break;
-
-      case O1:
-        if (value == null) {
-          unsetO1();
-        } else {
-          setO1((MetaException)value);
-        }
-        break;
-
-      }
-    }
-
-    public Object getFieldValue(_Fields field) {
-      switch (field) {
-      case SUCCESS:
-        return getSuccess();
-
-      case O1:
-        return getO1();
-
-      }
-      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 SUCCESS:
-        return isSetSuccess();
-      case O1:
-        return isSetO1();
-      }
-      throw new IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(Object that) {
-      if (that == null)
-        return false;
-      if (that instanceof get_runtime_stats_result)
-        return this.equals((get_runtime_stats_result)that);
-      return false;
-    }
-
-    public boolean equals(get_runtime_stats_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;
-      }
-
-      boolean this_present_o1 = true && this.isSetO1();
-      boolean that_present_o1 = true && that.isSetO1();
-      if (this_present_o1 || that_present_o1) {
-        if (!(this_present_o1 && that_present_o1))
-          return false;
-        if (!this.o1.equals(that.o1))
-          return false;
-      }
-
-      return true;
-    }
-
-    @Override
-    public int hashCode() {
-      List<Object> list = new ArrayList<Object>();
-
-      boolean present_success = true && (isSetSuccess());
-      list.add(present_success);
-      if (present_success)
-        list.add(success);
-
-      boolean present_o1 = true && (isSetO1());
-      list.add(present_o1);
-      if (present_o1)
-        list.add(o1);
-
-      return list.hashCode();
-    }
-
-    @Override
-    public int compareTo(get_runtime_stats_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;
-        }
-      }
-      lastComparison = Boolean.valueOf(isSetO1()).compareTo(other.isSetO1());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetO1()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o1, other.o1);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      return 0;
-    }
-
-    public _Fields fieldForId(int fieldId) {
-      return _Fields.fin

<TRUNCATED>

[6/6] hive git commit: Revert "HIVE-19171 : Persist runtime statistics in metastore (Zoltan Haindrich via Ashutosh Chauhan)"

Posted by pr...@apache.org.
Revert "HIVE-19171 : Persist runtime statistics in metastore (Zoltan Haindrich via Ashutosh Chauhan)"

This reverts commit b3e2d8a05f57a91b12b8347b2763a296c3480d97.


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

Branch: refs/heads/master
Commit: f0199500f00ae58cf1a9f73f5baebdc5d5eca417
Parents: 997ad34
Author: Prasanth Jayachandran <pr...@apache.org>
Authored: Mon Apr 23 15:07:41 2018 -0700
Committer: Prasanth Jayachandran <pr...@apache.org>
Committed: Mon Apr 23 15:07:41 2018 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/conf/HiveConf.java   |    6 +-
 .../listener/DummyRawStoreFailEvent.java        |   19 -
 .../org/apache/hadoop/hive/ql/QTestUtil.java    |    2 +-
 .../upgrade/derby/056-HIVE-19171.derby.sql      |   10 -
 .../ql/optimizer/signature/OpSignature.java     |   19 +-
 .../ql/optimizer/signature/OpTreeSignature.java |   24 +-
 .../signature/OpTreeSignatureFactory.java       |   12 +-
 .../ql/optimizer/signature/RuntimeStatsMap.java |   83 -
 .../signature/RuntimeStatsPersister.java        |   54 -
 .../ql/optimizer/signature/SignatureUtils.java  |   22 +-
 .../hadoop/hive/ql/plan/FileSinkDesc.java       |    7 +-
 .../hadoop/hive/ql/plan/HashTableSinkDesc.java  |    6 +-
 .../apache/hadoop/hive/ql/plan/JoinDesc.java    |    6 +-
 .../apache/hadoop/hive/ql/plan/MapJoinDesc.java |    6 +-
 .../hive/ql/plan/mapper/CachingStatsSource.java |    7 +-
 .../hive/ql/plan/mapper/EmptyStatsSource.java   |    2 +-
 .../ql/plan/mapper/MetastoreStatsConnector.java |  143 -
 .../hadoop/hive/ql/plan/mapper/PlanMapper.java  |  108 +-
 .../plan/mapper/SimpleRuntimeStatsSource.java   |   37 +-
 .../hive/ql/plan/mapper/StatsSources.java       |   86 +-
 .../hadoop/hive/ql/reexec/ReOptimizePlugin.java |   17 +-
 .../hadoop/hive/ql/stats/OperatorStats.java     |   33 +-
 .../signature/TestRuntimeStatsPersistence.java  |  165 -
 .../ql/plan/mapping/TestCounterMapping.java     |    7 +-
 .../ql/plan/mapping/TestReOptimization.java     |   85 +-
 .../apache/hive/service/server/HiveServer2.java |    3 -
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.cpp  | 5376 ++++++++----------
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.h    |  259 -
 .../ThriftHiveMetastore_server.skeleton.cpp     |   10 -
 .../gen/thrift/gen-cpp/hive_metastore_types.cpp |  376 +-
 .../gen/thrift/gen-cpp/hive_metastore_types.h   |   97 -
 .../metastore/api/GetRuntimeStatsRequest.java   |  283 -
 .../hadoop/hive/metastore/api/RuntimeStat.java  |  600 --
 .../hive/metastore/api/ThriftHiveMetastore.java | 2584 ++-------
 .../gen-php/metastore/ThriftHiveMetastore.php   |  481 --
 .../src/gen/thrift/gen-php/metastore/Types.php  |  171 -
 .../hive_metastore/ThriftHiveMetastore-remote   |   14 -
 .../hive_metastore/ThriftHiveMetastore.py       |  409 --
 .../gen/thrift/gen-py/hive_metastore/ttypes.py  |  141 -
 .../gen/thrift/gen-rb/hive_metastore_types.rb   |   37 -
 .../gen/thrift/gen-rb/thrift_hive_metastore.rb  |  119 -
 .../hadoop/hive/metastore/HiveMetaStore.java    |  113 +-
 .../hive/metastore/HiveMetaStoreClient.java     |  140 +-
 .../hadoop/hive/metastore/IMetaStoreClient.java |    9 -
 .../hadoop/hive/metastore/ObjectStore.java      |  248 +-
 .../apache/hadoop/hive/metastore/RawStore.java  |   15 +-
 .../hive/metastore/RuntimeStatsCleanerTask.java |   67 -
 .../hive/metastore/cache/CachedStore.java       |   21 +-
 .../hive/metastore/conf/MetastoreConf.java      |   34 +-
 .../hive/metastore/model/MRuntimeStat.java      |   59 -
 .../src/main/resources/package.jdo              |   14 -
 .../main/sql/derby/hive-schema-3.0.0.derby.sql  |   10 -
 .../sql/derby/upgrade-2.3.0-to-3.0.0.derby.sql  |   10 -
 .../main/sql/mssql/hive-schema-3.0.0.mssql.sql  |    9 -
 .../sql/mssql/upgrade-2.3.0-to-3.0.0.mssql.sql  |    9 -
 .../main/sql/mysql/hive-schema-3.0.0.mysql.sql  |   10 -
 .../sql/mysql/upgrade-2.3.0-to-3.0.0.mysql.sql  |    9 -
 .../sql/oracle/hive-schema-3.0.0.oracle.sql     |   10 -
 .../oracle/upgrade-2.3.0-to-3.0.0.oracle.sql    |    9 -
 .../sql/postgres/hive-schema-3.0.0.postgres.sql |   11 -
 .../upgrade-2.3.0-to-3.0.0.postgres.sql         |    9 -
 .../src/main/thrift/hive_metastore.thrift       |   12 -
 .../DummyRawStoreControlledCommit.java          |   18 -
 .../DummyRawStoreForJdoConnection.java          |   17 -
 .../HiveMetaStoreClientPreCatalog.java          |   10 -
 .../hive/metastore/client/TestRuntimeStats.java |  100 -
 66 files changed, 2963 insertions(+), 9936 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/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 f40c606..2403d7a 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -4270,15 +4270,15 @@ public class HiveConf extends Configuration {
         new StringSet("query", "hiveserver", "metastore"),
         "Sets the persistence scope of runtime statistics\n"
             + "  query: runtime statistics are only used during re-execution\n"
-            + "  hiveserver: runtime statistics are persisted in the hiveserver - all sessions share it\n"
-            + "  metastore: runtime statistics are persisted in the metastore as well"),
+            + "  hiveserver: runtime statistics are persisted in the hiveserver - all sessions share it"),
 
     HIVE_QUERY_MAX_REEXECUTION_COUNT("hive.query.reexecution.max.count", 1,
         "Maximum number of re-executions for a single query."),
     HIVE_QUERY_REEXECUTION_ALWAYS_COLLECT_OPERATOR_STATS("hive.query.reexecution.always.collect.operator.stats", false,
         "If sessionstats are enabled; this option can be used to collect statistics all the time"),
     HIVE_QUERY_REEXECUTION_STATS_CACHE_SIZE("hive.query.reexecution.stats.cache.size", 100_000,
-        "Size of the runtime statistics cache. Unit is: OperatorStat entry; a query plan consist ~100. See also: runtime.stats.max.entries"),
+        "Size of the runtime statistics cache. Unit is: OperatorStat entry; a query plan consist ~100"),
+
 
     HIVE_QUERY_RESULTS_CACHE_ENABLED("hive.query.results.cache.enabled", true,
         "If the query results cache is enabled. This will keep results of previously executed queries " +

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
----------------------------------------------------------------------
diff --git a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
index 8ecbaad..801de7a 100644
--- a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
+++ b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
@@ -69,7 +69,6 @@ import org.apache.hadoop.hive.metastore.api.WMTrigger;
 import org.apache.hadoop.hive.metastore.api.WMValidateResourcePlanResponse;
 import org.apache.hadoop.hive.metastore.api.Role;
 import org.apache.hadoop.hive.metastore.api.RolePrincipalGrant;
-import org.apache.hadoop.hive.metastore.api.RuntimeStat;
 import org.apache.hadoop.hive.metastore.api.SQLCheckConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLDefaultConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
@@ -296,7 +295,6 @@ public class DummyRawStoreFailEvent implements RawStore, Configurable {
     objectStore.updateCreationMetadata(catName, dbname, tablename, cm);
   }
 
-  @Override
   public void alterTable(String catName, String dbName, String name, Table newTable)
       throws InvalidObjectException, MetaException {
     if (shouldEventSucceed) {
@@ -1128,7 +1126,6 @@ public class DummyRawStoreFailEvent implements RawStore, Configurable {
     return null;
   }
 
-  @Override
   public void createISchema(ISchema schema) throws AlreadyExistsException, MetaException,
       NoSuchObjectException {
     objectStore.createISchema(schema);
@@ -1198,20 +1195,4 @@ public class DummyRawStoreFailEvent implements RawStore, Configurable {
   public void addSerde(SerDeInfo serde) throws AlreadyExistsException, MetaException {
     objectStore.addSerde(serde);
   }
-
-  @Override
-  public void addRuntimeStat(RuntimeStat stat) throws MetaException {
-    objectStore.addRuntimeStat(stat);
-  }
-
-  @Override
-  public List<RuntimeStat> getRuntimeStats() throws MetaException {
-    return objectStore.getRuntimeStats();
-  }
-
-  @Override
-  public int deleteRuntimeStats(int maxRetained, int maxRetainSecs) {
-    return 0;
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
index 750fc69..88022be 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
@@ -1067,7 +1067,7 @@ public class QTestUtil {
     clearTablesCreatedDuringTests();
     clearUDFsCreatedDuringTests();
     clearKeysCreatedInTests();
-    StatsSources.clearGlobalStats();
+    StatsSources.clearAllStats();
   }
 
   protected void clearSettingsCreatedInTests() throws IOException {

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/metastore/scripts/upgrade/derby/056-HIVE-19171.derby.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/derby/056-HIVE-19171.derby.sql b/metastore/scripts/upgrade/derby/056-HIVE-19171.derby.sql
deleted file mode 100644
index ef6c77b..0000000
--- a/metastore/scripts/upgrade/derby/056-HIVE-19171.derby.sql
+++ /dev/null
@@ -1,10 +0,0 @@
-
-
-CREATE TABLE "APP"."RUNTIME_STATS" (
-  "RS_ID" bigint primary key,
-  "CREATE_TIME" integer not null,
-  "WEIGHT" integer not null,
-  "PAYLOAD" BLOB
-);
-
-CREATE INDEX IDX_RUNTIME_STATS_CREATE_TIME ON RUNTIME_STATS(CREATE_TIME);

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/ql/src/java/org/apache/hadoop/hive/ql/optimizer/signature/OpSignature.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/signature/OpSignature.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/signature/OpSignature.java
index f626bd5..e87bbce 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/signature/OpSignature.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/signature/OpSignature.java
@@ -25,28 +25,18 @@ import java.util.Map.Entry;
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.plan.OperatorDesc;
 
-import com.fasterxml.jackson.annotation.JsonIdentityInfo;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.annotation.ObjectIdGenerators;
 import com.google.common.annotations.VisibleForTesting;
 
 /**
  * Signature of the operator(non-recursive).
  */
-@JsonIdentityInfo(generator = ObjectIdGenerators.IntSequenceGenerator.class, property = "@id")
-public final class OpSignature {
+public class OpSignature {
 
   /**
    * Holds the signature of the operator; the keys are are the methods name marked by {@link Signature}.
    */
-  @JsonProperty
   private Map<String, Object> sigMap;
 
-  // need this for Jackson to work
-  @SuppressWarnings("unused")
-  private OpSignature() {
-  }
-
   private OpSignature(Operator<? extends OperatorDesc> op) {
     sigMap = new HashMap<>();
     // FIXME: consider other operator info as well..not just conf?
@@ -80,7 +70,7 @@ public final class OpSignature {
 
   @VisibleForTesting
   public void proveEquals(OpSignature other) {
-    proveEquals(sigMap, other.sigMap);
+    proveEquals(sigMap,other.sigMap);
   }
 
   private static void proveEquals(Map<String, Object> m1, Map<String, Object> m2) {
@@ -113,9 +103,4 @@ public final class OpSignature {
     }
     return sb.toString();
   }
-
-  public Map<String, Object> getSigMap() {
-    return sigMap;
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/ql/src/java/org/apache/hadoop/hive/ql/optimizer/signature/OpTreeSignature.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/signature/OpTreeSignature.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/signature/OpTreeSignature.java
index f774158..c3dc848 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/signature/OpTreeSignature.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/signature/OpTreeSignature.java
@@ -24,28 +24,14 @@ import java.util.Objects;
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.plan.OperatorDesc;
 
-import com.fasterxml.jackson.annotation.JsonIdentityInfo;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.annotation.ObjectIdGenerators;
-
 /**
  * Operator tree signature.
  */
-@JsonIdentityInfo(generator = ObjectIdGenerators.IntSequenceGenerator.class, property = "@id")
-public final class OpTreeSignature {
-
-  @JsonProperty
+public class OpTreeSignature {
   private int hashCode;
-  @JsonProperty
   private OpSignature sig;
-  @JsonProperty
   private ArrayList<OpTreeSignature> parentSig;
 
-  // need this for Jackson to work
-  @SuppressWarnings("unused")
-  private OpTreeSignature() {
-  }
-
   OpTreeSignature(Operator<?> op, OpTreeSignatureFactory osf) {
     sig = OpSignature.of(op);
     parentSig = new ArrayList<>();
@@ -96,12 +82,4 @@ public final class OpTreeSignature {
     return sb.toString();
   }
 
-  public OpSignature getSig() {
-    return sig;
-  }
-
-  public ArrayList<OpTreeSignature> getParentSig() {
-    return parentSig;
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/ql/src/java/org/apache/hadoop/hive/ql/optimizer/signature/OpTreeSignatureFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/signature/OpTreeSignatureFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/signature/OpTreeSignatureFactory.java
index 80a3edf..3df5ee9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/signature/OpTreeSignatureFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/signature/OpTreeSignatureFactory.java
@@ -29,22 +29,22 @@ import org.apache.hadoop.hive.ql.plan.OperatorDesc;
  */
 public interface OpTreeSignatureFactory {
 
-  OpTreeSignature getSignature(Operator<? extends OperatorDesc> op);
+  public OpTreeSignature getSignature(Operator<? extends OperatorDesc> op);
 
-  OpTreeSignatureFactory DIRECT = new Direct();
+  static final OpTreeSignatureFactory DIRECT = new Direct();
 
-  static OpTreeSignatureFactory direct() {
+  public static OpTreeSignatureFactory direct() {
     return DIRECT;
   }
 
-  static OpTreeSignatureFactory newCache() {
+  public static OpTreeSignatureFactory newCache() {
     return new CachedFactory();
   }
 
   // FIXME: possible alternative: move both OpSignature/OpTreeSignature into
   // under some class as nested ones; and that way this factory level caching can be made "transparent"
 
-  class Direct implements OpTreeSignatureFactory {
+  static class Direct implements OpTreeSignatureFactory {
 
     @Override
     public OpTreeSignature getSignature(Operator<? extends OperatorDesc> op) {
@@ -53,7 +53,7 @@ public interface OpTreeSignatureFactory {
 
   }
 
-  class CachedFactory implements OpTreeSignatureFactory {
+  static class CachedFactory implements OpTreeSignatureFactory {
 
     Map<Operator<? extends OperatorDesc>, OpTreeSignature> cache = new IdentityHashMap<>();
 

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/ql/src/java/org/apache/hadoop/hive/ql/optimizer/signature/RuntimeStatsMap.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/signature/RuntimeStatsMap.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/signature/RuntimeStatsMap.java
deleted file mode 100644
index 195a8b1..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/signature/RuntimeStatsMap.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/*
- * 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.optimizer.signature;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-
-import org.apache.hadoop.hive.ql.stats.OperatorStats;
-
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.google.common.base.Objects;
-
-/**
- * This class makes it easier for jackson to comprehend the map type
- *
- * Instead of getting into convincing Jackson to store the map with serializers and typefactory tricks;
- * this class is a simple "repacker" to and from list.
- */
-public final class RuntimeStatsMap {
-  @JsonProperty
-  private List<OpTreeSignature> sigs;
-  @JsonProperty
-  private List<OperatorStats> ss;
-
-  RuntimeStatsMap() {
-  }
-
-
-  public RuntimeStatsMap(Map<OpTreeSignature, OperatorStats> input) {
-    sigs = new ArrayList<>(input.size());
-    ss = new ArrayList<>(input.size());
-    for (Entry<OpTreeSignature, OperatorStats> ent : input.entrySet()) {
-      sigs.add(ent.getKey());
-      ss.add(ent.getValue());
-    }
-  }
-
-  public Map<OpTreeSignature, OperatorStats> toMap() throws IOException {
-    if (sigs.size() != ss.size()) {
-      throw new IOException("constraint validation");
-    }
-    Map<OpTreeSignature, OperatorStats> ret = new HashMap<>();
-    for (int i = 0; i < sigs.size(); i++) {
-      ret.put(sigs.get(i), ss.get(i));
-    }
-    return ret;
-  }
-
-  @Override
-  public int hashCode() {
-    return Objects.hashCode(sigs, ss);
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (obj == null || obj.getClass() != RuntimeStatsMap.class) {
-      return false;
-    }
-    RuntimeStatsMap o = (RuntimeStatsMap) obj;
-    return Objects.equal(sigs, o.sigs) &&
-        Objects.equal(ss, o.ss);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/ql/src/java/org/apache/hadoop/hive/ql/optimizer/signature/RuntimeStatsPersister.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/signature/RuntimeStatsPersister.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/signature/RuntimeStatsPersister.java
deleted file mode 100644
index 696fe1f..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/signature/RuntimeStatsPersister.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * 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.optimizer.signature;
-
-import java.io.IOException;
-
-import com.fasterxml.jackson.databind.MapperFeature;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.SerializationFeature;
-
-/**
- * Enables to encode/decode runtime statistics values into textual form.
- */
-public class RuntimeStatsPersister {
-  public static final RuntimeStatsPersister INSTANCE = new RuntimeStatsPersister();
-
-  private final ObjectMapper om;
-
-  RuntimeStatsPersister() {
-    om = new ObjectMapper();
-    om.enableDefaultTyping(ObjectMapper.DefaultTyping.NON_FINAL);
-    om.configure(SerializationFeature.INDENT_OUTPUT, true);
-    om.configure(MapperFeature.REQUIRE_SETTERS_FOR_GETTERS, true);
-  }
-
-  public <T> String encode(T input) throws IOException {
-    return om.writeValueAsString(input);
-  }
-
-  public <T> T decode(String input, Class<T> clazz) throws IOException {
-    return om.readValue(input, clazz);
-  }
-
-  public <T> T decode(byte[] input, Class<T> clazz) throws IOException {
-    return om.readValue(input, clazz);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/ql/src/java/org/apache/hadoop/hive/ql/optimizer/signature/SignatureUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/signature/SignatureUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/signature/SignatureUtils.java
index f599d33..4f3e338 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/signature/SignatureUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/signature/SignatureUtils.java
@@ -23,15 +23,17 @@ import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
+
+import com.google.common.collect.Sets;
 
 /**
  * Enables to calculate the signature of an object.
  *
  * If the object has methods annotated with {@link Signature}, they will be used.
- * If the object has no methods marked with the annotation;
- * the object itself is used in the signature to prevent incorrect matches.
+ * If the object has no methods marked with the annotation; the object itself is used in the signature to prevent incorrect matches.
  */
-public final class SignatureUtils {
+public class SignatureUtils {
 
   private static Map<Class<?>, SignatureMapper> mappers = new HashMap<>();
 
@@ -40,24 +42,28 @@ public final class SignatureUtils {
     mapper.write(ret, o);
   }
 
-  /** Prevent construction. */
-  private SignatureUtils() {
-  }
-
   static class SignatureMapper {
 
+    static final Set<String> acceptedSignatureTypes = Sets.newHashSet();
+
     private List<Method> sigMethods;
 
     private String classLabel;
 
-    SignatureMapper(Class<?> o) {
+    public SignatureMapper(Class<?> o) {
       Method[] f = o.getMethods();
       sigMethods = new ArrayList<>();
       for (Method method : f) {
         if (method.isAnnotationPresent(Signature.class)) {
+          Class<?> rType = method.getReturnType();
+          String rTypeName = rType.getName();
+          if (!rType.isPrimitive() && acceptedSignatureTypes.contains(rTypeName)) {
+            throw new RuntimeException("unxepected type (" + rTypeName + ") used in signature");
+          }
           sigMethods.add(method);
         }
       }
+
       classLabel = o.getName();
     }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/ql/src/java/org/apache/hadoop/hive/ql/plan/FileSinkDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/FileSinkDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/FileSinkDesc.java
index fcb6de7..e15a49f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/FileSinkDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/FileSinkDesc.java
@@ -192,15 +192,11 @@ public class FileSinkDesc extends AbstractOperatorDesc implements IStatsGatherDe
   }
 
   @Explain(displayName = "directory", explainLevels = { Level.EXTENDED })
+  @Signature
   public Path getDirName() {
     return dirName;
   }
 
-  @Signature
-  public String getDirNameString() {
-    return dirName.toString();
-  }
-
   public void setDirName(final Path dirName) {
     this.dirName = dirName;
   }
@@ -220,6 +216,7 @@ public class FileSinkDesc extends AbstractOperatorDesc implements IStatsGatherDe
   }
 
   @Explain(displayName = "table", explainLevels = { Level.USER, Level.DEFAULT, Level.EXTENDED })
+  @Signature
   public TableDesc getTableInfo() {
     return tableInfo;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/ql/src/java/org/apache/hadoop/hive/ql/plan/HashTableSinkDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/HashTableSinkDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/HashTableSinkDesc.java
index d71ba5b..a61a47e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/HashTableSinkDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/HashTableSinkDesc.java
@@ -308,10 +308,10 @@ public class HashTableSinkDesc extends JoinDesc implements Serializable {
    */
   @Override
   @Explain(displayName = "keys")
-  public Map<String, String> getKeysString() {
-    Map<String, String> keyMap = new LinkedHashMap<>();
+  public Map<Byte, String> getKeysString() {
+    Map<Byte, String> keyMap = new LinkedHashMap<Byte, String>();
     for (Map.Entry<Byte, List<ExprNodeDesc>> k: getKeys().entrySet()) {
-      keyMap.put(String.valueOf(k.getKey()), PlanUtils.getExprListString(k.getValue()));
+      keyMap.put(k.getKey(), PlanUtils.getExprListString(k.getValue()));
     }
     return keyMap;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/ql/src/java/org/apache/hadoop/hive/ql/plan/JoinDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/JoinDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/JoinDesc.java
index 95990b8..e7ca7f6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/JoinDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/JoinDesc.java
@@ -231,14 +231,14 @@ public class JoinDesc extends AbstractOperatorDesc {
    */
   @Explain(displayName = "keys")
   @Signature
-  public Map<String, String> getKeysString() {
+  public Map<Byte, String> getKeysString() {
     if (joinKeys == null) {
       return null;
     }
 
-    Map<String, String> keyMap = new LinkedHashMap<String, String>();
+    Map<Byte, String> keyMap = new LinkedHashMap<Byte, String>();
     for (byte i = 0; i < joinKeys.length; i++) {
-      keyMap.put(String.valueOf(i), PlanUtils.getExprListString(Arrays.asList(joinKeys[i])));
+      keyMap.put(i, PlanUtils.getExprListString(Arrays.asList(joinKeys[i])));
     }
     return keyMap;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/ql/src/java/org/apache/hadoop/hive/ql/plan/MapJoinDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/MapJoinDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/MapJoinDesc.java
index dc4f085..54b705d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/MapJoinDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/MapJoinDesc.java
@@ -213,10 +213,10 @@ public class MapJoinDesc extends JoinDesc implements Serializable {
    */
   @Override
   @Explain(displayName = "keys")
-  public Map<String, String> getKeysString() {
-    Map<String, String> keyMap = new LinkedHashMap<>();
+  public Map<Byte, String> getKeysString() {
+    Map<Byte, String> keyMap = new LinkedHashMap<Byte, String>();
     for (Map.Entry<Byte, List<ExprNodeDesc>> k: getKeys().entrySet()) {
-      keyMap.put(String.valueOf(k.getKey()), PlanUtils.getExprListString(k.getValue()));
+      keyMap.put(k.getKey(), PlanUtils.getExprListString(k.getValue()));
     }
     return keyMap;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/CachingStatsSource.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/CachingStatsSource.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/CachingStatsSource.java
index 2841638..c515276 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/CachingStatsSource.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/CachingStatsSource.java
@@ -22,6 +22,8 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Optional;
 
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.optimizer.signature.OpTreeSignature;
 import org.apache.hadoop.hive.ql.stats.OperatorStats;
@@ -34,8 +36,9 @@ public class CachingStatsSource implements StatsSource {
 
   private final Cache<OpTreeSignature, OperatorStats> cache;
 
-  public CachingStatsSource(int cacheSize) {
-    cache = CacheBuilder.newBuilder().maximumSize(cacheSize).build();
+  public CachingStatsSource(HiveConf conf) {
+    int size = conf.getIntVar(ConfVars.HIVE_QUERY_REEXECUTION_STATS_CACHE_SIZE);
+    cache = CacheBuilder.newBuilder().maximumSize(size).build();
   }
 
   public void put(OpTreeSignature sig, OperatorStats opStat) {

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/EmptyStatsSource.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/EmptyStatsSource.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/EmptyStatsSource.java
index 624f107..19df13a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/EmptyStatsSource.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/EmptyStatsSource.java
@@ -24,7 +24,7 @@ import java.util.Optional;
 import org.apache.hadoop.hive.ql.optimizer.signature.OpTreeSignature;
 import org.apache.hadoop.hive.ql.stats.OperatorStats;
 
-public final class EmptyStatsSource implements StatsSource {
+public class EmptyStatsSource implements StatsSource {
 
   public static StatsSource INSTANCE = new EmptyStatsSource();
 

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/MetastoreStatsConnector.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/MetastoreStatsConnector.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/MetastoreStatsConnector.java
deleted file mode 100644
index 237c1cc..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/MetastoreStatsConnector.java
+++ /dev/null
@@ -1,143 +0,0 @@
-/*
- * 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.mapper;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import org.apache.commons.lang3.concurrent.BasicThreadFactory;
-import org.apache.hadoop.hive.metastore.api.RuntimeStat;
-import org.apache.hadoop.hive.ql.metadata.Hive;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-import org.apache.hadoop.hive.ql.optimizer.signature.OpTreeSignature;
-import org.apache.hadoop.hive.ql.optimizer.signature.RuntimeStatsMap;
-import org.apache.hadoop.hive.ql.optimizer.signature.RuntimeStatsPersister;
-import org.apache.hadoop.hive.ql.stats.OperatorStats;
-import org.apache.thrift.TException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.base.Charsets;
-
-/**
- * Decorates a StatSource to be loaded and persisted in the metastore as well.
- */
-class MetastoreStatsConnector implements StatsSource {
-
-  private static final Logger LOG = LoggerFactory.getLogger(MetastoreStatsConnector.class);
-
-  private final StatsSource ss;
-
-  private ExecutorService executor;
-
-  MetastoreStatsConnector(StatsSource ss) {
-    this.ss = ss;
-    executor = Executors.newSingleThreadExecutor(
-        new BasicThreadFactory.Builder()
-            .namingPattern("Metastore-RuntimeStats-Loader-%d")
-            .daemon(true)
-            .build());
-
-    executor.submit(new RuntimeStatsLoader());
-  }
-
-  private class RuntimeStatsLoader implements Runnable {
-
-    @Override
-    public void run() {
-      try {
-        List<RuntimeStat> rs = Hive.get().getMSC().getRuntimeStats();
-        for (RuntimeStat thriftStat : rs) {
-          try {
-            ss.putAll(decode(thriftStat));
-          } catch (IOException e) {
-            logException("Exception while loading runtime stats", e);
-          }
-        }
-      } catch (TException | HiveException e) {
-        logException("Exception while reading metastore runtime stats", e);
-      }
-    }
-  }
-
-  @Override
-  public boolean canProvideStatsFor(Class<?> clazz) {
-    return ss.canProvideStatsFor(clazz);
-  }
-
-  @Override
-  public Optional<OperatorStats> lookup(OpTreeSignature treeSig) {
-    return ss.lookup(treeSig);
-  }
-
-  @Override
-  public void putAll(Map<OpTreeSignature, OperatorStats> map) {
-    ss.putAll(map);
-    executor.submit(new RuntimeStatsSubmitter(map));
-  }
-
-  class RuntimeStatsSubmitter implements Runnable {
-
-    private Map<OpTreeSignature, OperatorStats> map;
-
-    public RuntimeStatsSubmitter(Map<OpTreeSignature, OperatorStats> map) {
-      this.map = map;
-    }
-
-    @Override
-    public void run() {
-      try {
-        RuntimeStat rec = encode(map);
-        Hive.get().getMSC().addRuntimeStat(rec);
-      } catch (TException | HiveException | IOException e) {
-        logException("Exception while persisting runtime stat", e);
-      }
-    }
-  }
-
-  private RuntimeStat encode(Map<OpTreeSignature, OperatorStats> map) throws IOException {
-    String payload = RuntimeStatsPersister.INSTANCE.encode(new RuntimeStatsMap(map));
-    RuntimeStat rs = new RuntimeStat();
-    rs.setWeight(map.size());
-    rs.setPayload(ByteBuffer.wrap(payload.getBytes(Charsets.UTF_8)));
-    return rs;
-  }
-
-  private Map<OpTreeSignature, OperatorStats> decode(RuntimeStat rs) throws IOException {
-    RuntimeStatsMap rsm = RuntimeStatsPersister.INSTANCE.decode(rs.getPayload(), RuntimeStatsMap.class);
-    return rsm.toMap();
-  }
-
-  public void destroy() {
-    executor.shutdown();
-  }
-
-  static void logException(String msg, Exception e) {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug(msg, e);
-    } else {
-      LOG.info(msg + ": " + e.getMessage());
-    }
-  }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/PlanMapper.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/PlanMapper.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/PlanMapper.java
index e932304..a372804 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/PlanMapper.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/PlanMapper.java
@@ -18,9 +18,7 @@
 
 package org.apache.hadoop.hive.ql.plan.mapper;
 
-import java.lang.reflect.Modifier;
 import java.util.ArrayList;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -36,7 +34,6 @@ import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.optimizer.signature.OpTreeSignature;
 import org.apache.hadoop.hive.ql.optimizer.signature.OpTreeSignatureFactory;
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.Sets;
 
 /**
  * Enables to connect related objects to eachother.
@@ -46,106 +43,7 @@ import com.google.common.collect.Sets;
 public class PlanMapper {
 
   Set<EquivGroup> groups = new HashSet<>();
-  private Map<Object, EquivGroup> objectMap = new CompositeMap<>(OpTreeSignature.class);
-
-  /**
-   * Specialized class which can compare by identity or value; based on the key type.
-   */
-  private static class CompositeMap<K, V> implements Map<K, V> {
-
-    Map<K, V> comparedMap = new HashMap<>();
-    Map<K, V> identityMap = new IdentityHashMap<>();
-    final Set<Class<?>> typeCompared;
-
-    CompositeMap(Class<?>... comparedTypes) {
-      for (Class<?> class1 : comparedTypes) {
-        if (!Modifier.isFinal(class1.getModifiers())) {
-          throw new RuntimeException(class1 + " is not final...for this to reliably work; it should be");
-        }
-      }
-      typeCompared = Sets.newHashSet(comparedTypes);
-    }
-
-    @Override
-    public int size() {
-      return comparedMap.size() + identityMap.size();
-    }
-
-    @Override
-    public boolean isEmpty() {
-      return comparedMap.isEmpty() && identityMap.isEmpty();
-    }
-
-    @Override
-    public boolean containsKey(Object key) {
-      return comparedMap.containsKey(key) || identityMap.containsKey(key);
-    }
-
-    @Override
-    public boolean containsValue(Object value) {
-      return comparedMap.containsValue(value) || identityMap.containsValue(value);
-    }
-
-    @Override
-    public V get(Object key) {
-      V v0 = comparedMap.get(key);
-      if (v0 != null) {
-        return v0;
-      }
-      return identityMap.get(key);
-    }
-
-    @Override
-    public V put(K key, V value) {
-      if (shouldCompare(key.getClass())) {
-        return comparedMap.put(key, value);
-      } else {
-        return identityMap.put(key, value);
-      }
-    }
-
-    @Override
-    public V remove(Object key) {
-      if (shouldCompare(key.getClass())) {
-        return comparedMap.remove(key);
-      } else {
-        return identityMap.remove(key);
-      }
-    }
-
-    private boolean shouldCompare(Class<?> key) {
-      return typeCompared.contains(key);
-    }
-
-    @Override
-    public void putAll(Map<? extends K, ? extends V> m) {
-      for (Entry<? extends K, ? extends V> e : m.entrySet()) {
-        put(e.getKey(), e.getValue());
-      }
-    }
-
-    @Override
-    public void clear() {
-      comparedMap.clear();
-      identityMap.clear();
-    }
-
-    @Override
-    public Set<K> keySet() {
-      return Sets.union(comparedMap.keySet(), identityMap.keySet());
-    }
-
-    @Override
-    public Collection<V> values() {
-      throw new UnsupportedOperationException("This method is not supported");
-    }
-
-    @Override
-    public Set<Entry<K, V>> entrySet() {
-      return Sets.union(comparedMap.entrySet(), identityMap.entrySet());
-    }
-
-  }
+  private Map<Object, EquivGroup> objectMap = new HashMap<>();
 
   /**
    * A set of objects which are representing the same thing.
@@ -157,7 +55,7 @@ public class PlanMapper {
    *   there might be more than one, since an optimization may replace an operator with a new one
    *   <li> Signature - to enable inter-plan look up of the same data
    *   <li> OperatorStats - collected runtime information
-   * </ul>
+   * <ul>
    */
   public class EquivGroup {
     Set<Object> members = new HashSet<>();
@@ -218,7 +116,7 @@ public class PlanMapper {
 
   private Object getKeyFor(Object o) {
     if (o instanceof Operator) {
-      Operator<?> operator = (Operator<?>) o;
+      Operator operator = (Operator) o;
       return signatureCache.getSignature(operator);
     }
     return o;

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/SimpleRuntimeStatsSource.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/SimpleRuntimeStatsSource.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/SimpleRuntimeStatsSource.java
index fb2b5f8..3d6c257 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/SimpleRuntimeStatsSource.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/SimpleRuntimeStatsSource.java
@@ -18,33 +18,48 @@
 
 package org.apache.hadoop.hive.ql.plan.mapper;
 
+import java.util.List;
 import java.util.Map;
+import java.util.NoSuchElementException;
 import java.util.Optional;
-import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.optimizer.signature.OpTreeSignature;
 import org.apache.hadoop.hive.ql.stats.OperatorStats;
 
-public class MapBackedStatsSource implements StatsSource {
+public class SimpleRuntimeStatsSource implements StatsSource {
 
-  private Map<OpTreeSignature, OperatorStats> map = new ConcurrentHashMap<>();
+  private final PlanMapper pm;
+
+
+  public SimpleRuntimeStatsSource(PlanMapper pm) {
+    this.pm = pm;
+  }
 
   @Override
-  public boolean canProvideStatsFor(Class<?> clazz) {
-    if (Operator.class.isAssignableFrom(clazz)) {
-      return true;
+  public Optional<OperatorStats> lookup(OpTreeSignature sig) {
+    try {
+      List<OperatorStats> v = pm.lookupAll(OperatorStats.class, sig);
+      if (v.size() > 0) {
+        return Optional.of(v.get(0));
+      }
+      return Optional.empty();
+    } catch (NoSuchElementException | IllegalArgumentException iae) {
+      return Optional.empty();
     }
-    return false;
   }
 
   @Override
-  public Optional<OperatorStats> lookup(OpTreeSignature treeSig) {
-    return Optional.ofNullable(map.get(treeSig));
+  public boolean canProvideStatsFor(Class<?> class1) {
+    if (Operator.class.isAssignableFrom(class1)) {
+      return true;
+    }
+    return false;
   }
 
   @Override
   public void putAll(Map<OpTreeSignature, OperatorStats> map) {
-    this.map.putAll(map);
+    throw new RuntimeException();
   }
-}
\ No newline at end of file
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/StatsSources.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/StatsSources.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/StatsSources.java
index 30b6a30..a4e33c3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/StatsSources.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/StatsSources.java
@@ -18,12 +18,14 @@
 
 package org.apache.hadoop.hive.ql.plan.mapper;
 
+import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.optimizer.signature.OpTreeSignature;
 import org.apache.hadoop.hive.ql.plan.mapper.PlanMapper.EquivGroup;
 import org.apache.hadoop.hive.ql.stats.OperatorStats;
@@ -31,50 +33,53 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.ImmutableMap.Builder;
 
 public class StatsSources {
 
-  private static final Logger LOG = LoggerFactory.getLogger(StatsSources.class);
+  public static class MapBackedStatsSource implements StatsSource {
 
-  static enum StatsSourceMode {
-    query, hiveserver, metastore;
-  }
+    private Map<OpTreeSignature, OperatorStats> map = new HashMap<>();
 
-  public static void initialize(HiveConf hiveConf) {
-    // requesting for the stats source will implicitly initialize it
-    getStatsSource(hiveConf);
-  }
+    @Override
+    public boolean canProvideStatsFor(Class<?> clazz) {
+      if (Operator.class.isAssignableFrom(clazz)) {
+        return true;
+      }
+      return false;
+    }
 
-  public static StatsSource getStatsSource(HiveConf conf) {
-    String mode = conf.getVar(ConfVars.HIVE_QUERY_REEXECUTION_STATS_PERSISTENCE);
-    int cacheSize = conf.getIntVar(ConfVars.HIVE_QUERY_REEXECUTION_STATS_CACHE_SIZE);
-    switch (mode) {
-    case "query":
-      return new MapBackedStatsSource();
-    case "hiveserver":
-      return StatsSources.globalStatsSource(cacheSize);
-    case "metastore":
-      return StatsSources.metastoreBackedStatsSource(StatsSources.globalStatsSource(cacheSize));
-    default:
-      throw new RuntimeException("Unknown StatsSource setting: " + mode);
+    @Override
+    public Optional<OperatorStats> lookup(OpTreeSignature treeSig) {
+      return Optional.ofNullable(map.get(treeSig));
     }
+
+    @Override
+    public void putAll(Map<OpTreeSignature, OperatorStats> map) {
+      map.putAll(map);
+    }
+
   }
 
+  private static final Logger LOG = LoggerFactory.getLogger(StatsSources.class);
+
   public static StatsSource getStatsSourceContaining(StatsSource currentStatsSource, PlanMapper pm) {
-    StatsSource statsSource = currentStatsSource;
-    if (currentStatsSource  == EmptyStatsSource.INSTANCE) {
-      statsSource = new MapBackedStatsSource();
+    if (currentStatsSource instanceof CachingStatsSource) {
+      CachingStatsSource sessionStatsSource = (CachingStatsSource) currentStatsSource;
+      loadFromPlanMapper(sessionStatsSource, pm);
+      return sessionStatsSource;
+    } else {
+      return new SimpleRuntimeStatsSource(pm);
     }
+  }
 
-    Map<OpTreeSignature, OperatorStats> statMap = extractStatMapFromPlanMapper(pm);
-    statsSource.putAll(statMap);
-    return statsSource;
+  public static void loadFromPlanMapper(CachingStatsSource sessionStatsSource, PlanMapper pm) {
+    Map<OpTreeSignature, OperatorStats> map = extractStatMapFromPlanMapper(pm);
+    sessionStatsSource.putAll(map);
   }
 
+
   private static Map<OpTreeSignature, OperatorStats> extractStatMapFromPlanMapper(PlanMapper pm) {
-    Builder<OpTreeSignature, OperatorStats> map = ImmutableMap.builder();
+    Map<OpTreeSignature, OperatorStats> map = new HashMap<OpTreeSignature, OperatorStats>();
     Iterator<EquivGroup> it = pm.iterateGroups();
     while (it.hasNext()) {
       EquivGroup e = it.next();
@@ -98,33 +103,20 @@ public class StatsSources {
         map.put(sig.get(0), stat.get(0));
       }
     }
-    return map.build();
+    return map;
   }
 
   private static StatsSource globalStatsSource;
-  private static MetastoreStatsConnector metastoreStatsConnector;
 
-  public static StatsSource globalStatsSource(int cacheSize) {
+  public static StatsSource globalStatsSource(HiveConf conf) {
     if (globalStatsSource == null) {
-      globalStatsSource = new CachingStatsSource(cacheSize);
+      globalStatsSource = new CachingStatsSource(conf);
     }
     return globalStatsSource;
   }
 
-  public static StatsSource metastoreBackedStatsSource(StatsSource parent) {
-    if (metastoreStatsConnector == null) {
-      metastoreStatsConnector = new MetastoreStatsConnector(parent);
-    }
-    return metastoreStatsConnector;
-  }
-
   @VisibleForTesting
-  public static void clearGlobalStats() {
-    if (metastoreStatsConnector != null) {
-      metastoreStatsConnector.destroy();
-    }
+  public static void clearAllStats() {
     globalStatsSource = null;
-    metastoreStatsConnector = null;
   }
-
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/ql/src/java/org/apache/hadoop/hive/ql/reexec/ReOptimizePlugin.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/reexec/ReOptimizePlugin.java b/ql/src/java/org/apache/hadoop/hive/ql/reexec/ReOptimizePlugin.java
index 8dc7387..409cc73 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/reexec/ReOptimizePlugin.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/reexec/ReOptimizePlugin.java
@@ -84,7 +84,22 @@ public class ReOptimizePlugin implements IReExecutionPlugin {
     alwaysCollectStats = driver.getConf().getBoolVar(ConfVars.HIVE_QUERY_REEXECUTION_ALWAYS_COLLECT_OPERATOR_STATS);
     statsReaderHook.setCollectOnSuccess(alwaysCollectStats);
 
-    coreDriver.setStatsSource(StatsSources.getStatsSource(driver.getConf()));
+    coreDriver.setStatsSource(getStatsSource(driver.getConf()));
+  }
+
+  static enum StatsSourceMode {
+    query, hiveserver;
+  }
+
+  private StatsSource getStatsSource(HiveConf conf) {
+    StatsSourceMode mode = StatsSourceMode.valueOf(conf.getVar(ConfVars.HIVE_QUERY_REEXECUTION_STATS_PERSISTENCE));
+    switch (mode) {
+    case query:
+      return new StatsSources.MapBackedStatsSource();
+    case hiveserver:
+      return StatsSources.globalStatsSource(conf);
+    }
+    throw new RuntimeException("Unknown StatsSource setting: " + mode);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/ql/src/java/org/apache/hadoop/hive/ql/stats/OperatorStats.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/stats/OperatorStats.java b/ql/src/java/org/apache/hadoop/hive/ql/stats/OperatorStats.java
index d70bb82..52e18a8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/stats/OperatorStats.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/stats/OperatorStats.java
@@ -6,9 +6,7 @@
  * 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
- *
+ * 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.
@@ -17,20 +15,10 @@
  */
 package org.apache.hadoop.hive.ql.stats;
 
-import com.google.common.base.Objects;
-
-/**
- * Holds information an operator's statistics.
- */
-public final class OperatorStats {
-  private String operatorId;
+public class OperatorStats {
+  private final String operatorId;
   private long outputRecords;
 
-  // for jackson
-  @SuppressWarnings("unused")
-  private OperatorStats() {
-  }
-
   public OperatorStats(final String opId) {
     this.operatorId = opId;
     this.outputRecords = -1;
@@ -52,19 +40,4 @@ public final class OperatorStats {
   public String toString() {
     return String.format("OperatorStats %s records: %d", operatorId, outputRecords);
   }
-
-  @Override
-  public int hashCode() {
-    return Objects.hashCode(operatorId, outputRecords);
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if(obj==null || obj.getClass()!= OperatorStats.class){
-      return false;
-    }
-    OperatorStats o = (OperatorStats) obj;
-    return Objects.equal(operatorId, o.operatorId) &&
-        Objects.equal(outputRecords, o.outputRecords);
-  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/ql/src/test/org/apache/hadoop/hive/ql/optimizer/signature/TestRuntimeStatsPersistence.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/optimizer/signature/TestRuntimeStatsPersistence.java b/ql/src/test/org/apache/hadoop/hive/ql/optimizer/signature/TestRuntimeStatsPersistence.java
deleted file mode 100644
index 627c2d8..0000000
--- a/ql/src/test/org/apache/hadoop/hive/ql/optimizer/signature/TestRuntimeStatsPersistence.java
+++ /dev/null
@@ -1,165 +0,0 @@
-/*
- * 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.optimizer.signature;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.hadoop.hive.ql.CompilationOpContext;
-import org.apache.hadoop.hive.ql.exec.Operator;
-import org.apache.hadoop.hive.ql.exec.OperatorFactory;
-import org.apache.hadoop.hive.ql.metadata.Table;
-import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
-import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc;
-import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
-import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
-import org.apache.hadoop.hive.ql.plan.FilterDesc;
-import org.apache.hadoop.hive.ql.plan.JoinCondDesc;
-import org.apache.hadoop.hive.ql.plan.OperatorDesc;
-import org.apache.hadoop.hive.ql.plan.TableScanDesc;
-import org.apache.hadoop.hive.ql.stats.OperatorStats;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDFConcat;
-import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
-import org.junit.Test;
-
-import com.google.common.collect.Lists;
-
-public class TestRuntimeStatsPersistence {
-
-  GenericUDF udf = new GenericUDFConcat();
-
-  CompilationOpContext cCtx = new CompilationOpContext();
-
-  private Operator<?> getFilTsOp(int i, int j) {
-    Operator<TableScanDesc> ts = getTsOp(i);
-    Operator<? extends OperatorDesc> fil = getFilterOp(j);
-
-    connectOperators(ts, fil);
-
-    return fil;
-  }
-
-  private void connectOperators(Operator<?> parent, Operator<?> child) {
-    parent.getChildOperators().add(child);
-    child.getParentOperators().add(parent);
-  }
-
-  @Test
-  public void checkPersistJoinCondDesc() throws Exception {
-    JoinCondDesc jcd = new JoinCondDesc(1, 2, 3);
-    JoinCondDesc jcd2 = persistenceLoop(jcd, JoinCondDesc.class);
-    assertEquals(jcd, jcd2);
-  }
-
-  OpTreeSignatureFactory signatureFactory = OpTreeSignatureFactory.newCache();
-
-  @Test
-  public void checkPersistingSigWorks() throws Exception {
-    OpSignature sig = OpSignature.of(getTsOp(3));
-    OpSignature sig2 = persistenceLoop(sig, OpSignature.class);
-    assertEquals(sig, sig2);
-  }
-
-  @Test
-  public void checkPersistingTreeSigWorks() throws Exception {
-    OpTreeSignature sig = signatureFactory.getSignature(getFilTsOp(3, 4));
-    OpTreeSignature sig2 = persistenceLoop(sig, OpTreeSignature.class);
-    assertEquals(sig, sig2);
-  }
-
-  @Test
-  public void checkCanStoreAsGraph() throws Exception {
-
-    Operator<?> ts = getTsOp(0);
-    Operator<?> fil1 = getFilterOp(1);
-    Operator<?> fil2 = getFilterOp(2);
-    Operator<?> fil3 = getFilterOp(3);
-
-    connectOperators(ts, fil1);
-    connectOperators(ts, fil2);
-    connectOperators(fil1, fil3);
-    connectOperators(fil2, fil3);
-
-    OpTreeSignature sig = signatureFactory.getSignature(fil3);
-    OpTreeSignature sig2 = persistenceLoop(sig, OpTreeSignature.class);
-
-    assertEquals(sig, sig2);
-
-    OpTreeSignature o0 = sig.getParentSig().get(0).getParentSig().get(0);
-    OpTreeSignature o1 = sig.getParentSig().get(1).getParentSig().get(0);
-    assertTrue("these have to be the same instance", o0 == o1);
-
-    OpTreeSignature p0 = sig2.getParentSig().get(0).getParentSig().get(0);
-    OpTreeSignature p1 = sig2.getParentSig().get(1).getParentSig().get(0);
-
-    assertTrue("these have to be the same instance", p0 == p1);
-
-    assertEquals(p0, p1);
-
-  }
-
-  @Test
-  public void checkCanStoreMap() throws Exception {
-
-    Map<OpTreeSignature, OperatorStats> map = new HashMap<>();
-    map.put(signatureFactory.getSignature(getTsOp(0)), new OperatorStats("ts0"));
-    map.put(signatureFactory.getSignature(getTsOp(1)), new OperatorStats("ts1"));
-
-    RuntimeStatsMap rsm = new RuntimeStatsMap(map);
-
-    RuntimeStatsMap rsm2 = persistenceLoop(rsm, RuntimeStatsMap.class);
-    OpTreeSignature k1 = rsm.toMap().keySet().iterator().next();
-    OpTreeSignature k2 = rsm2.toMap().keySet().iterator().next();
-    assertEquals(k1, k2);
-    assertEquals(rsm, rsm2);
-  }
-
-  private <T> T persistenceLoop(T sig, Class<T> clazz) throws IOException {
-    RuntimeStatsPersister sp = RuntimeStatsPersister.INSTANCE;
-    String stored = sp.encode(sig);
-    System.out.println(stored);
-    T sig2 = sp.decode(stored, clazz);
-    return sig2;
-  }
-
-  private Operator<? extends OperatorDesc> getFilterOp(int constVal) {
-    ExprNodeDesc pred = new ExprNodeConstantDesc(constVal);
-    FilterDesc fd = new FilterDesc(pred, true);
-    Operator<? extends OperatorDesc> op = OperatorFactory.get(cCtx, fd);
-    return op;
-  }
-
-  private Operator<TableScanDesc> getTsOp(int i) {
-    Table tblMetadata = new Table("db", "table");
-    TableScanDesc desc = new TableScanDesc("alias"/*+ cCtx.nextOperatorId()*/, tblMetadata);
-    List<ExprNodeDesc> as =
-        Lists.newArrayList(new ExprNodeConstantDesc(TypeInfoFactory.intTypeInfo, Integer.valueOf(i)),
-            new ExprNodeColumnDesc(TypeInfoFactory.intTypeInfo, "c1", "aa", false));
-    ExprNodeGenericFuncDesc f1 = new ExprNodeGenericFuncDesc(TypeInfoFactory.intTypeInfo, udf, as);
-    desc.setFilterExpr(f1);
-    Operator<TableScanDesc> ts = OperatorFactory.get(cCtx, desc);
-    return ts;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/ql/src/test/org/apache/hadoop/hive/ql/plan/mapping/TestCounterMapping.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/plan/mapping/TestCounterMapping.java b/ql/src/test/org/apache/hadoop/hive/ql/plan/mapping/TestCounterMapping.java
index e8a7a1b..8126970 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/plan/mapping/TestCounterMapping.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/plan/mapping/TestCounterMapping.java
@@ -33,11 +33,10 @@ import org.apache.hadoop.hive.ql.exec.FilterOperator;
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveFilter;
 import org.apache.hadoop.hive.ql.parse.ParseException;
-import org.apache.hadoop.hive.ql.plan.mapper.EmptyStatsSource;
 import org.apache.hadoop.hive.ql.plan.mapper.PlanMapper;
-import org.apache.hadoop.hive.ql.plan.mapper.StatsSources;
-import org.apache.hadoop.hive.ql.plan.mapper.PlanMapper.EquivGroup;
+import org.apache.hadoop.hive.ql.plan.mapper.SimpleRuntimeStatsSource;
 import org.apache.hadoop.hive.ql.reexec.ReExecDriver;
+import org.apache.hadoop.hive.ql.plan.mapper.PlanMapper.EquivGroup;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.ql.stats.OperatorStats;
 import org.apache.hadoop.hive.ql.stats.OperatorStatsReaderHook;
@@ -130,7 +129,7 @@ public class TestCounterMapping {
     FilterOperator filter1 = filters1.get(0);
 
     driver = createDriver();
-    ((ReExecDriver) driver).setStatsSource(StatsSources.getStatsSourceContaining(EmptyStatsSource.INSTANCE, pm1));
+    ((ReExecDriver) driver).setStatsSource(new SimpleRuntimeStatsSource(pm1));
 
     PlanMapper pm2 = getMapperForQuery(driver, query);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/ql/src/test/org/apache/hadoop/hive/ql/plan/mapping/TestReOptimization.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/plan/mapping/TestReOptimization.java b/ql/src/test/org/apache/hadoop/hive/ql/plan/mapping/TestReOptimization.java
index 8bec56f..b726300 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/plan/mapping/TestReOptimization.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/plan/mapping/TestReOptimization.java
@@ -27,18 +27,15 @@ import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.ql.DriverFactory;
 import org.apache.hadoop.hive.ql.IDriver;
-import org.apache.hadoop.hive.ql.exec.CommonJoinOperator;
 import org.apache.hadoop.hive.ql.exec.FilterOperator;
 import org.apache.hadoop.hive.ql.plan.Statistics;
 import org.apache.hadoop.hive.ql.plan.mapper.PlanMapper;
-import org.apache.hadoop.hive.ql.plan.mapper.StatsSources;
 import org.apache.hadoop.hive.ql.plan.mapper.PlanMapper.EquivGroup;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.ql.stats.OperatorStats;
 import org.apache.hadoop.hive.ql.stats.OperatorStatsReaderHook;
 import org.apache.hive.testutils.HiveTestEnvSetup;
-import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
@@ -58,7 +55,7 @@ public class TestReOptimization {
   public static void beforeClass() throws Exception {
     IDriver driver = createDriver("");
     dropTables(driver);
-    String[] cmds = {
+    String cmds[] = {
         // @formatter:off
         "create table tu(id_uv int,id_uw int,u int)",
         "create table tv(id_uv int,v int)",
@@ -81,13 +78,8 @@ public class TestReOptimization {
     dropTables(driver);
   }
 
-  @After
-  public void after() {
-    StatsSources.clearGlobalStats();
-  }
-
   public static void dropTables(IDriver driver) throws Exception {
-    String[] tables = new String[] {"tu", "tv", "tw" };
+    String tables[] = { "tu", "tv", "tw" };
     for (String t : tables) {
       int ret = driver.run("drop table if exists " + t).getResponseCode();
       assertEquals("Checking command success", 0, ret);
@@ -106,9 +98,7 @@ public class TestReOptimization {
   @Test
   public void testStatsAreSetInReopt() throws Exception {
     IDriver driver = createDriver("overlay,reoptimize");
-    String query = "select assert_true_oom(${hiveconf:zzz} > sum(u*v))"
-        + " from tu join tv on (tu.id_uv=tv.id_uv)"
-        + " where u<10 and v>1";
+    String query = "select assert_true_oom(${hiveconf:zzz} > sum(u*v)) from tu join tv on (tu.id_uv=tv.id_uv) where u<10 and v>1";
 
     PlanMapper pm = getMapperForQuery(driver, query);
     Iterator<EquivGroup> itG = pm.iterateGroups();
@@ -143,7 +133,7 @@ public class TestReOptimization {
     IDriver driver = createDriver("overlay,reoptimize");
     String query =
         "select assert_true_oom(${hiveconf:zzz}>sum(1)) from tu join tv on (tu.id_uv=tv.id_uv) where u<10 and v>1";
-    getMapperForQuery(driver, query);
+    PlanMapper pm = getMapperForQuery(driver, query);
 
   }
 
@@ -153,72 +143,8 @@ public class TestReOptimization {
     String query =
         "select assert_true(${hiveconf:zzz}>sum(1)) from tu join tv on (tu.id_uv=tv.id_uv) where u<10 and v>1";
 
-    getMapperForQuery(driver, query);
-    assertEquals(1, driver.getContext().getExecutionIndex());
-  }
-
-  @Test
-  public void testStatCachingQuery() throws Exception {
-    HiveConf conf = env_setup.getTestCtx().hiveConf;
-    conf.setVar(ConfVars.HIVE_QUERY_REEXECUTION_STATS_PERSISTENCE, "query");
-    conf.setBoolVar(ConfVars.HIVE_QUERY_REEXECUTION_ALWAYS_COLLECT_OPERATOR_STATS, true);
-
-    checkRuntimeStatsReuse(false, false, false);
-  }
-
-  @Test
-  public void testStatCachingHS2() throws Exception {
-    HiveConf conf = env_setup.getTestCtx().hiveConf;
-    conf.setVar(ConfVars.HIVE_QUERY_REEXECUTION_STATS_PERSISTENCE, "hiveserver");
-    conf.setBoolVar(ConfVars.HIVE_QUERY_REEXECUTION_ALWAYS_COLLECT_OPERATOR_STATS, true);
-
-    checkRuntimeStatsReuse(true, true, false);
-  }
-
-  @Test
-  public void testStatCachingMetaStore() throws Exception {
-    HiveConf conf = env_setup.getTestCtx().hiveConf;
-    conf.setVar(ConfVars.HIVE_QUERY_REEXECUTION_STATS_PERSISTENCE, "metastore");
-    conf.setBoolVar(ConfVars.HIVE_QUERY_REEXECUTION_ALWAYS_COLLECT_OPERATOR_STATS, true);
-
-    checkRuntimeStatsReuse(true, true, true);
-  }
-
-  private void checkRuntimeStatsReuse(
-      boolean expectInSameSession,
-      boolean expectNewHs2Session,
-      boolean expectHs2Instance) throws CommandProcessorResponse {
-    {
-      // same session
-      IDriver driver = createDriver("reoptimize");
-      checkUsageOfRuntimeStats(driver, false);
-      driver = DriverFactory.newDriver(env_setup.getTestCtx().hiveConf);
-      checkUsageOfRuntimeStats(driver, expectInSameSession);
-    }
-    {
-      // new session
-      IDriver driver = createDriver("reoptimize");
-      checkUsageOfRuntimeStats(driver, expectNewHs2Session);
-    }
-    StatsSources.clearGlobalStats();
-    {
-      // new hs2 instance session
-      IDriver driver = createDriver("reoptimize");
-      checkUsageOfRuntimeStats(driver, expectHs2Instance);
-    }
-  }
-
-  @SuppressWarnings("rawtypes")
-  private void checkUsageOfRuntimeStats(IDriver driver, boolean expected) throws CommandProcessorResponse {
-    String query = "select sum(u) from tu join tv on (tu.id_uv=tv.id_uv) where u<10 and v>1";
     PlanMapper pm = getMapperForQuery(driver, query);
     assertEquals(1, driver.getContext().getExecutionIndex());
-    List<CommonJoinOperator> allJoin = pm.getAll(CommonJoinOperator.class);
-    CommonJoinOperator join = allJoin.iterator().next();
-    Statistics joinStat = join.getStatistics();
-
-    assertEquals("expectation of the usage of runtime stats doesn't match", expected,
-        joinStat.isRuntimeStats());
   }
 
   @Test
@@ -226,7 +152,7 @@ public class TestReOptimization {
 
     IDriver driver = createDriver("overlay,reoptimize");
     String query = "explain reoptimization select 1 from tu join tv on (tu.id_uv=tv.id_uv) where u<10 and v>1";
-    getMapperForQuery(driver, query);
+    PlanMapper pm = getMapperForQuery(driver, query);
     List<String> res = new ArrayList<>();
     List<String> res1 = new ArrayList<>();
     while (driver.getResults(res1)) {
@@ -239,7 +165,6 @@ public class TestReOptimization {
 
   }
 
-
   private static IDriver createDriver(String strategies) {
     HiveConf conf = env_setup.getTestCtx().hiveConf;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/service/src/java/org/apache/hive/service/server/HiveServer2.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/server/HiveServer2.java b/service/src/java/org/apache/hive/service/server/HiveServer2.java
index e373628..1642357 100644
--- a/service/src/java/org/apache/hive/service/server/HiveServer2.java
+++ b/service/src/java/org/apache/hive/service/server/HiveServer2.java
@@ -81,7 +81,6 @@ import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.metadata.HiveMaterializedViewsRegistry;
 import org.apache.hadoop.hive.ql.metadata.events.NotificationEventPoll;
-import org.apache.hadoop.hive.ql.plan.mapper.StatsSources;
 import org.apache.hadoop.hive.ql.session.ClearDanglingScratchDir;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.ql.util.ZooKeeperHiveHelper;
@@ -248,8 +247,6 @@ public class HiveServer2 extends CompositeService {
     // Create views registry
     HiveMaterializedViewsRegistry.get().init();
 
-    StatsSources.initialize(hiveConf);
-
     // Setup cache if enabled.
     if (hiveConf.getBoolVar(HiveConf.ConfVars.HIVE_QUERY_RESULTS_CACHE_ENABLED)) {
       try {


[4/6] hive git commit: Revert "HIVE-19171 : Persist runtime statistics in metastore (Zoltan Haindrich via Ashutosh Chauhan)"

Posted by pr...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
index da66951..802d8e3 100644
--- a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
+++ b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
@@ -224,8 +224,6 @@ class ThriftHiveMetastoreIf : virtual public  ::facebook::fb303::FacebookService
   virtual void get_serde(SerDeInfo& _return, const GetSerdeRequest& rqst) = 0;
   virtual void get_lock_materialization_rebuild(LockResponse& _return, const std::string& dbName, const std::string& tableName, const int64_t txnId) = 0;
   virtual bool heartbeat_lock_materialization_rebuild(const std::string& dbName, const std::string& tableName, const int64_t txnId) = 0;
-  virtual void add_runtime_stats(const RuntimeStat& stat) = 0;
-  virtual void get_runtime_stats(std::vector<RuntimeStat> & _return, const GetRuntimeStatsRequest& rqst) = 0;
 };
 
 class ThriftHiveMetastoreIfFactory : virtual public  ::facebook::fb303::FacebookServiceIfFactory {
@@ -889,12 +887,6 @@ class ThriftHiveMetastoreNull : virtual public ThriftHiveMetastoreIf , virtual p
     bool _return = false;
     return _return;
   }
-  void add_runtime_stats(const RuntimeStat& /* stat */) {
-    return;
-  }
-  void get_runtime_stats(std::vector<RuntimeStat> & /* _return */, const GetRuntimeStatsRequest& /* rqst */) {
-    return;
-  }
 };
 
 typedef struct _ThriftHiveMetastore_getMetaConf_args__isset {
@@ -25652,222 +25644,6 @@ class ThriftHiveMetastore_heartbeat_lock_materialization_rebuild_presult {
 
 };
 
-typedef struct _ThriftHiveMetastore_add_runtime_stats_args__isset {
-  _ThriftHiveMetastore_add_runtime_stats_args__isset() : stat(false) {}
-  bool stat :1;
-} _ThriftHiveMetastore_add_runtime_stats_args__isset;
-
-class ThriftHiveMetastore_add_runtime_stats_args {
- public:
-
-  ThriftHiveMetastore_add_runtime_stats_args(const ThriftHiveMetastore_add_runtime_stats_args&);
-  ThriftHiveMetastore_add_runtime_stats_args& operator=(const ThriftHiveMetastore_add_runtime_stats_args&);
-  ThriftHiveMetastore_add_runtime_stats_args() {
-  }
-
-  virtual ~ThriftHiveMetastore_add_runtime_stats_args() throw();
-  RuntimeStat stat;
-
-  _ThriftHiveMetastore_add_runtime_stats_args__isset __isset;
-
-  void __set_stat(const RuntimeStat& val);
-
-  bool operator == (const ThriftHiveMetastore_add_runtime_stats_args & rhs) const
-  {
-    if (!(stat == rhs.stat))
-      return false;
-    return true;
-  }
-  bool operator != (const ThriftHiveMetastore_add_runtime_stats_args &rhs) const {
-    return !(*this == rhs);
-  }
-
-  bool operator < (const ThriftHiveMetastore_add_runtime_stats_args & ) const;
-
-  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
-  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
-
-};
-
-
-class ThriftHiveMetastore_add_runtime_stats_pargs {
- public:
-
-
-  virtual ~ThriftHiveMetastore_add_runtime_stats_pargs() throw();
-  const RuntimeStat* stat;
-
-  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
-
-};
-
-typedef struct _ThriftHiveMetastore_add_runtime_stats_result__isset {
-  _ThriftHiveMetastore_add_runtime_stats_result__isset() : o1(false) {}
-  bool o1 :1;
-} _ThriftHiveMetastore_add_runtime_stats_result__isset;
-
-class ThriftHiveMetastore_add_runtime_stats_result {
- public:
-
-  ThriftHiveMetastore_add_runtime_stats_result(const ThriftHiveMetastore_add_runtime_stats_result&);
-  ThriftHiveMetastore_add_runtime_stats_result& operator=(const ThriftHiveMetastore_add_runtime_stats_result&);
-  ThriftHiveMetastore_add_runtime_stats_result() {
-  }
-
-  virtual ~ThriftHiveMetastore_add_runtime_stats_result() throw();
-  MetaException o1;
-
-  _ThriftHiveMetastore_add_runtime_stats_result__isset __isset;
-
-  void __set_o1(const MetaException& val);
-
-  bool operator == (const ThriftHiveMetastore_add_runtime_stats_result & rhs) const
-  {
-    if (!(o1 == rhs.o1))
-      return false;
-    return true;
-  }
-  bool operator != (const ThriftHiveMetastore_add_runtime_stats_result &rhs) const {
-    return !(*this == rhs);
-  }
-
-  bool operator < (const ThriftHiveMetastore_add_runtime_stats_result & ) const;
-
-  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
-  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
-
-};
-
-typedef struct _ThriftHiveMetastore_add_runtime_stats_presult__isset {
-  _ThriftHiveMetastore_add_runtime_stats_presult__isset() : o1(false) {}
-  bool o1 :1;
-} _ThriftHiveMetastore_add_runtime_stats_presult__isset;
-
-class ThriftHiveMetastore_add_runtime_stats_presult {
- public:
-
-
-  virtual ~ThriftHiveMetastore_add_runtime_stats_presult() throw();
-  MetaException o1;
-
-  _ThriftHiveMetastore_add_runtime_stats_presult__isset __isset;
-
-  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
-
-};
-
-typedef struct _ThriftHiveMetastore_get_runtime_stats_args__isset {
-  _ThriftHiveMetastore_get_runtime_stats_args__isset() : rqst(false) {}
-  bool rqst :1;
-} _ThriftHiveMetastore_get_runtime_stats_args__isset;
-
-class ThriftHiveMetastore_get_runtime_stats_args {
- public:
-
-  ThriftHiveMetastore_get_runtime_stats_args(const ThriftHiveMetastore_get_runtime_stats_args&);
-  ThriftHiveMetastore_get_runtime_stats_args& operator=(const ThriftHiveMetastore_get_runtime_stats_args&);
-  ThriftHiveMetastore_get_runtime_stats_args() {
-  }
-
-  virtual ~ThriftHiveMetastore_get_runtime_stats_args() throw();
-  GetRuntimeStatsRequest rqst;
-
-  _ThriftHiveMetastore_get_runtime_stats_args__isset __isset;
-
-  void __set_rqst(const GetRuntimeStatsRequest& val);
-
-  bool operator == (const ThriftHiveMetastore_get_runtime_stats_args & rhs) const
-  {
-    if (!(rqst == rhs.rqst))
-      return false;
-    return true;
-  }
-  bool operator != (const ThriftHiveMetastore_get_runtime_stats_args &rhs) const {
-    return !(*this == rhs);
-  }
-
-  bool operator < (const ThriftHiveMetastore_get_runtime_stats_args & ) const;
-
-  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
-  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
-
-};
-
-
-class ThriftHiveMetastore_get_runtime_stats_pargs {
- public:
-
-
-  virtual ~ThriftHiveMetastore_get_runtime_stats_pargs() throw();
-  const GetRuntimeStatsRequest* rqst;
-
-  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
-
-};
-
-typedef struct _ThriftHiveMetastore_get_runtime_stats_result__isset {
-  _ThriftHiveMetastore_get_runtime_stats_result__isset() : success(false), o1(false) {}
-  bool success :1;
-  bool o1 :1;
-} _ThriftHiveMetastore_get_runtime_stats_result__isset;
-
-class ThriftHiveMetastore_get_runtime_stats_result {
- public:
-
-  ThriftHiveMetastore_get_runtime_stats_result(const ThriftHiveMetastore_get_runtime_stats_result&);
-  ThriftHiveMetastore_get_runtime_stats_result& operator=(const ThriftHiveMetastore_get_runtime_stats_result&);
-  ThriftHiveMetastore_get_runtime_stats_result() {
-  }
-
-  virtual ~ThriftHiveMetastore_get_runtime_stats_result() throw();
-  std::vector<RuntimeStat>  success;
-  MetaException o1;
-
-  _ThriftHiveMetastore_get_runtime_stats_result__isset __isset;
-
-  void __set_success(const std::vector<RuntimeStat> & val);
-
-  void __set_o1(const MetaException& val);
-
-  bool operator == (const ThriftHiveMetastore_get_runtime_stats_result & rhs) const
-  {
-    if (!(success == rhs.success))
-      return false;
-    if (!(o1 == rhs.o1))
-      return false;
-    return true;
-  }
-  bool operator != (const ThriftHiveMetastore_get_runtime_stats_result &rhs) const {
-    return !(*this == rhs);
-  }
-
-  bool operator < (const ThriftHiveMetastore_get_runtime_stats_result & ) const;
-
-  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
-  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
-
-};
-
-typedef struct _ThriftHiveMetastore_get_runtime_stats_presult__isset {
-  _ThriftHiveMetastore_get_runtime_stats_presult__isset() : success(false), o1(false) {}
-  bool success :1;
-  bool o1 :1;
-} _ThriftHiveMetastore_get_runtime_stats_presult__isset;
-
-class ThriftHiveMetastore_get_runtime_stats_presult {
- public:
-
-
-  virtual ~ThriftHiveMetastore_get_runtime_stats_presult() throw();
-  std::vector<RuntimeStat> * success;
-  MetaException o1;
-
-  _ThriftHiveMetastore_get_runtime_stats_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) :
@@ -26485,12 +26261,6 @@ class ThriftHiveMetastoreClient : virtual public ThriftHiveMetastoreIf, public
   bool heartbeat_lock_materialization_rebuild(const std::string& dbName, const std::string& tableName, const int64_t txnId);
   void send_heartbeat_lock_materialization_rebuild(const std::string& dbName, const std::string& tableName, const int64_t txnId);
   bool recv_heartbeat_lock_materialization_rebuild();
-  void add_runtime_stats(const RuntimeStat& stat);
-  void send_add_runtime_stats(const RuntimeStat& stat);
-  void recv_add_runtime_stats();
-  void get_runtime_stats(std::vector<RuntimeStat> & _return, const GetRuntimeStatsRequest& rqst);
-  void send_get_runtime_stats(const GetRuntimeStatsRequest& rqst);
-  void recv_get_runtime_stats(std::vector<RuntimeStat> & _return);
 };
 
 class ThriftHiveMetastoreProcessor : public  ::facebook::fb303::FacebookServiceProcessor {
@@ -26703,8 +26473,6 @@ class ThriftHiveMetastoreProcessor : public  ::facebook::fb303::FacebookServiceP
   void process_get_serde(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_get_lock_materialization_rebuild(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_heartbeat_lock_materialization_rebuild(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
-  void process_add_runtime_stats(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
-  void process_get_runtime_stats(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),
@@ -26911,8 +26679,6 @@ class ThriftHiveMetastoreProcessor : public  ::facebook::fb303::FacebookServiceP
     processMap_["get_serde"] = &ThriftHiveMetastoreProcessor::process_get_serde;
     processMap_["get_lock_materialization_rebuild"] = &ThriftHiveMetastoreProcessor::process_get_lock_materialization_rebuild;
     processMap_["heartbeat_lock_materialization_rebuild"] = &ThriftHiveMetastoreProcessor::process_heartbeat_lock_materialization_rebuild;
-    processMap_["add_runtime_stats"] = &ThriftHiveMetastoreProcessor::process_add_runtime_stats;
-    processMap_["get_runtime_stats"] = &ThriftHiveMetastoreProcessor::process_get_runtime_stats;
   }
 
   virtual ~ThriftHiveMetastoreProcessor() {}
@@ -28888,25 +28654,6 @@ class ThriftHiveMetastoreMultiface : virtual public ThriftHiveMetastoreIf, publi
     return ifaces_[i]->heartbeat_lock_materialization_rebuild(dbName, tableName, txnId);
   }
 
-  void add_runtime_stats(const RuntimeStat& stat) {
-    size_t sz = ifaces_.size();
-    size_t i = 0;
-    for (; i < (sz - 1); ++i) {
-      ifaces_[i]->add_runtime_stats(stat);
-    }
-    ifaces_[i]->add_runtime_stats(stat);
-  }
-
-  void get_runtime_stats(std::vector<RuntimeStat> & _return, const GetRuntimeStatsRequest& rqst) {
-    size_t sz = ifaces_.size();
-    size_t i = 0;
-    for (; i < (sz - 1); ++i) {
-      ifaces_[i]->get_runtime_stats(_return, rqst);
-    }
-    ifaces_[i]->get_runtime_stats(_return, rqst);
-    return;
-  }
-
 };
 
 // The 'concurrent' client is a thread safe client that correctly handles
@@ -29529,12 +29276,6 @@ class ThriftHiveMetastoreConcurrentClient : virtual public ThriftHiveMetastoreIf
   bool heartbeat_lock_materialization_rebuild(const std::string& dbName, const std::string& tableName, const int64_t txnId);
   int32_t send_heartbeat_lock_materialization_rebuild(const std::string& dbName, const std::string& tableName, const int64_t txnId);
   bool recv_heartbeat_lock_materialization_rebuild(const int32_t seqid);
-  void add_runtime_stats(const RuntimeStat& stat);
-  int32_t send_add_runtime_stats(const RuntimeStat& stat);
-  void recv_add_runtime_stats(const int32_t seqid);
-  void get_runtime_stats(std::vector<RuntimeStat> & _return, const GetRuntimeStatsRequest& rqst);
-  int32_t send_get_runtime_stats(const GetRuntimeStatsRequest& rqst);
-  void recv_get_runtime_stats(std::vector<RuntimeStat> & _return, const int32_t seqid);
 };
 
 #ifdef _WIN32

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
index 3b0b38c..c0a39f8 100644
--- a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
+++ b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
@@ -1032,16 +1032,6 @@ class ThriftHiveMetastoreHandler : virtual public ThriftHiveMetastoreIf {
     printf("heartbeat_lock_materialization_rebuild\n");
   }
 
-  void add_runtime_stats(const RuntimeStat& stat) {
-    // Your implementation goes here
-    printf("add_runtime_stats\n");
-  }
-
-  void get_runtime_stats(std::vector<RuntimeStat> & _return, const GetRuntimeStatsRequest& rqst) {
-    // Your implementation goes here
-    printf("get_runtime_stats\n");
-  }
-
 };
 
 int main(int argc, char **argv) {

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp b/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
index 2fab857..052b595 100644
--- a/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
+++ b/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
@@ -30551,202 +30551,6 @@ void GetSerdeRequest::printTo(std::ostream& out) const {
 }
 
 
-RuntimeStat::~RuntimeStat() throw() {
-}
-
-
-void RuntimeStat::__set_createTime(const int32_t val) {
-  this->createTime = val;
-__isset.createTime = true;
-}
-
-void RuntimeStat::__set_weight(const int32_t val) {
-  this->weight = val;
-}
-
-void RuntimeStat::__set_payload(const std::string& val) {
-  this->payload = val;
-}
-
-uint32_t RuntimeStat::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_weight = false;
-  bool isset_payload = 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_I32) {
-          xfer += iprot->readI32(this->createTime);
-          this->__isset.createTime = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 2:
-        if (ftype == ::apache::thrift::protocol::T_I32) {
-          xfer += iprot->readI32(this->weight);
-          isset_weight = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 3:
-        if (ftype == ::apache::thrift::protocol::T_STRING) {
-          xfer += iprot->readBinary(this->payload);
-          isset_payload = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      default:
-        xfer += iprot->skip(ftype);
-        break;
-    }
-    xfer += iprot->readFieldEnd();
-  }
-
-  xfer += iprot->readStructEnd();
-
-  if (!isset_weight)
-    throw TProtocolException(TProtocolException::INVALID_DATA);
-  if (!isset_payload)
-    throw TProtocolException(TProtocolException::INVALID_DATA);
-  return xfer;
-}
-
-uint32_t RuntimeStat::write(::apache::thrift::protocol::TProtocol* oprot) const {
-  uint32_t xfer = 0;
-  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("RuntimeStat");
-
-  if (this->__isset.createTime) {
-    xfer += oprot->writeFieldBegin("createTime", ::apache::thrift::protocol::T_I32, 1);
-    xfer += oprot->writeI32(this->createTime);
-    xfer += oprot->writeFieldEnd();
-  }
-  xfer += oprot->writeFieldBegin("weight", ::apache::thrift::protocol::T_I32, 2);
-  xfer += oprot->writeI32(this->weight);
-  xfer += oprot->writeFieldEnd();
-
-  xfer += oprot->writeFieldBegin("payload", ::apache::thrift::protocol::T_STRING, 3);
-  xfer += oprot->writeBinary(this->payload);
-  xfer += oprot->writeFieldEnd();
-
-  xfer += oprot->writeFieldStop();
-  xfer += oprot->writeStructEnd();
-  return xfer;
-}
-
-void swap(RuntimeStat &a, RuntimeStat &b) {
-  using ::std::swap;
-  swap(a.createTime, b.createTime);
-  swap(a.weight, b.weight);
-  swap(a.payload, b.payload);
-  swap(a.__isset, b.__isset);
-}
-
-RuntimeStat::RuntimeStat(const RuntimeStat& other1157) {
-  createTime = other1157.createTime;
-  weight = other1157.weight;
-  payload = other1157.payload;
-  __isset = other1157.__isset;
-}
-RuntimeStat& RuntimeStat::operator=(const RuntimeStat& other1158) {
-  createTime = other1158.createTime;
-  weight = other1158.weight;
-  payload = other1158.payload;
-  __isset = other1158.__isset;
-  return *this;
-}
-void RuntimeStat::printTo(std::ostream& out) const {
-  using ::apache::thrift::to_string;
-  out << "RuntimeStat(";
-  out << "createTime="; (__isset.createTime ? (out << to_string(createTime)) : (out << "<null>"));
-  out << ", " << "weight=" << to_string(weight);
-  out << ", " << "payload=" << to_string(payload);
-  out << ")";
-}
-
-
-GetRuntimeStatsRequest::~GetRuntimeStatsRequest() throw() {
-}
-
-
-uint32_t GetRuntimeStatsRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
-
-  apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
-  uint32_t xfer = 0;
-  std::string fname;
-  ::apache::thrift::protocol::TType ftype;
-  int16_t fid;
-
-  xfer += iprot->readStructBegin(fname);
-
-  using ::apache::thrift::protocol::TProtocolException;
-
-
-  while (true)
-  {
-    xfer += iprot->readFieldBegin(fname, ftype, fid);
-    if (ftype == ::apache::thrift::protocol::T_STOP) {
-      break;
-    }
-    xfer += iprot->skip(ftype);
-    xfer += iprot->readFieldEnd();
-  }
-
-  xfer += iprot->readStructEnd();
-
-  return xfer;
-}
-
-uint32_t GetRuntimeStatsRequest::write(::apache::thrift::protocol::TProtocol* oprot) const {
-  uint32_t xfer = 0;
-  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
-  xfer += oprot->writeStructBegin("GetRuntimeStatsRequest");
-
-  xfer += oprot->writeFieldStop();
-  xfer += oprot->writeStructEnd();
-  return xfer;
-}
-
-void swap(GetRuntimeStatsRequest &a, GetRuntimeStatsRequest &b) {
-  using ::std::swap;
-  (void) a;
-  (void) b;
-}
-
-GetRuntimeStatsRequest::GetRuntimeStatsRequest(const GetRuntimeStatsRequest& other1159) {
-  (void) other1159;
-}
-GetRuntimeStatsRequest& GetRuntimeStatsRequest::operator=(const GetRuntimeStatsRequest& other1160) {
-  (void) other1160;
-  return *this;
-}
-void GetRuntimeStatsRequest::printTo(std::ostream& out) const {
-  using ::apache::thrift::to_string;
-  out << "GetRuntimeStatsRequest(";
-  out << ")";
-}
-
-
 MetaException::~MetaException() throw() {
 }
 
@@ -30816,13 +30620,13 @@ void swap(MetaException &a, MetaException &b) {
   swap(a.__isset, b.__isset);
 }
 
-MetaException::MetaException(const MetaException& other1161) : TException() {
-  message = other1161.message;
-  __isset = other1161.__isset;
+MetaException::MetaException(const MetaException& other1157) : TException() {
+  message = other1157.message;
+  __isset = other1157.__isset;
 }
-MetaException& MetaException::operator=(const MetaException& other1162) {
-  message = other1162.message;
-  __isset = other1162.__isset;
+MetaException& MetaException::operator=(const MetaException& other1158) {
+  message = other1158.message;
+  __isset = other1158.__isset;
   return *this;
 }
 void MetaException::printTo(std::ostream& out) const {
@@ -30913,13 +30717,13 @@ void swap(UnknownTableException &a, UnknownTableException &b) {
   swap(a.__isset, b.__isset);
 }
 
-UnknownTableException::UnknownTableException(const UnknownTableException& other1163) : TException() {
-  message = other1163.message;
-  __isset = other1163.__isset;
+UnknownTableException::UnknownTableException(const UnknownTableException& other1159) : TException() {
+  message = other1159.message;
+  __isset = other1159.__isset;
 }
-UnknownTableException& UnknownTableException::operator=(const UnknownTableException& other1164) {
-  message = other1164.message;
-  __isset = other1164.__isset;
+UnknownTableException& UnknownTableException::operator=(const UnknownTableException& other1160) {
+  message = other1160.message;
+  __isset = other1160.__isset;
   return *this;
 }
 void UnknownTableException::printTo(std::ostream& out) const {
@@ -31010,13 +30814,13 @@ void swap(UnknownDBException &a, UnknownDBException &b) {
   swap(a.__isset, b.__isset);
 }
 
-UnknownDBException::UnknownDBException(const UnknownDBException& other1165) : TException() {
-  message = other1165.message;
-  __isset = other1165.__isset;
+UnknownDBException::UnknownDBException(const UnknownDBException& other1161) : TException() {
+  message = other1161.message;
+  __isset = other1161.__isset;
 }
-UnknownDBException& UnknownDBException::operator=(const UnknownDBException& other1166) {
-  message = other1166.message;
-  __isset = other1166.__isset;
+UnknownDBException& UnknownDBException::operator=(const UnknownDBException& other1162) {
+  message = other1162.message;
+  __isset = other1162.__isset;
   return *this;
 }
 void UnknownDBException::printTo(std::ostream& out) const {
@@ -31107,13 +30911,13 @@ void swap(AlreadyExistsException &a, AlreadyExistsException &b) {
   swap(a.__isset, b.__isset);
 }
 
-AlreadyExistsException::AlreadyExistsException(const AlreadyExistsException& other1167) : TException() {
-  message = other1167.message;
-  __isset = other1167.__isset;
+AlreadyExistsException::AlreadyExistsException(const AlreadyExistsException& other1163) : TException() {
+  message = other1163.message;
+  __isset = other1163.__isset;
 }
-AlreadyExistsException& AlreadyExistsException::operator=(const AlreadyExistsException& other1168) {
-  message = other1168.message;
-  __isset = other1168.__isset;
+AlreadyExistsException& AlreadyExistsException::operator=(const AlreadyExistsException& other1164) {
+  message = other1164.message;
+  __isset = other1164.__isset;
   return *this;
 }
 void AlreadyExistsException::printTo(std::ostream& out) const {
@@ -31204,13 +31008,13 @@ void swap(InvalidPartitionException &a, InvalidPartitionException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidPartitionException::InvalidPartitionException(const InvalidPartitionException& other1169) : TException() {
-  message = other1169.message;
-  __isset = other1169.__isset;
+InvalidPartitionException::InvalidPartitionException(const InvalidPartitionException& other1165) : TException() {
+  message = other1165.message;
+  __isset = other1165.__isset;
 }
-InvalidPartitionException& InvalidPartitionException::operator=(const InvalidPartitionException& other1170) {
-  message = other1170.message;
-  __isset = other1170.__isset;
+InvalidPartitionException& InvalidPartitionException::operator=(const InvalidPartitionException& other1166) {
+  message = other1166.message;
+  __isset = other1166.__isset;
   return *this;
 }
 void InvalidPartitionException::printTo(std::ostream& out) const {
@@ -31301,13 +31105,13 @@ void swap(UnknownPartitionException &a, UnknownPartitionException &b) {
   swap(a.__isset, b.__isset);
 }
 
-UnknownPartitionException::UnknownPartitionException(const UnknownPartitionException& other1171) : TException() {
-  message = other1171.message;
-  __isset = other1171.__isset;
+UnknownPartitionException::UnknownPartitionException(const UnknownPartitionException& other1167) : TException() {
+  message = other1167.message;
+  __isset = other1167.__isset;
 }
-UnknownPartitionException& UnknownPartitionException::operator=(const UnknownPartitionException& other1172) {
-  message = other1172.message;
-  __isset = other1172.__isset;
+UnknownPartitionException& UnknownPartitionException::operator=(const UnknownPartitionException& other1168) {
+  message = other1168.message;
+  __isset = other1168.__isset;
   return *this;
 }
 void UnknownPartitionException::printTo(std::ostream& out) const {
@@ -31398,13 +31202,13 @@ void swap(InvalidObjectException &a, InvalidObjectException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidObjectException::InvalidObjectException(const InvalidObjectException& other1173) : TException() {
-  message = other1173.message;
-  __isset = other1173.__isset;
+InvalidObjectException::InvalidObjectException(const InvalidObjectException& other1169) : TException() {
+  message = other1169.message;
+  __isset = other1169.__isset;
 }
-InvalidObjectException& InvalidObjectException::operator=(const InvalidObjectException& other1174) {
-  message = other1174.message;
-  __isset = other1174.__isset;
+InvalidObjectException& InvalidObjectException::operator=(const InvalidObjectException& other1170) {
+  message = other1170.message;
+  __isset = other1170.__isset;
   return *this;
 }
 void InvalidObjectException::printTo(std::ostream& out) const {
@@ -31495,13 +31299,13 @@ void swap(NoSuchObjectException &a, NoSuchObjectException &b) {
   swap(a.__isset, b.__isset);
 }
 
-NoSuchObjectException::NoSuchObjectException(const NoSuchObjectException& other1175) : TException() {
-  message = other1175.message;
-  __isset = other1175.__isset;
+NoSuchObjectException::NoSuchObjectException(const NoSuchObjectException& other1171) : TException() {
+  message = other1171.message;
+  __isset = other1171.__isset;
 }
-NoSuchObjectException& NoSuchObjectException::operator=(const NoSuchObjectException& other1176) {
-  message = other1176.message;
-  __isset = other1176.__isset;
+NoSuchObjectException& NoSuchObjectException::operator=(const NoSuchObjectException& other1172) {
+  message = other1172.message;
+  __isset = other1172.__isset;
   return *this;
 }
 void NoSuchObjectException::printTo(std::ostream& out) const {
@@ -31592,13 +31396,13 @@ void swap(InvalidOperationException &a, InvalidOperationException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidOperationException::InvalidOperationException(const InvalidOperationException& other1177) : TException() {
-  message = other1177.message;
-  __isset = other1177.__isset;
+InvalidOperationException::InvalidOperationException(const InvalidOperationException& other1173) : TException() {
+  message = other1173.message;
+  __isset = other1173.__isset;
 }
-InvalidOperationException& InvalidOperationException::operator=(const InvalidOperationException& other1178) {
-  message = other1178.message;
-  __isset = other1178.__isset;
+InvalidOperationException& InvalidOperationException::operator=(const InvalidOperationException& other1174) {
+  message = other1174.message;
+  __isset = other1174.__isset;
   return *this;
 }
 void InvalidOperationException::printTo(std::ostream& out) const {
@@ -31689,13 +31493,13 @@ void swap(ConfigValSecurityException &a, ConfigValSecurityException &b) {
   swap(a.__isset, b.__isset);
 }
 
-ConfigValSecurityException::ConfigValSecurityException(const ConfigValSecurityException& other1179) : TException() {
-  message = other1179.message;
-  __isset = other1179.__isset;
+ConfigValSecurityException::ConfigValSecurityException(const ConfigValSecurityException& other1175) : TException() {
+  message = other1175.message;
+  __isset = other1175.__isset;
 }
-ConfigValSecurityException& ConfigValSecurityException::operator=(const ConfigValSecurityException& other1180) {
-  message = other1180.message;
-  __isset = other1180.__isset;
+ConfigValSecurityException& ConfigValSecurityException::operator=(const ConfigValSecurityException& other1176) {
+  message = other1176.message;
+  __isset = other1176.__isset;
   return *this;
 }
 void ConfigValSecurityException::printTo(std::ostream& out) const {
@@ -31786,13 +31590,13 @@ void swap(InvalidInputException &a, InvalidInputException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidInputException::InvalidInputException(const InvalidInputException& other1181) : TException() {
-  message = other1181.message;
-  __isset = other1181.__isset;
+InvalidInputException::InvalidInputException(const InvalidInputException& other1177) : TException() {
+  message = other1177.message;
+  __isset = other1177.__isset;
 }
-InvalidInputException& InvalidInputException::operator=(const InvalidInputException& other1182) {
-  message = other1182.message;
-  __isset = other1182.__isset;
+InvalidInputException& InvalidInputException::operator=(const InvalidInputException& other1178) {
+  message = other1178.message;
+  __isset = other1178.__isset;
   return *this;
 }
 void InvalidInputException::printTo(std::ostream& out) const {
@@ -31883,13 +31687,13 @@ void swap(NoSuchTxnException &a, NoSuchTxnException &b) {
   swap(a.__isset, b.__isset);
 }
 
-NoSuchTxnException::NoSuchTxnException(const NoSuchTxnException& other1183) : TException() {
-  message = other1183.message;
-  __isset = other1183.__isset;
+NoSuchTxnException::NoSuchTxnException(const NoSuchTxnException& other1179) : TException() {
+  message = other1179.message;
+  __isset = other1179.__isset;
 }
-NoSuchTxnException& NoSuchTxnException::operator=(const NoSuchTxnException& other1184) {
-  message = other1184.message;
-  __isset = other1184.__isset;
+NoSuchTxnException& NoSuchTxnException::operator=(const NoSuchTxnException& other1180) {
+  message = other1180.message;
+  __isset = other1180.__isset;
   return *this;
 }
 void NoSuchTxnException::printTo(std::ostream& out) const {
@@ -31980,13 +31784,13 @@ void swap(TxnAbortedException &a, TxnAbortedException &b) {
   swap(a.__isset, b.__isset);
 }
 
-TxnAbortedException::TxnAbortedException(const TxnAbortedException& other1185) : TException() {
-  message = other1185.message;
-  __isset = other1185.__isset;
+TxnAbortedException::TxnAbortedException(const TxnAbortedException& other1181) : TException() {
+  message = other1181.message;
+  __isset = other1181.__isset;
 }
-TxnAbortedException& TxnAbortedException::operator=(const TxnAbortedException& other1186) {
-  message = other1186.message;
-  __isset = other1186.__isset;
+TxnAbortedException& TxnAbortedException::operator=(const TxnAbortedException& other1182) {
+  message = other1182.message;
+  __isset = other1182.__isset;
   return *this;
 }
 void TxnAbortedException::printTo(std::ostream& out) const {
@@ -32077,13 +31881,13 @@ void swap(TxnOpenException &a, TxnOpenException &b) {
   swap(a.__isset, b.__isset);
 }
 
-TxnOpenException::TxnOpenException(const TxnOpenException& other1187) : TException() {
-  message = other1187.message;
-  __isset = other1187.__isset;
+TxnOpenException::TxnOpenException(const TxnOpenException& other1183) : TException() {
+  message = other1183.message;
+  __isset = other1183.__isset;
 }
-TxnOpenException& TxnOpenException::operator=(const TxnOpenException& other1188) {
-  message = other1188.message;
-  __isset = other1188.__isset;
+TxnOpenException& TxnOpenException::operator=(const TxnOpenException& other1184) {
+  message = other1184.message;
+  __isset = other1184.__isset;
   return *this;
 }
 void TxnOpenException::printTo(std::ostream& out) const {
@@ -32174,13 +31978,13 @@ void swap(NoSuchLockException &a, NoSuchLockException &b) {
   swap(a.__isset, b.__isset);
 }
 
-NoSuchLockException::NoSuchLockException(const NoSuchLockException& other1189) : TException() {
-  message = other1189.message;
-  __isset = other1189.__isset;
+NoSuchLockException::NoSuchLockException(const NoSuchLockException& other1185) : TException() {
+  message = other1185.message;
+  __isset = other1185.__isset;
 }
-NoSuchLockException& NoSuchLockException::operator=(const NoSuchLockException& other1190) {
-  message = other1190.message;
-  __isset = other1190.__isset;
+NoSuchLockException& NoSuchLockException::operator=(const NoSuchLockException& other1186) {
+  message = other1186.message;
+  __isset = other1186.__isset;
   return *this;
 }
 void NoSuchLockException::printTo(std::ostream& out) const {

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h b/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
index cd78f58..6b872a6 100644
--- a/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
+++ b/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
@@ -659,10 +659,6 @@ class SetSchemaVersionStateRequest;
 
 class GetSerdeRequest;
 
-class RuntimeStat;
-
-class GetRuntimeStatsRequest;
-
 class MetaException;
 
 class UnknownTableException;
@@ -12938,99 +12934,6 @@ inline std::ostream& operator<<(std::ostream& out, const GetSerdeRequest& obj)
   return out;
 }
 
-typedef struct _RuntimeStat__isset {
-  _RuntimeStat__isset() : createTime(false) {}
-  bool createTime :1;
-} _RuntimeStat__isset;
-
-class RuntimeStat {
- public:
-
-  RuntimeStat(const RuntimeStat&);
-  RuntimeStat& operator=(const RuntimeStat&);
-  RuntimeStat() : createTime(0), weight(0), payload() {
-  }
-
-  virtual ~RuntimeStat() throw();
-  int32_t createTime;
-  int32_t weight;
-  std::string payload;
-
-  _RuntimeStat__isset __isset;
-
-  void __set_createTime(const int32_t val);
-
-  void __set_weight(const int32_t val);
-
-  void __set_payload(const std::string& val);
-
-  bool operator == (const RuntimeStat & rhs) const
-  {
-    if (__isset.createTime != rhs.__isset.createTime)
-      return false;
-    else if (__isset.createTime && !(createTime == rhs.createTime))
-      return false;
-    if (!(weight == rhs.weight))
-      return false;
-    if (!(payload == rhs.payload))
-      return false;
-    return true;
-  }
-  bool operator != (const RuntimeStat &rhs) const {
-    return !(*this == rhs);
-  }
-
-  bool operator < (const RuntimeStat & ) 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(RuntimeStat &a, RuntimeStat &b);
-
-inline std::ostream& operator<<(std::ostream& out, const RuntimeStat& obj)
-{
-  obj.printTo(out);
-  return out;
-}
-
-
-class GetRuntimeStatsRequest {
- public:
-
-  GetRuntimeStatsRequest(const GetRuntimeStatsRequest&);
-  GetRuntimeStatsRequest& operator=(const GetRuntimeStatsRequest&);
-  GetRuntimeStatsRequest() {
-  }
-
-  virtual ~GetRuntimeStatsRequest() throw();
-
-  bool operator == (const GetRuntimeStatsRequest & /* rhs */) const
-  {
-    return true;
-  }
-  bool operator != (const GetRuntimeStatsRequest &rhs) const {
-    return !(*this == rhs);
-  }
-
-  bool operator < (const GetRuntimeStatsRequest & ) 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(GetRuntimeStatsRequest &a, GetRuntimeStatsRequest &b);
-
-inline std::ostream& operator<<(std::ostream& out, const GetRuntimeStatsRequest& obj)
-{
-  obj.printTo(out);
-  return out;
-}
-
 typedef struct _MetaException__isset {
   _MetaException__isset() : message(false) {}
   bool message :1;

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetRuntimeStatsRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetRuntimeStatsRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetRuntimeStatsRequest.java
deleted file mode 100644
index e5ce97e..0000000
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetRuntimeStatsRequest.java
+++ /dev/null
@@ -1,283 +0,0 @@
-/**
- * Autogenerated by Thrift Compiler (0.9.3)
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- *  @generated
- */
-package org.apache.hadoop.hive.metastore.api;
-
-import org.apache.thrift.scheme.IScheme;
-import org.apache.thrift.scheme.SchemeFactory;
-import org.apache.thrift.scheme.StandardScheme;
-
-import org.apache.thrift.scheme.TupleScheme;
-import org.apache.thrift.protocol.TTupleProtocol;
-import org.apache.thrift.protocol.TProtocolException;
-import org.apache.thrift.EncodingUtils;
-import org.apache.thrift.TException;
-import org.apache.thrift.async.AsyncMethodCallback;
-import org.apache.thrift.server.AbstractNonblockingServer.*;
-import java.util.List;
-import java.util.ArrayList;
-import java.util.Map;
-import java.util.HashMap;
-import java.util.EnumMap;
-import java.util.Set;
-import java.util.HashSet;
-import java.util.EnumSet;
-import java.util.Collections;
-import java.util.BitSet;
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-import javax.annotation.Generated;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
-@org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public class GetRuntimeStatsRequest implements org.apache.thrift.TBase<GetRuntimeStatsRequest, GetRuntimeStatsRequest._Fields>, java.io.Serializable, Cloneable, Comparable<GetRuntimeStatsRequest> {
-  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetRuntimeStatsRequest");
-
-
-  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
-  static {
-    schemes.put(StandardScheme.class, new GetRuntimeStatsRequestStandardSchemeFactory());
-    schemes.put(TupleScheme.class, new GetRuntimeStatsRequestTupleSchemeFactory());
-  }
-
-
-  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
-  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-;
-
-    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) {
-        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;
-    }
-  }
-  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);
-    metaDataMap = Collections.unmodifiableMap(tmpMap);
-    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetRuntimeStatsRequest.class, metaDataMap);
-  }
-
-  public GetRuntimeStatsRequest() {
-  }
-
-  /**
-   * Performs a deep copy on <i>other</i>.
-   */
-  public GetRuntimeStatsRequest(GetRuntimeStatsRequest other) {
-  }
-
-  public GetRuntimeStatsRequest deepCopy() {
-    return new GetRuntimeStatsRequest(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 GetRuntimeStatsRequest)
-      return this.equals((GetRuntimeStatsRequest)that);
-    return false;
-  }
-
-  public boolean equals(GetRuntimeStatsRequest 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(GetRuntimeStatsRequest 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("GetRuntimeStatsRequest(");
-    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 GetRuntimeStatsRequestStandardSchemeFactory implements SchemeFactory {
-    public GetRuntimeStatsRequestStandardScheme getScheme() {
-      return new GetRuntimeStatsRequestStandardScheme();
-    }
-  }
-
-  private static class GetRuntimeStatsRequestStandardScheme extends StandardScheme<GetRuntimeStatsRequest> {
-
-    public void read(org.apache.thrift.protocol.TProtocol iprot, GetRuntimeStatsRequest 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, GetRuntimeStatsRequest struct) throws org.apache.thrift.TException {
-      struct.validate();
-
-      oprot.writeStructBegin(STRUCT_DESC);
-      oprot.writeFieldStop();
-      oprot.writeStructEnd();
-    }
-
-  }
-
-  private static class GetRuntimeStatsRequestTupleSchemeFactory implements SchemeFactory {
-    public GetRuntimeStatsRequestTupleScheme getScheme() {
-      return new GetRuntimeStatsRequestTupleScheme();
-    }
-  }
-
-  private static class GetRuntimeStatsRequestTupleScheme extends TupleScheme<GetRuntimeStatsRequest> {
-
-    @Override
-    public void write(org.apache.thrift.protocol.TProtocol prot, GetRuntimeStatsRequest struct) throws org.apache.thrift.TException {
-      TTupleProtocol oprot = (TTupleProtocol) prot;
-    }
-
-    @Override
-    public void read(org.apache.thrift.protocol.TProtocol prot, GetRuntimeStatsRequest struct) throws org.apache.thrift.TException {
-      TTupleProtocol iprot = (TTupleProtocol) prot;
-    }
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/RuntimeStat.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/RuntimeStat.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/RuntimeStat.java
deleted file mode 100644
index b48718c..0000000
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/RuntimeStat.java
+++ /dev/null
@@ -1,600 +0,0 @@
-/**
- * Autogenerated by Thrift Compiler (0.9.3)
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- *  @generated
- */
-package org.apache.hadoop.hive.metastore.api;
-
-import org.apache.thrift.scheme.IScheme;
-import org.apache.thrift.scheme.SchemeFactory;
-import org.apache.thrift.scheme.StandardScheme;
-
-import org.apache.thrift.scheme.TupleScheme;
-import org.apache.thrift.protocol.TTupleProtocol;
-import org.apache.thrift.protocol.TProtocolException;
-import org.apache.thrift.EncodingUtils;
-import org.apache.thrift.TException;
-import org.apache.thrift.async.AsyncMethodCallback;
-import org.apache.thrift.server.AbstractNonblockingServer.*;
-import java.util.List;
-import java.util.ArrayList;
-import java.util.Map;
-import java.util.HashMap;
-import java.util.EnumMap;
-import java.util.Set;
-import java.util.HashSet;
-import java.util.EnumSet;
-import java.util.Collections;
-import java.util.BitSet;
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-import javax.annotation.Generated;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
-@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
-@org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public class RuntimeStat implements org.apache.thrift.TBase<RuntimeStat, RuntimeStat._Fields>, java.io.Serializable, Cloneable, Comparable<RuntimeStat> {
-  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("RuntimeStat");
-
-  private static final org.apache.thrift.protocol.TField CREATE_TIME_FIELD_DESC = new org.apache.thrift.protocol.TField("createTime", org.apache.thrift.protocol.TType.I32, (short)1);
-  private static final org.apache.thrift.protocol.TField WEIGHT_FIELD_DESC = new org.apache.thrift.protocol.TField("weight", org.apache.thrift.protocol.TType.I32, (short)2);
-  private static final org.apache.thrift.protocol.TField PAYLOAD_FIELD_DESC = new org.apache.thrift.protocol.TField("payload", org.apache.thrift.protocol.TType.STRING, (short)3);
-
-  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
-  static {
-    schemes.put(StandardScheme.class, new RuntimeStatStandardSchemeFactory());
-    schemes.put(TupleScheme.class, new RuntimeStatTupleSchemeFactory());
-  }
-
-  private int createTime; // optional
-  private int weight; // required
-  private ByteBuffer payload; // 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 {
-    CREATE_TIME((short)1, "createTime"),
-    WEIGHT((short)2, "weight"),
-    PAYLOAD((short)3, "payload");
-
-    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: // CREATE_TIME
-          return CREATE_TIME;
-        case 2: // WEIGHT
-          return WEIGHT;
-        case 3: // PAYLOAD
-          return PAYLOAD;
-        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 __CREATETIME_ISSET_ID = 0;
-  private static final int __WEIGHT_ISSET_ID = 1;
-  private byte __isset_bitfield = 0;
-  private static final _Fields optionals[] = {_Fields.CREATE_TIME};
-  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.CREATE_TIME, new org.apache.thrift.meta_data.FieldMetaData("createTime", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
-        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
-    tmpMap.put(_Fields.WEIGHT, new org.apache.thrift.meta_data.FieldMetaData("weight", org.apache.thrift.TFieldRequirementType.REQUIRED, 
-        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
-    tmpMap.put(_Fields.PAYLOAD, new org.apache.thrift.meta_data.FieldMetaData("payload", org.apache.thrift.TFieldRequirementType.REQUIRED, 
-        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING        , true)));
-    metaDataMap = Collections.unmodifiableMap(tmpMap);
-    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(RuntimeStat.class, metaDataMap);
-  }
-
-  public RuntimeStat() {
-  }
-
-  public RuntimeStat(
-    int weight,
-    ByteBuffer payload)
-  {
-    this();
-    this.weight = weight;
-    setWeightIsSet(true);
-    this.payload = org.apache.thrift.TBaseHelper.copyBinary(payload);
-  }
-
-  /**
-   * Performs a deep copy on <i>other</i>.
-   */
-  public RuntimeStat(RuntimeStat other) {
-    __isset_bitfield = other.__isset_bitfield;
-    this.createTime = other.createTime;
-    this.weight = other.weight;
-    if (other.isSetPayload()) {
-      this.payload = org.apache.thrift.TBaseHelper.copyBinary(other.payload);
-    }
-  }
-
-  public RuntimeStat deepCopy() {
-    return new RuntimeStat(this);
-  }
-
-  @Override
-  public void clear() {
-    setCreateTimeIsSet(false);
-    this.createTime = 0;
-    setWeightIsSet(false);
-    this.weight = 0;
-    this.payload = null;
-  }
-
-  public int getCreateTime() {
-    return this.createTime;
-  }
-
-  public void setCreateTime(int createTime) {
-    this.createTime = createTime;
-    setCreateTimeIsSet(true);
-  }
-
-  public void unsetCreateTime() {
-    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __CREATETIME_ISSET_ID);
-  }
-
-  /** Returns true if field createTime is set (has been assigned a value) and false otherwise */
-  public boolean isSetCreateTime() {
-    return EncodingUtils.testBit(__isset_bitfield, __CREATETIME_ISSET_ID);
-  }
-
-  public void setCreateTimeIsSet(boolean value) {
-    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __CREATETIME_ISSET_ID, value);
-  }
-
-  public int getWeight() {
-    return this.weight;
-  }
-
-  public void setWeight(int weight) {
-    this.weight = weight;
-    setWeightIsSet(true);
-  }
-
-  public void unsetWeight() {
-    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __WEIGHT_ISSET_ID);
-  }
-
-  /** Returns true if field weight is set (has been assigned a value) and false otherwise */
-  public boolean isSetWeight() {
-    return EncodingUtils.testBit(__isset_bitfield, __WEIGHT_ISSET_ID);
-  }
-
-  public void setWeightIsSet(boolean value) {
-    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __WEIGHT_ISSET_ID, value);
-  }
-
-  public byte[] getPayload() {
-    setPayload(org.apache.thrift.TBaseHelper.rightSize(payload));
-    return payload == null ? null : payload.array();
-  }
-
-  public ByteBuffer bufferForPayload() {
-    return org.apache.thrift.TBaseHelper.copyBinary(payload);
-  }
-
-  public void setPayload(byte[] payload) {
-    this.payload = payload == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(payload, payload.length));
-  }
-
-  public void setPayload(ByteBuffer payload) {
-    this.payload = org.apache.thrift.TBaseHelper.copyBinary(payload);
-  }
-
-  public void unsetPayload() {
-    this.payload = null;
-  }
-
-  /** Returns true if field payload is set (has been assigned a value) and false otherwise */
-  public boolean isSetPayload() {
-    return this.payload != null;
-  }
-
-  public void setPayloadIsSet(boolean value) {
-    if (!value) {
-      this.payload = null;
-    }
-  }
-
-  public void setFieldValue(_Fields field, Object value) {
-    switch (field) {
-    case CREATE_TIME:
-      if (value == null) {
-        unsetCreateTime();
-      } else {
-        setCreateTime((Integer)value);
-      }
-      break;
-
-    case WEIGHT:
-      if (value == null) {
-        unsetWeight();
-      } else {
-        setWeight((Integer)value);
-      }
-      break;
-
-    case PAYLOAD:
-      if (value == null) {
-        unsetPayload();
-      } else {
-        setPayload((ByteBuffer)value);
-      }
-      break;
-
-    }
-  }
-
-  public Object getFieldValue(_Fields field) {
-    switch (field) {
-    case CREATE_TIME:
-      return getCreateTime();
-
-    case WEIGHT:
-      return getWeight();
-
-    case PAYLOAD:
-      return getPayload();
-
-    }
-    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 CREATE_TIME:
-      return isSetCreateTime();
-    case WEIGHT:
-      return isSetWeight();
-    case PAYLOAD:
-      return isSetPayload();
-    }
-    throw new IllegalStateException();
-  }
-
-  @Override
-  public boolean equals(Object that) {
-    if (that == null)
-      return false;
-    if (that instanceof RuntimeStat)
-      return this.equals((RuntimeStat)that);
-    return false;
-  }
-
-  public boolean equals(RuntimeStat that) {
-    if (that == null)
-      return false;
-
-    boolean this_present_createTime = true && this.isSetCreateTime();
-    boolean that_present_createTime = true && that.isSetCreateTime();
-    if (this_present_createTime || that_present_createTime) {
-      if (!(this_present_createTime && that_present_createTime))
-        return false;
-      if (this.createTime != that.createTime)
-        return false;
-    }
-
-    boolean this_present_weight = true;
-    boolean that_present_weight = true;
-    if (this_present_weight || that_present_weight) {
-      if (!(this_present_weight && that_present_weight))
-        return false;
-      if (this.weight != that.weight)
-        return false;
-    }
-
-    boolean this_present_payload = true && this.isSetPayload();
-    boolean that_present_payload = true && that.isSetPayload();
-    if (this_present_payload || that_present_payload) {
-      if (!(this_present_payload && that_present_payload))
-        return false;
-      if (!this.payload.equals(that.payload))
-        return false;
-    }
-
-    return true;
-  }
-
-  @Override
-  public int hashCode() {
-    List<Object> list = new ArrayList<Object>();
-
-    boolean present_createTime = true && (isSetCreateTime());
-    list.add(present_createTime);
-    if (present_createTime)
-      list.add(createTime);
-
-    boolean present_weight = true;
-    list.add(present_weight);
-    if (present_weight)
-      list.add(weight);
-
-    boolean present_payload = true && (isSetPayload());
-    list.add(present_payload);
-    if (present_payload)
-      list.add(payload);
-
-    return list.hashCode();
-  }
-
-  @Override
-  public int compareTo(RuntimeStat other) {
-    if (!getClass().equals(other.getClass())) {
-      return getClass().getName().compareTo(other.getClass().getName());
-    }
-
-    int lastComparison = 0;
-
-    lastComparison = Boolean.valueOf(isSetCreateTime()).compareTo(other.isSetCreateTime());
-    if (lastComparison != 0) {
-      return lastComparison;
-    }
-    if (isSetCreateTime()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.createTime, other.createTime);
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-    }
-    lastComparison = Boolean.valueOf(isSetWeight()).compareTo(other.isSetWeight());
-    if (lastComparison != 0) {
-      return lastComparison;
-    }
-    if (isSetWeight()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.weight, other.weight);
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-    }
-    lastComparison = Boolean.valueOf(isSetPayload()).compareTo(other.isSetPayload());
-    if (lastComparison != 0) {
-      return lastComparison;
-    }
-    if (isSetPayload()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.payload, other.payload);
-      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("RuntimeStat(");
-    boolean first = true;
-
-    if (isSetCreateTime()) {
-      sb.append("createTime:");
-      sb.append(this.createTime);
-      first = false;
-    }
-    if (!first) sb.append(", ");
-    sb.append("weight:");
-    sb.append(this.weight);
-    first = false;
-    if (!first) sb.append(", ");
-    sb.append("payload:");
-    if (this.payload == null) {
-      sb.append("null");
-    } else {
-      org.apache.thrift.TBaseHelper.toString(this.payload, sb);
-    }
-    first = false;
-    sb.append(")");
-    return sb.toString();
-  }
-
-  public void validate() throws org.apache.thrift.TException {
-    // check for required fields
-    if (!isSetWeight()) {
-      throw new org.apache.thrift.protocol.TProtocolException("Required field 'weight' is unset! Struct:" + toString());
-    }
-
-    if (!isSetPayload()) {
-      throw new org.apache.thrift.protocol.TProtocolException("Required field 'payload' 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 RuntimeStatStandardSchemeFactory implements SchemeFactory {
-    public RuntimeStatStandardScheme getScheme() {
-      return new RuntimeStatStandardScheme();
-    }
-  }
-
-  private static class RuntimeStatStandardScheme extends StandardScheme<RuntimeStat> {
-
-    public void read(org.apache.thrift.protocol.TProtocol iprot, RuntimeStat 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: // CREATE_TIME
-            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
-              struct.createTime = iprot.readI32();
-              struct.setCreateTimeIsSet(true);
-            } else { 
-              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-            }
-            break;
-          case 2: // WEIGHT
-            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
-              struct.weight = iprot.readI32();
-              struct.setWeightIsSet(true);
-            } else { 
-              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-            }
-            break;
-          case 3: // PAYLOAD
-            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
-              struct.payload = iprot.readBinary();
-              struct.setPayloadIsSet(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, RuntimeStat struct) throws org.apache.thrift.TException {
-      struct.validate();
-
-      oprot.writeStructBegin(STRUCT_DESC);
-      if (struct.isSetCreateTime()) {
-        oprot.writeFieldBegin(CREATE_TIME_FIELD_DESC);
-        oprot.writeI32(struct.createTime);
-        oprot.writeFieldEnd();
-      }
-      oprot.writeFieldBegin(WEIGHT_FIELD_DESC);
-      oprot.writeI32(struct.weight);
-      oprot.writeFieldEnd();
-      if (struct.payload != null) {
-        oprot.writeFieldBegin(PAYLOAD_FIELD_DESC);
-        oprot.writeBinary(struct.payload);
-        oprot.writeFieldEnd();
-      }
-      oprot.writeFieldStop();
-      oprot.writeStructEnd();
-    }
-
-  }
-
-  private static class RuntimeStatTupleSchemeFactory implements SchemeFactory {
-    public RuntimeStatTupleScheme getScheme() {
-      return new RuntimeStatTupleScheme();
-    }
-  }
-
-  private static class RuntimeStatTupleScheme extends TupleScheme<RuntimeStat> {
-
-    @Override
-    public void write(org.apache.thrift.protocol.TProtocol prot, RuntimeStat struct) throws org.apache.thrift.TException {
-      TTupleProtocol oprot = (TTupleProtocol) prot;
-      oprot.writeI32(struct.weight);
-      oprot.writeBinary(struct.payload);
-      BitSet optionals = new BitSet();
-      if (struct.isSetCreateTime()) {
-        optionals.set(0);
-      }
-      oprot.writeBitSet(optionals, 1);
-      if (struct.isSetCreateTime()) {
-        oprot.writeI32(struct.createTime);
-      }
-    }
-
-    @Override
-    public void read(org.apache.thrift.protocol.TProtocol prot, RuntimeStat struct) throws org.apache.thrift.TException {
-      TTupleProtocol iprot = (TTupleProtocol) prot;
-      struct.weight = iprot.readI32();
-      struct.setWeightIsSet(true);
-      struct.payload = iprot.readBinary();
-      struct.setPayloadIsSet(true);
-      BitSet incoming = iprot.readBitSet(1);
-      if (incoming.get(0)) {
-        struct.createTime = iprot.readI32();
-        struct.setCreateTimeIsSet(true);
-      }
-    }
-  }
-
-}
-


[2/6] hive git commit: Revert "HIVE-19171 : Persist runtime statistics in metastore (Zoltan Haindrich via Ashutosh Chauhan)"

Posted by pr...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/standalone-metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php b/standalone-metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
index 1c1d58e..9c94942 100644
--- a/standalone-metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
+++ b/standalone-metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
@@ -1528,17 +1528,6 @@ interface ThriftHiveMetastoreIf extends \FacebookServiceIf {
    * @return bool
    */
   public function heartbeat_lock_materialization_rebuild($dbName, $tableName, $txnId);
-  /**
-   * @param \metastore\RuntimeStat $stat
-   * @throws \metastore\MetaException
-   */
-  public function add_runtime_stats(\metastore\RuntimeStat $stat);
-  /**
-   * @param \metastore\GetRuntimeStatsRequest $rqst
-   * @return \metastore\RuntimeStat[]
-   * @throws \metastore\MetaException
-   */
-  public function get_runtime_stats(\metastore\GetRuntimeStatsRequest $rqst);
 }
 
 class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metastore\ThriftHiveMetastoreIf {
@@ -13018,111 +13007,6 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas
     throw new \Exception("heartbeat_lock_materialization_rebuild failed: unknown result");
   }
 
-  public function add_runtime_stats(\metastore\RuntimeStat $stat)
-  {
-    $this->send_add_runtime_stats($stat);
-    $this->recv_add_runtime_stats();
-  }
-
-  public function send_add_runtime_stats(\metastore\RuntimeStat $stat)
-  {
-    $args = new \metastore\ThriftHiveMetastore_add_runtime_stats_args();
-    $args->stat = $stat;
-    $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
-    if ($bin_accel)
-    {
-      thrift_protocol_write_binary($this->output_, 'add_runtime_stats', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
-    }
-    else
-    {
-      $this->output_->writeMessageBegin('add_runtime_stats', TMessageType::CALL, $this->seqid_);
-      $args->write($this->output_);
-      $this->output_->writeMessageEnd();
-      $this->output_->getTransport()->flush();
-    }
-  }
-
-  public function recv_add_runtime_stats()
-  {
-    $bin_accel = ($this->input_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_read_binary');
-    if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_add_runtime_stats_result', $this->input_->isStrictRead());
-    else
-    {
-      $rseqid = 0;
-      $fname = null;
-      $mtype = 0;
-
-      $this->input_->readMessageBegin($fname, $mtype, $rseqid);
-      if ($mtype == TMessageType::EXCEPTION) {
-        $x = new TApplicationException();
-        $x->read($this->input_);
-        $this->input_->readMessageEnd();
-        throw $x;
-      }
-      $result = new \metastore\ThriftHiveMetastore_add_runtime_stats_result();
-      $result->read($this->input_);
-      $this->input_->readMessageEnd();
-    }
-    if ($result->o1 !== null) {
-      throw $result->o1;
-    }
-    return;
-  }
-
-  public function get_runtime_stats(\metastore\GetRuntimeStatsRequest $rqst)
-  {
-    $this->send_get_runtime_stats($rqst);
-    return $this->recv_get_runtime_stats();
-  }
-
-  public function send_get_runtime_stats(\metastore\GetRuntimeStatsRequest $rqst)
-  {
-    $args = new \metastore\ThriftHiveMetastore_get_runtime_stats_args();
-    $args->rqst = $rqst;
-    $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
-    if ($bin_accel)
-    {
-      thrift_protocol_write_binary($this->output_, 'get_runtime_stats', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
-    }
-    else
-    {
-      $this->output_->writeMessageBegin('get_runtime_stats', TMessageType::CALL, $this->seqid_);
-      $args->write($this->output_);
-      $this->output_->writeMessageEnd();
-      $this->output_->getTransport()->flush();
-    }
-  }
-
-  public function recv_get_runtime_stats()
-  {
-    $bin_accel = ($this->input_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_read_binary');
-    if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_get_runtime_stats_result', $this->input_->isStrictRead());
-    else
-    {
-      $rseqid = 0;
-      $fname = null;
-      $mtype = 0;
-
-      $this->input_->readMessageBegin($fname, $mtype, $rseqid);
-      if ($mtype == TMessageType::EXCEPTION) {
-        $x = new TApplicationException();
-        $x->read($this->input_);
-        $this->input_->readMessageEnd();
-        throw $x;
-      }
-      $result = new \metastore\ThriftHiveMetastore_get_runtime_stats_result();
-      $result->read($this->input_);
-      $this->input_->readMessageEnd();
-    }
-    if ($result->success !== null) {
-      return $result->success;
-    }
-    if ($result->o1 !== null) {
-      throw $result->o1;
-    }
-    throw new \Exception("get_runtime_stats failed: unknown result");
-  }
-
 }
 
 // HELPER FUNCTIONS AND STRUCTURES
@@ -58614,369 +58498,4 @@ class ThriftHiveMetastore_heartbeat_lock_materialization_rebuild_result {
 
 }
 
-class ThriftHiveMetastore_add_runtime_stats_args {
-  static $_TSPEC;
-
-  /**
-   * @var \metastore\RuntimeStat
-   */
-  public $stat = null;
-
-  public function __construct($vals=null) {
-    if (!isset(self::$_TSPEC)) {
-      self::$_TSPEC = array(
-        1 => array(
-          'var' => 'stat',
-          'type' => TType::STRUCT,
-          'class' => '\metastore\RuntimeStat',
-          ),
-        );
-    }
-    if (is_array($vals)) {
-      if (isset($vals['stat'])) {
-        $this->stat = $vals['stat'];
-      }
-    }
-  }
-
-  public function getName() {
-    return 'ThriftHiveMetastore_add_runtime_stats_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->stat = new \metastore\RuntimeStat();
-            $xfer += $this->stat->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_add_runtime_stats_args');
-    if ($this->stat !== null) {
-      if (!is_object($this->stat)) {
-        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
-      }
-      $xfer += $output->writeFieldBegin('stat', TType::STRUCT, 1);
-      $xfer += $this->stat->write($output);
-      $xfer += $output->writeFieldEnd();
-    }
-    $xfer += $output->writeFieldStop();
-    $xfer += $output->writeStructEnd();
-    return $xfer;
-  }
-
-}
-
-class ThriftHiveMetastore_add_runtime_stats_result {
-  static $_TSPEC;
-
-  /**
-   * @var \metastore\MetaException
-   */
-  public $o1 = null;
-
-  public function __construct($vals=null) {
-    if (!isset(self::$_TSPEC)) {
-      self::$_TSPEC = array(
-        1 => array(
-          'var' => 'o1',
-          'type' => TType::STRUCT,
-          'class' => '\metastore\MetaException',
-          ),
-        );
-    }
-    if (is_array($vals)) {
-      if (isset($vals['o1'])) {
-        $this->o1 = $vals['o1'];
-      }
-    }
-  }
-
-  public function getName() {
-    return 'ThriftHiveMetastore_add_runtime_stats_result';
-  }
-
-  public function read($input)
-  {
-    $xfer = 0;
-    $fname = null;
-    $ftype = 0;
-    $fid = 0;
-    $xfer += $input->readStructBegin($fname);
-    while (true)
-    {
-      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
-      if ($ftype == TType::STOP) {
-        break;
-      }
-      switch ($fid)
-      {
-        case 1:
-          if ($ftype == TType::STRUCT) {
-            $this->o1 = new \metastore\MetaException();
-            $xfer += $this->o1->read($input);
-          } else {
-            $xfer += $input->skip($ftype);
-          }
-          break;
-        default:
-          $xfer += $input->skip($ftype);
-          break;
-      }
-      $xfer += $input->readFieldEnd();
-    }
-    $xfer += $input->readStructEnd();
-    return $xfer;
-  }
-
-  public function write($output) {
-    $xfer = 0;
-    $xfer += $output->writeStructBegin('ThriftHiveMetastore_add_runtime_stats_result');
-    if ($this->o1 !== null) {
-      $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1);
-      $xfer += $this->o1->write($output);
-      $xfer += $output->writeFieldEnd();
-    }
-    $xfer += $output->writeFieldStop();
-    $xfer += $output->writeStructEnd();
-    return $xfer;
-  }
-
-}
-
-class ThriftHiveMetastore_get_runtime_stats_args {
-  static $_TSPEC;
-
-  /**
-   * @var \metastore\GetRuntimeStatsRequest
-   */
-  public $rqst = null;
-
-  public function __construct($vals=null) {
-    if (!isset(self::$_TSPEC)) {
-      self::$_TSPEC = array(
-        1 => array(
-          'var' => 'rqst',
-          'type' => TType::STRUCT,
-          'class' => '\metastore\GetRuntimeStatsRequest',
-          ),
-        );
-    }
-    if (is_array($vals)) {
-      if (isset($vals['rqst'])) {
-        $this->rqst = $vals['rqst'];
-      }
-    }
-  }
-
-  public function getName() {
-    return 'ThriftHiveMetastore_get_runtime_stats_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->rqst = new \metastore\GetRuntimeStatsRequest();
-            $xfer += $this->rqst->read($input);
-          } else {
-            $xfer += $input->skip($ftype);
-          }
-          break;
-        default:
-          $xfer += $input->skip($ftype);
-          break;
-      }
-      $xfer += $input->readFieldEnd();
-    }
-    $xfer += $input->readStructEnd();
-    return $xfer;
-  }
-
-  public function write($output) {
-    $xfer = 0;
-    $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_runtime_stats_args');
-    if ($this->rqst !== null) {
-      if (!is_object($this->rqst)) {
-        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
-      }
-      $xfer += $output->writeFieldBegin('rqst', TType::STRUCT, 1);
-      $xfer += $this->rqst->write($output);
-      $xfer += $output->writeFieldEnd();
-    }
-    $xfer += $output->writeFieldStop();
-    $xfer += $output->writeStructEnd();
-    return $xfer;
-  }
-
-}
-
-class ThriftHiveMetastore_get_runtime_stats_result {
-  static $_TSPEC;
-
-  /**
-   * @var \metastore\RuntimeStat[]
-   */
-  public $success = null;
-  /**
-   * @var \metastore\MetaException
-   */
-  public $o1 = null;
-
-  public function __construct($vals=null) {
-    if (!isset(self::$_TSPEC)) {
-      self::$_TSPEC = array(
-        0 => array(
-          'var' => 'success',
-          'type' => TType::LST,
-          'etype' => TType::STRUCT,
-          'elem' => array(
-            'type' => TType::STRUCT,
-            'class' => '\metastore\RuntimeStat',
-            ),
-          ),
-        1 => array(
-          'var' => 'o1',
-          'type' => TType::STRUCT,
-          'class' => '\metastore\MetaException',
-          ),
-        );
-    }
-    if (is_array($vals)) {
-      if (isset($vals['success'])) {
-        $this->success = $vals['success'];
-      }
-      if (isset($vals['o1'])) {
-        $this->o1 = $vals['o1'];
-      }
-    }
-  }
-
-  public function getName() {
-    return 'ThriftHiveMetastore_get_runtime_stats_result';
-  }
-
-  public function read($input)
-  {
-    $xfer = 0;
-    $fname = null;
-    $ftype = 0;
-    $fid = 0;
-    $xfer += $input->readStructBegin($fname);
-    while (true)
-    {
-      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
-      if ($ftype == TType::STOP) {
-        break;
-      }
-      switch ($fid)
-      {
-        case 0:
-          if ($ftype == TType::LST) {
-            $this->success = array();
-            $_size1324 = 0;
-            $_etype1327 = 0;
-            $xfer += $input->readListBegin($_etype1327, $_size1324);
-            for ($_i1328 = 0; $_i1328 < $_size1324; ++$_i1328)
-            {
-              $elem1329 = null;
-              $elem1329 = new \metastore\RuntimeStat();
-              $xfer += $elem1329->read($input);
-              $this->success []= $elem1329;
-            }
-            $xfer += $input->readListEnd();
-          } else {
-            $xfer += $input->skip($ftype);
-          }
-          break;
-        case 1:
-          if ($ftype == TType::STRUCT) {
-            $this->o1 = new \metastore\MetaException();
-            $xfer += $this->o1->read($input);
-          } else {
-            $xfer += $input->skip($ftype);
-          }
-          break;
-        default:
-          $xfer += $input->skip($ftype);
-          break;
-      }
-      $xfer += $input->readFieldEnd();
-    }
-    $xfer += $input->readStructEnd();
-    return $xfer;
-  }
-
-  public function write($output) {
-    $xfer = 0;
-    $xfer += $output->writeStructBegin('ThriftHiveMetastore_get_runtime_stats_result');
-    if ($this->success !== null) {
-      if (!is_array($this->success)) {
-        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
-      }
-      $xfer += $output->writeFieldBegin('success', TType::LST, 0);
-      {
-        $output->writeListBegin(TType::STRUCT, count($this->success));
-        {
-          foreach ($this->success as $iter1330)
-          {
-            $xfer += $iter1330->write($output);
-          }
-        }
-        $output->writeListEnd();
-      }
-      $xfer += $output->writeFieldEnd();
-    }
-    if ($this->o1 !== null) {
-      $xfer += $output->writeFieldBegin('o1', TType::STRUCT, 1);
-      $xfer += $this->o1->write($output);
-      $xfer += $output->writeFieldEnd();
-    }
-    $xfer += $output->writeFieldStop();
-    $xfer += $output->writeStructEnd();
-    return $xfer;
-  }
-
-}
-
 

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php b/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php
index c9ebfaf..1625788 100644
--- a/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php
+++ b/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php
@@ -29722,177 +29722,6 @@ class GetSerdeRequest {
 
 }
 
-class RuntimeStat {
-  static $_TSPEC;
-
-  /**
-   * @var int
-   */
-  public $createTime = null;
-  /**
-   * @var int
-   */
-  public $weight = null;
-  /**
-   * @var string
-   */
-  public $payload = null;
-
-  public function __construct($vals=null) {
-    if (!isset(self::$_TSPEC)) {
-      self::$_TSPEC = array(
-        1 => array(
-          'var' => 'createTime',
-          'type' => TType::I32,
-          ),
-        2 => array(
-          'var' => 'weight',
-          'type' => TType::I32,
-          ),
-        3 => array(
-          'var' => 'payload',
-          'type' => TType::STRING,
-          ),
-        );
-    }
-    if (is_array($vals)) {
-      if (isset($vals['createTime'])) {
-        $this->createTime = $vals['createTime'];
-      }
-      if (isset($vals['weight'])) {
-        $this->weight = $vals['weight'];
-      }
-      if (isset($vals['payload'])) {
-        $this->payload = $vals['payload'];
-      }
-    }
-  }
-
-  public function getName() {
-    return 'RuntimeStat';
-  }
-
-  public function read($input)
-  {
-    $xfer = 0;
-    $fname = null;
-    $ftype = 0;
-    $fid = 0;
-    $xfer += $input->readStructBegin($fname);
-    while (true)
-    {
-      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
-      if ($ftype == TType::STOP) {
-        break;
-      }
-      switch ($fid)
-      {
-        case 1:
-          if ($ftype == TType::I32) {
-            $xfer += $input->readI32($this->createTime);
-          } else {
-            $xfer += $input->skip($ftype);
-          }
-          break;
-        case 2:
-          if ($ftype == TType::I32) {
-            $xfer += $input->readI32($this->weight);
-          } else {
-            $xfer += $input->skip($ftype);
-          }
-          break;
-        case 3:
-          if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->payload);
-          } 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('RuntimeStat');
-    if ($this->createTime !== null) {
-      $xfer += $output->writeFieldBegin('createTime', TType::I32, 1);
-      $xfer += $output->writeI32($this->createTime);
-      $xfer += $output->writeFieldEnd();
-    }
-    if ($this->weight !== null) {
-      $xfer += $output->writeFieldBegin('weight', TType::I32, 2);
-      $xfer += $output->writeI32($this->weight);
-      $xfer += $output->writeFieldEnd();
-    }
-    if ($this->payload !== null) {
-      $xfer += $output->writeFieldBegin('payload', TType::STRING, 3);
-      $xfer += $output->writeString($this->payload);
-      $xfer += $output->writeFieldEnd();
-    }
-    $xfer += $output->writeFieldStop();
-    $xfer += $output->writeStructEnd();
-    return $xfer;
-  }
-
-}
-
-class GetRuntimeStatsRequest {
-  static $_TSPEC;
-
-
-  public function __construct() {
-    if (!isset(self::$_TSPEC)) {
-      self::$_TSPEC = array(
-        );
-    }
-  }
-
-  public function getName() {
-    return 'GetRuntimeStatsRequest';
-  }
-
-  public function read($input)
-  {
-    $xfer = 0;
-    $fname = null;
-    $ftype = 0;
-    $fid = 0;
-    $xfer += $input->readStructBegin($fname);
-    while (true)
-    {
-      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
-      if ($ftype == TType::STOP) {
-        break;
-      }
-      switch ($fid)
-      {
-        default:
-          $xfer += $input->skip($ftype);
-          break;
-      }
-      $xfer += $input->readFieldEnd();
-    }
-    $xfer += $input->readStructEnd();
-    return $xfer;
-  }
-
-  public function write($output) {
-    $xfer = 0;
-    $xfer += $output->writeStructBegin('GetRuntimeStatsRequest');
-    $xfer += $output->writeFieldStop();
-    $xfer += $output->writeStructEnd();
-    return $xfer;
-  }
-
-}
-
 class MetaException extends TException {
   static $_TSPEC;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote b/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
index a231e9c..079c7fc 100755
--- a/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
+++ b/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
@@ -226,8 +226,6 @@ if len(sys.argv) <= 1 or sys.argv[1] == '--help':
   print('  SerDeInfo get_serde(GetSerdeRequest rqst)')
   print('  LockResponse get_lock_materialization_rebuild(string dbName, string tableName, i64 txnId)')
   print('  bool heartbeat_lock_materialization_rebuild(string dbName, string tableName, i64 txnId)')
-  print('  void add_runtime_stats(RuntimeStat stat)')
-  print('   get_runtime_stats(GetRuntimeStatsRequest rqst)')
   print('  string getName()')
   print('  string getVersion()')
   print('  fb_status getStatus()')
@@ -1509,18 +1507,6 @@ elif cmd == 'heartbeat_lock_materialization_rebuild':
     sys.exit(1)
   pp.pprint(client.heartbeat_lock_materialization_rebuild(args[0],args[1],eval(args[2]),))
 
-elif cmd == 'add_runtime_stats':
-  if len(args) != 1:
-    print('add_runtime_stats requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.add_runtime_stats(eval(args[0]),))
-
-elif cmd == 'get_runtime_stats':
-  if len(args) != 1:
-    print('get_runtime_stats requires 1 args')
-    sys.exit(1)
-  pp.pprint(client.get_runtime_stats(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/f0199500/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py b/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
index d94951b..d241414 100644
--- a/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
+++ b/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
@@ -1566,20 +1566,6 @@ class Iface(fb303.FacebookService.Iface):
     """
     pass
 
-  def add_runtime_stats(self, stat):
-    """
-    Parameters:
-     - stat
-    """
-    pass
-
-  def get_runtime_stats(self, rqst):
-    """
-    Parameters:
-     - rqst
-    """
-    pass
-
 
 class Client(fb303.FacebookService.Client, Iface):
   """
@@ -8813,70 +8799,6 @@ class Client(fb303.FacebookService.Client, Iface):
       return result.success
     raise TApplicationException(TApplicationException.MISSING_RESULT, "heartbeat_lock_materialization_rebuild failed: unknown result")
 
-  def add_runtime_stats(self, stat):
-    """
-    Parameters:
-     - stat
-    """
-    self.send_add_runtime_stats(stat)
-    self.recv_add_runtime_stats()
-
-  def send_add_runtime_stats(self, stat):
-    self._oprot.writeMessageBegin('add_runtime_stats', TMessageType.CALL, self._seqid)
-    args = add_runtime_stats_args()
-    args.stat = stat
-    args.write(self._oprot)
-    self._oprot.writeMessageEnd()
-    self._oprot.trans.flush()
-
-  def recv_add_runtime_stats(self):
-    iprot = self._iprot
-    (fname, mtype, rseqid) = iprot.readMessageBegin()
-    if mtype == TMessageType.EXCEPTION:
-      x = TApplicationException()
-      x.read(iprot)
-      iprot.readMessageEnd()
-      raise x
-    result = add_runtime_stats_result()
-    result.read(iprot)
-    iprot.readMessageEnd()
-    if result.o1 is not None:
-      raise result.o1
-    return
-
-  def get_runtime_stats(self, rqst):
-    """
-    Parameters:
-     - rqst
-    """
-    self.send_get_runtime_stats(rqst)
-    return self.recv_get_runtime_stats()
-
-  def send_get_runtime_stats(self, rqst):
-    self._oprot.writeMessageBegin('get_runtime_stats', TMessageType.CALL, self._seqid)
-    args = get_runtime_stats_args()
-    args.rqst = rqst
-    args.write(self._oprot)
-    self._oprot.writeMessageEnd()
-    self._oprot.trans.flush()
-
-  def recv_get_runtime_stats(self):
-    iprot = self._iprot
-    (fname, mtype, rseqid) = iprot.readMessageBegin()
-    if mtype == TMessageType.EXCEPTION:
-      x = TApplicationException()
-      x.read(iprot)
-      iprot.readMessageEnd()
-      raise x
-    result = get_runtime_stats_result()
-    result.read(iprot)
-    iprot.readMessageEnd()
-    if result.success is not None:
-      return result.success
-    if result.o1 is not None:
-      raise result.o1
-    raise TApplicationException(TApplicationException.MISSING_RESULT, "get_runtime_stats failed: unknown result")
-
 
 class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
   def __init__(self, handler):
@@ -9083,8 +9005,6 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
     self._processMap["get_serde"] = Processor.process_get_serde
     self._processMap["get_lock_materialization_rebuild"] = Processor.process_get_lock_materialization_rebuild
     self._processMap["heartbeat_lock_materialization_rebuild"] = Processor.process_heartbeat_lock_materialization_rebuild
-    self._processMap["add_runtime_stats"] = Processor.process_add_runtime_stats
-    self._processMap["get_runtime_stats"] = Processor.process_get_runtime_stats
 
   def process(self, iprot, oprot):
     (name, type, seqid) = iprot.readMessageBegin()
@@ -14097,50 +14017,6 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
     oprot.writeMessageEnd()
     oprot.trans.flush()
 
-  def process_add_runtime_stats(self, seqid, iprot, oprot):
-    args = add_runtime_stats_args()
-    args.read(iprot)
-    iprot.readMessageEnd()
-    result = add_runtime_stats_result()
-    try:
-      self._handler.add_runtime_stats(args.stat)
-      msg_type = TMessageType.REPLY
-    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
-      raise
-    except MetaException as o1:
-      msg_type = TMessageType.REPLY
-      result.o1 = o1
-    except Exception as ex:
-      msg_type = TMessageType.EXCEPTION
-      logging.exception(ex)
-      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
-    oprot.writeMessageBegin("add_runtime_stats", msg_type, seqid)
-    result.write(oprot)
-    oprot.writeMessageEnd()
-    oprot.trans.flush()
-
-  def process_get_runtime_stats(self, seqid, iprot, oprot):
-    args = get_runtime_stats_args()
-    args.read(iprot)
-    iprot.readMessageEnd()
-    result = get_runtime_stats_result()
-    try:
-      result.success = self._handler.get_runtime_stats(args.rqst)
-      msg_type = TMessageType.REPLY
-    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
-      raise
-    except MetaException as o1:
-      msg_type = TMessageType.REPLY
-      result.o1 = o1
-    except Exception as ex:
-      msg_type = TMessageType.EXCEPTION
-      logging.exception(ex)
-      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
-    oprot.writeMessageBegin("get_runtime_stats", msg_type, seqid)
-    result.write(oprot)
-    oprot.writeMessageEnd()
-    oprot.trans.flush()
-
 
 # HELPER FUNCTIONS AND STRUCTURES
 
@@ -47844,288 +47720,3 @@ class heartbeat_lock_materialization_rebuild_result:
 
   def __ne__(self, other):
     return not (self == other)
-
-class add_runtime_stats_args:
-  """
-  Attributes:
-   - stat
-  """
-
-  thrift_spec = (
-    None, # 0
-    (1, TType.STRUCT, 'stat', (RuntimeStat, RuntimeStat.thrift_spec), None, ), # 1
-  )
-
-  def __init__(self, stat=None,):
-    self.stat = stat
-
-  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.stat = RuntimeStat()
-          self.stat.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('add_runtime_stats_args')
-    if self.stat is not None:
-      oprot.writeFieldBegin('stat', TType.STRUCT, 1)
-      self.stat.write(oprot)
-      oprot.writeFieldEnd()
-    oprot.writeFieldStop()
-    oprot.writeStructEnd()
-
-  def validate(self):
-    return
-
-
-  def __hash__(self):
-    value = 17
-    value = (value * 31) ^ hash(self.stat)
-    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 add_runtime_stats_result:
-  """
-  Attributes:
-   - o1
-  """
-
-  thrift_spec = (
-    None, # 0
-    (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1
-  )
-
-  def __init__(self, o1=None,):
-    self.o1 = o1
-
-  def read(self, iprot):
-    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
-      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
-      return
-    iprot.readStructBegin()
-    while True:
-      (fname, ftype, fid) = iprot.readFieldBegin()
-      if ftype == TType.STOP:
-        break
-      if fid == 1:
-        if ftype == TType.STRUCT:
-          self.o1 = MetaException()
-          self.o1.read(iprot)
-        else:
-          iprot.skip(ftype)
-      else:
-        iprot.skip(ftype)
-      iprot.readFieldEnd()
-    iprot.readStructEnd()
-
-  def write(self, oprot):
-    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
-      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
-      return
-    oprot.writeStructBegin('add_runtime_stats_result')
-    if self.o1 is not None:
-      oprot.writeFieldBegin('o1', TType.STRUCT, 1)
-      self.o1.write(oprot)
-      oprot.writeFieldEnd()
-    oprot.writeFieldStop()
-    oprot.writeStructEnd()
-
-  def validate(self):
-    return
-
-
-  def __hash__(self):
-    value = 17
-    value = (value * 31) ^ hash(self.o1)
-    return value
-
-  def __repr__(self):
-    L = ['%s=%r' % (key, value)
-      for key, value in self.__dict__.iteritems()]
-    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
-
-  def __eq__(self, other):
-    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
-
-  def __ne__(self, other):
-    return not (self == other)
-
-class get_runtime_stats_args:
-  """
-  Attributes:
-   - rqst
-  """
-
-  thrift_spec = (
-    None, # 0
-    (1, TType.STRUCT, 'rqst', (GetRuntimeStatsRequest, GetRuntimeStatsRequest.thrift_spec), None, ), # 1
-  )
-
-  def __init__(self, rqst=None,):
-    self.rqst = rqst
-
-  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.rqst = GetRuntimeStatsRequest()
-          self.rqst.read(iprot)
-        else:
-          iprot.skip(ftype)
-      else:
-        iprot.skip(ftype)
-      iprot.readFieldEnd()
-    iprot.readStructEnd()
-
-  def write(self, oprot):
-    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
-      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
-      return
-    oprot.writeStructBegin('get_runtime_stats_args')
-    if self.rqst is not None:
-      oprot.writeFieldBegin('rqst', TType.STRUCT, 1)
-      self.rqst.write(oprot)
-      oprot.writeFieldEnd()
-    oprot.writeFieldStop()
-    oprot.writeStructEnd()
-
-  def validate(self):
-    return
-
-
-  def __hash__(self):
-    value = 17
-    value = (value * 31) ^ hash(self.rqst)
-    return value
-
-  def __repr__(self):
-    L = ['%s=%r' % (key, value)
-      for key, value in self.__dict__.iteritems()]
-    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
-
-  def __eq__(self, other):
-    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
-
-  def __ne__(self, other):
-    return not (self == other)
-
-class get_runtime_stats_result:
-  """
-  Attributes:
-   - success
-   - o1
-  """
-
-  thrift_spec = (
-    (0, TType.LIST, 'success', (TType.STRUCT,(RuntimeStat, RuntimeStat.thrift_spec)), None, ), # 0
-    (1, TType.STRUCT, 'o1', (MetaException, MetaException.thrift_spec), None, ), # 1
-  )
-
-  def __init__(self, success=None, o1=None,):
-    self.success = success
-    self.o1 = o1
-
-  def read(self, iprot):
-    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
-      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
-      return
-    iprot.readStructBegin()
-    while True:
-      (fname, ftype, fid) = iprot.readFieldBegin()
-      if ftype == TType.STOP:
-        break
-      if fid == 0:
-        if ftype == TType.LIST:
-          self.success = []
-          (_etype1323, _size1320) = iprot.readListBegin()
-          for _i1324 in xrange(_size1320):
-            _elem1325 = RuntimeStat()
-            _elem1325.read(iprot)
-            self.success.append(_elem1325)
-          iprot.readListEnd()
-        else:
-          iprot.skip(ftype)
-      elif fid == 1:
-        if ftype == TType.STRUCT:
-          self.o1 = MetaException()
-          self.o1.read(iprot)
-        else:
-          iprot.skip(ftype)
-      else:
-        iprot.skip(ftype)
-      iprot.readFieldEnd()
-    iprot.readStructEnd()
-
-  def write(self, oprot):
-    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
-      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
-      return
-    oprot.writeStructBegin('get_runtime_stats_result')
-    if self.success is not None:
-      oprot.writeFieldBegin('success', TType.LIST, 0)
-      oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1326 in self.success:
-        iter1326.write(oprot)
-      oprot.writeListEnd()
-      oprot.writeFieldEnd()
-    if self.o1 is not None:
-      oprot.writeFieldBegin('o1', TType.STRUCT, 1)
-      self.o1.write(oprot)
-      oprot.writeFieldEnd()
-    oprot.writeFieldStop()
-    oprot.writeStructEnd()
-
-  def validate(self):
-    return
-
-
-  def __hash__(self):
-    value = 17
-    value = (value * 31) ^ hash(self.success)
-    value = (value * 31) ^ hash(self.o1)
-    return value
-
-  def __repr__(self):
-    L = ['%s=%r' % (key, value)
-      for key, value in self.__dict__.iteritems()]
-    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
-
-  def __eq__(self, other):
-    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
-
-  def __ne__(self, other):
-    return not (self == other)

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py b/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
index b1e577a..1d09cb8 100644
--- a/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
+++ b/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
@@ -21236,147 +21236,6 @@ class GetSerdeRequest:
   def __ne__(self, other):
     return not (self == other)
 
-class RuntimeStat:
-  """
-  Attributes:
-   - createTime
-   - weight
-   - payload
-  """
-
-  thrift_spec = (
-    None, # 0
-    (1, TType.I32, 'createTime', None, None, ), # 1
-    (2, TType.I32, 'weight', None, None, ), # 2
-    (3, TType.STRING, 'payload', None, None, ), # 3
-  )
-
-  def __init__(self, createTime=None, weight=None, payload=None,):
-    self.createTime = createTime
-    self.weight = weight
-    self.payload = payload
-
-  def read(self, iprot):
-    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
-      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
-      return
-    iprot.readStructBegin()
-    while True:
-      (fname, ftype, fid) = iprot.readFieldBegin()
-      if ftype == TType.STOP:
-        break
-      if fid == 1:
-        if ftype == TType.I32:
-          self.createTime = iprot.readI32()
-        else:
-          iprot.skip(ftype)
-      elif fid == 2:
-        if ftype == TType.I32:
-          self.weight = iprot.readI32()
-        else:
-          iprot.skip(ftype)
-      elif fid == 3:
-        if ftype == TType.STRING:
-          self.payload = iprot.readString()
-        else:
-          iprot.skip(ftype)
-      else:
-        iprot.skip(ftype)
-      iprot.readFieldEnd()
-    iprot.readStructEnd()
-
-  def write(self, oprot):
-    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
-      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
-      return
-    oprot.writeStructBegin('RuntimeStat')
-    if self.createTime is not None:
-      oprot.writeFieldBegin('createTime', TType.I32, 1)
-      oprot.writeI32(self.createTime)
-      oprot.writeFieldEnd()
-    if self.weight is not None:
-      oprot.writeFieldBegin('weight', TType.I32, 2)
-      oprot.writeI32(self.weight)
-      oprot.writeFieldEnd()
-    if self.payload is not None:
-      oprot.writeFieldBegin('payload', TType.STRING, 3)
-      oprot.writeString(self.payload)
-      oprot.writeFieldEnd()
-    oprot.writeFieldStop()
-    oprot.writeStructEnd()
-
-  def validate(self):
-    if self.weight is None:
-      raise TProtocol.TProtocolException(message='Required field weight is unset!')
-    if self.payload is None:
-      raise TProtocol.TProtocolException(message='Required field payload is unset!')
-    return
-
-
-  def __hash__(self):
-    value = 17
-    value = (value * 31) ^ hash(self.createTime)
-    value = (value * 31) ^ hash(self.weight)
-    value = (value * 31) ^ hash(self.payload)
-    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 GetRuntimeStatsRequest:
-
-  thrift_spec = (
-  )
-
-  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
-      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('GetRuntimeStatsRequest')
-    oprot.writeFieldStop()
-    oprot.writeStructEnd()
-
-  def validate(self):
-    return
-
-
-  def __hash__(self):
-    value = 17
-    return value
-
-  def __repr__(self):
-    L = ['%s=%r' % (key, value)
-      for key, value in self.__dict__.iteritems()]
-    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
-
-  def __eq__(self, other):
-    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
-
-  def __ne__(self, other):
-    return not (self == other)
-
 class MetaException(TException):
   """
   Attributes:

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/standalone-metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb b/standalone-metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
index 2687ce5..47e8d0f 100644
--- a/standalone-metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
+++ b/standalone-metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
@@ -4825,43 +4825,6 @@ class GetSerdeRequest
   ::Thrift::Struct.generate_accessors self
 end
 
-class RuntimeStat
-  include ::Thrift::Struct, ::Thrift::Struct_Union
-  CREATETIME = 1
-  WEIGHT = 2
-  PAYLOAD = 3
-
-  FIELDS = {
-    CREATETIME => {:type => ::Thrift::Types::I32, :name => 'createTime', :optional => true},
-    WEIGHT => {:type => ::Thrift::Types::I32, :name => 'weight'},
-    PAYLOAD => {:type => ::Thrift::Types::STRING, :name => 'payload', :binary => true}
-  }
-
-  def struct_fields; FIELDS; end
-
-  def validate
-    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field weight is unset!') unless @weight
-    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field payload is unset!') unless @payload
-  end
-
-  ::Thrift::Struct.generate_accessors self
-end
-
-class GetRuntimeStatsRequest
-  include ::Thrift::Struct, ::Thrift::Struct_Union
-
-  FIELDS = {
-
-  }
-
-  def struct_fields; FIELDS; end
-
-  def validate
-  end
-
-  ::Thrift::Struct.generate_accessors self
-end
-
 class MetaException < ::Thrift::Exception
   include ::Thrift::Struct, ::Thrift::Struct_Union
   def initialize(message=nil)

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/standalone-metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb b/standalone-metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
index 4de8bd3..58ebd29 100644
--- a/standalone-metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
+++ b/standalone-metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
@@ -3378,37 +3378,6 @@ module ThriftHiveMetastore
       raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'heartbeat_lock_materialization_rebuild failed: unknown result')
     end
 
-    def add_runtime_stats(stat)
-      send_add_runtime_stats(stat)
-      recv_add_runtime_stats()
-    end
-
-    def send_add_runtime_stats(stat)
-      send_message('add_runtime_stats', Add_runtime_stats_args, :stat => stat)
-    end
-
-    def recv_add_runtime_stats()
-      result = receive_message(Add_runtime_stats_result)
-      raise result.o1 unless result.o1.nil?
-      return
-    end
-
-    def get_runtime_stats(rqst)
-      send_get_runtime_stats(rqst)
-      return recv_get_runtime_stats()
-    end
-
-    def send_get_runtime_stats(rqst)
-      send_message('get_runtime_stats', Get_runtime_stats_args, :rqst => rqst)
-    end
-
-    def recv_get_runtime_stats()
-      result = receive_message(Get_runtime_stats_result)
-      return result.success unless result.success.nil?
-      raise result.o1 unless result.o1.nil?
-      raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'get_runtime_stats failed: unknown result')
-    end
-
   end
 
   class Processor < ::FacebookService::Processor 
@@ -5950,28 +5919,6 @@ module ThriftHiveMetastore
       write_result(result, oprot, 'heartbeat_lock_materialization_rebuild', seqid)
     end
 
-    def process_add_runtime_stats(seqid, iprot, oprot)
-      args = read_args(iprot, Add_runtime_stats_args)
-      result = Add_runtime_stats_result.new()
-      begin
-        @handler.add_runtime_stats(args.stat)
-      rescue ::MetaException => o1
-        result.o1 = o1
-      end
-      write_result(result, oprot, 'add_runtime_stats', seqid)
-    end
-
-    def process_get_runtime_stats(seqid, iprot, oprot)
-      args = read_args(iprot, Get_runtime_stats_args)
-      result = Get_runtime_stats_result.new()
-      begin
-        result.success = @handler.get_runtime_stats(args.rqst)
-      rescue ::MetaException => o1
-        result.o1 = o1
-      end
-      write_result(result, oprot, 'get_runtime_stats', seqid)
-    end
-
   end
 
   # HELPER FUNCTIONS AND STRUCTURES
@@ -13470,71 +13417,5 @@ module ThriftHiveMetastore
     ::Thrift::Struct.generate_accessors self
   end
 
-  class Add_runtime_stats_args
-    include ::Thrift::Struct, ::Thrift::Struct_Union
-    STAT = 1
-
-    FIELDS = {
-      STAT => {:type => ::Thrift::Types::STRUCT, :name => 'stat', :class => ::RuntimeStat}
-    }
-
-    def struct_fields; FIELDS; end
-
-    def validate
-    end
-
-    ::Thrift::Struct.generate_accessors self
-  end
-
-  class Add_runtime_stats_result
-    include ::Thrift::Struct, ::Thrift::Struct_Union
-    O1 = 1
-
-    FIELDS = {
-      O1 => {:type => ::Thrift::Types::STRUCT, :name => 'o1', :class => ::MetaException}
-    }
-
-    def struct_fields; FIELDS; end
-
-    def validate
-    end
-
-    ::Thrift::Struct.generate_accessors self
-  end
-
-  class Get_runtime_stats_args
-    include ::Thrift::Struct, ::Thrift::Struct_Union
-    RQST = 1
-
-    FIELDS = {
-      RQST => {:type => ::Thrift::Types::STRUCT, :name => 'rqst', :class => ::GetRuntimeStatsRequest}
-    }
-
-    def struct_fields; FIELDS; end
-
-    def validate
-    end
-
-    ::Thrift::Struct.generate_accessors self
-  end
-
-  class Get_runtime_stats_result
-    include ::Thrift::Struct, ::Thrift::Struct_Union
-    SUCCESS = 0
-    O1 = 1
-
-    FIELDS = {
-      SUCCESS => {:type => ::Thrift::Types::LIST, :name => 'success', :element => {:type => ::Thrift::Types::STRUCT, :class => ::RuntimeStat}},
-      O1 => {:type => ::Thrift::Types::STRUCT, :name => 'o1', :class => ::MetaException}
-    }
-
-    def struct_fields; FIELDS; end
-
-    def validate
-    end
-
-    ::Thrift::Struct.generate_accessors self
-  end
-
 end
 

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
index 1450e29..cd50e1b 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@ -50,6 +50,7 @@ import java.util.Map.Entry;
 import java.util.Objects;
 import java.util.Properties;
 import java.util.Set;
+import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
@@ -1236,9 +1237,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       startFunction("create_database", ": " + db.toString());
       boolean success = false;
       Exception ex = null;
-      if (!db.isSetCatalogName()) {
-        db.setCatalogName(getDefaultCatalog(conf));
-      }
+      if (!db.isSetCatalogName()) db.setCatalogName(getDefaultCatalog(conf));
       try {
         try {
           if (null != get_database_core(db.getCatalogName(), db.getName())) {
@@ -1754,9 +1753,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       Path tblPath = null;
       boolean success = false, madeDir = false;
       try {
-        if (!tbl.isSetCatName()) {
-          tbl.setCatName(getDefaultCatalog(conf));
-        }
+        if (!tbl.isSetCatName()) tbl.setCatName(getDefaultCatalog(conf));
         firePreEvent(new PreCreateTableEvent(tbl, this));
 
         ms.openTransaction();
@@ -3397,14 +3394,10 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         return result;
       }
       try {
-        if (!request.isSetCatName()) {
-          request.setCatName(getDefaultCatalog(conf));
-        }
+        if (!request.isSetCatName()) request.setCatName(getDefaultCatalog(conf));
         // Make sure all of the partitions have the catalog set as well
         request.getParts().forEach(p -> {
-          if (!p.isSetCatName()) {
-            p.setCatName(getDefaultCatalog(conf));
-          }
+          if (!p.isSetCatName()) p.setCatName(getDefaultCatalog(conf));
         });
         List<Partition> parts = add_partitions_core(getMS(), request.getCatName(), request.getDbName(),
             request.getTblName(), request.getParts(), request.isIfNotExists());
@@ -3436,9 +3429,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         // Old API assumed all partitions belong to the same table; keep the same assumption
         if (!parts.get(0).isSetCatName()) {
           String defaultCat = getDefaultCatalog(conf);
-          for (Partition p : parts) {
-            p.setCatName(defaultCat);
-          }
+          for (Partition p : parts) p.setCatName(defaultCat);
         }
         ret = add_partitions_core(getMS(), parts.get(0).getCatName(), parts.get(0).getDbName(),
             parts.get(0).getTableName(), parts, false).size();
@@ -3749,9 +3740,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       boolean success = false;
       Table tbl = null;
       Map<String, String> transactionalListenerResponses = Collections.emptyMap();
-      if (!part.isSetCatName()) {
-        part.setCatName(getDefaultCatalog(conf));
-      }
+      if (!part.isSetCatName()) part.setCatName(getDefaultCatalog(conf));
       try {
         ms.openTransaction();
         tbl = ms.getTable(part.getCatName(), part.getDbName(), part.getTableName());
@@ -4759,9 +4748,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       }
 
       // Make sure the new partition has the catalog value set
-      if (!new_part.isSetCatName()) {
-        new_part.setCatName(catName);
-      }
+      if (!new_part.isSetCatName()) new_part.setCatName(catName);
 
       Partition oldPart = null;
       Exception ex = null;
@@ -4834,9 +4821,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       try {
         for (Partition tmpPart : new_parts) {
           // Make sure the catalog name is set in the new partition
-          if (!tmpPart.isSetCatName()) {
-            tmpPart.setCatName(getDefaultCatalog(conf));
-          }
+          if (!tmpPart.isSetCatName()) tmpPart.setCatName(getDefaultCatalog(conf));
           firePreEvent(new PreAlterPartitionEvent(parsedDbName[DB_NAME], tbl_name, null, tmpPart, this));
         }
         oldParts = alterHandler.alterPartitions(getMS(), wh, parsedDbName[CAT_NAME],
@@ -4941,9 +4926,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         }
       }
       // Set the catalog name if it hasn't been set in the new table
-      if (!newTable.isSetCatName()) {
-        newTable.setCatName(catName);
-      }
+      if (!newTable.isSetCatName()) newTable.setCatName(catName);
 
       boolean success = false;
       Exception ex = null;
@@ -8012,7 +7995,6 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       }
     }
 
-    @Override
     public WMAlterPoolResponse alter_wm_pool(WMAlterPoolRequest request)
         throws AlreadyExistsException, NoSuchObjectException, InvalidObjectException, MetaException,
         TException {
@@ -8038,7 +8020,6 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       }
     }
 
-    @Override
     public WMDropPoolResponse drop_wm_pool(WMDropPoolRequest request)
         throws NoSuchObjectException, InvalidOperationException, MetaException, TException {
       try {
@@ -8050,7 +8031,6 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       }
     }
 
-    @Override
     public WMCreateOrUpdateMappingResponse create_or_update_wm_mapping(
         WMCreateOrUpdateMappingRequest request) throws AlreadyExistsException,
         NoSuchObjectException, InvalidObjectException, MetaException, TException {
@@ -8063,7 +8043,6 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       }
     }
 
-    @Override
     public WMDropMappingResponse drop_wm_mapping(WMDropMappingRequest request)
         throws NoSuchObjectException, InvalidOperationException, MetaException, TException {
       try {
@@ -8075,7 +8054,6 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       }
     }
 
-    @Override
     public WMCreateOrDropTriggerToPoolMappingResponse create_or_drop_wm_trigger_to_pool_mapping(
         WMCreateOrDropTriggerToPoolMappingRequest request) throws AlreadyExistsException,
         NoSuchObjectException, InvalidObjectException, MetaException, TException {
@@ -8094,7 +8072,6 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       }
     }
 
-    @Override
     public void create_ischema(ISchema schema) throws TException {
       startFunction("create_ischema", ": " + schema.getName());
       boolean success = false;
@@ -8114,9 +8091,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
           }
           success = ms.commitTransaction();
         } finally {
-          if (!success) {
-            ms.rollbackTransaction();
-          }
+          if (!success) ms.rollbackTransaction();
           if (!listeners.isEmpty()) {
             MetaStoreListenerNotifier.notifyEvent(listeners, EventType.CREATE_ISCHEMA,
                 new CreateISchemaEvent(success, this, schema), null,
@@ -8155,9 +8130,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
           }
           success = ms.commitTransaction();
         } finally {
-          if (!success) {
-            ms.rollbackTransaction();
-          }
+          if (!success) ms.rollbackTransaction();
           if (!listeners.isEmpty()) {
             MetaStoreListenerNotifier.notifyEvent(listeners, EventType.ALTER_ISCHEMA,
                 new AlterISchemaEvent(success, this, oldSchema, rqst.getNewSchema()), null,
@@ -8222,9 +8195,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
           }
           success = ms.commitTransaction();
         } finally {
-          if (!success) {
-            ms.rollbackTransaction();
-          }
+          if (!success) ms.rollbackTransaction();
           if (!listeners.isEmpty()) {
             MetaStoreListenerNotifier.notifyEvent(listeners, EventType.DROP_ISCHEMA,
                 new DropISchemaEvent(success, this, schema), null,
@@ -8264,9 +8235,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
           }
           success = ms.commitTransaction();
         } finally {
-          if (!success) {
-            ms.rollbackTransaction();
-          }
+          if (!success) ms.rollbackTransaction();
           if (!listeners.isEmpty()) {
             MetaStoreListenerNotifier.notifyEvent(listeners, EventType.ADD_SCHEMA_VERSION,
                 new AddSchemaVersionEvent(success, this, schemaVersion), null,
@@ -8368,9 +8337,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
           }
           success = ms.commitTransaction();
         } finally {
-          if (!success) {
-            ms.rollbackTransaction();
-          }
+          if (!success) ms.rollbackTransaction();
           if (!listeners.isEmpty()) {
             MetaStoreListenerNotifier.notifyEvent(listeners, EventType.DROP_SCHEMA_VERSION,
                 new DropSchemaVersionEvent(success, this, schemaVersion), null,
@@ -8439,9 +8406,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
           }
           success = ms.commitTransaction();
         } finally {
-          if (!success) {
-            ms.rollbackTransaction();
-          }
+          if (!success) ms.rollbackTransaction();
           if (!listeners.isEmpty()) {
             MetaStoreListenerNotifier.notifyEvent(listeners, EventType.ALTER_SCHEMA_VERSION,
                 new AlterSchemaVersionEvent(success, this, oldSchemaVersion, newSchemaVersion), null,
@@ -8483,9 +8448,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
           }
           success = ms.commitTransaction();
         } finally {
-          if (!success) {
-            ms.rollbackTransaction();
-          }
+          if (!success) ms.rollbackTransaction();
           if (!listeners.isEmpty()) {
             MetaStoreListenerNotifier.notifyEvent(listeners, EventType.ALTER_SCHEMA_VERSION,
                 new AlterSchemaVersionEvent(success, this, oldSchemaVersion, newSchemaVersion), null,
@@ -8516,9 +8479,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         ex = e;
         throw e;
       } finally {
-        if (!success) {
-          ms.rollbackTransaction();
-        }
+        if (!success) ms.rollbackTransaction();
         endFunction("create_serde", success, ex);
       }
     }
@@ -8554,44 +8515,6 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         throws TException {
       return MaterializationsRebuildLockHandler.get().refreshLockResource(dbName, tableName, txnId);
     }
-
-    @Override
-    public void add_runtime_stats(RuntimeStat stat) throws TException {
-      startFunction("store_runtime_stats");
-      Exception ex = null;
-      boolean success = false;
-      RawStore ms = getMS();
-      try {
-        ms.openTransaction();
-        ms.addRuntimeStat(stat);
-        success = ms.commitTransaction();
-      } catch (Exception e) {
-        LOG.error("Caught exception", e);
-        ex = e;
-        throw e;
-      } finally {
-        if (!success) {
-          ms.rollbackTransaction();
-        }
-        endFunction("store_runtime_stats", success, ex);
-      }
-    }
-
-    @Override
-    public List<RuntimeStat> get_runtime_stats(GetRuntimeStatsRequest rqst) throws TException {
-      startFunction("get_runtime_stats");
-      Exception ex = null;
-      try {
-        List<RuntimeStat> res = getMS().getRuntimeStats();
-        return res;
-      } catch (MetaException e) {
-        LOG.error("Caught exception", e);
-        ex = e;
-        throw e;
-      } finally {
-        endFunction("get_runtime_stats", ex == null, ex);
-      }
-    }
   }
 
   private static IHMSHandler newRetryingHMSHandler(IHMSHandler baseHandler, Configuration conf)

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
index 1c8d223..feae991 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
@@ -649,9 +649,7 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
 
   public Partition add_partition(Partition new_part, EnvironmentContext envContext)
       throws TException {
-    if (new_part != null && !new_part.isSetCatName()) {
-      new_part.setCatName(getDefaultCatalog(conf));
-    }
+    if (new_part != null && !new_part.isSetCatName()) new_part.setCatName(getDefaultCatalog(conf));
     Partition p = client.add_partition_with_environment_context(new_part, envContext);
     return deepCopy(p);
   }
@@ -699,9 +697,7 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
     if (partitionSpec == null) {
       throw new MetaException("PartitionSpec cannot be null.");
     }
-    if (partitionSpec.getCatName() == null) {
-      partitionSpec.setCatName(getDefaultCatalog(conf));
-    }
+    if (partitionSpec.getCatName() == null) partitionSpec.setCatName(getDefaultCatalog(conf));
     return client.add_partitions_pspec(partitionSpec.toPartitionSpec());
   }
 
@@ -804,9 +800,7 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
   @Override
   public void createDatabase(Database db)
       throws AlreadyExistsException, InvalidObjectException, MetaException, TException {
-    if (!db.isSetCatalogName()) {
-      db.setCatalogName(getDefaultCatalog(conf));
-    }
+    if (!db.isSetCatalogName()) db.setCatalogName(getDefaultCatalog(conf));
     client.create_database(db);
   }
 
@@ -825,9 +819,7 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
 
   public void createTable(Table tbl, EnvironmentContext envContext) throws AlreadyExistsException,
       InvalidObjectException, MetaException, NoSuchObjectException, TException {
-    if (!tbl.isSetCatName()) {
-      tbl.setCatName(getDefaultCatalog(conf));
-    }
+    if (!tbl.isSetCatName()) tbl.setCatName(getDefaultCatalog(conf));
     HiveMetaHook hook = getHook(tbl);
     if (hook != null) {
       hook.preCreateTable(tbl);
@@ -852,7 +844,6 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
     }
   }
 
-  @Override
   public void createTableWithConstraints(Table tbl,
     List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys,
     List<SQLUniqueConstraint> uniqueConstraints,
@@ -865,24 +856,12 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
     if (!tbl.isSetCatName()) {
       String defaultCat = getDefaultCatalog(conf);
       tbl.setCatName(defaultCat);
-      if (primaryKeys != null) {
-        primaryKeys.forEach(pk -> pk.setCatName(defaultCat));
-      }
-      if (foreignKeys != null) {
-        foreignKeys.forEach(fk -> fk.setCatName(defaultCat));
-      }
-      if (uniqueConstraints != null) {
-        uniqueConstraints.forEach(uc -> uc.setCatName(defaultCat));
-      }
-      if (notNullConstraints != null) {
-        notNullConstraints.forEach(nn -> nn.setCatName(defaultCat));
-      }
-      if (defaultConstraints != null) {
-        defaultConstraints.forEach(def -> def.setCatName(defaultCat));
-      }
-      if (checkConstraints != null) {
-        checkConstraints.forEach(cc -> cc.setCatName(defaultCat));
-      }
+      if (primaryKeys != null) primaryKeys.forEach(pk -> pk.setCatName(defaultCat));
+      if (foreignKeys != null) foreignKeys.forEach(fk -> fk.setCatName(defaultCat));
+      if (uniqueConstraints != null) uniqueConstraints.forEach(uc -> uc.setCatName(defaultCat));
+      if (notNullConstraints != null) notNullConstraints.forEach(nn -> nn.setCatName(defaultCat));
+      if (defaultConstraints != null) defaultConstraints.forEach(def -> def.setCatName(defaultCat));
+      if (checkConstraints != null) checkConstraints.forEach(cc -> cc.setCatName(defaultCat));
     }
     HiveMetaHook hook = getHook(tbl);
     if (hook != null) {
@@ -1016,7 +995,6 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
     dropDatabase(getDefaultCatalog(conf), name, deleteData, ignoreUnknownDb, cascade);
   }
 
-  @Override
   public void dropDatabase(String catalogName, String dbName, boolean deleteData,
                            boolean ignoreUnknownDb, boolean cascade)
       throws NoSuchObjectException, InvalidOperationException, MetaException, TException {
@@ -1543,9 +1521,7 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
   @Override
   public PartitionValuesResponse listPartitionValues(PartitionValuesRequest request)
       throws MetaException, TException, NoSuchObjectException {
-    if (!request.isSetCatName()) {
-      request.setCatName(getDefaultCatalog(conf));
-    }
+    if (!request.isSetCatName()) request.setCatName(getDefaultCatalog(conf));
     return client.get_partition_values(request);
   }
 
@@ -1850,71 +1826,55 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
 
   @Override
   public List<SQLPrimaryKey> getPrimaryKeys(PrimaryKeysRequest req) throws TException {
-    if (!req.isSetCatName()) {
-      req.setCatName(getDefaultCatalog(conf));
-    }
+    if (!req.isSetCatName()) req.setCatName(getDefaultCatalog(conf));
     return client.get_primary_keys(req).getPrimaryKeys();
   }
 
   @Override
   public List<SQLForeignKey> getForeignKeys(ForeignKeysRequest req) throws MetaException,
     NoSuchObjectException, TException {
-    if (!req.isSetCatName()) {
-      req.setCatName(getDefaultCatalog(conf));
-    }
+    if (!req.isSetCatName()) req.setCatName(getDefaultCatalog(conf));
     return client.get_foreign_keys(req).getForeignKeys();
   }
 
   @Override
   public List<SQLUniqueConstraint> getUniqueConstraints(UniqueConstraintsRequest req)
     throws MetaException, NoSuchObjectException, TException {
-    if (!req.isSetCatName()) {
-      req.setCatName(getDefaultCatalog(conf));
-    }
+    if (!req.isSetCatName()) req.setCatName(getDefaultCatalog(conf));
     return client.get_unique_constraints(req).getUniqueConstraints();
   }
 
   @Override
   public List<SQLNotNullConstraint> getNotNullConstraints(NotNullConstraintsRequest req)
     throws MetaException, NoSuchObjectException, TException {
-    if (!req.isSetCatName()) {
-      req.setCatName(getDefaultCatalog(conf));
-    }
+    if (!req.isSetCatName()) req.setCatName(getDefaultCatalog(conf));
     return client.get_not_null_constraints(req).getNotNullConstraints();
   }
 
   @Override
   public List<SQLDefaultConstraint> getDefaultConstraints(DefaultConstraintsRequest req)
       throws MetaException, NoSuchObjectException, TException {
-    if (!req.isSetCatName()) {
-      req.setCatName(getDefaultCatalog(conf));
-    }
+    if (!req.isSetCatName()) req.setCatName(getDefaultCatalog(conf));
     return client.get_default_constraints(req).getDefaultConstraints();
   }
 
   @Override
   public List<SQLCheckConstraint> getCheckConstraints(CheckConstraintsRequest req)
       throws MetaException, NoSuchObjectException, TException {
-    if (!req.isSetCatName()) {
-      req.setCatName(getDefaultCatalog(conf));
-    }
+    if (!req.isSetCatName()) req.setCatName(getDefaultCatalog(conf));
     return client.get_check_constraints(req).getCheckConstraints();
   }
 
   /** {@inheritDoc} */
   @Override
   public boolean updateTableColumnStatistics(ColumnStatistics statsObj) throws TException {
-    if (!statsObj.getStatsDesc().isSetCatName()) {
-      statsObj.getStatsDesc().setCatName(getDefaultCatalog(conf));
-    }
+    if (!statsObj.getStatsDesc().isSetCatName()) statsObj.getStatsDesc().setCatName(getDefaultCatalog(conf));
     return client.update_table_column_statistics(statsObj);
   }
 
   @Override
   public boolean updatePartitionColumnStatistics(ColumnStatistics statsObj) throws TException {
-    if (!statsObj.getStatsDesc().isSetCatName()) {
-      statsObj.getStatsDesc().setCatName(getDefaultCatalog(conf));
-    }
+    if (!statsObj.getStatsDesc().isSetCatName()) statsObj.getStatsDesc().setCatName(getDefaultCatalog(conf));
     return client.update_partition_column_statistics(statsObj);
   }
 
@@ -1922,9 +1882,7 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
   public boolean setPartitionColumnStatistics(SetPartitionsStatsRequest request) throws TException {
     String defaultCat = getDefaultCatalog(conf);
     for (ColumnStatistics stats : request.getColStats()) {
-      if (!stats.getStatsDesc().isSetCatName()) {
-        stats.getStatsDesc().setCatName(defaultCat);
-      }
+      if (!stats.getStatsDesc().isSetCatName()) stats.getStatsDesc().setCatName(defaultCat);
     }
     return client.set_aggr_stats_for(request);
   }
@@ -2238,9 +2196,7 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
       throws MetaException, TException {
     String defaultCat = getDefaultCatalog(conf);
     for (HiveObjectPrivilege priv : privileges.getPrivileges()) {
-      if (!priv.getHiveObject().isSetCatName()) {
-        priv.getHiveObject().setCatName(defaultCat);
-      }
+      if (!priv.getHiveObject().isSetCatName()) priv.getHiveObject().setCatName(defaultCat);
     }
     GrantRevokePrivilegeRequest req = new GrantRevokePrivilegeRequest();
     req.setRequestType(GrantRevokeType.GRANT);
@@ -2273,9 +2229,7 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
       TException {
     String defaultCat = getDefaultCatalog(conf);
     for (HiveObjectPrivilege priv : privileges.getPrivileges()) {
-      if (!priv.getHiveObject().isSetCatName()) {
-        priv.getHiveObject().setCatName(defaultCat);
-      }
+      if (!priv.getHiveObject().isSetCatName()) priv.getHiveObject().setCatName(defaultCat);
     }
     GrantRevokePrivilegeRequest req = new GrantRevokePrivilegeRequest();
     req.setRequestType(GrantRevokeType.REVOKE);
@@ -2292,9 +2246,7 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
   public PrincipalPrivilegeSet get_privilege_set(HiveObjectRef hiveObject,
       String userName, List<String> groupNames) throws MetaException,
       TException {
-    if (!hiveObject.isSetCatName()) {
-      hiveObject.setCatName(getDefaultCatalog(conf));
-    }
+    if (!hiveObject.isSetCatName()) hiveObject.setCatName(getDefaultCatalog(conf));
     return client.get_privilege_set(hiveObject, userName, groupNames);
   }
 
@@ -2302,9 +2254,7 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
   public List<HiveObjectPrivilege> list_privileges(String principalName,
       PrincipalType principalType, HiveObjectRef hiveObject)
       throws MetaException, TException {
-    if (!hiveObject.isSetCatName()) {
-      hiveObject.setCatName(getDefaultCatalog(conf));
-    }
+    if (!hiveObject.isSetCatName()) hiveObject.setCatName(getDefaultCatalog(conf));
     return client.list_privileges(principalName, principalType, hiveObject);
   }
 
@@ -2677,18 +2627,14 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
   @Override
   public NotificationEventsCountResponse getNotificationEventsCount(NotificationEventsCountRequest rqst)
           throws TException {
-    if (!rqst.isSetCatName()) {
-      rqst.setCatName(getDefaultCatalog(conf));
-    }
+    if (!rqst.isSetCatName()) rqst.setCatName(getDefaultCatalog(conf));
     return client.get_notification_events_count(rqst);
   }
 
   @InterfaceAudience.LimitedPrivate({"Apache Hive, HCatalog"})
   @Override
   public FireEventResponse fireListenerEvent(FireEventRequest rqst) throws TException {
-    if (!rqst.isSetCatName()) {
-      rqst.setCatName(getDefaultCatalog(conf));
-    }
+    if (!rqst.isSetCatName()) rqst.setCatName(getDefaultCatalog(conf));
     return client.fire_listener_event(rqst);
   }
 
@@ -2761,9 +2707,7 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
     if (func == null) {
       throw new MetaException("Function cannot be null.");
     }
-    if (!func.isSetCatName()) {
-      func.setCatName(getDefaultCatalog(conf));
-    }
+    if (!func.isSetCatName()) func.setCatName(getDefaultCatalog(conf));
     client.create_function(func);
   }
 
@@ -3143,11 +3087,8 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
     client.create_or_drop_wm_trigger_to_pool_mapping(request);
   }
 
-  @Override
   public void createISchema(ISchema schema) throws TException {
-    if (!schema.isSetCatName()) {
-      schema.setCatName(getDefaultCatalog(conf));
-    }
+    if (!schema.isSetCatName()) schema.setCatName(getDefaultCatalog(conf));
     client.create_ischema(schema);
   }
 
@@ -3168,13 +3109,10 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
 
   @Override
   public void addSchemaVersion(SchemaVersion schemaVersion) throws TException {
-    if (!schemaVersion.getSchema().isSetCatName()) {
-      schemaVersion.getSchema().setCatName(getDefaultCatalog(conf));
-    }
+    if (!schemaVersion.getSchema().isSetCatName()) schemaVersion.getSchema().setCatName(getDefaultCatalog(conf));
     client.add_schema_version(schemaVersion);
   }
 
-  @Override
   public SchemaVersion getSchemaVersion(String catName, String dbName, String schemaName, int version) throws TException {
     return client.get_schema_version(new SchemaVersionDescriptor(new ISchemaName(catName, dbName, schemaName), version));
   }
@@ -3224,13 +3162,9 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
   }
 
   private short shrinkMaxtoShort(int max) {
-    if (max < 0) {
-      return -1;
-    } else if (max <= Short.MAX_VALUE) {
-      return (short)max;
-    } else {
-      return Short.MAX_VALUE;
-    }
+    if (max < 0) return -1;
+    else if (max <= Short.MAX_VALUE) return (short)max;
+    else return Short.MAX_VALUE;
   }
 
   @Override
@@ -3242,14 +3176,4 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
   public boolean heartbeatLockMaterializationRebuild(String dbName, String tableName, long txnId) throws TException {
     return client.heartbeat_lock_materialization_rebuild(dbName, tableName, txnId);
   }
-
-  @Override
-  public void addRuntimeStat(RuntimeStat stat) throws TException {
-    client.add_runtime_stats(stat);
-  }
-
-  @Override
-  public List<RuntimeStat> getRuntimeStats() throws TException {
-    return client.get_runtime_stats(new GetRuntimeStatsRequest());
-  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
index aee416d..27f8775 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
@@ -94,7 +94,6 @@ import org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet;
 import org.apache.hadoop.hive.metastore.api.PrincipalType;
 import org.apache.hadoop.hive.metastore.api.PrivilegeBag;
 import org.apache.hadoop.hive.metastore.api.Role;
-import org.apache.hadoop.hive.metastore.api.RuntimeStat;
 import org.apache.hadoop.hive.metastore.api.SQLCheckConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLDefaultConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
@@ -1892,7 +1891,6 @@ public interface IMetaStoreClient {
    * @throws TException Thrift transport error.
    * @deprecated Use {@link #dropPartitions(String, String, String, List, boolean, boolean, boolean)}
    */
-  @Deprecated
   List<Partition> dropPartitions(String dbName, String tblName,
       List<ObjectPair<Integer, byte[]>> partExprs, boolean deleteData,
       boolean ifExists, boolean needResults) throws NoSuchObjectException, MetaException, TException;
@@ -3643,11 +3641,4 @@ public interface IMetaStoreClient {
    * @return true if the lock could be renewed, false otherwise
    */
   boolean heartbeatLockMaterializationRebuild(String dbName, String tableName, long txnId) throws TException;
-
-  /** Adds a RuntimeStat for metastore persistence. */
-  void addRuntimeStat(RuntimeStat stat) throws TException;
-
-  /** Reads runtime statistics. */
-  List<RuntimeStat> getRuntimeStats() throws TException;
-
 }


[5/6] hive git commit: Revert "HIVE-19171 : Persist runtime statistics in metastore (Zoltan Haindrich via Ashutosh Chauhan)"

Posted by pr...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/f0199500/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
index 4787703..dfa13a0 100644
--- a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
+++ b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
@@ -2107,14 +2107,14 @@ uint32_t ThriftHiveMetastore_get_databases_result::read(::apache::thrift::protoc
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1191;
-            ::apache::thrift::protocol::TType _etype1194;
-            xfer += iprot->readListBegin(_etype1194, _size1191);
-            this->success.resize(_size1191);
-            uint32_t _i1195;
-            for (_i1195 = 0; _i1195 < _size1191; ++_i1195)
+            uint32_t _size1187;
+            ::apache::thrift::protocol::TType _etype1190;
+            xfer += iprot->readListBegin(_etype1190, _size1187);
+            this->success.resize(_size1187);
+            uint32_t _i1191;
+            for (_i1191 = 0; _i1191 < _size1187; ++_i1191)
             {
-              xfer += iprot->readString(this->success[_i1195]);
+              xfer += iprot->readString(this->success[_i1191]);
             }
             xfer += iprot->readListEnd();
           }
@@ -2153,10 +2153,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 _iter1196;
-      for (_iter1196 = this->success.begin(); _iter1196 != this->success.end(); ++_iter1196)
+      std::vector<std::string> ::const_iterator _iter1192;
+      for (_iter1192 = this->success.begin(); _iter1192 != this->success.end(); ++_iter1192)
       {
-        xfer += oprot->writeString((*_iter1196));
+        xfer += oprot->writeString((*_iter1192));
       }
       xfer += oprot->writeListEnd();
     }
@@ -2201,14 +2201,14 @@ uint32_t ThriftHiveMetastore_get_databases_presult::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1197;
-            ::apache::thrift::protocol::TType _etype1200;
-            xfer += iprot->readListBegin(_etype1200, _size1197);
-            (*(this->success)).resize(_size1197);
-            uint32_t _i1201;
-            for (_i1201 = 0; _i1201 < _size1197; ++_i1201)
+            uint32_t _size1193;
+            ::apache::thrift::protocol::TType _etype1196;
+            xfer += iprot->readListBegin(_etype1196, _size1193);
+            (*(this->success)).resize(_size1193);
+            uint32_t _i1197;
+            for (_i1197 = 0; _i1197 < _size1193; ++_i1197)
             {
-              xfer += iprot->readString((*(this->success))[_i1201]);
+              xfer += iprot->readString((*(this->success))[_i1197]);
             }
             xfer += iprot->readListEnd();
           }
@@ -2325,14 +2325,14 @@ uint32_t ThriftHiveMetastore_get_all_databases_result::read(::apache::thrift::pr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1202;
-            ::apache::thrift::protocol::TType _etype1205;
-            xfer += iprot->readListBegin(_etype1205, _size1202);
-            this->success.resize(_size1202);
-            uint32_t _i1206;
-            for (_i1206 = 0; _i1206 < _size1202; ++_i1206)
+            uint32_t _size1198;
+            ::apache::thrift::protocol::TType _etype1201;
+            xfer += iprot->readListBegin(_etype1201, _size1198);
+            this->success.resize(_size1198);
+            uint32_t _i1202;
+            for (_i1202 = 0; _i1202 < _size1198; ++_i1202)
             {
-              xfer += iprot->readString(this->success[_i1206]);
+              xfer += iprot->readString(this->success[_i1202]);
             }
             xfer += iprot->readListEnd();
           }
@@ -2371,10 +2371,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 _iter1207;
-      for (_iter1207 = this->success.begin(); _iter1207 != this->success.end(); ++_iter1207)
+      std::vector<std::string> ::const_iterator _iter1203;
+      for (_iter1203 = this->success.begin(); _iter1203 != this->success.end(); ++_iter1203)
       {
-        xfer += oprot->writeString((*_iter1207));
+        xfer += oprot->writeString((*_iter1203));
       }
       xfer += oprot->writeListEnd();
     }
@@ -2419,14 +2419,14 @@ uint32_t ThriftHiveMetastore_get_all_databases_presult::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1208;
-            ::apache::thrift::protocol::TType _etype1211;
-            xfer += iprot->readListBegin(_etype1211, _size1208);
-            (*(this->success)).resize(_size1208);
-            uint32_t _i1212;
-            for (_i1212 = 0; _i1212 < _size1208; ++_i1212)
+            uint32_t _size1204;
+            ::apache::thrift::protocol::TType _etype1207;
+            xfer += iprot->readListBegin(_etype1207, _size1204);
+            (*(this->success)).resize(_size1204);
+            uint32_t _i1208;
+            for (_i1208 = 0; _i1208 < _size1204; ++_i1208)
             {
-              xfer += iprot->readString((*(this->success))[_i1212]);
+              xfer += iprot->readString((*(this->success))[_i1208]);
             }
             xfer += iprot->readListEnd();
           }
@@ -3488,17 +3488,17 @@ uint32_t ThriftHiveMetastore_get_type_all_result::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->success.clear();
-            uint32_t _size1213;
-            ::apache::thrift::protocol::TType _ktype1214;
-            ::apache::thrift::protocol::TType _vtype1215;
-            xfer += iprot->readMapBegin(_ktype1214, _vtype1215, _size1213);
-            uint32_t _i1217;
-            for (_i1217 = 0; _i1217 < _size1213; ++_i1217)
+            uint32_t _size1209;
+            ::apache::thrift::protocol::TType _ktype1210;
+            ::apache::thrift::protocol::TType _vtype1211;
+            xfer += iprot->readMapBegin(_ktype1210, _vtype1211, _size1209);
+            uint32_t _i1213;
+            for (_i1213 = 0; _i1213 < _size1209; ++_i1213)
             {
-              std::string _key1218;
-              xfer += iprot->readString(_key1218);
-              Type& _val1219 = this->success[_key1218];
-              xfer += _val1219.read(iprot);
+              std::string _key1214;
+              xfer += iprot->readString(_key1214);
+              Type& _val1215 = this->success[_key1214];
+              xfer += _val1215.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -3537,11 +3537,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 _iter1220;
-      for (_iter1220 = this->success.begin(); _iter1220 != this->success.end(); ++_iter1220)
+      std::map<std::string, Type> ::const_iterator _iter1216;
+      for (_iter1216 = this->success.begin(); _iter1216 != this->success.end(); ++_iter1216)
       {
-        xfer += oprot->writeString(_iter1220->first);
-        xfer += _iter1220->second.write(oprot);
+        xfer += oprot->writeString(_iter1216->first);
+        xfer += _iter1216->second.write(oprot);
       }
       xfer += oprot->writeMapEnd();
     }
@@ -3586,17 +3586,17 @@ uint32_t ThriftHiveMetastore_get_type_all_presult::read(::apache::thrift::protoc
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             (*(this->success)).clear();
-            uint32_t _size1221;
-            ::apache::thrift::protocol::TType _ktype1222;
-            ::apache::thrift::protocol::TType _vtype1223;
-            xfer += iprot->readMapBegin(_ktype1222, _vtype1223, _size1221);
-            uint32_t _i1225;
-            for (_i1225 = 0; _i1225 < _size1221; ++_i1225)
+            uint32_t _size1217;
+            ::apache::thrift::protocol::TType _ktype1218;
+            ::apache::thrift::protocol::TType _vtype1219;
+            xfer += iprot->readMapBegin(_ktype1218, _vtype1219, _size1217);
+            uint32_t _i1221;
+            for (_i1221 = 0; _i1221 < _size1217; ++_i1221)
             {
-              std::string _key1226;
-              xfer += iprot->readString(_key1226);
-              Type& _val1227 = (*(this->success))[_key1226];
-              xfer += _val1227.read(iprot);
+              std::string _key1222;
+              xfer += iprot->readString(_key1222);
+              Type& _val1223 = (*(this->success))[_key1222];
+              xfer += _val1223.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -3750,14 +3750,14 @@ uint32_t ThriftHiveMetastore_get_fields_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1228;
-            ::apache::thrift::protocol::TType _etype1231;
-            xfer += iprot->readListBegin(_etype1231, _size1228);
-            this->success.resize(_size1228);
-            uint32_t _i1232;
-            for (_i1232 = 0; _i1232 < _size1228; ++_i1232)
+            uint32_t _size1224;
+            ::apache::thrift::protocol::TType _etype1227;
+            xfer += iprot->readListBegin(_etype1227, _size1224);
+            this->success.resize(_size1224);
+            uint32_t _i1228;
+            for (_i1228 = 0; _i1228 < _size1224; ++_i1228)
             {
-              xfer += this->success[_i1232].read(iprot);
+              xfer += this->success[_i1228].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3812,10 +3812,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 _iter1233;
-      for (_iter1233 = this->success.begin(); _iter1233 != this->success.end(); ++_iter1233)
+      std::vector<FieldSchema> ::const_iterator _iter1229;
+      for (_iter1229 = this->success.begin(); _iter1229 != this->success.end(); ++_iter1229)
       {
-        xfer += (*_iter1233).write(oprot);
+        xfer += (*_iter1229).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -3868,14 +3868,14 @@ uint32_t ThriftHiveMetastore_get_fields_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1234;
-            ::apache::thrift::protocol::TType _etype1237;
-            xfer += iprot->readListBegin(_etype1237, _size1234);
-            (*(this->success)).resize(_size1234);
-            uint32_t _i1238;
-            for (_i1238 = 0; _i1238 < _size1234; ++_i1238)
+            uint32_t _size1230;
+            ::apache::thrift::protocol::TType _etype1233;
+            xfer += iprot->readListBegin(_etype1233, _size1230);
+            (*(this->success)).resize(_size1230);
+            uint32_t _i1234;
+            for (_i1234 = 0; _i1234 < _size1230; ++_i1234)
             {
-              xfer += (*(this->success))[_i1238].read(iprot);
+              xfer += (*(this->success))[_i1234].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4061,14 +4061,14 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_result::read(::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1239;
-            ::apache::thrift::protocol::TType _etype1242;
-            xfer += iprot->readListBegin(_etype1242, _size1239);
-            this->success.resize(_size1239);
-            uint32_t _i1243;
-            for (_i1243 = 0; _i1243 < _size1239; ++_i1243)
+            uint32_t _size1235;
+            ::apache::thrift::protocol::TType _etype1238;
+            xfer += iprot->readListBegin(_etype1238, _size1235);
+            this->success.resize(_size1235);
+            uint32_t _i1239;
+            for (_i1239 = 0; _i1239 < _size1235; ++_i1239)
             {
-              xfer += this->success[_i1243].read(iprot);
+              xfer += this->success[_i1239].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4123,10 +4123,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 _iter1244;
-      for (_iter1244 = this->success.begin(); _iter1244 != this->success.end(); ++_iter1244)
+      std::vector<FieldSchema> ::const_iterator _iter1240;
+      for (_iter1240 = this->success.begin(); _iter1240 != this->success.end(); ++_iter1240)
       {
-        xfer += (*_iter1244).write(oprot);
+        xfer += (*_iter1240).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -4179,14 +4179,14 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_presult::read(:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1245;
-            ::apache::thrift::protocol::TType _etype1248;
-            xfer += iprot->readListBegin(_etype1248, _size1245);
-            (*(this->success)).resize(_size1245);
-            uint32_t _i1249;
-            for (_i1249 = 0; _i1249 < _size1245; ++_i1249)
+            uint32_t _size1241;
+            ::apache::thrift::protocol::TType _etype1244;
+            xfer += iprot->readListBegin(_etype1244, _size1241);
+            (*(this->success)).resize(_size1241);
+            uint32_t _i1245;
+            for (_i1245 = 0; _i1245 < _size1241; ++_i1245)
             {
-              xfer += (*(this->success))[_i1249].read(iprot);
+              xfer += (*(this->success))[_i1245].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4356,14 +4356,14 @@ uint32_t ThriftHiveMetastore_get_schema_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1250;
-            ::apache::thrift::protocol::TType _etype1253;
-            xfer += iprot->readListBegin(_etype1253, _size1250);
-            this->success.resize(_size1250);
-            uint32_t _i1254;
-            for (_i1254 = 0; _i1254 < _size1250; ++_i1254)
+            uint32_t _size1246;
+            ::apache::thrift::protocol::TType _etype1249;
+            xfer += iprot->readListBegin(_etype1249, _size1246);
+            this->success.resize(_size1246);
+            uint32_t _i1250;
+            for (_i1250 = 0; _i1250 < _size1246; ++_i1250)
             {
-              xfer += this->success[_i1254].read(iprot);
+              xfer += this->success[_i1250].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4418,10 +4418,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 _iter1255;
-      for (_iter1255 = this->success.begin(); _iter1255 != this->success.end(); ++_iter1255)
+      std::vector<FieldSchema> ::const_iterator _iter1251;
+      for (_iter1251 = this->success.begin(); _iter1251 != this->success.end(); ++_iter1251)
       {
-        xfer += (*_iter1255).write(oprot);
+        xfer += (*_iter1251).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -4474,14 +4474,14 @@ uint32_t ThriftHiveMetastore_get_schema_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1256;
-            ::apache::thrift::protocol::TType _etype1259;
-            xfer += iprot->readListBegin(_etype1259, _size1256);
-            (*(this->success)).resize(_size1256);
-            uint32_t _i1260;
-            for (_i1260 = 0; _i1260 < _size1256; ++_i1260)
+            uint32_t _size1252;
+            ::apache::thrift::protocol::TType _etype1255;
+            xfer += iprot->readListBegin(_etype1255, _size1252);
+            (*(this->success)).resize(_size1252);
+            uint32_t _i1256;
+            for (_i1256 = 0; _i1256 < _size1252; ++_i1256)
             {
-              xfer += (*(this->success))[_i1260].read(iprot);
+              xfer += (*(this->success))[_i1256].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4667,14 +4667,14 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_result::read(::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1261;
-            ::apache::thrift::protocol::TType _etype1264;
-            xfer += iprot->readListBegin(_etype1264, _size1261);
-            this->success.resize(_size1261);
-            uint32_t _i1265;
-            for (_i1265 = 0; _i1265 < _size1261; ++_i1265)
+            uint32_t _size1257;
+            ::apache::thrift::protocol::TType _etype1260;
+            xfer += iprot->readListBegin(_etype1260, _size1257);
+            this->success.resize(_size1257);
+            uint32_t _i1261;
+            for (_i1261 = 0; _i1261 < _size1257; ++_i1261)
             {
-              xfer += this->success[_i1265].read(iprot);
+              xfer += this->success[_i1261].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4729,10 +4729,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 _iter1266;
-      for (_iter1266 = this->success.begin(); _iter1266 != this->success.end(); ++_iter1266)
+      std::vector<FieldSchema> ::const_iterator _iter1262;
+      for (_iter1262 = this->success.begin(); _iter1262 != this->success.end(); ++_iter1262)
       {
-        xfer += (*_iter1266).write(oprot);
+        xfer += (*_iter1262).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -4785,14 +4785,14 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_presult::read(:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1267;
-            ::apache::thrift::protocol::TType _etype1270;
-            xfer += iprot->readListBegin(_etype1270, _size1267);
-            (*(this->success)).resize(_size1267);
-            uint32_t _i1271;
-            for (_i1271 = 0; _i1271 < _size1267; ++_i1271)
+            uint32_t _size1263;
+            ::apache::thrift::protocol::TType _etype1266;
+            xfer += iprot->readListBegin(_etype1266, _size1263);
+            (*(this->success)).resize(_size1263);
+            uint32_t _i1267;
+            for (_i1267 = 0; _i1267 < _size1263; ++_i1267)
             {
-              xfer += (*(this->success))[_i1271].read(iprot);
+              xfer += (*(this->success))[_i1267].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5385,14 +5385,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->primaryKeys.clear();
-            uint32_t _size1272;
-            ::apache::thrift::protocol::TType _etype1275;
-            xfer += iprot->readListBegin(_etype1275, _size1272);
-            this->primaryKeys.resize(_size1272);
-            uint32_t _i1276;
-            for (_i1276 = 0; _i1276 < _size1272; ++_i1276)
+            uint32_t _size1268;
+            ::apache::thrift::protocol::TType _etype1271;
+            xfer += iprot->readListBegin(_etype1271, _size1268);
+            this->primaryKeys.resize(_size1268);
+            uint32_t _i1272;
+            for (_i1272 = 0; _i1272 < _size1268; ++_i1272)
             {
-              xfer += this->primaryKeys[_i1276].read(iprot);
+              xfer += this->primaryKeys[_i1272].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5405,14 +5405,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->foreignKeys.clear();
-            uint32_t _size1277;
-            ::apache::thrift::protocol::TType _etype1280;
-            xfer += iprot->readListBegin(_etype1280, _size1277);
-            this->foreignKeys.resize(_size1277);
-            uint32_t _i1281;
-            for (_i1281 = 0; _i1281 < _size1277; ++_i1281)
+            uint32_t _size1273;
+            ::apache::thrift::protocol::TType _etype1276;
+            xfer += iprot->readListBegin(_etype1276, _size1273);
+            this->foreignKeys.resize(_size1273);
+            uint32_t _i1277;
+            for (_i1277 = 0; _i1277 < _size1273; ++_i1277)
             {
-              xfer += this->foreignKeys[_i1281].read(iprot);
+              xfer += this->foreignKeys[_i1277].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5425,14 +5425,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->uniqueConstraints.clear();
-            uint32_t _size1282;
-            ::apache::thrift::protocol::TType _etype1285;
-            xfer += iprot->readListBegin(_etype1285, _size1282);
-            this->uniqueConstraints.resize(_size1282);
-            uint32_t _i1286;
-            for (_i1286 = 0; _i1286 < _size1282; ++_i1286)
+            uint32_t _size1278;
+            ::apache::thrift::protocol::TType _etype1281;
+            xfer += iprot->readListBegin(_etype1281, _size1278);
+            this->uniqueConstraints.resize(_size1278);
+            uint32_t _i1282;
+            for (_i1282 = 0; _i1282 < _size1278; ++_i1282)
             {
-              xfer += this->uniqueConstraints[_i1286].read(iprot);
+              xfer += this->uniqueConstraints[_i1282].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5445,14 +5445,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->notNullConstraints.clear();
-            uint32_t _size1287;
-            ::apache::thrift::protocol::TType _etype1290;
-            xfer += iprot->readListBegin(_etype1290, _size1287);
-            this->notNullConstraints.resize(_size1287);
-            uint32_t _i1291;
-            for (_i1291 = 0; _i1291 < _size1287; ++_i1291)
+            uint32_t _size1283;
+            ::apache::thrift::protocol::TType _etype1286;
+            xfer += iprot->readListBegin(_etype1286, _size1283);
+            this->notNullConstraints.resize(_size1283);
+            uint32_t _i1287;
+            for (_i1287 = 0; _i1287 < _size1283; ++_i1287)
             {
-              xfer += this->notNullConstraints[_i1291].read(iprot);
+              xfer += this->notNullConstraints[_i1287].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5465,14 +5465,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->defaultConstraints.clear();
-            uint32_t _size1292;
-            ::apache::thrift::protocol::TType _etype1295;
-            xfer += iprot->readListBegin(_etype1295, _size1292);
-            this->defaultConstraints.resize(_size1292);
-            uint32_t _i1296;
-            for (_i1296 = 0; _i1296 < _size1292; ++_i1296)
+            uint32_t _size1288;
+            ::apache::thrift::protocol::TType _etype1291;
+            xfer += iprot->readListBegin(_etype1291, _size1288);
+            this->defaultConstraints.resize(_size1288);
+            uint32_t _i1292;
+            for (_i1292 = 0; _i1292 < _size1288; ++_i1292)
             {
-              xfer += this->defaultConstraints[_i1296].read(iprot);
+              xfer += this->defaultConstraints[_i1292].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5485,14 +5485,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->checkConstraints.clear();
-            uint32_t _size1297;
-            ::apache::thrift::protocol::TType _etype1300;
-            xfer += iprot->readListBegin(_etype1300, _size1297);
-            this->checkConstraints.resize(_size1297);
-            uint32_t _i1301;
-            for (_i1301 = 0; _i1301 < _size1297; ++_i1301)
+            uint32_t _size1293;
+            ::apache::thrift::protocol::TType _etype1296;
+            xfer += iprot->readListBegin(_etype1296, _size1293);
+            this->checkConstraints.resize(_size1293);
+            uint32_t _i1297;
+            for (_i1297 = 0; _i1297 < _size1293; ++_i1297)
             {
-              xfer += this->checkConstraints[_i1301].read(iprot);
+              xfer += this->checkConstraints[_i1297].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5525,10 +5525,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache:
   xfer += oprot->writeFieldBegin("primaryKeys", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->primaryKeys.size()));
-    std::vector<SQLPrimaryKey> ::const_iterator _iter1302;
-    for (_iter1302 = this->primaryKeys.begin(); _iter1302 != this->primaryKeys.end(); ++_iter1302)
+    std::vector<SQLPrimaryKey> ::const_iterator _iter1298;
+    for (_iter1298 = this->primaryKeys.begin(); _iter1298 != this->primaryKeys.end(); ++_iter1298)
     {
-      xfer += (*_iter1302).write(oprot);
+      xfer += (*_iter1298).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5537,10 +5537,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache:
   xfer += oprot->writeFieldBegin("foreignKeys", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->foreignKeys.size()));
-    std::vector<SQLForeignKey> ::const_iterator _iter1303;
-    for (_iter1303 = this->foreignKeys.begin(); _iter1303 != this->foreignKeys.end(); ++_iter1303)
+    std::vector<SQLForeignKey> ::const_iterator _iter1299;
+    for (_iter1299 = this->foreignKeys.begin(); _iter1299 != this->foreignKeys.end(); ++_iter1299)
     {
-      xfer += (*_iter1303).write(oprot);
+      xfer += (*_iter1299).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5549,10 +5549,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache:
   xfer += oprot->writeFieldBegin("uniqueConstraints", ::apache::thrift::protocol::T_LIST, 4);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->uniqueConstraints.size()));
-    std::vector<SQLUniqueConstraint> ::const_iterator _iter1304;
-    for (_iter1304 = this->uniqueConstraints.begin(); _iter1304 != this->uniqueConstraints.end(); ++_iter1304)
+    std::vector<SQLUniqueConstraint> ::const_iterator _iter1300;
+    for (_iter1300 = this->uniqueConstraints.begin(); _iter1300 != this->uniqueConstraints.end(); ++_iter1300)
     {
-      xfer += (*_iter1304).write(oprot);
+      xfer += (*_iter1300).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5561,10 +5561,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache:
   xfer += oprot->writeFieldBegin("notNullConstraints", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->notNullConstraints.size()));
-    std::vector<SQLNotNullConstraint> ::const_iterator _iter1305;
-    for (_iter1305 = this->notNullConstraints.begin(); _iter1305 != this->notNullConstraints.end(); ++_iter1305)
+    std::vector<SQLNotNullConstraint> ::const_iterator _iter1301;
+    for (_iter1301 = this->notNullConstraints.begin(); _iter1301 != this->notNullConstraints.end(); ++_iter1301)
     {
-      xfer += (*_iter1305).write(oprot);
+      xfer += (*_iter1301).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5573,10 +5573,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache:
   xfer += oprot->writeFieldBegin("defaultConstraints", ::apache::thrift::protocol::T_LIST, 6);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->defaultConstraints.size()));
-    std::vector<SQLDefaultConstraint> ::const_iterator _iter1306;
-    for (_iter1306 = this->defaultConstraints.begin(); _iter1306 != this->defaultConstraints.end(); ++_iter1306)
+    std::vector<SQLDefaultConstraint> ::const_iterator _iter1302;
+    for (_iter1302 = this->defaultConstraints.begin(); _iter1302 != this->defaultConstraints.end(); ++_iter1302)
     {
-      xfer += (*_iter1306).write(oprot);
+      xfer += (*_iter1302).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5585,10 +5585,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache:
   xfer += oprot->writeFieldBegin("checkConstraints", ::apache::thrift::protocol::T_LIST, 7);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->checkConstraints.size()));
-    std::vector<SQLCheckConstraint> ::const_iterator _iter1307;
-    for (_iter1307 = this->checkConstraints.begin(); _iter1307 != this->checkConstraints.end(); ++_iter1307)
+    std::vector<SQLCheckConstraint> ::const_iterator _iter1303;
+    for (_iter1303 = this->checkConstraints.begin(); _iter1303 != this->checkConstraints.end(); ++_iter1303)
     {
-      xfer += (*_iter1307).write(oprot);
+      xfer += (*_iter1303).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5616,10 +5616,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache
   xfer += oprot->writeFieldBegin("primaryKeys", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->primaryKeys)).size()));
-    std::vector<SQLPrimaryKey> ::const_iterator _iter1308;
-    for (_iter1308 = (*(this->primaryKeys)).begin(); _iter1308 != (*(this->primaryKeys)).end(); ++_iter1308)
+    std::vector<SQLPrimaryKey> ::const_iterator _iter1304;
+    for (_iter1304 = (*(this->primaryKeys)).begin(); _iter1304 != (*(this->primaryKeys)).end(); ++_iter1304)
     {
-      xfer += (*_iter1308).write(oprot);
+      xfer += (*_iter1304).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5628,10 +5628,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache
   xfer += oprot->writeFieldBegin("foreignKeys", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->foreignKeys)).size()));
-    std::vector<SQLForeignKey> ::const_iterator _iter1309;
-    for (_iter1309 = (*(this->foreignKeys)).begin(); _iter1309 != (*(this->foreignKeys)).end(); ++_iter1309)
+    std::vector<SQLForeignKey> ::const_iterator _iter1305;
+    for (_iter1305 = (*(this->foreignKeys)).begin(); _iter1305 != (*(this->foreignKeys)).end(); ++_iter1305)
     {
-      xfer += (*_iter1309).write(oprot);
+      xfer += (*_iter1305).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5640,10 +5640,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache
   xfer += oprot->writeFieldBegin("uniqueConstraints", ::apache::thrift::protocol::T_LIST, 4);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->uniqueConstraints)).size()));
-    std::vector<SQLUniqueConstraint> ::const_iterator _iter1310;
-    for (_iter1310 = (*(this->uniqueConstraints)).begin(); _iter1310 != (*(this->uniqueConstraints)).end(); ++_iter1310)
+    std::vector<SQLUniqueConstraint> ::const_iterator _iter1306;
+    for (_iter1306 = (*(this->uniqueConstraints)).begin(); _iter1306 != (*(this->uniqueConstraints)).end(); ++_iter1306)
     {
-      xfer += (*_iter1310).write(oprot);
+      xfer += (*_iter1306).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5652,10 +5652,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache
   xfer += oprot->writeFieldBegin("notNullConstraints", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->notNullConstraints)).size()));
-    std::vector<SQLNotNullConstraint> ::const_iterator _iter1311;
-    for (_iter1311 = (*(this->notNullConstraints)).begin(); _iter1311 != (*(this->notNullConstraints)).end(); ++_iter1311)
+    std::vector<SQLNotNullConstraint> ::const_iterator _iter1307;
+    for (_iter1307 = (*(this->notNullConstraints)).begin(); _iter1307 != (*(this->notNullConstraints)).end(); ++_iter1307)
     {
-      xfer += (*_iter1311).write(oprot);
+      xfer += (*_iter1307).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5664,10 +5664,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache
   xfer += oprot->writeFieldBegin("defaultConstraints", ::apache::thrift::protocol::T_LIST, 6);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->defaultConstraints)).size()));
-    std::vector<SQLDefaultConstraint> ::const_iterator _iter1312;
-    for (_iter1312 = (*(this->defaultConstraints)).begin(); _iter1312 != (*(this->defaultConstraints)).end(); ++_iter1312)
+    std::vector<SQLDefaultConstraint> ::const_iterator _iter1308;
+    for (_iter1308 = (*(this->defaultConstraints)).begin(); _iter1308 != (*(this->defaultConstraints)).end(); ++_iter1308)
     {
-      xfer += (*_iter1312).write(oprot);
+      xfer += (*_iter1308).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5676,10 +5676,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache
   xfer += oprot->writeFieldBegin("checkConstraints", ::apache::thrift::protocol::T_LIST, 7);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->checkConstraints)).size()));
-    std::vector<SQLCheckConstraint> ::const_iterator _iter1313;
-    for (_iter1313 = (*(this->checkConstraints)).begin(); _iter1313 != (*(this->checkConstraints)).end(); ++_iter1313)
+    std::vector<SQLCheckConstraint> ::const_iterator _iter1309;
+    for (_iter1309 = (*(this->checkConstraints)).begin(); _iter1309 != (*(this->checkConstraints)).end(); ++_iter1309)
     {
-      xfer += (*_iter1313).write(oprot);
+      xfer += (*_iter1309).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -7847,14 +7847,14 @@ uint32_t ThriftHiveMetastore_truncate_table_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partNames.clear();
-            uint32_t _size1314;
-            ::apache::thrift::protocol::TType _etype1317;
-            xfer += iprot->readListBegin(_etype1317, _size1314);
-            this->partNames.resize(_size1314);
-            uint32_t _i1318;
-            for (_i1318 = 0; _i1318 < _size1314; ++_i1318)
+            uint32_t _size1310;
+            ::apache::thrift::protocol::TType _etype1313;
+            xfer += iprot->readListBegin(_etype1313, _size1310);
+            this->partNames.resize(_size1310);
+            uint32_t _i1314;
+            for (_i1314 = 0; _i1314 < _size1310; ++_i1314)
             {
-              xfer += iprot->readString(this->partNames[_i1318]);
+              xfer += iprot->readString(this->partNames[_i1314]);
             }
             xfer += iprot->readListEnd();
           }
@@ -7891,10 +7891,10 @@ uint32_t ThriftHiveMetastore_truncate_table_args::write(::apache::thrift::protoc
   xfer += oprot->writeFieldBegin("partNames", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partNames.size()));
-    std::vector<std::string> ::const_iterator _iter1319;
-    for (_iter1319 = this->partNames.begin(); _iter1319 != this->partNames.end(); ++_iter1319)
+    std::vector<std::string> ::const_iterator _iter1315;
+    for (_iter1315 = this->partNames.begin(); _iter1315 != this->partNames.end(); ++_iter1315)
     {
-      xfer += oprot->writeString((*_iter1319));
+      xfer += oprot->writeString((*_iter1315));
     }
     xfer += oprot->writeListEnd();
   }
@@ -7926,10 +7926,10 @@ uint32_t ThriftHiveMetastore_truncate_table_pargs::write(::apache::thrift::proto
   xfer += oprot->writeFieldBegin("partNames", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->partNames)).size()));
-    std::vector<std::string> ::const_iterator _iter1320;
-    for (_iter1320 = (*(this->partNames)).begin(); _iter1320 != (*(this->partNames)).end(); ++_iter1320)
+    std::vector<std::string> ::const_iterator _iter1316;
+    for (_iter1316 = (*(this->partNames)).begin(); _iter1316 != (*(this->partNames)).end(); ++_iter1316)
     {
-      xfer += oprot->writeString((*_iter1320));
+      xfer += oprot->writeString((*_iter1316));
     }
     xfer += oprot->writeListEnd();
   }
@@ -8173,14 +8173,14 @@ uint32_t ThriftHiveMetastore_get_tables_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1321;
-            ::apache::thrift::protocol::TType _etype1324;
-            xfer += iprot->readListBegin(_etype1324, _size1321);
-            this->success.resize(_size1321);
-            uint32_t _i1325;
-            for (_i1325 = 0; _i1325 < _size1321; ++_i1325)
+            uint32_t _size1317;
+            ::apache::thrift::protocol::TType _etype1320;
+            xfer += iprot->readListBegin(_etype1320, _size1317);
+            this->success.resize(_size1317);
+            uint32_t _i1321;
+            for (_i1321 = 0; _i1321 < _size1317; ++_i1321)
             {
-              xfer += iprot->readString(this->success[_i1325]);
+              xfer += iprot->readString(this->success[_i1321]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8219,10 +8219,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 _iter1326;
-      for (_iter1326 = this->success.begin(); _iter1326 != this->success.end(); ++_iter1326)
+      std::vector<std::string> ::const_iterator _iter1322;
+      for (_iter1322 = this->success.begin(); _iter1322 != this->success.end(); ++_iter1322)
       {
-        xfer += oprot->writeString((*_iter1326));
+        xfer += oprot->writeString((*_iter1322));
       }
       xfer += oprot->writeListEnd();
     }
@@ -8267,14 +8267,14 @@ uint32_t ThriftHiveMetastore_get_tables_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1327;
-            ::apache::thrift::protocol::TType _etype1330;
-            xfer += iprot->readListBegin(_etype1330, _size1327);
-            (*(this->success)).resize(_size1327);
-            uint32_t _i1331;
-            for (_i1331 = 0; _i1331 < _size1327; ++_i1331)
+            uint32_t _size1323;
+            ::apache::thrift::protocol::TType _etype1326;
+            xfer += iprot->readListBegin(_etype1326, _size1323);
+            (*(this->success)).resize(_size1323);
+            uint32_t _i1327;
+            for (_i1327 = 0; _i1327 < _size1323; ++_i1327)
             {
-              xfer += iprot->readString((*(this->success))[_i1331]);
+              xfer += iprot->readString((*(this->success))[_i1327]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8444,14 +8444,14 @@ uint32_t ThriftHiveMetastore_get_tables_by_type_result::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1332;
-            ::apache::thrift::protocol::TType _etype1335;
-            xfer += iprot->readListBegin(_etype1335, _size1332);
-            this->success.resize(_size1332);
-            uint32_t _i1336;
-            for (_i1336 = 0; _i1336 < _size1332; ++_i1336)
+            uint32_t _size1328;
+            ::apache::thrift::protocol::TType _etype1331;
+            xfer += iprot->readListBegin(_etype1331, _size1328);
+            this->success.resize(_size1328);
+            uint32_t _i1332;
+            for (_i1332 = 0; _i1332 < _size1328; ++_i1332)
             {
-              xfer += iprot->readString(this->success[_i1336]);
+              xfer += iprot->readString(this->success[_i1332]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8490,10 +8490,10 @@ uint32_t ThriftHiveMetastore_get_tables_by_type_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 _iter1337;
-      for (_iter1337 = this->success.begin(); _iter1337 != this->success.end(); ++_iter1337)
+      std::vector<std::string> ::const_iterator _iter1333;
+      for (_iter1333 = this->success.begin(); _iter1333 != this->success.end(); ++_iter1333)
       {
-        xfer += oprot->writeString((*_iter1337));
+        xfer += oprot->writeString((*_iter1333));
       }
       xfer += oprot->writeListEnd();
     }
@@ -8538,14 +8538,14 @@ uint32_t ThriftHiveMetastore_get_tables_by_type_presult::read(::apache::thrift::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1338;
-            ::apache::thrift::protocol::TType _etype1341;
-            xfer += iprot->readListBegin(_etype1341, _size1338);
-            (*(this->success)).resize(_size1338);
-            uint32_t _i1342;
-            for (_i1342 = 0; _i1342 < _size1338; ++_i1342)
+            uint32_t _size1334;
+            ::apache::thrift::protocol::TType _etype1337;
+            xfer += iprot->readListBegin(_etype1337, _size1334);
+            (*(this->success)).resize(_size1334);
+            uint32_t _i1338;
+            for (_i1338 = 0; _i1338 < _size1334; ++_i1338)
             {
-              xfer += iprot->readString((*(this->success))[_i1342]);
+              xfer += iprot->readString((*(this->success))[_i1338]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8683,14 +8683,14 @@ uint32_t ThriftHiveMetastore_get_materialized_views_for_rewriting_result::read(:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1343;
-            ::apache::thrift::protocol::TType _etype1346;
-            xfer += iprot->readListBegin(_etype1346, _size1343);
-            this->success.resize(_size1343);
-            uint32_t _i1347;
-            for (_i1347 = 0; _i1347 < _size1343; ++_i1347)
+            uint32_t _size1339;
+            ::apache::thrift::protocol::TType _etype1342;
+            xfer += iprot->readListBegin(_etype1342, _size1339);
+            this->success.resize(_size1339);
+            uint32_t _i1343;
+            for (_i1343 = 0; _i1343 < _size1339; ++_i1343)
             {
-              xfer += iprot->readString(this->success[_i1347]);
+              xfer += iprot->readString(this->success[_i1343]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8729,10 +8729,10 @@ uint32_t ThriftHiveMetastore_get_materialized_views_for_rewriting_result::write(
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter1348;
-      for (_iter1348 = this->success.begin(); _iter1348 != this->success.end(); ++_iter1348)
+      std::vector<std::string> ::const_iterator _iter1344;
+      for (_iter1344 = this->success.begin(); _iter1344 != this->success.end(); ++_iter1344)
       {
-        xfer += oprot->writeString((*_iter1348));
+        xfer += oprot->writeString((*_iter1344));
       }
       xfer += oprot->writeListEnd();
     }
@@ -8777,14 +8777,14 @@ uint32_t ThriftHiveMetastore_get_materialized_views_for_rewriting_presult::read(
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1349;
-            ::apache::thrift::protocol::TType _etype1352;
-            xfer += iprot->readListBegin(_etype1352, _size1349);
-            (*(this->success)).resize(_size1349);
-            uint32_t _i1353;
-            for (_i1353 = 0; _i1353 < _size1349; ++_i1353)
+            uint32_t _size1345;
+            ::apache::thrift::protocol::TType _etype1348;
+            xfer += iprot->readListBegin(_etype1348, _size1345);
+            (*(this->success)).resize(_size1345);
+            uint32_t _i1349;
+            for (_i1349 = 0; _i1349 < _size1345; ++_i1349)
             {
-              xfer += iprot->readString((*(this->success))[_i1353]);
+              xfer += iprot->readString((*(this->success))[_i1349]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8859,14 +8859,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 _size1354;
-            ::apache::thrift::protocol::TType _etype1357;
-            xfer += iprot->readListBegin(_etype1357, _size1354);
-            this->tbl_types.resize(_size1354);
-            uint32_t _i1358;
-            for (_i1358 = 0; _i1358 < _size1354; ++_i1358)
+            uint32_t _size1350;
+            ::apache::thrift::protocol::TType _etype1353;
+            xfer += iprot->readListBegin(_etype1353, _size1350);
+            this->tbl_types.resize(_size1350);
+            uint32_t _i1354;
+            for (_i1354 = 0; _i1354 < _size1350; ++_i1354)
             {
-              xfer += iprot->readString(this->tbl_types[_i1358]);
+              xfer += iprot->readString(this->tbl_types[_i1354]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8903,10 +8903,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 _iter1359;
-    for (_iter1359 = this->tbl_types.begin(); _iter1359 != this->tbl_types.end(); ++_iter1359)
+    std::vector<std::string> ::const_iterator _iter1355;
+    for (_iter1355 = this->tbl_types.begin(); _iter1355 != this->tbl_types.end(); ++_iter1355)
     {
-      xfer += oprot->writeString((*_iter1359));
+      xfer += oprot->writeString((*_iter1355));
     }
     xfer += oprot->writeListEnd();
   }
@@ -8938,10 +8938,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 _iter1360;
-    for (_iter1360 = (*(this->tbl_types)).begin(); _iter1360 != (*(this->tbl_types)).end(); ++_iter1360)
+    std::vector<std::string> ::const_iterator _iter1356;
+    for (_iter1356 = (*(this->tbl_types)).begin(); _iter1356 != (*(this->tbl_types)).end(); ++_iter1356)
     {
-      xfer += oprot->writeString((*_iter1360));
+      xfer += oprot->writeString((*_iter1356));
     }
     xfer += oprot->writeListEnd();
   }
@@ -8982,14 +8982,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1361;
-            ::apache::thrift::protocol::TType _etype1364;
-            xfer += iprot->readListBegin(_etype1364, _size1361);
-            this->success.resize(_size1361);
-            uint32_t _i1365;
-            for (_i1365 = 0; _i1365 < _size1361; ++_i1365)
+            uint32_t _size1357;
+            ::apache::thrift::protocol::TType _etype1360;
+            xfer += iprot->readListBegin(_etype1360, _size1357);
+            this->success.resize(_size1357);
+            uint32_t _i1361;
+            for (_i1361 = 0; _i1361 < _size1357; ++_i1361)
             {
-              xfer += this->success[_i1365].read(iprot);
+              xfer += this->success[_i1361].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -9028,10 +9028,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 _iter1366;
-      for (_iter1366 = this->success.begin(); _iter1366 != this->success.end(); ++_iter1366)
+      std::vector<TableMeta> ::const_iterator _iter1362;
+      for (_iter1362 = this->success.begin(); _iter1362 != this->success.end(); ++_iter1362)
       {
-        xfer += (*_iter1366).write(oprot);
+        xfer += (*_iter1362).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -9076,14 +9076,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1367;
-            ::apache::thrift::protocol::TType _etype1370;
-            xfer += iprot->readListBegin(_etype1370, _size1367);
-            (*(this->success)).resize(_size1367);
-            uint32_t _i1371;
-            for (_i1371 = 0; _i1371 < _size1367; ++_i1371)
+            uint32_t _size1363;
+            ::apache::thrift::protocol::TType _etype1366;
+            xfer += iprot->readListBegin(_etype1366, _size1363);
+            (*(this->success)).resize(_size1363);
+            uint32_t _i1367;
+            for (_i1367 = 0; _i1367 < _size1363; ++_i1367)
             {
-              xfer += (*(this->success))[_i1371].read(iprot);
+              xfer += (*(this->success))[_i1367].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -9221,14 +9221,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1372;
-            ::apache::thrift::protocol::TType _etype1375;
-            xfer += iprot->readListBegin(_etype1375, _size1372);
-            this->success.resize(_size1372);
-            uint32_t _i1376;
-            for (_i1376 = 0; _i1376 < _size1372; ++_i1376)
+            uint32_t _size1368;
+            ::apache::thrift::protocol::TType _etype1371;
+            xfer += iprot->readListBegin(_etype1371, _size1368);
+            this->success.resize(_size1368);
+            uint32_t _i1372;
+            for (_i1372 = 0; _i1372 < _size1368; ++_i1372)
             {
-              xfer += iprot->readString(this->success[_i1376]);
+              xfer += iprot->readString(this->success[_i1372]);
             }
             xfer += iprot->readListEnd();
           }
@@ -9267,10 +9267,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 _iter1377;
-      for (_iter1377 = this->success.begin(); _iter1377 != this->success.end(); ++_iter1377)
+      std::vector<std::string> ::const_iterator _iter1373;
+      for (_iter1373 = this->success.begin(); _iter1373 != this->success.end(); ++_iter1373)
       {
-        xfer += oprot->writeString((*_iter1377));
+        xfer += oprot->writeString((*_iter1373));
       }
       xfer += oprot->writeListEnd();
     }
@@ -9315,14 +9315,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1378;
-            ::apache::thrift::protocol::TType _etype1381;
-            xfer += iprot->readListBegin(_etype1381, _size1378);
-            (*(this->success)).resize(_size1378);
-            uint32_t _i1382;
-            for (_i1382 = 0; _i1382 < _size1378; ++_i1382)
+            uint32_t _size1374;
+            ::apache::thrift::protocol::TType _etype1377;
+            xfer += iprot->readListBegin(_etype1377, _size1374);
+            (*(this->success)).resize(_size1374);
+            uint32_t _i1378;
+            for (_i1378 = 0; _i1378 < _size1374; ++_i1378)
             {
-              xfer += iprot->readString((*(this->success))[_i1382]);
+              xfer += iprot->readString((*(this->success))[_i1378]);
             }
             xfer += iprot->readListEnd();
           }
@@ -9632,14 +9632,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 _size1383;
-            ::apache::thrift::protocol::TType _etype1386;
-            xfer += iprot->readListBegin(_etype1386, _size1383);
-            this->tbl_names.resize(_size1383);
-            uint32_t _i1387;
-            for (_i1387 = 0; _i1387 < _size1383; ++_i1387)
+            uint32_t _size1379;
+            ::apache::thrift::protocol::TType _etype1382;
+            xfer += iprot->readListBegin(_etype1382, _size1379);
+            this->tbl_names.resize(_size1379);
+            uint32_t _i1383;
+            for (_i1383 = 0; _i1383 < _size1379; ++_i1383)
             {
-              xfer += iprot->readString(this->tbl_names[_i1387]);
+              xfer += iprot->readString(this->tbl_names[_i1383]);
             }
             xfer += iprot->readListEnd();
           }
@@ -9672,10 +9672,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 _iter1388;
-    for (_iter1388 = this->tbl_names.begin(); _iter1388 != this->tbl_names.end(); ++_iter1388)
+    std::vector<std::string> ::const_iterator _iter1384;
+    for (_iter1384 = this->tbl_names.begin(); _iter1384 != this->tbl_names.end(); ++_iter1384)
     {
-      xfer += oprot->writeString((*_iter1388));
+      xfer += oprot->writeString((*_iter1384));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9703,10 +9703,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 _iter1389;
-    for (_iter1389 = (*(this->tbl_names)).begin(); _iter1389 != (*(this->tbl_names)).end(); ++_iter1389)
+    std::vector<std::string> ::const_iterator _iter1385;
+    for (_iter1385 = (*(this->tbl_names)).begin(); _iter1385 != (*(this->tbl_names)).end(); ++_iter1385)
     {
-      xfer += oprot->writeString((*_iter1389));
+      xfer += oprot->writeString((*_iter1385));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9747,14 +9747,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 _size1390;
-            ::apache::thrift::protocol::TType _etype1393;
-            xfer += iprot->readListBegin(_etype1393, _size1390);
-            this->success.resize(_size1390);
-            uint32_t _i1394;
-            for (_i1394 = 0; _i1394 < _size1390; ++_i1394)
+            uint32_t _size1386;
+            ::apache::thrift::protocol::TType _etype1389;
+            xfer += iprot->readListBegin(_etype1389, _size1386);
+            this->success.resize(_size1386);
+            uint32_t _i1390;
+            for (_i1390 = 0; _i1390 < _size1386; ++_i1390)
             {
-              xfer += this->success[_i1394].read(iprot);
+              xfer += this->success[_i1390].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -9785,10 +9785,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 _iter1395;
-      for (_iter1395 = this->success.begin(); _iter1395 != this->success.end(); ++_iter1395)
+      std::vector<Table> ::const_iterator _iter1391;
+      for (_iter1391 = this->success.begin(); _iter1391 != this->success.end(); ++_iter1391)
       {
-        xfer += (*_iter1395).write(oprot);
+        xfer += (*_iter1391).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -9829,14 +9829,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 _size1396;
-            ::apache::thrift::protocol::TType _etype1399;
-            xfer += iprot->readListBegin(_etype1399, _size1396);
-            (*(this->success)).resize(_size1396);
-            uint32_t _i1400;
-            for (_i1400 = 0; _i1400 < _size1396; ++_i1400)
+            uint32_t _size1392;
+            ::apache::thrift::protocol::TType _etype1395;
+            xfer += iprot->readListBegin(_etype1395, _size1392);
+            (*(this->success)).resize(_size1392);
+            uint32_t _i1396;
+            for (_i1396 = 0; _i1396 < _size1392; ++_i1396)
             {
-              xfer += (*(this->success))[_i1400].read(iprot);
+              xfer += (*(this->success))[_i1396].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -10369,14 +10369,14 @@ uint32_t ThriftHiveMetastore_get_materialization_invalidation_info_args::read(::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->tbl_names.clear();
-            uint32_t _size1401;
-            ::apache::thrift::protocol::TType _etype1404;
-            xfer += iprot->readListBegin(_etype1404, _size1401);
-            this->tbl_names.resize(_size1401);
-            uint32_t _i1405;
-            for (_i1405 = 0; _i1405 < _size1401; ++_i1405)
+            uint32_t _size1397;
+            ::apache::thrift::protocol::TType _etype1400;
+            xfer += iprot->readListBegin(_etype1400, _size1397);
+            this->tbl_names.resize(_size1397);
+            uint32_t _i1401;
+            for (_i1401 = 0; _i1401 < _size1397; ++_i1401)
             {
-              xfer += iprot->readString(this->tbl_names[_i1405]);
+              xfer += iprot->readString(this->tbl_names[_i1401]);
             }
             xfer += iprot->readListEnd();
           }
@@ -10409,10 +10409,10 @@ uint32_t ThriftHiveMetastore_get_materialization_invalidation_info_args::write(:
   xfer += oprot->writeFieldBegin("tbl_names", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->tbl_names.size()));
-    std::vector<std::string> ::const_iterator _iter1406;
-    for (_iter1406 = this->tbl_names.begin(); _iter1406 != this->tbl_names.end(); ++_iter1406)
+    std::vector<std::string> ::const_iterator _iter1402;
+    for (_iter1402 = this->tbl_names.begin(); _iter1402 != this->tbl_names.end(); ++_iter1402)
     {
-      xfer += oprot->writeString((*_iter1406));
+      xfer += oprot->writeString((*_iter1402));
     }
     xfer += oprot->writeListEnd();
   }
@@ -10440,10 +10440,10 @@ uint32_t ThriftHiveMetastore_get_materialization_invalidation_info_pargs::write(
   xfer += oprot->writeFieldBegin("tbl_names", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->tbl_names)).size()));
-    std::vector<std::string> ::const_iterator _iter1407;
-    for (_iter1407 = (*(this->tbl_names)).begin(); _iter1407 != (*(this->tbl_names)).end(); ++_iter1407)
+    std::vector<std::string> ::const_iterator _iter1403;
+    for (_iter1403 = (*(this->tbl_names)).begin(); _iter1403 != (*(this->tbl_names)).end(); ++_iter1403)
     {
-      xfer += oprot->writeString((*_iter1407));
+      xfer += oprot->writeString((*_iter1403));
     }
     xfer += oprot->writeListEnd();
   }
@@ -10484,17 +10484,17 @@ uint32_t ThriftHiveMetastore_get_materialization_invalidation_info_result::read(
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->success.clear();
-            uint32_t _size1408;
-            ::apache::thrift::protocol::TType _ktype1409;
-            ::apache::thrift::protocol::TType _vtype1410;
-            xfer += iprot->readMapBegin(_ktype1409, _vtype1410, _size1408);
-            uint32_t _i1412;
-            for (_i1412 = 0; _i1412 < _size1408; ++_i1412)
+            uint32_t _size1404;
+            ::apache::thrift::protocol::TType _ktype1405;
+            ::apache::thrift::protocol::TType _vtype1406;
+            xfer += iprot->readMapBegin(_ktype1405, _vtype1406, _size1404);
+            uint32_t _i1408;
+            for (_i1408 = 0; _i1408 < _size1404; ++_i1408)
             {
-              std::string _key1413;
-              xfer += iprot->readString(_key1413);
-              Materialization& _val1414 = this->success[_key1413];
-              xfer += _val1414.read(iprot);
+              std::string _key1409;
+              xfer += iprot->readString(_key1409);
+              Materialization& _val1410 = this->success[_key1409];
+              xfer += _val1410.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -10549,11 +10549,11 @@ uint32_t ThriftHiveMetastore_get_materialization_invalidation_info_result::write
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_MAP, 0);
     {
       xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::map<std::string, Materialization> ::const_iterator _iter1415;
-      for (_iter1415 = this->success.begin(); _iter1415 != this->success.end(); ++_iter1415)
+      std::map<std::string, Materialization> ::const_iterator _iter1411;
+      for (_iter1411 = this->success.begin(); _iter1411 != this->success.end(); ++_iter1411)
       {
-        xfer += oprot->writeString(_iter1415->first);
-        xfer += _iter1415->second.write(oprot);
+        xfer += oprot->writeString(_iter1411->first);
+        xfer += _iter1411->second.write(oprot);
       }
       xfer += oprot->writeMapEnd();
     }
@@ -10606,17 +10606,17 @@ uint32_t ThriftHiveMetastore_get_materialization_invalidation_info_presult::read
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             (*(this->success)).clear();
-            uint32_t _size1416;
-            ::apache::thrift::protocol::TType _ktype1417;
-            ::apache::thrift::protocol::TType _vtype1418;
-            xfer += iprot->readMapBegin(_ktype1417, _vtype1418, _size1416);
-            uint32_t _i1420;
-            for (_i1420 = 0; _i1420 < _size1416; ++_i1420)
+            uint32_t _size1412;
+            ::apache::thrift::protocol::TType _ktype1413;
+            ::apache::thrift::protocol::TType _vtype1414;
+            xfer += iprot->readMapBegin(_ktype1413, _vtype1414, _size1412);
+            uint32_t _i1416;
+            for (_i1416 = 0; _i1416 < _size1412; ++_i1416)
             {
-              std::string _key1421;
-              xfer += iprot->readString(_key1421);
-              Materialization& _val1422 = (*(this->success))[_key1421];
-              xfer += _val1422.read(iprot);
+              std::string _key1417;
+              xfer += iprot->readString(_key1417);
+              Materialization& _val1418 = (*(this->success))[_key1417];
+              xfer += _val1418.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -11077,14 +11077,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 _size1423;
-            ::apache::thrift::protocol::TType _etype1426;
-            xfer += iprot->readListBegin(_etype1426, _size1423);
-            this->success.resize(_size1423);
-            uint32_t _i1427;
-            for (_i1427 = 0; _i1427 < _size1423; ++_i1427)
+            uint32_t _size1419;
+            ::apache::thrift::protocol::TType _etype1422;
+            xfer += iprot->readListBegin(_etype1422, _size1419);
+            this->success.resize(_size1419);
+            uint32_t _i1423;
+            for (_i1423 = 0; _i1423 < _size1419; ++_i1423)
             {
-              xfer += iprot->readString(this->success[_i1427]);
+              xfer += iprot->readString(this->success[_i1423]);
             }
             xfer += iprot->readListEnd();
           }
@@ -11139,10 +11139,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 _iter1428;
-      for (_iter1428 = this->success.begin(); _iter1428 != this->success.end(); ++_iter1428)
+      std::vector<std::string> ::const_iterator _iter1424;
+      for (_iter1424 = this->success.begin(); _iter1424 != this->success.end(); ++_iter1424)
       {
-        xfer += oprot->writeString((*_iter1428));
+        xfer += oprot->writeString((*_iter1424));
       }
       xfer += oprot->writeListEnd();
     }
@@ -11195,14 +11195,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 _size1429;
-            ::apache::thrift::protocol::TType _etype1432;
-            xfer += iprot->readListBegin(_etype1432, _size1429);
-            (*(this->success)).resize(_size1429);
-            uint32_t _i1433;
-            for (_i1433 = 0; _i1433 < _size1429; ++_i1433)
+            uint32_t _size1425;
+            ::apache::thrift::protocol::TType _etype1428;
+            xfer += iprot->readListBegin(_etype1428, _size1425);
+            (*(this->success)).resize(_size1425);
+            uint32_t _i1429;
+            for (_i1429 = 0; _i1429 < _size1425; ++_i1429)
             {
-              xfer += iprot->readString((*(this->success))[_i1433]);
+              xfer += iprot->readString((*(this->success))[_i1429]);
             }
             xfer += iprot->readListEnd();
           }
@@ -12536,14 +12536,14 @@ uint32_t ThriftHiveMetastore_add_partitions_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->new_parts.clear();
-            uint32_t _size1434;
-            ::apache::thrift::protocol::TType _etype1437;
-            xfer += iprot->readListBegin(_etype1437, _size1434);
-            this->new_parts.resize(_size1434);
-            uint32_t _i1438;
-            for (_i1438 = 0; _i1438 < _size1434; ++_i1438)
+            uint32_t _size1430;
+            ::apache::thrift::protocol::TType _etype1433;
+            xfer += iprot->readListBegin(_etype1433, _size1430);
+            this->new_parts.resize(_size1430);
+            uint32_t _i1434;
+            for (_i1434 = 0; _i1434 < _size1430; ++_i1434)
             {
-              xfer += this->new_parts[_i1438].read(iprot);
+              xfer += this->new_parts[_i1434].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -12572,10 +12572,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 _iter1439;
-    for (_iter1439 = this->new_parts.begin(); _iter1439 != this->new_parts.end(); ++_iter1439)
+    std::vector<Partition> ::const_iterator _iter1435;
+    for (_iter1435 = this->new_parts.begin(); _iter1435 != this->new_parts.end(); ++_iter1435)
     {
-      xfer += (*_iter1439).write(oprot);
+      xfer += (*_iter1435).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -12599,10 +12599,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 _iter1440;
-    for (_iter1440 = (*(this->new_parts)).begin(); _iter1440 != (*(this->new_parts)).end(); ++_iter1440)
+    std::vector<Partition> ::const_iterator _iter1436;
+    for (_iter1436 = (*(this->new_parts)).begin(); _iter1436 != (*(this->new_parts)).end(); ++_iter1436)
     {
-      xfer += (*_iter1440).write(oprot);
+      xfer += (*_iter1436).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -12811,14 +12811,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 _size1441;
-            ::apache::thrift::protocol::TType _etype1444;
-            xfer += iprot->readListBegin(_etype1444, _size1441);
-            this->new_parts.resize(_size1441);
-            uint32_t _i1445;
-            for (_i1445 = 0; _i1445 < _size1441; ++_i1445)
+            uint32_t _size1437;
+            ::apache::thrift::protocol::TType _etype1440;
+            xfer += iprot->readListBegin(_etype1440, _size1437);
+            this->new_parts.resize(_size1437);
+            uint32_t _i1441;
+            for (_i1441 = 0; _i1441 < _size1437; ++_i1441)
             {
-              xfer += this->new_parts[_i1445].read(iprot);
+              xfer += this->new_parts[_i1441].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -12847,10 +12847,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 _iter1446;
-    for (_iter1446 = this->new_parts.begin(); _iter1446 != this->new_parts.end(); ++_iter1446)
+    std::vector<PartitionSpec> ::const_iterator _iter1442;
+    for (_iter1442 = this->new_parts.begin(); _iter1442 != this->new_parts.end(); ++_iter1442)
     {
-      xfer += (*_iter1446).write(oprot);
+      xfer += (*_iter1442).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -12874,10 +12874,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 _iter1447;
-    for (_iter1447 = (*(this->new_parts)).begin(); _iter1447 != (*(this->new_parts)).end(); ++_iter1447)
+    std::vector<PartitionSpec> ::const_iterator _iter1443;
+    for (_iter1443 = (*(this->new_parts)).begin(); _iter1443 != (*(this->new_parts)).end(); ++_iter1443)
     {
-      xfer += (*_iter1447).write(oprot);
+      xfer += (*_iter1443).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -13102,14 +13102,14 @@ uint32_t ThriftHiveMetastore_append_partition_args::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1448;
-            ::apache::thrift::protocol::TType _etype1451;
-            xfer += iprot->readListBegin(_etype1451, _size1448);
-            this->part_vals.resize(_size1448);
-            uint32_t _i1452;
-            for (_i1452 = 0; _i1452 < _size1448; ++_i1452)
+            uint32_t _size1444;
+            ::apache::thrift::protocol::TType _etype1447;
+            xfer += iprot->readListBegin(_etype1447, _size1444);
+            this->part_vals.resize(_size1444);
+            uint32_t _i1448;
+            for (_i1448 = 0; _i1448 < _size1444; ++_i1448)
             {
-              xfer += iprot->readString(this->part_vals[_i1452]);
+              xfer += iprot->readString(this->part_vals[_i1448]);
             }
             xfer += iprot->readListEnd();
           }
@@ -13146,10 +13146,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 _iter1453;
-    for (_iter1453 = this->part_vals.begin(); _iter1453 != this->part_vals.end(); ++_iter1453)
+    std::vector<std::string> ::const_iterator _iter1449;
+    for (_iter1449 = this->part_vals.begin(); _iter1449 != this->part_vals.end(); ++_iter1449)
     {
-      xfer += oprot->writeString((*_iter1453));
+      xfer += oprot->writeString((*_iter1449));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13181,10 +13181,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 _iter1454;
-    for (_iter1454 = (*(this->part_vals)).begin(); _iter1454 != (*(this->part_vals)).end(); ++_iter1454)
+    std::vector<std::string> ::const_iterator _iter1450;
+    for (_iter1450 = (*(this->part_vals)).begin(); _iter1450 != (*(this->part_vals)).end(); ++_iter1450)
     {
-      xfer += oprot->writeString((*_iter1454));
+      xfer += oprot->writeString((*_iter1450));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13656,14 +13656,14 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_args::rea
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1455;
-            ::apache::thrift::protocol::TType _etype1458;
-            xfer += iprot->readListBegin(_etype1458, _size1455);
-            this->part_vals.resize(_size1455);
-            uint32_t _i1459;
-            for (_i1459 = 0; _i1459 < _size1455; ++_i1459)
+            uint32_t _size1451;
+            ::apache::thrift::protocol::TType _etype1454;
+            xfer += iprot->readListBegin(_etype1454, _size1451);
+            this->part_vals.resize(_size1451);
+            uint32_t _i1455;
+            for (_i1455 = 0; _i1455 < _size1451; ++_i1455)
             {
-              xfer += iprot->readString(this->part_vals[_i1459]);
+              xfer += iprot->readString(this->part_vals[_i1455]);
             }
             xfer += iprot->readListEnd();
           }
@@ -13708,10 +13708,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 _iter1460;
-    for (_iter1460 = this->part_vals.begin(); _iter1460 != this->part_vals.end(); ++_iter1460)
+    std::vector<std::string> ::const_iterator _iter1456;
+    for (_iter1456 = this->part_vals.begin(); _iter1456 != this->part_vals.end(); ++_iter1456)
     {
-      xfer += oprot->writeString((*_iter1460));
+      xfer += oprot->writeString((*_iter1456));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13747,10 +13747,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 _iter1461;
-    for (_iter1461 = (*(this->part_vals)).begin(); _iter1461 != (*(this->part_vals)).end(); ++_iter1461)
+    std::vector<std::string> ::const_iterator _iter1457;
+    for (_iter1457 = (*(this->part_vals)).begin(); _iter1457 != (*(this->part_vals)).end(); ++_iter1457)
     {
-      xfer += oprot->writeString((*_iter1461));
+      xfer += oprot->writeString((*_iter1457));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14553,14 +14553,14 @@ uint32_t ThriftHiveMetastore_drop_partition_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1462;
-            ::apache::thrift::protocol::TType _etype1465;
-            xfer += iprot->readListBegin(_etype1465, _size1462);
-            this->part_vals.resize(_size1462);
-            uint32_t _i1466;
-            for (_i1466 = 0; _i1466 < _size1462; ++_i1466)
+            uint32_t _size1458;
+            ::apache::thrift::protocol::TType _etype1461;
+            xfer += iprot->readListBegin(_etype1461, _size1458);
+            this->part_vals.resize(_size1458);
+            uint32_t _i1462;
+            for (_i1462 = 0; _i1462 < _size1458; ++_i1462)
             {
-              xfer += iprot->readString(this->part_vals[_i1466]);
+              xfer += iprot->readString(this->part_vals[_i1462]);
             }
             xfer += iprot->readListEnd();
           }
@@ -14605,10 +14605,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 _iter1467;
-    for (_iter1467 = this->part_vals.begin(); _iter1467 != this->part_vals.end(); ++_iter1467)
+    std::vector<std::string> ::const_iterator _iter1463;
+    for (_iter1463 = this->part_vals.begin(); _iter1463 != this->part_vals.end(); ++_iter1463)
     {
-      xfer += oprot->writeString((*_iter1467));
+      xfer += oprot->writeString((*_iter1463));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14644,10 +14644,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 _iter1468;
-    for (_iter1468 = (*(this->part_vals)).begin(); _iter1468 != (*(this->part_vals)).end(); ++_iter1468)
+    std::vector<std::string> ::const_iterator _iter1464;
+    for (_iter1464 = (*(this->part_vals)).begin(); _iter1464 != (*(this->part_vals)).end(); ++_iter1464)
     {
-      xfer += oprot->writeString((*_iter1468));
+      xfer += oprot->writeString((*_iter1464));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14856,14 +14856,14 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_args::read(
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1469;
-            ::apache::thrift::protocol::TType _etype1472;
-            xfer += iprot->readListBegin(_etype1472, _size1469);
-            this->part_vals.resize(_size1469);
-            uint32_t _i1473;
-            for (_i1473 = 0; _i1473 < _size1469; ++_i1473)
+            uint32_t _size1465;
+            ::apache::thrift::protocol::TType _etype1468;
+            xfer += iprot->readListBegin(_etype1468, _size1465);
+            this->part_vals.resize(_size1465);
+            uint32_t _i1469;
+            for (_i1469 = 0; _i1469 < _size1465; ++_i1469)
             {
-              xfer += iprot->readString(this->part_vals[_i1473]);
+              xfer += iprot->readString(this->part_vals[_i1469]);
             }
             xfer += iprot->readListEnd();
           }
@@ -14916,10 +14916,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 _iter1474;
-    for (_iter1474 = this->part_vals.begin(); _iter1474 != this->part_vals.end(); ++_iter1474)
+    std::vector<std::string> ::const_iterator _iter1470;
+    for (_iter1470 = this->part_vals.begin(); _iter1470 != this->part_vals.end(); ++_iter1470)
     {
-      xfer += oprot->writeString((*_iter1474));
+      xfer += oprot->writeString((*_iter1470));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14959,10 +14959,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 _iter1475;
-    for (_iter1475 = (*(this->part_vals)).begin(); _iter1475 != (*(this->part_vals)).end(); ++_iter1475)
+    std::vector<std::string> ::const_iterator _iter1471;
+    for (_iter1471 = (*(this->part_vals)).begin(); _iter1471 != (*(this->part_vals)).end(); ++_iter1471)
     {
-      xfer += oprot->writeString((*_iter1475));
+      xfer += oprot->writeString((*_iter1471));
     }
     xfer += oprot->writeListEnd();
   }
@@ -15968,14 +15968,14 @@ uint32_t ThriftHiveMetastore_get_partition_args::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1476;
-            ::apache::thrift::protocol::TType _etype1479;
-            xfer += iprot->readListBegin(_etype1479, _size1476);
-            this->part_vals.resize(_size1476);
-            uint32_t _i1480;
-            for (_i1480 = 0; _i1480 < _size1476; ++_i1480)
+            uint32_t _size1472;
+            ::apache::thrift::protocol::TType _etype1475;
+            xfer += iprot->readListBegin(_etype1475, _size1472);
+            this->part_vals.resize(_size1472);
+            uint32_t _i1476;
+            for (_i1476 = 0; _i1476 < _size1472; ++_i1476)
             {
-              xfer += iprot->readString(this->part_vals[_i1480]);
+              xfer += iprot->readString(this->part_vals[_i1476]);
             }
             xfer += iprot->readListEnd();
           }
@@ -16012,10 +16012,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 _iter1481;
-    for (_iter1481 = this->part_vals.begin(); _iter1481 != this->part_vals.end(); ++_iter1481)
+    std::vector<std::string> ::const_iterator _iter1477;
+    for (_iter1477 = this->part_vals.begin(); _iter1477 != this->part_vals.end(); ++_iter1477)
     {
-      xfer += oprot->writeString((*_iter1481));
+      xfer += oprot->writeString((*_iter1477));
     }
     xfer += oprot->writeListEnd();
   }
@@ -16047,10 +16047,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 _iter1482;
-    for (_iter1482 = (*(this->part_vals)).begin(); _iter1482 != (*(this->part_vals)).end(); ++_iter1482)
+    std::vector<std::string> ::const_iterator _iter1478;
+    for (_iter1478 = (*(this->part_vals)).begin(); _iter1478 != (*(this->part_vals)).end(); ++_iter1478)
     {
-      xfer += oprot->writeString((*_iter1482));
+      xfer += oprot->writeString((*_iter1478));
     }
     xfer += oprot->writeListEnd();
   }


<TRUNCATED>