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

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

Repository: hive
Updated Branches:
  refs/heads/master 4f67bebe1 -> b3e2d8a05


http://git-wip-us.apache.org/repos/asf/hive/blob/b3e2d8a0/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 125d5a7..184ecb6 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,6 +55,7 @@ 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;
@@ -83,7 +84,6 @@ 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,6 +124,7 @@ 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;
@@ -186,6 +187,7 @@ import org.apache.hadoop.hive.metastore.model.MPartitionPrivilege;
 import org.apache.hadoop.hive.metastore.model.MResourceUri;
 import org.apache.hadoop.hive.metastore.model.MRole;
 import org.apache.hadoop.hive.metastore.model.MRoleMap;
+import org.apache.hadoop.hive.metastore.model.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;
@@ -210,7 +212,6 @@ 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;
@@ -809,7 +810,9 @@ public class ObjectStore implements RawStore, Configurable {
       pm.makePersistent(mCat);
       committed = commitTransaction();
     } finally {
-      if (!committed) rollbackTransaction();
+      if (!committed) {
+        rollbackTransaction();
+      }
     }
   }
 
@@ -832,7 +835,9 @@ public class ObjectStore implements RawStore, Configurable {
       pm.makePersistent(mCat);
       committed = commitTransaction();
     } finally {
-      if (!committed) rollbackTransaction();
+      if (!committed) {
+        rollbackTransaction();
+      }
     }
   }
 
@@ -840,7 +845,9 @@ 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);
   }
 
@@ -874,11 +881,15 @@ 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();
+      }
     }
   }
 
@@ -903,14 +914,18 @@ 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;
   }
 
@@ -1983,10 +1998,18 @@ 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;
   }
 
@@ -3679,7 +3702,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 {
@@ -9998,7 +10021,9 @@ public class ObjectStore implements RawStore, Configurable {
       pm.makePersistent(mSchema);
       committed = commitTransaction();
     } finally {
-      if (!committed) rollbackTransaction();
+      if (!committed) {
+        rollbackTransaction();
+      }
     }
   }
 
@@ -10017,11 +10042,17 @@ 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();
+      }
     }
   }
 
@@ -10035,7 +10066,9 @@ public class ObjectStore implements RawStore, Configurable {
       committed = commitTransaction();
       return schema;
     } finally {
-      if (!committed) rollbackTransaction();
+      if (!committed) {
+        rollbackTransaction();
+      }
     }
   }
 
@@ -10054,7 +10087,9 @@ public class ObjectStore implements RawStore, Configurable {
       pm.retrieve(mSchema);
       return mSchema;
     } finally {
-      if (query != null) query.closeAll();
+      if (query != null) {
+        query.closeAll();
+      }
     }
   }
 
@@ -10071,7 +10106,9 @@ public class ObjectStore implements RawStore, Configurable {
       }
       committed = commitTransaction();
     } finally {
-      if (!committed) rollbackTransaction();
+      if (!committed) {
+        rollbackTransaction();
+      }
     }
   }
 
@@ -10096,7 +10133,9 @@ public class ObjectStore implements RawStore, Configurable {
       pm.makePersistent(mSchemaVersion);
       committed = commitTransaction();
     } finally {
-      if (!committed) rollbackTransaction();;
+      if (!committed) {
+        rollbackTransaction();
+      }
     }
   }
 
@@ -10113,11 +10152,17 @@ 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();
+      }
     }
   }
 
@@ -10132,7 +10177,9 @@ public class ObjectStore implements RawStore, Configurable {
       committed = commitTransaction();
       return schemaVersion;
     } finally {
-      if (!committed) rollbackTransaction();;
+      if (!committed) {
+        rollbackTransaction();
+      }
     }
   }
 
@@ -10152,11 +10199,15 @@ 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();
+      }
     }
   }
 
@@ -10180,7 +10231,9 @@ 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();
@@ -10206,11 +10259,15 @@ 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();
@@ -10232,8 +10289,12 @@ 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 ");
@@ -10259,12 +10320,16 @@ 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();
@@ -10291,7 +10356,9 @@ public class ObjectStore implements RawStore, Configurable {
       }
       committed = commitTransaction();
     } finally {
-      if (!committed) rollbackTransaction();
+      if (!committed) {
+        rollbackTransaction();
+      }
     }
   }
 
@@ -10308,7 +10375,9 @@ public class ObjectStore implements RawStore, Configurable {
       committed = commitTransaction();
       return serde;
     } finally {
-      if (!committed) rollbackTransaction();;
+      if (!committed) {
+        rollbackTransaction();
+      }
     }
   }
 
@@ -10322,7 +10391,9 @@ public class ObjectStore implements RawStore, Configurable {
       pm.retrieve(mSerDeInfo);
       return mSerDeInfo;
     } finally {
-      if (query != null) query.closeAll();
+      if (query != null) {
+        query.closeAll();
+      }
     }
   }
 
@@ -10338,7 +10409,9 @@ public class ObjectStore implements RawStore, Configurable {
       pm.makePersistent(mSerde);
       committed = commitTransaction();
     } finally {
-      if (!committed) rollbackTransaction();
+      if (!committed) {
+        rollbackTransaction();
+      }
     }
 
   }
@@ -10355,7 +10428,9 @@ 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(),
@@ -10363,8 +10438,12 @@ 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;
   }
 
@@ -10385,19 +10464,33 @@ 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;
   }
 
@@ -11507,4 +11600,61 @@ 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/b3e2d8a0/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 f6c46ee..2c9f2e5 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java
@@ -18,7 +18,6 @@
 
 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;
@@ -38,7 +37,6 @@ 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;
@@ -46,7 +44,6 @@ 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;
@@ -66,8 +63,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;
@@ -81,7 +78,6 @@ import org.apache.hadoop.hive.metastore.api.Type;
 import org.apache.hadoop.hive.metastore.api.UnknownDBException;
 import org.apache.hadoop.hive.metastore.api.UnknownPartitionException;
 import org.apache.hadoop.hive.metastore.api.UnknownTableException;
-import org.apache.hadoop.hive.metastore.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;
@@ -1629,4 +1625,13 @@ 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/b3e2d8a0/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
new file mode 100644
index 0000000..202058e
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RuntimeStatsCleanerTask.java
@@ -0,0 +1,67 @@
+/*
+ * 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/b3e2d8a0/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 ebdcbc2..92d000b 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,6 +89,7 @@ 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;
@@ -220,8 +221,9 @@ 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);
@@ -2175,6 +2177,7 @@ public class CachedStore implements RawStore, Configurable {
     return rawStore.addNotNullConstraints(nns);
   }
 
+  @Override
   public List<String> addDefaultConstraints(List<SQLDefaultConstraint> nns)
       throws InvalidObjectException, MetaException {
     // TODO constraintCache
@@ -2195,6 +2198,7 @@ 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);
@@ -2466,4 +2470,19 @@ 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/b3e2d8a0/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 59749e4..552eeca 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,6 +24,7 @@ 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;
@@ -577,6 +578,15 @@ 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",
@@ -732,10 +742,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() + "," +
+        EventCleanerTask.class.getName() + "," + RuntimeStatsCleanerTask.class.getName() + "," +
         "org.apache.hadoop.hive.metastore.repl.DumpDirCleanerTask" + "," +
         MaterializationsCacheCleanerTask.class.getName() + "," +
-        MaterializationsRebuildLockCleanerTask.class.getName(),
+            MaterializationsRebuildLockCleanerTask.class.getName() + "," + RuntimeStatsCleanerTask.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()),
@@ -1127,16 +1137,22 @@ 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.
@@ -1268,8 +1284,12 @@ 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/b3e2d8a0/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
new file mode 100644
index 0000000..054ce7c
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MRuntimeStat.java
@@ -0,0 +1,59 @@
+/*
+ * 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/b3e2d8a0/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 9ddf598..221192e 100644
--- a/standalone-metastore/src/main/resources/package.jdo
+++ b/standalone-metastore/src/main/resources/package.jdo
@@ -1339,6 +1339,20 @@
         <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/b3e2d8a0/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 adfa4c5..48d28cb 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,6 +676,16 @@ 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/b3e2d8a0/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 a75b740..ed6c4cd 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,6 +238,16 @@ 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/b3e2d8a0/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 91c581c..6e31b16 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,6 +1230,15 @@ 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/b3e2d8a0/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 87f5884..c2504d3 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,6 +306,15 @@ 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/b3e2d8a0/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 7e2a57a..4309911 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,6 +1154,16 @@ 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/b3e2d8a0/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 5ba68ca..e01b4da 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,6 +279,15 @@ 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/b3e2d8a0/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 f9e1a19..a45c7bb 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,6 +1124,16 @@ 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/b3e2d8a0/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 a769d24..327800b 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,6 +297,15 @@ 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/b3e2d8a0/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 6fed072..2484744 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,6 +1811,17 @@ 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/b3e2d8a0/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 7b6b3b7..63932a9 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,6 +314,15 @@ 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/b3e2d8a0/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 12e4e40..c56a4f9 100644
--- a/standalone-metastore/src/main/thrift/hive_metastore.thrift
+++ b/standalone-metastore/src/main/thrift/hive_metastore.thrift
@@ -1517,6 +1517,15 @@ struct GetSerdeRequest {
   1: string serdeName
 }
 
+struct RuntimeStat {
+  1: optional i32 createTime,
+  2: required i32 weight,
+  3: required binary payload
+}
+
+struct GetRuntimeStatsRequest {
+}
+
 // Exceptions.
 
 exception MetaException {
@@ -2171,6 +2180,9 @@ 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/b3e2d8a0/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 304f567..defc68f 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,6 +66,7 @@ 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;
@@ -280,6 +281,7 @@ 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);
   }
@@ -1092,6 +1094,7 @@ public class DummyRawStoreControlledCommit implements RawStore, Configurable {
     return null;
   }
 
+  @Override
   public void createISchema(ISchema schema) throws AlreadyExistsException, MetaException,
       NoSuchObjectException {
     objectStore.createISchema(schema);
@@ -1161,4 +1164,19 @@ 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/b3e2d8a0/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 85c6727..20c5d8a 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,6 +64,7 @@ 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;
@@ -276,6 +277,7 @@ public class DummyRawStoreForJdoConnection implements RawStore {
       throws MetaException {
   }
 
+  @Override
   public List<String> getTables(String catName, String dbName, String pattern) throws MetaException {
     return Collections.emptyList();
   }
@@ -1080,6 +1082,7 @@ public class DummyRawStoreForJdoConnection implements RawStore {
     return null;
   }
 
+  @Override
   public void createISchema(ISchema schema) throws AlreadyExistsException, MetaException {
 
   }
@@ -1148,4 +1151,18 @@ 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/b3e2d8a0/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 cb51763..bf87cfc 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,4 +3352,14 @@ 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/b3e2d8a0/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
new file mode 100644
index 0000000..2db7a8b
--- /dev/null
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/client/TestRuntimeStats.java
@@ -0,0 +1,100 @@
+/**
+ * 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;
+  }
+
+}


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

Posted by ha...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/b3e2d8a0/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 9c94942..1c1d58e 100644
--- a/standalone-metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
+++ b/standalone-metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
@@ -1528,6 +1528,17 @@ 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 {
@@ -13007,6 +13018,111 @@ 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
@@ -58498,4 +58614,369 @@ 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/b3e2d8a0/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 1625788..c9ebfaf 100644
--- a/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php
+++ b/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php
@@ -29722,6 +29722,177 @@ 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/b3e2d8a0/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 079c7fc..a231e9c 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,6 +226,8 @@ 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()')
@@ -1507,6 +1509,18 @@ 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/b3e2d8a0/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 d241414..d94951b 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,6 +1566,20 @@ 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):
   """
@@ -8799,6 +8813,70 @@ 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):
@@ -9005,6 +9083,8 @@ 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()
@@ -14017,6 +14097,50 @@ 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
 
@@ -47720,3 +47844,288 @@ 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/b3e2d8a0/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 1d09cb8..b1e577a 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,6 +21236,147 @@ 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/b3e2d8a0/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 47e8d0f..2687ce5 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,6 +4825,43 @@ 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/b3e2d8a0/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 58ebd29..4de8bd3 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,6 +3378,37 @@ 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 
@@ -5919,6 +5950,28 @@ 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
@@ -13417,5 +13470,71 @@ 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/b3e2d8a0/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 cd50e1b..1450e29 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,7 +50,6 @@ 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;
@@ -1237,7 +1236,9 @@ 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())) {
@@ -1753,7 +1754,9 @@ 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();
@@ -3394,10 +3397,14 @@ 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());
@@ -3429,7 +3436,9 @@ 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();
@@ -3740,7 +3749,9 @@ 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());
@@ -4748,7 +4759,9 @@ 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;
@@ -4821,7 +4834,9 @@ 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],
@@ -4926,7 +4941,9 @@ 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;
@@ -7995,6 +8012,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       }
     }
 
+    @Override
     public WMAlterPoolResponse alter_wm_pool(WMAlterPoolRequest request)
         throws AlreadyExistsException, NoSuchObjectException, InvalidObjectException, MetaException,
         TException {
@@ -8020,6 +8038,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       }
     }
 
+    @Override
     public WMDropPoolResponse drop_wm_pool(WMDropPoolRequest request)
         throws NoSuchObjectException, InvalidOperationException, MetaException, TException {
       try {
@@ -8031,6 +8050,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       }
     }
 
+    @Override
     public WMCreateOrUpdateMappingResponse create_or_update_wm_mapping(
         WMCreateOrUpdateMappingRequest request) throws AlreadyExistsException,
         NoSuchObjectException, InvalidObjectException, MetaException, TException {
@@ -8043,6 +8063,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       }
     }
 
+    @Override
     public WMDropMappingResponse drop_wm_mapping(WMDropMappingRequest request)
         throws NoSuchObjectException, InvalidOperationException, MetaException, TException {
       try {
@@ -8054,6 +8075,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       }
     }
 
+    @Override
     public WMCreateOrDropTriggerToPoolMappingResponse create_or_drop_wm_trigger_to_pool_mapping(
         WMCreateOrDropTriggerToPoolMappingRequest request) throws AlreadyExistsException,
         NoSuchObjectException, InvalidObjectException, MetaException, TException {
@@ -8072,6 +8094,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       }
     }
 
+    @Override
     public void create_ischema(ISchema schema) throws TException {
       startFunction("create_ischema", ": " + schema.getName());
       boolean success = false;
@@ -8091,7 +8114,9 @@ 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,
@@ -8130,7 +8155,9 @@ 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,
@@ -8195,7 +8222,9 @@ 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,
@@ -8235,7 +8264,9 @@ 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,
@@ -8337,7 +8368,9 @@ 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,
@@ -8406,7 +8439,9 @@ 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,
@@ -8448,7 +8483,9 @@ 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,
@@ -8479,7 +8516,9 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         ex = e;
         throw e;
       } finally {
-        if (!success) ms.rollbackTransaction();
+        if (!success) {
+          ms.rollbackTransaction();
+        }
         endFunction("create_serde", success, ex);
       }
     }
@@ -8515,6 +8554,44 @@ 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/b3e2d8a0/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 feae991..1c8d223 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,7 +649,9 @@ 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);
   }
@@ -697,7 +699,9 @@ 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());
   }
 
@@ -800,7 +804,9 @@ 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);
   }
 
@@ -819,7 +825,9 @@ 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);
@@ -844,6 +852,7 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
     }
   }
 
+  @Override
   public void createTableWithConstraints(Table tbl,
     List<SQLPrimaryKey> primaryKeys, List<SQLForeignKey> foreignKeys,
     List<SQLUniqueConstraint> uniqueConstraints,
@@ -856,12 +865,24 @@ 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) {
@@ -995,6 +1016,7 @@ 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 {
@@ -1521,7 +1543,9 @@ 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);
   }
 
@@ -1826,55 +1850,71 @@ 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);
   }
 
@@ -1882,7 +1922,9 @@ 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);
   }
@@ -2196,7 +2238,9 @@ 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);
@@ -2229,7 +2273,9 @@ 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);
@@ -2246,7 +2292,9 @@ 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);
   }
 
@@ -2254,7 +2302,9 @@ 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);
   }
 
@@ -2627,14 +2677,18 @@ 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);
   }
 
@@ -2707,7 +2761,9 @@ 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);
   }
 
@@ -3087,8 +3143,11 @@ 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);
   }
 
@@ -3109,10 +3168,13 @@ 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));
   }
@@ -3162,9 +3224,13 @@ 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
@@ -3176,4 +3242,14 @@ 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/b3e2d8a0/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 27f8775..aee416d 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,6 +94,7 @@ 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;
@@ -1891,6 +1892,7 @@ 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;
@@ -3641,4 +3643,11 @@ 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;
+
 }


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

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

Signed-off-by: Ashutosh Chauhan <ha...@apache.org>


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

Branch: refs/heads/master
Commit: b3e2d8a05f57a91b12b8347b2763a296c3480d97
Parents: 4f67beb
Author: Zoltan Haindrich <ki...@rxd.hu>
Authored: Mon Apr 23 13:36:11 2018 -0700
Committer: Ashutosh Chauhan <ha...@apache.org>
Committed: Mon Apr 23 13:36:11 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, 9936 insertions(+), 2963 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/b3e2d8a0/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 2403d7a..f40c606 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"),
+            + "  hiveserver: runtime statistics are persisted in the hiveserver - all sessions share it\n"
+            + "  metastore: runtime statistics are persisted in the metastore as well"),
 
     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"),
-
+        "Size of the runtime statistics cache. Unit is: OperatorStat entry; a query plan consist ~100. See also: runtime.stats.max.entries"),
 
     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/b3e2d8a0/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 801de7a..8ecbaad 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,6 +69,7 @@ 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;
@@ -295,6 +296,7 @@ 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) {
@@ -1126,6 +1128,7 @@ public class DummyRawStoreFailEvent implements RawStore, Configurable {
     return null;
   }
 
+  @Override
   public void createISchema(ISchema schema) throws AlreadyExistsException, MetaException,
       NoSuchObjectException {
     objectStore.createISchema(schema);
@@ -1195,4 +1198,20 @@ 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/b3e2d8a0/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 88022be..750fc69 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.clearAllStats();
+    StatsSources.clearGlobalStats();
   }
 
   protected void clearSettingsCreatedInTests() throws IOException {

http://git-wip-us.apache.org/repos/asf/hive/blob/b3e2d8a0/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
new file mode 100644
index 0000000..ef6c77b
--- /dev/null
+++ b/metastore/scripts/upgrade/derby/056-HIVE-19171.derby.sql
@@ -0,0 +1,10 @@
+
+
+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/b3e2d8a0/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 e87bbce..f626bd5 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,18 +25,28 @@ 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).
  */
-public class OpSignature {
+@JsonIdentityInfo(generator = ObjectIdGenerators.IntSequenceGenerator.class, property = "@id")
+public final 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?
@@ -70,7 +80,7 @@ public 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) {
@@ -103,4 +113,9 @@ public class OpSignature {
     }
     return sb.toString();
   }
+
+  public Map<String, Object> getSigMap() {
+    return sigMap;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/b3e2d8a0/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 c3dc848..f774158 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,14 +24,28 @@ 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.
  */
-public class OpTreeSignature {
+@JsonIdentityInfo(generator = ObjectIdGenerators.IntSequenceGenerator.class, property = "@id")
+public final class OpTreeSignature {
+
+  @JsonProperty
   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<>();
@@ -82,4 +96,12 @@ public 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/b3e2d8a0/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 3df5ee9..80a3edf 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 {
 
-  public OpTreeSignature getSignature(Operator<? extends OperatorDesc> op);
+  OpTreeSignature getSignature(Operator<? extends OperatorDesc> op);
 
-  static final OpTreeSignatureFactory DIRECT = new Direct();
+  OpTreeSignatureFactory DIRECT = new Direct();
 
-  public static OpTreeSignatureFactory direct() {
+  static OpTreeSignatureFactory direct() {
     return DIRECT;
   }
 
-  public static OpTreeSignatureFactory newCache() {
+  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"
 
-  static class Direct implements OpTreeSignatureFactory {
+  class Direct implements OpTreeSignatureFactory {
 
     @Override
     public OpTreeSignature getSignature(Operator<? extends OperatorDesc> op) {
@@ -53,7 +53,7 @@ public interface OpTreeSignatureFactory {
 
   }
 
-  static class CachedFactory implements OpTreeSignatureFactory {
+  class CachedFactory implements OpTreeSignatureFactory {
 
     Map<Operator<? extends OperatorDesc>, OpTreeSignature> cache = new IdentityHashMap<>();
 

http://git-wip-us.apache.org/repos/asf/hive/blob/b3e2d8a0/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
new file mode 100644
index 0000000..195a8b1
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/signature/RuntimeStatsMap.java
@@ -0,0 +1,83 @@
+/*
+ * 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/b3e2d8a0/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
new file mode 100644
index 0000000..696fe1f
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/signature/RuntimeStatsPersister.java
@@ -0,0 +1,54 @@
+/*
+ * 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/b3e2d8a0/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 4f3e338..f599d33 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,17 +23,15 @@ 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 class SignatureUtils {
+public final class SignatureUtils {
 
   private static Map<Class<?>, SignatureMapper> mappers = new HashMap<>();
 
@@ -42,28 +40,24 @@ public class SignatureUtils {
     mapper.write(ret, o);
   }
 
-  static class SignatureMapper {
+  /** Prevent construction. */
+  private SignatureUtils() {
+  }
 
-    static final Set<String> acceptedSignatureTypes = Sets.newHashSet();
+  static class SignatureMapper {
 
     private List<Method> sigMethods;
 
     private String classLabel;
 
-    public SignatureMapper(Class<?> o) {
+    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/b3e2d8a0/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 e15a49f..fcb6de7 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,11 +192,15 @@ 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;
   }
@@ -216,7 +220,6 @@ 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/b3e2d8a0/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 a61a47e..d71ba5b 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<Byte, String> getKeysString() {
-    Map<Byte, String> keyMap = new LinkedHashMap<Byte, String>();
+  public Map<String, String> getKeysString() {
+    Map<String, String> keyMap = new LinkedHashMap<>();
     for (Map.Entry<Byte, List<ExprNodeDesc>> k: getKeys().entrySet()) {
-      keyMap.put(k.getKey(), PlanUtils.getExprListString(k.getValue()));
+      keyMap.put(String.valueOf(k.getKey()), PlanUtils.getExprListString(k.getValue()));
     }
     return keyMap;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/b3e2d8a0/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 e7ca7f6..95990b8 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<Byte, String> getKeysString() {
+  public Map<String, String> getKeysString() {
     if (joinKeys == null) {
       return null;
     }
 
-    Map<Byte, String> keyMap = new LinkedHashMap<Byte, String>();
+    Map<String, String> keyMap = new LinkedHashMap<String, String>();
     for (byte i = 0; i < joinKeys.length; i++) {
-      keyMap.put(i, PlanUtils.getExprListString(Arrays.asList(joinKeys[i])));
+      keyMap.put(String.valueOf(i), PlanUtils.getExprListString(Arrays.asList(joinKeys[i])));
     }
     return keyMap;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/b3e2d8a0/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 54b705d..dc4f085 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<Byte, String> getKeysString() {
-    Map<Byte, String> keyMap = new LinkedHashMap<Byte, String>();
+  public Map<String, String> getKeysString() {
+    Map<String, String> keyMap = new LinkedHashMap<>();
     for (Map.Entry<Byte, List<ExprNodeDesc>> k: getKeys().entrySet()) {
-      keyMap.put(k.getKey(), PlanUtils.getExprListString(k.getValue()));
+      keyMap.put(String.valueOf(k.getKey()), PlanUtils.getExprListString(k.getValue()));
     }
     return keyMap;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/b3e2d8a0/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 c515276..2841638 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,8 +22,6 @@ 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;
@@ -36,9 +34,8 @@ public class CachingStatsSource implements StatsSource {
 
   private final Cache<OpTreeSignature, OperatorStats> cache;
 
-  public CachingStatsSource(HiveConf conf) {
-    int size = conf.getIntVar(ConfVars.HIVE_QUERY_REEXECUTION_STATS_CACHE_SIZE);
-    cache = CacheBuilder.newBuilder().maximumSize(size).build();
+  public CachingStatsSource(int cacheSize) {
+    cache = CacheBuilder.newBuilder().maximumSize(cacheSize).build();
   }
 
   public void put(OpTreeSignature sig, OperatorStats opStat) {

http://git-wip-us.apache.org/repos/asf/hive/blob/b3e2d8a0/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 19df13a..624f107 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 class EmptyStatsSource implements StatsSource {
+public final class EmptyStatsSource implements StatsSource {
 
   public static StatsSource INSTANCE = new EmptyStatsSource();
 

http://git-wip-us.apache.org/repos/asf/hive/blob/b3e2d8a0/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
new file mode 100644
index 0000000..237c1cc
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/MetastoreStatsConnector.java
@@ -0,0 +1,143 @@
+/*
+ * 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/b3e2d8a0/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 a372804..e932304 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,7 +18,9 @@
 
 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;
@@ -34,6 +36,7 @@ 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.
@@ -43,7 +46,106 @@ import com.google.common.annotations.VisibleForTesting;
 public class PlanMapper {
 
   Set<EquivGroup> groups = new HashSet<>();
-  private Map<Object, EquivGroup> objectMap = new HashMap<>();
+  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());
+    }
+
+  }
 
   /**
    * A set of objects which are representing the same thing.
@@ -55,7 +157,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<>();
@@ -116,7 +218,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/b3e2d8a0/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 3d6c257..fb2b5f8 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,48 +18,33 @@
 
 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 SimpleRuntimeStatsSource implements StatsSource {
+public class MapBackedStatsSource implements StatsSource {
 
-  private final PlanMapper pm;
-
-
-  public SimpleRuntimeStatsSource(PlanMapper pm) {
-    this.pm = pm;
-  }
+  private Map<OpTreeSignature, OperatorStats> map = new ConcurrentHashMap<>();
 
   @Override
-  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();
+  public boolean canProvideStatsFor(Class<?> clazz) {
+    if (Operator.class.isAssignableFrom(clazz)) {
+      return true;
     }
+    return false;
   }
 
   @Override
-  public boolean canProvideStatsFor(Class<?> class1) {
-    if (Operator.class.isAssignableFrom(class1)) {
-      return true;
-    }
-    return false;
+  public Optional<OperatorStats> lookup(OpTreeSignature treeSig) {
+    return Optional.ofNullable(map.get(treeSig));
   }
 
   @Override
   public void putAll(Map<OpTreeSignature, OperatorStats> map) {
-    throw new RuntimeException();
+    this.map.putAll(map);
   }
-
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/b3e2d8a0/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 a4e33c3..30b6a30 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,14 +18,12 @@
 
 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.ql.exec.Operator;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 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;
@@ -33,53 +31,50 @@ 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 {
 
-  public static class MapBackedStatsSource implements StatsSource {
-
-    private Map<OpTreeSignature, OperatorStats> map = new HashMap<>();
+  private static final Logger LOG = LoggerFactory.getLogger(StatsSources.class);
 
-    @Override
-    public boolean canProvideStatsFor(Class<?> clazz) {
-      if (Operator.class.isAssignableFrom(clazz)) {
-        return true;
-      }
-      return false;
-    }
+  static enum StatsSourceMode {
+    query, hiveserver, metastore;
+  }
 
-    @Override
-    public Optional<OperatorStats> lookup(OpTreeSignature treeSig) {
-      return Optional.ofNullable(map.get(treeSig));
-    }
+  public static void initialize(HiveConf hiveConf) {
+    // requesting for the stats source will implicitly initialize it
+    getStatsSource(hiveConf);
+  }
 
-    @Override
-    public void putAll(Map<OpTreeSignature, OperatorStats> map) {
-      map.putAll(map);
+  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);
     }
-
   }
 
-  private static final Logger LOG = LoggerFactory.getLogger(StatsSources.class);
-
   public static StatsSource getStatsSourceContaining(StatsSource currentStatsSource, PlanMapper pm) {
-    if (currentStatsSource instanceof CachingStatsSource) {
-      CachingStatsSource sessionStatsSource = (CachingStatsSource) currentStatsSource;
-      loadFromPlanMapper(sessionStatsSource, pm);
-      return sessionStatsSource;
-    } else {
-      return new SimpleRuntimeStatsSource(pm);
+    StatsSource statsSource = currentStatsSource;
+    if (currentStatsSource  == EmptyStatsSource.INSTANCE) {
+      statsSource = new MapBackedStatsSource();
     }
-  }
 
-  public static void loadFromPlanMapper(CachingStatsSource sessionStatsSource, PlanMapper pm) {
-    Map<OpTreeSignature, OperatorStats> map = extractStatMapFromPlanMapper(pm);
-    sessionStatsSource.putAll(map);
+    Map<OpTreeSignature, OperatorStats> statMap = extractStatMapFromPlanMapper(pm);
+    statsSource.putAll(statMap);
+    return statsSource;
   }
 
-
   private static Map<OpTreeSignature, OperatorStats> extractStatMapFromPlanMapper(PlanMapper pm) {
-    Map<OpTreeSignature, OperatorStats> map = new HashMap<OpTreeSignature, OperatorStats>();
+    Builder<OpTreeSignature, OperatorStats> map = ImmutableMap.builder();
     Iterator<EquivGroup> it = pm.iterateGroups();
     while (it.hasNext()) {
       EquivGroup e = it.next();
@@ -103,20 +98,33 @@ public class StatsSources {
         map.put(sig.get(0), stat.get(0));
       }
     }
-    return map;
+    return map.build();
   }
 
   private static StatsSource globalStatsSource;
+  private static MetastoreStatsConnector metastoreStatsConnector;
 
-  public static StatsSource globalStatsSource(HiveConf conf) {
+  public static StatsSource globalStatsSource(int cacheSize) {
     if (globalStatsSource == null) {
-      globalStatsSource = new CachingStatsSource(conf);
+      globalStatsSource = new CachingStatsSource(cacheSize);
     }
     return globalStatsSource;
   }
 
+  public static StatsSource metastoreBackedStatsSource(StatsSource parent) {
+    if (metastoreStatsConnector == null) {
+      metastoreStatsConnector = new MetastoreStatsConnector(parent);
+    }
+    return metastoreStatsConnector;
+  }
+
   @VisibleForTesting
-  public static void clearAllStats() {
+  public static void clearGlobalStats() {
+    if (metastoreStatsConnector != null) {
+      metastoreStatsConnector.destroy();
+    }
     globalStatsSource = null;
+    metastoreStatsConnector = null;
   }
+
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/b3e2d8a0/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 409cc73..8dc7387 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,22 +84,7 @@ public class ReOptimizePlugin implements IReExecutionPlugin {
     alwaysCollectStats = driver.getConf().getBoolVar(ConfVars.HIVE_QUERY_REEXECUTION_ALWAYS_COLLECT_OPERATOR_STATS);
     statsReaderHook.setCollectOnSuccess(alwaysCollectStats);
 
-    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);
+    coreDriver.setStatsSource(StatsSources.getStatsSource(driver.getConf()));
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/b3e2d8a0/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 52e18a8..d70bb82 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,7 +6,9 @@
  * 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.
@@ -15,10 +17,20 @@
  */
 package org.apache.hadoop.hive.ql.stats;
 
-public class OperatorStats {
-  private final String operatorId;
+import com.google.common.base.Objects;
+
+/**
+ * Holds information an operator's statistics.
+ */
+public final class OperatorStats {
+  private String operatorId;
   private long outputRecords;
 
+  // for jackson
+  @SuppressWarnings("unused")
+  private OperatorStats() {
+  }
+
   public OperatorStats(final String opId) {
     this.operatorId = opId;
     this.outputRecords = -1;
@@ -40,4 +52,19 @@ public 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/b3e2d8a0/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
new file mode 100644
index 0000000..627c2d8
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/optimizer/signature/TestRuntimeStatsPersistence.java
@@ -0,0 +1,165 @@
+/*
+ * 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/b3e2d8a0/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 8126970..e8a7a1b 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,10 +33,11 @@ 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.SimpleRuntimeStatsSource;
-import org.apache.hadoop.hive.ql.reexec.ReExecDriver;
+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.reexec.ReExecDriver;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.ql.stats.OperatorStats;
 import org.apache.hadoop.hive.ql.stats.OperatorStatsReaderHook;
@@ -129,7 +130,7 @@ public class TestCounterMapping {
     FilterOperator filter1 = filters1.get(0);
 
     driver = createDriver();
-    ((ReExecDriver) driver).setStatsSource(new SimpleRuntimeStatsSource(pm1));
+    ((ReExecDriver) driver).setStatsSource(StatsSources.getStatsSourceContaining(EmptyStatsSource.INSTANCE, pm1));
 
     PlanMapper pm2 = getMapperForQuery(driver, query);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/b3e2d8a0/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 b726300..8bec56f 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,15 +27,18 @@ 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;
@@ -55,7 +58,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)",
@@ -78,8 +81,13 @@ public class TestReOptimization {
     dropTables(driver);
   }
 
+  @After
+  public void after() {
+    StatsSources.clearGlobalStats();
+  }
+
   public static void dropTables(IDriver driver) throws Exception {
-    String tables[] = { "tu", "tv", "tw" };
+    String[] tables = new String[] {"tu", "tv", "tw" };
     for (String t : tables) {
       int ret = driver.run("drop table if exists " + t).getResponseCode();
       assertEquals("Checking command success", 0, ret);
@@ -98,7 +106,9 @@ 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();
@@ -133,7 +143,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";
-    PlanMapper pm = getMapperForQuery(driver, query);
+    getMapperForQuery(driver, query);
 
   }
 
@@ -143,8 +153,72 @@ 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
@@ -152,7 +226,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";
-    PlanMapper pm = getMapperForQuery(driver, query);
+    getMapperForQuery(driver, query);
     List<String> res = new ArrayList<>();
     List<String> res1 = new ArrayList<>();
     while (driver.getResults(res1)) {
@@ -165,6 +239,7 @@ 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/b3e2d8a0/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 1642357..e373628 100644
--- a/service/src/java/org/apache/hive/service/server/HiveServer2.java
+++ b/service/src/java/org/apache/hive/service/server/HiveServer2.java
@@ -81,6 +81,7 @@ 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;
@@ -247,6 +248,8 @@ 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 {


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

Posted by ha...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/b3e2d8a0/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 dfa13a0..4787703 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 _size1187;
-            ::apache::thrift::protocol::TType _etype1190;
-            xfer += iprot->readListBegin(_etype1190, _size1187);
-            this->success.resize(_size1187);
-            uint32_t _i1191;
-            for (_i1191 = 0; _i1191 < _size1187; ++_i1191)
+            uint32_t _size1191;
+            ::apache::thrift::protocol::TType _etype1194;
+            xfer += iprot->readListBegin(_etype1194, _size1191);
+            this->success.resize(_size1191);
+            uint32_t _i1195;
+            for (_i1195 = 0; _i1195 < _size1191; ++_i1195)
             {
-              xfer += iprot->readString(this->success[_i1191]);
+              xfer += iprot->readString(this->success[_i1195]);
             }
             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 _iter1192;
-      for (_iter1192 = this->success.begin(); _iter1192 != this->success.end(); ++_iter1192)
+      std::vector<std::string> ::const_iterator _iter1196;
+      for (_iter1196 = this->success.begin(); _iter1196 != this->success.end(); ++_iter1196)
       {
-        xfer += oprot->writeString((*_iter1192));
+        xfer += oprot->writeString((*_iter1196));
       }
       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 _size1193;
-            ::apache::thrift::protocol::TType _etype1196;
-            xfer += iprot->readListBegin(_etype1196, _size1193);
-            (*(this->success)).resize(_size1193);
-            uint32_t _i1197;
-            for (_i1197 = 0; _i1197 < _size1193; ++_i1197)
+            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)
             {
-              xfer += iprot->readString((*(this->success))[_i1197]);
+              xfer += iprot->readString((*(this->success))[_i1201]);
             }
             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 _size1198;
-            ::apache::thrift::protocol::TType _etype1201;
-            xfer += iprot->readListBegin(_etype1201, _size1198);
-            this->success.resize(_size1198);
-            uint32_t _i1202;
-            for (_i1202 = 0; _i1202 < _size1198; ++_i1202)
+            uint32_t _size1202;
+            ::apache::thrift::protocol::TType _etype1205;
+            xfer += iprot->readListBegin(_etype1205, _size1202);
+            this->success.resize(_size1202);
+            uint32_t _i1206;
+            for (_i1206 = 0; _i1206 < _size1202; ++_i1206)
             {
-              xfer += iprot->readString(this->success[_i1202]);
+              xfer += iprot->readString(this->success[_i1206]);
             }
             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 _iter1203;
-      for (_iter1203 = this->success.begin(); _iter1203 != this->success.end(); ++_iter1203)
+      std::vector<std::string> ::const_iterator _iter1207;
+      for (_iter1207 = this->success.begin(); _iter1207 != this->success.end(); ++_iter1207)
       {
-        xfer += oprot->writeString((*_iter1203));
+        xfer += oprot->writeString((*_iter1207));
       }
       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 _size1204;
-            ::apache::thrift::protocol::TType _etype1207;
-            xfer += iprot->readListBegin(_etype1207, _size1204);
-            (*(this->success)).resize(_size1204);
-            uint32_t _i1208;
-            for (_i1208 = 0; _i1208 < _size1204; ++_i1208)
+            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)
             {
-              xfer += iprot->readString((*(this->success))[_i1208]);
+              xfer += iprot->readString((*(this->success))[_i1212]);
             }
             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 _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)
+            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)
             {
-              std::string _key1214;
-              xfer += iprot->readString(_key1214);
-              Type& _val1215 = this->success[_key1214];
-              xfer += _val1215.read(iprot);
+              std::string _key1218;
+              xfer += iprot->readString(_key1218);
+              Type& _val1219 = this->success[_key1218];
+              xfer += _val1219.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 _iter1216;
-      for (_iter1216 = this->success.begin(); _iter1216 != this->success.end(); ++_iter1216)
+      std::map<std::string, Type> ::const_iterator _iter1220;
+      for (_iter1220 = this->success.begin(); _iter1220 != this->success.end(); ++_iter1220)
       {
-        xfer += oprot->writeString(_iter1216->first);
-        xfer += _iter1216->second.write(oprot);
+        xfer += oprot->writeString(_iter1220->first);
+        xfer += _iter1220->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 _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)
+            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)
             {
-              std::string _key1222;
-              xfer += iprot->readString(_key1222);
-              Type& _val1223 = (*(this->success))[_key1222];
-              xfer += _val1223.read(iprot);
+              std::string _key1226;
+              xfer += iprot->readString(_key1226);
+              Type& _val1227 = (*(this->success))[_key1226];
+              xfer += _val1227.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 _size1224;
-            ::apache::thrift::protocol::TType _etype1227;
-            xfer += iprot->readListBegin(_etype1227, _size1224);
-            this->success.resize(_size1224);
-            uint32_t _i1228;
-            for (_i1228 = 0; _i1228 < _size1224; ++_i1228)
+            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)
             {
-              xfer += this->success[_i1228].read(iprot);
+              xfer += this->success[_i1232].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 _iter1229;
-      for (_iter1229 = this->success.begin(); _iter1229 != this->success.end(); ++_iter1229)
+      std::vector<FieldSchema> ::const_iterator _iter1233;
+      for (_iter1233 = this->success.begin(); _iter1233 != this->success.end(); ++_iter1233)
       {
-        xfer += (*_iter1229).write(oprot);
+        xfer += (*_iter1233).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 _size1230;
-            ::apache::thrift::protocol::TType _etype1233;
-            xfer += iprot->readListBegin(_etype1233, _size1230);
-            (*(this->success)).resize(_size1230);
-            uint32_t _i1234;
-            for (_i1234 = 0; _i1234 < _size1230; ++_i1234)
+            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)
             {
-              xfer += (*(this->success))[_i1234].read(iprot);
+              xfer += (*(this->success))[_i1238].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 _size1235;
-            ::apache::thrift::protocol::TType _etype1238;
-            xfer += iprot->readListBegin(_etype1238, _size1235);
-            this->success.resize(_size1235);
-            uint32_t _i1239;
-            for (_i1239 = 0; _i1239 < _size1235; ++_i1239)
+            uint32_t _size1239;
+            ::apache::thrift::protocol::TType _etype1242;
+            xfer += iprot->readListBegin(_etype1242, _size1239);
+            this->success.resize(_size1239);
+            uint32_t _i1243;
+            for (_i1243 = 0; _i1243 < _size1239; ++_i1243)
             {
-              xfer += this->success[_i1239].read(iprot);
+              xfer += this->success[_i1243].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 _iter1240;
-      for (_iter1240 = this->success.begin(); _iter1240 != this->success.end(); ++_iter1240)
+      std::vector<FieldSchema> ::const_iterator _iter1244;
+      for (_iter1244 = this->success.begin(); _iter1244 != this->success.end(); ++_iter1244)
       {
-        xfer += (*_iter1240).write(oprot);
+        xfer += (*_iter1244).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 _size1241;
-            ::apache::thrift::protocol::TType _etype1244;
-            xfer += iprot->readListBegin(_etype1244, _size1241);
-            (*(this->success)).resize(_size1241);
-            uint32_t _i1245;
-            for (_i1245 = 0; _i1245 < _size1241; ++_i1245)
+            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)
             {
-              xfer += (*(this->success))[_i1245].read(iprot);
+              xfer += (*(this->success))[_i1249].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 _size1246;
-            ::apache::thrift::protocol::TType _etype1249;
-            xfer += iprot->readListBegin(_etype1249, _size1246);
-            this->success.resize(_size1246);
-            uint32_t _i1250;
-            for (_i1250 = 0; _i1250 < _size1246; ++_i1250)
+            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)
             {
-              xfer += this->success[_i1250].read(iprot);
+              xfer += this->success[_i1254].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 _iter1251;
-      for (_iter1251 = this->success.begin(); _iter1251 != this->success.end(); ++_iter1251)
+      std::vector<FieldSchema> ::const_iterator _iter1255;
+      for (_iter1255 = this->success.begin(); _iter1255 != this->success.end(); ++_iter1255)
       {
-        xfer += (*_iter1251).write(oprot);
+        xfer += (*_iter1255).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 _size1252;
-            ::apache::thrift::protocol::TType _etype1255;
-            xfer += iprot->readListBegin(_etype1255, _size1252);
-            (*(this->success)).resize(_size1252);
-            uint32_t _i1256;
-            for (_i1256 = 0; _i1256 < _size1252; ++_i1256)
+            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)
             {
-              xfer += (*(this->success))[_i1256].read(iprot);
+              xfer += (*(this->success))[_i1260].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 _size1257;
-            ::apache::thrift::protocol::TType _etype1260;
-            xfer += iprot->readListBegin(_etype1260, _size1257);
-            this->success.resize(_size1257);
-            uint32_t _i1261;
-            for (_i1261 = 0; _i1261 < _size1257; ++_i1261)
+            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)
             {
-              xfer += this->success[_i1261].read(iprot);
+              xfer += this->success[_i1265].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 _iter1262;
-      for (_iter1262 = this->success.begin(); _iter1262 != this->success.end(); ++_iter1262)
+      std::vector<FieldSchema> ::const_iterator _iter1266;
+      for (_iter1266 = this->success.begin(); _iter1266 != this->success.end(); ++_iter1266)
       {
-        xfer += (*_iter1262).write(oprot);
+        xfer += (*_iter1266).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 _size1263;
-            ::apache::thrift::protocol::TType _etype1266;
-            xfer += iprot->readListBegin(_etype1266, _size1263);
-            (*(this->success)).resize(_size1263);
-            uint32_t _i1267;
-            for (_i1267 = 0; _i1267 < _size1263; ++_i1267)
+            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)
             {
-              xfer += (*(this->success))[_i1267].read(iprot);
+              xfer += (*(this->success))[_i1271].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 _size1268;
-            ::apache::thrift::protocol::TType _etype1271;
-            xfer += iprot->readListBegin(_etype1271, _size1268);
-            this->primaryKeys.resize(_size1268);
-            uint32_t _i1272;
-            for (_i1272 = 0; _i1272 < _size1268; ++_i1272)
+            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)
             {
-              xfer += this->primaryKeys[_i1272].read(iprot);
+              xfer += this->primaryKeys[_i1276].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 _size1273;
-            ::apache::thrift::protocol::TType _etype1276;
-            xfer += iprot->readListBegin(_etype1276, _size1273);
-            this->foreignKeys.resize(_size1273);
-            uint32_t _i1277;
-            for (_i1277 = 0; _i1277 < _size1273; ++_i1277)
+            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)
             {
-              xfer += this->foreignKeys[_i1277].read(iprot);
+              xfer += this->foreignKeys[_i1281].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 _size1278;
-            ::apache::thrift::protocol::TType _etype1281;
-            xfer += iprot->readListBegin(_etype1281, _size1278);
-            this->uniqueConstraints.resize(_size1278);
-            uint32_t _i1282;
-            for (_i1282 = 0; _i1282 < _size1278; ++_i1282)
+            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)
             {
-              xfer += this->uniqueConstraints[_i1282].read(iprot);
+              xfer += this->uniqueConstraints[_i1286].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 _size1283;
-            ::apache::thrift::protocol::TType _etype1286;
-            xfer += iprot->readListBegin(_etype1286, _size1283);
-            this->notNullConstraints.resize(_size1283);
-            uint32_t _i1287;
-            for (_i1287 = 0; _i1287 < _size1283; ++_i1287)
+            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)
             {
-              xfer += this->notNullConstraints[_i1287].read(iprot);
+              xfer += this->notNullConstraints[_i1291].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 _size1288;
-            ::apache::thrift::protocol::TType _etype1291;
-            xfer += iprot->readListBegin(_etype1291, _size1288);
-            this->defaultConstraints.resize(_size1288);
-            uint32_t _i1292;
-            for (_i1292 = 0; _i1292 < _size1288; ++_i1292)
+            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)
             {
-              xfer += this->defaultConstraints[_i1292].read(iprot);
+              xfer += this->defaultConstraints[_i1296].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 _size1293;
-            ::apache::thrift::protocol::TType _etype1296;
-            xfer += iprot->readListBegin(_etype1296, _size1293);
-            this->checkConstraints.resize(_size1293);
-            uint32_t _i1297;
-            for (_i1297 = 0; _i1297 < _size1293; ++_i1297)
+            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)
             {
-              xfer += this->checkConstraints[_i1297].read(iprot);
+              xfer += this->checkConstraints[_i1301].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 _iter1298;
-    for (_iter1298 = this->primaryKeys.begin(); _iter1298 != this->primaryKeys.end(); ++_iter1298)
+    std::vector<SQLPrimaryKey> ::const_iterator _iter1302;
+    for (_iter1302 = this->primaryKeys.begin(); _iter1302 != this->primaryKeys.end(); ++_iter1302)
     {
-      xfer += (*_iter1298).write(oprot);
+      xfer += (*_iter1302).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 _iter1299;
-    for (_iter1299 = this->foreignKeys.begin(); _iter1299 != this->foreignKeys.end(); ++_iter1299)
+    std::vector<SQLForeignKey> ::const_iterator _iter1303;
+    for (_iter1303 = this->foreignKeys.begin(); _iter1303 != this->foreignKeys.end(); ++_iter1303)
     {
-      xfer += (*_iter1299).write(oprot);
+      xfer += (*_iter1303).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 _iter1300;
-    for (_iter1300 = this->uniqueConstraints.begin(); _iter1300 != this->uniqueConstraints.end(); ++_iter1300)
+    std::vector<SQLUniqueConstraint> ::const_iterator _iter1304;
+    for (_iter1304 = this->uniqueConstraints.begin(); _iter1304 != this->uniqueConstraints.end(); ++_iter1304)
     {
-      xfer += (*_iter1300).write(oprot);
+      xfer += (*_iter1304).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 _iter1301;
-    for (_iter1301 = this->notNullConstraints.begin(); _iter1301 != this->notNullConstraints.end(); ++_iter1301)
+    std::vector<SQLNotNullConstraint> ::const_iterator _iter1305;
+    for (_iter1305 = this->notNullConstraints.begin(); _iter1305 != this->notNullConstraints.end(); ++_iter1305)
     {
-      xfer += (*_iter1301).write(oprot);
+      xfer += (*_iter1305).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 _iter1302;
-    for (_iter1302 = this->defaultConstraints.begin(); _iter1302 != this->defaultConstraints.end(); ++_iter1302)
+    std::vector<SQLDefaultConstraint> ::const_iterator _iter1306;
+    for (_iter1306 = this->defaultConstraints.begin(); _iter1306 != this->defaultConstraints.end(); ++_iter1306)
     {
-      xfer += (*_iter1302).write(oprot);
+      xfer += (*_iter1306).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 _iter1303;
-    for (_iter1303 = this->checkConstraints.begin(); _iter1303 != this->checkConstraints.end(); ++_iter1303)
+    std::vector<SQLCheckConstraint> ::const_iterator _iter1307;
+    for (_iter1307 = this->checkConstraints.begin(); _iter1307 != this->checkConstraints.end(); ++_iter1307)
     {
-      xfer += (*_iter1303).write(oprot);
+      xfer += (*_iter1307).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 _iter1304;
-    for (_iter1304 = (*(this->primaryKeys)).begin(); _iter1304 != (*(this->primaryKeys)).end(); ++_iter1304)
+    std::vector<SQLPrimaryKey> ::const_iterator _iter1308;
+    for (_iter1308 = (*(this->primaryKeys)).begin(); _iter1308 != (*(this->primaryKeys)).end(); ++_iter1308)
     {
-      xfer += (*_iter1304).write(oprot);
+      xfer += (*_iter1308).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 _iter1305;
-    for (_iter1305 = (*(this->foreignKeys)).begin(); _iter1305 != (*(this->foreignKeys)).end(); ++_iter1305)
+    std::vector<SQLForeignKey> ::const_iterator _iter1309;
+    for (_iter1309 = (*(this->foreignKeys)).begin(); _iter1309 != (*(this->foreignKeys)).end(); ++_iter1309)
     {
-      xfer += (*_iter1305).write(oprot);
+      xfer += (*_iter1309).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 _iter1306;
-    for (_iter1306 = (*(this->uniqueConstraints)).begin(); _iter1306 != (*(this->uniqueConstraints)).end(); ++_iter1306)
+    std::vector<SQLUniqueConstraint> ::const_iterator _iter1310;
+    for (_iter1310 = (*(this->uniqueConstraints)).begin(); _iter1310 != (*(this->uniqueConstraints)).end(); ++_iter1310)
     {
-      xfer += (*_iter1306).write(oprot);
+      xfer += (*_iter1310).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 _iter1307;
-    for (_iter1307 = (*(this->notNullConstraints)).begin(); _iter1307 != (*(this->notNullConstraints)).end(); ++_iter1307)
+    std::vector<SQLNotNullConstraint> ::const_iterator _iter1311;
+    for (_iter1311 = (*(this->notNullConstraints)).begin(); _iter1311 != (*(this->notNullConstraints)).end(); ++_iter1311)
     {
-      xfer += (*_iter1307).write(oprot);
+      xfer += (*_iter1311).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 _iter1308;
-    for (_iter1308 = (*(this->defaultConstraints)).begin(); _iter1308 != (*(this->defaultConstraints)).end(); ++_iter1308)
+    std::vector<SQLDefaultConstraint> ::const_iterator _iter1312;
+    for (_iter1312 = (*(this->defaultConstraints)).begin(); _iter1312 != (*(this->defaultConstraints)).end(); ++_iter1312)
     {
-      xfer += (*_iter1308).write(oprot);
+      xfer += (*_iter1312).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 _iter1309;
-    for (_iter1309 = (*(this->checkConstraints)).begin(); _iter1309 != (*(this->checkConstraints)).end(); ++_iter1309)
+    std::vector<SQLCheckConstraint> ::const_iterator _iter1313;
+    for (_iter1313 = (*(this->checkConstraints)).begin(); _iter1313 != (*(this->checkConstraints)).end(); ++_iter1313)
     {
-      xfer += (*_iter1309).write(oprot);
+      xfer += (*_iter1313).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 _size1310;
-            ::apache::thrift::protocol::TType _etype1313;
-            xfer += iprot->readListBegin(_etype1313, _size1310);
-            this->partNames.resize(_size1310);
-            uint32_t _i1314;
-            for (_i1314 = 0; _i1314 < _size1310; ++_i1314)
+            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)
             {
-              xfer += iprot->readString(this->partNames[_i1314]);
+              xfer += iprot->readString(this->partNames[_i1318]);
             }
             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 _iter1315;
-    for (_iter1315 = this->partNames.begin(); _iter1315 != this->partNames.end(); ++_iter1315)
+    std::vector<std::string> ::const_iterator _iter1319;
+    for (_iter1319 = this->partNames.begin(); _iter1319 != this->partNames.end(); ++_iter1319)
     {
-      xfer += oprot->writeString((*_iter1315));
+      xfer += oprot->writeString((*_iter1319));
     }
     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 _iter1316;
-    for (_iter1316 = (*(this->partNames)).begin(); _iter1316 != (*(this->partNames)).end(); ++_iter1316)
+    std::vector<std::string> ::const_iterator _iter1320;
+    for (_iter1320 = (*(this->partNames)).begin(); _iter1320 != (*(this->partNames)).end(); ++_iter1320)
     {
-      xfer += oprot->writeString((*_iter1316));
+      xfer += oprot->writeString((*_iter1320));
     }
     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 _size1317;
-            ::apache::thrift::protocol::TType _etype1320;
-            xfer += iprot->readListBegin(_etype1320, _size1317);
-            this->success.resize(_size1317);
-            uint32_t _i1321;
-            for (_i1321 = 0; _i1321 < _size1317; ++_i1321)
+            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)
             {
-              xfer += iprot->readString(this->success[_i1321]);
+              xfer += iprot->readString(this->success[_i1325]);
             }
             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 _iter1322;
-      for (_iter1322 = this->success.begin(); _iter1322 != this->success.end(); ++_iter1322)
+      std::vector<std::string> ::const_iterator _iter1326;
+      for (_iter1326 = this->success.begin(); _iter1326 != this->success.end(); ++_iter1326)
       {
-        xfer += oprot->writeString((*_iter1322));
+        xfer += oprot->writeString((*_iter1326));
       }
       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 _size1323;
-            ::apache::thrift::protocol::TType _etype1326;
-            xfer += iprot->readListBegin(_etype1326, _size1323);
-            (*(this->success)).resize(_size1323);
-            uint32_t _i1327;
-            for (_i1327 = 0; _i1327 < _size1323; ++_i1327)
+            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)
             {
-              xfer += iprot->readString((*(this->success))[_i1327]);
+              xfer += iprot->readString((*(this->success))[_i1331]);
             }
             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 _size1328;
-            ::apache::thrift::protocol::TType _etype1331;
-            xfer += iprot->readListBegin(_etype1331, _size1328);
-            this->success.resize(_size1328);
-            uint32_t _i1332;
-            for (_i1332 = 0; _i1332 < _size1328; ++_i1332)
+            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)
             {
-              xfer += iprot->readString(this->success[_i1332]);
+              xfer += iprot->readString(this->success[_i1336]);
             }
             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 _iter1333;
-      for (_iter1333 = this->success.begin(); _iter1333 != this->success.end(); ++_iter1333)
+      std::vector<std::string> ::const_iterator _iter1337;
+      for (_iter1337 = this->success.begin(); _iter1337 != this->success.end(); ++_iter1337)
       {
-        xfer += oprot->writeString((*_iter1333));
+        xfer += oprot->writeString((*_iter1337));
       }
       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 _size1334;
-            ::apache::thrift::protocol::TType _etype1337;
-            xfer += iprot->readListBegin(_etype1337, _size1334);
-            (*(this->success)).resize(_size1334);
-            uint32_t _i1338;
-            for (_i1338 = 0; _i1338 < _size1334; ++_i1338)
+            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)
             {
-              xfer += iprot->readString((*(this->success))[_i1338]);
+              xfer += iprot->readString((*(this->success))[_i1342]);
             }
             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 _size1339;
-            ::apache::thrift::protocol::TType _etype1342;
-            xfer += iprot->readListBegin(_etype1342, _size1339);
-            this->success.resize(_size1339);
-            uint32_t _i1343;
-            for (_i1343 = 0; _i1343 < _size1339; ++_i1343)
+            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)
             {
-              xfer += iprot->readString(this->success[_i1343]);
+              xfer += iprot->readString(this->success[_i1347]);
             }
             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 _iter1344;
-      for (_iter1344 = this->success.begin(); _iter1344 != this->success.end(); ++_iter1344)
+      std::vector<std::string> ::const_iterator _iter1348;
+      for (_iter1348 = this->success.begin(); _iter1348 != this->success.end(); ++_iter1348)
       {
-        xfer += oprot->writeString((*_iter1344));
+        xfer += oprot->writeString((*_iter1348));
       }
       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 _size1345;
-            ::apache::thrift::protocol::TType _etype1348;
-            xfer += iprot->readListBegin(_etype1348, _size1345);
-            (*(this->success)).resize(_size1345);
-            uint32_t _i1349;
-            for (_i1349 = 0; _i1349 < _size1345; ++_i1349)
+            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)
             {
-              xfer += iprot->readString((*(this->success))[_i1349]);
+              xfer += iprot->readString((*(this->success))[_i1353]);
             }
             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 _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)
+            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)
             {
-              xfer += iprot->readString(this->tbl_types[_i1354]);
+              xfer += iprot->readString(this->tbl_types[_i1358]);
             }
             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 _iter1355;
-    for (_iter1355 = this->tbl_types.begin(); _iter1355 != this->tbl_types.end(); ++_iter1355)
+    std::vector<std::string> ::const_iterator _iter1359;
+    for (_iter1359 = this->tbl_types.begin(); _iter1359 != this->tbl_types.end(); ++_iter1359)
     {
-      xfer += oprot->writeString((*_iter1355));
+      xfer += oprot->writeString((*_iter1359));
     }
     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 _iter1356;
-    for (_iter1356 = (*(this->tbl_types)).begin(); _iter1356 != (*(this->tbl_types)).end(); ++_iter1356)
+    std::vector<std::string> ::const_iterator _iter1360;
+    for (_iter1360 = (*(this->tbl_types)).begin(); _iter1360 != (*(this->tbl_types)).end(); ++_iter1360)
     {
-      xfer += oprot->writeString((*_iter1356));
+      xfer += oprot->writeString((*_iter1360));
     }
     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 _size1357;
-            ::apache::thrift::protocol::TType _etype1360;
-            xfer += iprot->readListBegin(_etype1360, _size1357);
-            this->success.resize(_size1357);
-            uint32_t _i1361;
-            for (_i1361 = 0; _i1361 < _size1357; ++_i1361)
+            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)
             {
-              xfer += this->success[_i1361].read(iprot);
+              xfer += this->success[_i1365].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 _iter1362;
-      for (_iter1362 = this->success.begin(); _iter1362 != this->success.end(); ++_iter1362)
+      std::vector<TableMeta> ::const_iterator _iter1366;
+      for (_iter1366 = this->success.begin(); _iter1366 != this->success.end(); ++_iter1366)
       {
-        xfer += (*_iter1362).write(oprot);
+        xfer += (*_iter1366).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 _size1363;
-            ::apache::thrift::protocol::TType _etype1366;
-            xfer += iprot->readListBegin(_etype1366, _size1363);
-            (*(this->success)).resize(_size1363);
-            uint32_t _i1367;
-            for (_i1367 = 0; _i1367 < _size1363; ++_i1367)
+            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)
             {
-              xfer += (*(this->success))[_i1367].read(iprot);
+              xfer += (*(this->success))[_i1371].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 _size1368;
-            ::apache::thrift::protocol::TType _etype1371;
-            xfer += iprot->readListBegin(_etype1371, _size1368);
-            this->success.resize(_size1368);
-            uint32_t _i1372;
-            for (_i1372 = 0; _i1372 < _size1368; ++_i1372)
+            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)
             {
-              xfer += iprot->readString(this->success[_i1372]);
+              xfer += iprot->readString(this->success[_i1376]);
             }
             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 _iter1373;
-      for (_iter1373 = this->success.begin(); _iter1373 != this->success.end(); ++_iter1373)
+      std::vector<std::string> ::const_iterator _iter1377;
+      for (_iter1377 = this->success.begin(); _iter1377 != this->success.end(); ++_iter1377)
       {
-        xfer += oprot->writeString((*_iter1373));
+        xfer += oprot->writeString((*_iter1377));
       }
       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 _size1374;
-            ::apache::thrift::protocol::TType _etype1377;
-            xfer += iprot->readListBegin(_etype1377, _size1374);
-            (*(this->success)).resize(_size1374);
-            uint32_t _i1378;
-            for (_i1378 = 0; _i1378 < _size1374; ++_i1378)
+            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)
             {
-              xfer += iprot->readString((*(this->success))[_i1378]);
+              xfer += iprot->readString((*(this->success))[_i1382]);
             }
             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 _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)
+            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)
             {
-              xfer += iprot->readString(this->tbl_names[_i1383]);
+              xfer += iprot->readString(this->tbl_names[_i1387]);
             }
             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 _iter1384;
-    for (_iter1384 = this->tbl_names.begin(); _iter1384 != this->tbl_names.end(); ++_iter1384)
+    std::vector<std::string> ::const_iterator _iter1388;
+    for (_iter1388 = this->tbl_names.begin(); _iter1388 != this->tbl_names.end(); ++_iter1388)
     {
-      xfer += oprot->writeString((*_iter1384));
+      xfer += oprot->writeString((*_iter1388));
     }
     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 _iter1385;
-    for (_iter1385 = (*(this->tbl_names)).begin(); _iter1385 != (*(this->tbl_names)).end(); ++_iter1385)
+    std::vector<std::string> ::const_iterator _iter1389;
+    for (_iter1389 = (*(this->tbl_names)).begin(); _iter1389 != (*(this->tbl_names)).end(); ++_iter1389)
     {
-      xfer += oprot->writeString((*_iter1385));
+      xfer += oprot->writeString((*_iter1389));
     }
     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 _size1386;
-            ::apache::thrift::protocol::TType _etype1389;
-            xfer += iprot->readListBegin(_etype1389, _size1386);
-            this->success.resize(_size1386);
-            uint32_t _i1390;
-            for (_i1390 = 0; _i1390 < _size1386; ++_i1390)
+            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)
             {
-              xfer += this->success[_i1390].read(iprot);
+              xfer += this->success[_i1394].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 _iter1391;
-      for (_iter1391 = this->success.begin(); _iter1391 != this->success.end(); ++_iter1391)
+      std::vector<Table> ::const_iterator _iter1395;
+      for (_iter1395 = this->success.begin(); _iter1395 != this->success.end(); ++_iter1395)
       {
-        xfer += (*_iter1391).write(oprot);
+        xfer += (*_iter1395).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 _size1392;
-            ::apache::thrift::protocol::TType _etype1395;
-            xfer += iprot->readListBegin(_etype1395, _size1392);
-            (*(this->success)).resize(_size1392);
-            uint32_t _i1396;
-            for (_i1396 = 0; _i1396 < _size1392; ++_i1396)
+            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)
             {
-              xfer += (*(this->success))[_i1396].read(iprot);
+              xfer += (*(this->success))[_i1400].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 _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)
+            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)
             {
-              xfer += iprot->readString(this->tbl_names[_i1401]);
+              xfer += iprot->readString(this->tbl_names[_i1405]);
             }
             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 _iter1402;
-    for (_iter1402 = this->tbl_names.begin(); _iter1402 != this->tbl_names.end(); ++_iter1402)
+    std::vector<std::string> ::const_iterator _iter1406;
+    for (_iter1406 = this->tbl_names.begin(); _iter1406 != this->tbl_names.end(); ++_iter1406)
     {
-      xfer += oprot->writeString((*_iter1402));
+      xfer += oprot->writeString((*_iter1406));
     }
     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 _iter1403;
-    for (_iter1403 = (*(this->tbl_names)).begin(); _iter1403 != (*(this->tbl_names)).end(); ++_iter1403)
+    std::vector<std::string> ::const_iterator _iter1407;
+    for (_iter1407 = (*(this->tbl_names)).begin(); _iter1407 != (*(this->tbl_names)).end(); ++_iter1407)
     {
-      xfer += oprot->writeString((*_iter1403));
+      xfer += oprot->writeString((*_iter1407));
     }
     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 _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)
+            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)
             {
-              std::string _key1409;
-              xfer += iprot->readString(_key1409);
-              Materialization& _val1410 = this->success[_key1409];
-              xfer += _val1410.read(iprot);
+              std::string _key1413;
+              xfer += iprot->readString(_key1413);
+              Materialization& _val1414 = this->success[_key1413];
+              xfer += _val1414.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 _iter1411;
-      for (_iter1411 = this->success.begin(); _iter1411 != this->success.end(); ++_iter1411)
+      std::map<std::string, Materialization> ::const_iterator _iter1415;
+      for (_iter1415 = this->success.begin(); _iter1415 != this->success.end(); ++_iter1415)
       {
-        xfer += oprot->writeString(_iter1411->first);
-        xfer += _iter1411->second.write(oprot);
+        xfer += oprot->writeString(_iter1415->first);
+        xfer += _iter1415->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 _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)
+            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)
             {
-              std::string _key1417;
-              xfer += iprot->readString(_key1417);
-              Materialization& _val1418 = (*(this->success))[_key1417];
-              xfer += _val1418.read(iprot);
+              std::string _key1421;
+              xfer += iprot->readString(_key1421);
+              Materialization& _val1422 = (*(this->success))[_key1421];
+              xfer += _val1422.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 _size1419;
-            ::apache::thrift::protocol::TType _etype1422;
-            xfer += iprot->readListBegin(_etype1422, _size1419);
-            this->success.resize(_size1419);
-            uint32_t _i1423;
-            for (_i1423 = 0; _i1423 < _size1419; ++_i1423)
+            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)
             {
-              xfer += iprot->readString(this->success[_i1423]);
+              xfer += iprot->readString(this->success[_i1427]);
             }
             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 _iter1424;
-      for (_iter1424 = this->success.begin(); _iter1424 != this->success.end(); ++_iter1424)
+      std::vector<std::string> ::const_iterator _iter1428;
+      for (_iter1428 = this->success.begin(); _iter1428 != this->success.end(); ++_iter1428)
       {
-        xfer += oprot->writeString((*_iter1424));
+        xfer += oprot->writeString((*_iter1428));
       }
       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 _size1425;
-            ::apache::thrift::protocol::TType _etype1428;
-            xfer += iprot->readListBegin(_etype1428, _size1425);
-            (*(this->success)).resize(_size1425);
-            uint32_t _i1429;
-            for (_i1429 = 0; _i1429 < _size1425; ++_i1429)
+            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)
             {
-              xfer += iprot->readString((*(this->success))[_i1429]);
+              xfer += iprot->readString((*(this->success))[_i1433]);
             }
             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 _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)
+            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)
             {
-              xfer += this->new_parts[_i1434].read(iprot);
+              xfer += this->new_parts[_i1438].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 _iter1435;
-    for (_iter1435 = this->new_parts.begin(); _iter1435 != this->new_parts.end(); ++_iter1435)
+    std::vector<Partition> ::const_iterator _iter1439;
+    for (_iter1439 = this->new_parts.begin(); _iter1439 != this->new_parts.end(); ++_iter1439)
     {
-      xfer += (*_iter1435).write(oprot);
+      xfer += (*_iter1439).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 _iter1436;
-    for (_iter1436 = (*(this->new_parts)).begin(); _iter1436 != (*(this->new_parts)).end(); ++_iter1436)
+    std::vector<Partition> ::const_iterator _iter1440;
+    for (_iter1440 = (*(this->new_parts)).begin(); _iter1440 != (*(this->new_parts)).end(); ++_iter1440)
     {
-      xfer += (*_iter1436).write(oprot);
+      xfer += (*_iter1440).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 _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)
+            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)
             {
-              xfer += this->new_parts[_i1441].read(iprot);
+              xfer += this->new_parts[_i1445].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 _iter1442;
-    for (_iter1442 = this->new_parts.begin(); _iter1442 != this->new_parts.end(); ++_iter1442)
+    std::vector<PartitionSpec> ::const_iterator _iter1446;
+    for (_iter1446 = this->new_parts.begin(); _iter1446 != this->new_parts.end(); ++_iter1446)
     {
-      xfer += (*_iter1442).write(oprot);
+      xfer += (*_iter1446).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 _iter1443;
-    for (_iter1443 = (*(this->new_parts)).begin(); _iter1443 != (*(this->new_parts)).end(); ++_iter1443)
+    std::vector<PartitionSpec> ::const_iterator _iter1447;
+    for (_iter1447 = (*(this->new_parts)).begin(); _iter1447 != (*(this->new_parts)).end(); ++_iter1447)
     {
-      xfer += (*_iter1443).write(oprot);
+      xfer += (*_iter1447).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 _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)
+            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)
             {
-              xfer += iprot->readString(this->part_vals[_i1448]);
+              xfer += iprot->readString(this->part_vals[_i1452]);
             }
             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 _iter1449;
-    for (_iter1449 = this->part_vals.begin(); _iter1449 != this->part_vals.end(); ++_iter1449)
+    std::vector<std::string> ::const_iterator _iter1453;
+    for (_iter1453 = this->part_vals.begin(); _iter1453 != this->part_vals.end(); ++_iter1453)
     {
-      xfer += oprot->writeString((*_iter1449));
+      xfer += oprot->writeString((*_iter1453));
     }
     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 _iter1450;
-    for (_iter1450 = (*(this->part_vals)).begin(); _iter1450 != (*(this->part_vals)).end(); ++_iter1450)
+    std::vector<std::string> ::const_iterator _iter1454;
+    for (_iter1454 = (*(this->part_vals)).begin(); _iter1454 != (*(this->part_vals)).end(); ++_iter1454)
     {
-      xfer += oprot->writeString((*_iter1450));
+      xfer += oprot->writeString((*_iter1454));
     }
     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 _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)
+            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)
             {
-              xfer += iprot->readString(this->part_vals[_i1455]);
+              xfer += iprot->readString(this->part_vals[_i1459]);
             }
             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 _iter1456;
-    for (_iter1456 = this->part_vals.begin(); _iter1456 != this->part_vals.end(); ++_iter1456)
+    std::vector<std::string> ::const_iterator _iter1460;
+    for (_iter1460 = this->part_vals.begin(); _iter1460 != this->part_vals.end(); ++_iter1460)
     {
-      xfer += oprot->writeString((*_iter1456));
+      xfer += oprot->writeString((*_iter1460));
     }
     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 _iter1457;
-    for (_iter1457 = (*(this->part_vals)).begin(); _iter1457 != (*(this->part_vals)).end(); ++_iter1457)
+    std::vector<std::string> ::const_iterator _iter1461;
+    for (_iter1461 = (*(this->part_vals)).begin(); _iter1461 != (*(this->part_vals)).end(); ++_iter1461)
     {
-      xfer += oprot->writeString((*_iter1457));
+      xfer += oprot->writeString((*_iter1461));
     }
     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 _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)
+            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)
             {
-              xfer += iprot->readString(this->part_vals[_i1462]);
+              xfer += iprot->readString(this->part_vals[_i1466]);
             }
             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 _iter1463;
-    for (_iter1463 = this->part_vals.begin(); _iter1463 != this->part_vals.end(); ++_iter1463)
+    std::vector<std::string> ::const_iterator _iter1467;
+    for (_iter1467 = this->part_vals.begin(); _iter1467 != this->part_vals.end(); ++_iter1467)
     {
-      xfer += oprot->writeString((*_iter1463));
+      xfer += oprot->writeString((*_iter1467));
     }
     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 _iter1464;
-    for (_iter1464 = (*(this->part_vals)).begin(); _iter1464 != (*(this->part_vals)).end(); ++_iter1464)
+    std::vector<std::string> ::const_iterator _iter1468;
+    for (_iter1468 = (*(this->part_vals)).begin(); _iter1468 != (*(this->part_vals)).end(); ++_iter1468)
     {
-      xfer += oprot->writeString((*_iter1464));
+      xfer += oprot->writeString((*_iter1468));
     }
     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 _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)
+            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)
             {
-              xfer += iprot->readString(this->part_vals[_i1469]);
+              xfer += iprot->readString(this->part_vals[_i1473]);
             }
             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 _iter1470;
-    for (_iter1470 = this->part_vals.begin(); _iter1470 != this->part_vals.end(); ++_iter1470)
+    std::vector<std::string> ::const_iterator _iter1474;
+    for (_iter1474 = this->part_vals.begin(); _iter1474 != this->part_vals.end(); ++_iter1474)
     {
-      xfer += oprot->writeString((*_iter1470));
+      xfer += oprot->writeString((*_iter1474));
     }
     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 _iter1471;
-    for (_iter1471 = (*(this->part_vals)).begin(); _iter1471 != (*(this->part_vals)).end(); ++_iter1471)
+    std::vector<std::string> ::const_iterator _iter1475;
+    for (_iter1475 = (*(this->part_vals)).begin(); _iter1475 != (*(this->part_vals)).end(); ++_iter1475)
     {
-      xfer += oprot->writeString((*_iter1471));
+      xfer += oprot->writeString((*_iter1475));
     }
     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 _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)
+            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)
             {
-              xfer += iprot->readString(this->part_vals[_i1476]);
+              xfer += iprot->readString(this->part_vals[_i1480]);
             }
             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 _iter1477;
-    for (_iter1477 = this->part_vals.begin(); _iter1477 != this->part_vals.end(); ++_iter1477)
+    std::vector<std::string> ::const_iterator _iter1481;
+    for (_iter1481 = this->part_vals.begin(); _iter1481 != this->part_vals.end(); ++_iter1481)
     {
-      xfer += oprot->writeString((*_iter1477));
+      xfer += oprot->writeString((*_iter1481));
     }
     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 _iter1478;
-    for (_iter1478 = (*(this->part_vals)).begin(); _iter1478 != (*(this->part_vals)).end(); ++_iter1478)
+    std::vector<std::string> ::const_iterator _iter1482;
+    for (_iter1482 = (*(this->part_vals)).begin(); _iter1482 != (*(this->part_vals)).end(); ++_iter1482)
     {
-      xfer += oprot->writeString((*_iter1478));
+      xfer += oprot->writeString((*_iter1482));
     }
     xfer += oprot->writeListEnd();
   }


<TRUNCATED>

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

Posted by ha...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/b3e2d8a0/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 a354f27..e2f0e82 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,6 +446,10 @@ 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 {
@@ -854,6 +858,10 @@ 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 {
@@ -6692,6 +6700,55 @@ 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> {
@@ -13660,6 +13717,70 @@ 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 {
@@ -13875,6 +13996,8 @@ 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;
     }
 
@@ -19068,6 +19191,54 @@ 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> {
@@ -19283,6 +19454,8 @@ 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;
     }
 
@@ -31671,385 +31844,137 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-  }
-
-  @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);
-        }
-      }
-
-      /**
-       * 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;
-        }
-      }
-
-      /**
-       * 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;
+    @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");
       }
 
-      /**
-       * Find the _Fields constant that matches name, or null if its not found.
-       */
-      public static _Fields findByName(String name) {
-        return byName.get(name);
+      public add_runtime_stats_args getEmptyArgsInstance() {
+        return new add_runtime_stats_args();
       }
 
-      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();
+      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();
+          }
+        };
       }
-      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)
+      protected boolean isOneway() {
         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);
+      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);
       }
     }
 
-    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);
+    @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");
       }
-    }
 
-    private static class getMetaConf_argsStandardSchemeFactory implements SchemeFactory {
-      public getMetaConf_argsStandardScheme getScheme() {
-        return new getMetaConf_argsStandardScheme();
+      public get_runtime_stats_args getEmptyArgsInstance() {
+        return new get_runtime_stats_args();
       }
-    }
-
-    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 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();
           }
-          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);
+          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();
           }
-          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);
-        }
+      protected boolean isOneway() {
+        return false;
       }
 
-      @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);
-        }
+      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);
       }
     }
 
   }
 
-  @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");
+  @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 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 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_resultStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new getMetaConf_resultTupleSchemeFactory());
+      schemes.put(StandardScheme.class, new getMetaConf_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new getMetaConf_argsTupleSchemeFactory());
     }
 
-    private String success; // required
-    private MetaException o1; // required
+    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 {
-      SUCCESS((short)0, "success"),
-      O1((short)1, "o1");
+      KEY((short)1, "key");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -32064,10 +31989,371 @@ import org.slf4j.LoggerFactory;
        */
       public static _Fields findByThriftId(int fieldId) {
         switch(fieldId) {
-          case 0: // SUCCESS
-            return SUCCESS;
-          case 1: // O1
-            return O1;
+          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;
           default:
             return null;
         }
@@ -233506,4 +233792,1608 @@ 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>

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

Posted by ha...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/b3e2d8a0/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 802d8e3..da66951 100644
--- a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
+++ b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
@@ -224,6 +224,8 @@ 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 {
@@ -887,6 +889,12 @@ 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 {
@@ -25644,6 +25652,222 @@ 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) :
@@ -26261,6 +26485,12 @@ 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 {
@@ -26473,6 +26703,8 @@ 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),
@@ -26679,6 +26911,8 @@ 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() {}
@@ -28654,6 +28888,25 @@ 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
@@ -29276,6 +29529,12 @@ 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/b3e2d8a0/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 c0a39f8..3b0b38c 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,6 +1032,16 @@ 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/b3e2d8a0/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 052b595..2fab857 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,6 +30551,202 @@ 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() {
 }
 
@@ -30620,13 +30816,13 @@ void swap(MetaException &a, MetaException &b) {
   swap(a.__isset, b.__isset);
 }
 
-MetaException::MetaException(const MetaException& other1157) : TException() {
-  message = other1157.message;
-  __isset = other1157.__isset;
+MetaException::MetaException(const MetaException& other1161) : TException() {
+  message = other1161.message;
+  __isset = other1161.__isset;
 }
-MetaException& MetaException::operator=(const MetaException& other1158) {
-  message = other1158.message;
-  __isset = other1158.__isset;
+MetaException& MetaException::operator=(const MetaException& other1162) {
+  message = other1162.message;
+  __isset = other1162.__isset;
   return *this;
 }
 void MetaException::printTo(std::ostream& out) const {
@@ -30717,13 +30913,13 @@ void swap(UnknownTableException &a, UnknownTableException &b) {
   swap(a.__isset, b.__isset);
 }
 
-UnknownTableException::UnknownTableException(const UnknownTableException& other1159) : TException() {
-  message = other1159.message;
-  __isset = other1159.__isset;
+UnknownTableException::UnknownTableException(const UnknownTableException& other1163) : TException() {
+  message = other1163.message;
+  __isset = other1163.__isset;
 }
-UnknownTableException& UnknownTableException::operator=(const UnknownTableException& other1160) {
-  message = other1160.message;
-  __isset = other1160.__isset;
+UnknownTableException& UnknownTableException::operator=(const UnknownTableException& other1164) {
+  message = other1164.message;
+  __isset = other1164.__isset;
   return *this;
 }
 void UnknownTableException::printTo(std::ostream& out) const {
@@ -30814,13 +31010,13 @@ void swap(UnknownDBException &a, UnknownDBException &b) {
   swap(a.__isset, b.__isset);
 }
 
-UnknownDBException::UnknownDBException(const UnknownDBException& other1161) : TException() {
-  message = other1161.message;
-  __isset = other1161.__isset;
+UnknownDBException::UnknownDBException(const UnknownDBException& other1165) : TException() {
+  message = other1165.message;
+  __isset = other1165.__isset;
 }
-UnknownDBException& UnknownDBException::operator=(const UnknownDBException& other1162) {
-  message = other1162.message;
-  __isset = other1162.__isset;
+UnknownDBException& UnknownDBException::operator=(const UnknownDBException& other1166) {
+  message = other1166.message;
+  __isset = other1166.__isset;
   return *this;
 }
 void UnknownDBException::printTo(std::ostream& out) const {
@@ -30911,13 +31107,13 @@ void swap(AlreadyExistsException &a, AlreadyExistsException &b) {
   swap(a.__isset, b.__isset);
 }
 
-AlreadyExistsException::AlreadyExistsException(const AlreadyExistsException& other1163) : TException() {
-  message = other1163.message;
-  __isset = other1163.__isset;
+AlreadyExistsException::AlreadyExistsException(const AlreadyExistsException& other1167) : TException() {
+  message = other1167.message;
+  __isset = other1167.__isset;
 }
-AlreadyExistsException& AlreadyExistsException::operator=(const AlreadyExistsException& other1164) {
-  message = other1164.message;
-  __isset = other1164.__isset;
+AlreadyExistsException& AlreadyExistsException::operator=(const AlreadyExistsException& other1168) {
+  message = other1168.message;
+  __isset = other1168.__isset;
   return *this;
 }
 void AlreadyExistsException::printTo(std::ostream& out) const {
@@ -31008,13 +31204,13 @@ void swap(InvalidPartitionException &a, InvalidPartitionException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidPartitionException::InvalidPartitionException(const InvalidPartitionException& other1165) : TException() {
-  message = other1165.message;
-  __isset = other1165.__isset;
+InvalidPartitionException::InvalidPartitionException(const InvalidPartitionException& other1169) : TException() {
+  message = other1169.message;
+  __isset = other1169.__isset;
 }
-InvalidPartitionException& InvalidPartitionException::operator=(const InvalidPartitionException& other1166) {
-  message = other1166.message;
-  __isset = other1166.__isset;
+InvalidPartitionException& InvalidPartitionException::operator=(const InvalidPartitionException& other1170) {
+  message = other1170.message;
+  __isset = other1170.__isset;
   return *this;
 }
 void InvalidPartitionException::printTo(std::ostream& out) const {
@@ -31105,13 +31301,13 @@ void swap(UnknownPartitionException &a, UnknownPartitionException &b) {
   swap(a.__isset, b.__isset);
 }
 
-UnknownPartitionException::UnknownPartitionException(const UnknownPartitionException& other1167) : TException() {
-  message = other1167.message;
-  __isset = other1167.__isset;
+UnknownPartitionException::UnknownPartitionException(const UnknownPartitionException& other1171) : TException() {
+  message = other1171.message;
+  __isset = other1171.__isset;
 }
-UnknownPartitionException& UnknownPartitionException::operator=(const UnknownPartitionException& other1168) {
-  message = other1168.message;
-  __isset = other1168.__isset;
+UnknownPartitionException& UnknownPartitionException::operator=(const UnknownPartitionException& other1172) {
+  message = other1172.message;
+  __isset = other1172.__isset;
   return *this;
 }
 void UnknownPartitionException::printTo(std::ostream& out) const {
@@ -31202,13 +31398,13 @@ void swap(InvalidObjectException &a, InvalidObjectException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidObjectException::InvalidObjectException(const InvalidObjectException& other1169) : TException() {
-  message = other1169.message;
-  __isset = other1169.__isset;
+InvalidObjectException::InvalidObjectException(const InvalidObjectException& other1173) : TException() {
+  message = other1173.message;
+  __isset = other1173.__isset;
 }
-InvalidObjectException& InvalidObjectException::operator=(const InvalidObjectException& other1170) {
-  message = other1170.message;
-  __isset = other1170.__isset;
+InvalidObjectException& InvalidObjectException::operator=(const InvalidObjectException& other1174) {
+  message = other1174.message;
+  __isset = other1174.__isset;
   return *this;
 }
 void InvalidObjectException::printTo(std::ostream& out) const {
@@ -31299,13 +31495,13 @@ void swap(NoSuchObjectException &a, NoSuchObjectException &b) {
   swap(a.__isset, b.__isset);
 }
 
-NoSuchObjectException::NoSuchObjectException(const NoSuchObjectException& other1171) : TException() {
-  message = other1171.message;
-  __isset = other1171.__isset;
+NoSuchObjectException::NoSuchObjectException(const NoSuchObjectException& other1175) : TException() {
+  message = other1175.message;
+  __isset = other1175.__isset;
 }
-NoSuchObjectException& NoSuchObjectException::operator=(const NoSuchObjectException& other1172) {
-  message = other1172.message;
-  __isset = other1172.__isset;
+NoSuchObjectException& NoSuchObjectException::operator=(const NoSuchObjectException& other1176) {
+  message = other1176.message;
+  __isset = other1176.__isset;
   return *this;
 }
 void NoSuchObjectException::printTo(std::ostream& out) const {
@@ -31396,13 +31592,13 @@ void swap(InvalidOperationException &a, InvalidOperationException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidOperationException::InvalidOperationException(const InvalidOperationException& other1173) : TException() {
-  message = other1173.message;
-  __isset = other1173.__isset;
+InvalidOperationException::InvalidOperationException(const InvalidOperationException& other1177) : TException() {
+  message = other1177.message;
+  __isset = other1177.__isset;
 }
-InvalidOperationException& InvalidOperationException::operator=(const InvalidOperationException& other1174) {
-  message = other1174.message;
-  __isset = other1174.__isset;
+InvalidOperationException& InvalidOperationException::operator=(const InvalidOperationException& other1178) {
+  message = other1178.message;
+  __isset = other1178.__isset;
   return *this;
 }
 void InvalidOperationException::printTo(std::ostream& out) const {
@@ -31493,13 +31689,13 @@ void swap(ConfigValSecurityException &a, ConfigValSecurityException &b) {
   swap(a.__isset, b.__isset);
 }
 
-ConfigValSecurityException::ConfigValSecurityException(const ConfigValSecurityException& other1175) : TException() {
-  message = other1175.message;
-  __isset = other1175.__isset;
+ConfigValSecurityException::ConfigValSecurityException(const ConfigValSecurityException& other1179) : TException() {
+  message = other1179.message;
+  __isset = other1179.__isset;
 }
-ConfigValSecurityException& ConfigValSecurityException::operator=(const ConfigValSecurityException& other1176) {
-  message = other1176.message;
-  __isset = other1176.__isset;
+ConfigValSecurityException& ConfigValSecurityException::operator=(const ConfigValSecurityException& other1180) {
+  message = other1180.message;
+  __isset = other1180.__isset;
   return *this;
 }
 void ConfigValSecurityException::printTo(std::ostream& out) const {
@@ -31590,13 +31786,13 @@ void swap(InvalidInputException &a, InvalidInputException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidInputException::InvalidInputException(const InvalidInputException& other1177) : TException() {
-  message = other1177.message;
-  __isset = other1177.__isset;
+InvalidInputException::InvalidInputException(const InvalidInputException& other1181) : TException() {
+  message = other1181.message;
+  __isset = other1181.__isset;
 }
-InvalidInputException& InvalidInputException::operator=(const InvalidInputException& other1178) {
-  message = other1178.message;
-  __isset = other1178.__isset;
+InvalidInputException& InvalidInputException::operator=(const InvalidInputException& other1182) {
+  message = other1182.message;
+  __isset = other1182.__isset;
   return *this;
 }
 void InvalidInputException::printTo(std::ostream& out) const {
@@ -31687,13 +31883,13 @@ void swap(NoSuchTxnException &a, NoSuchTxnException &b) {
   swap(a.__isset, b.__isset);
 }
 
-NoSuchTxnException::NoSuchTxnException(const NoSuchTxnException& other1179) : TException() {
-  message = other1179.message;
-  __isset = other1179.__isset;
+NoSuchTxnException::NoSuchTxnException(const NoSuchTxnException& other1183) : TException() {
+  message = other1183.message;
+  __isset = other1183.__isset;
 }
-NoSuchTxnException& NoSuchTxnException::operator=(const NoSuchTxnException& other1180) {
-  message = other1180.message;
-  __isset = other1180.__isset;
+NoSuchTxnException& NoSuchTxnException::operator=(const NoSuchTxnException& other1184) {
+  message = other1184.message;
+  __isset = other1184.__isset;
   return *this;
 }
 void NoSuchTxnException::printTo(std::ostream& out) const {
@@ -31784,13 +31980,13 @@ void swap(TxnAbortedException &a, TxnAbortedException &b) {
   swap(a.__isset, b.__isset);
 }
 
-TxnAbortedException::TxnAbortedException(const TxnAbortedException& other1181) : TException() {
-  message = other1181.message;
-  __isset = other1181.__isset;
+TxnAbortedException::TxnAbortedException(const TxnAbortedException& other1185) : TException() {
+  message = other1185.message;
+  __isset = other1185.__isset;
 }
-TxnAbortedException& TxnAbortedException::operator=(const TxnAbortedException& other1182) {
-  message = other1182.message;
-  __isset = other1182.__isset;
+TxnAbortedException& TxnAbortedException::operator=(const TxnAbortedException& other1186) {
+  message = other1186.message;
+  __isset = other1186.__isset;
   return *this;
 }
 void TxnAbortedException::printTo(std::ostream& out) const {
@@ -31881,13 +32077,13 @@ void swap(TxnOpenException &a, TxnOpenException &b) {
   swap(a.__isset, b.__isset);
 }
 
-TxnOpenException::TxnOpenException(const TxnOpenException& other1183) : TException() {
-  message = other1183.message;
-  __isset = other1183.__isset;
+TxnOpenException::TxnOpenException(const TxnOpenException& other1187) : TException() {
+  message = other1187.message;
+  __isset = other1187.__isset;
 }
-TxnOpenException& TxnOpenException::operator=(const TxnOpenException& other1184) {
-  message = other1184.message;
-  __isset = other1184.__isset;
+TxnOpenException& TxnOpenException::operator=(const TxnOpenException& other1188) {
+  message = other1188.message;
+  __isset = other1188.__isset;
   return *this;
 }
 void TxnOpenException::printTo(std::ostream& out) const {
@@ -31978,13 +32174,13 @@ void swap(NoSuchLockException &a, NoSuchLockException &b) {
   swap(a.__isset, b.__isset);
 }
 
-NoSuchLockException::NoSuchLockException(const NoSuchLockException& other1185) : TException() {
-  message = other1185.message;
-  __isset = other1185.__isset;
+NoSuchLockException::NoSuchLockException(const NoSuchLockException& other1189) : TException() {
+  message = other1189.message;
+  __isset = other1189.__isset;
 }
-NoSuchLockException& NoSuchLockException::operator=(const NoSuchLockException& other1186) {
-  message = other1186.message;
-  __isset = other1186.__isset;
+NoSuchLockException& NoSuchLockException::operator=(const NoSuchLockException& other1190) {
+  message = other1190.message;
+  __isset = other1190.__isset;
   return *this;
 }
 void NoSuchLockException::printTo(std::ostream& out) const {

http://git-wip-us.apache.org/repos/asf/hive/blob/b3e2d8a0/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 6b872a6..cd78f58 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,6 +659,10 @@ class SetSchemaVersionStateRequest;
 
 class GetSerdeRequest;
 
+class RuntimeStat;
+
+class GetRuntimeStatsRequest;
+
 class MetaException;
 
 class UnknownTableException;
@@ -12934,6 +12938,99 @@ 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/b3e2d8a0/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
new file mode 100644
index 0000000..e5ce97e
--- /dev/null
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetRuntimeStatsRequest.java
@@ -0,0 +1,283 @@
+/**
+ * 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/b3e2d8a0/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
new file mode 100644
index 0000000..b48718c
--- /dev/null
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/RuntimeStat.java
@@ -0,0 +1,600 @@
+/**
+ * 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);
+      }
+    }
+  }
+
+}
+