You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by vg...@apache.org on 2020/02/06 17:59:59 UTC

[hive] branch master updated: HIVE-22783: Add test for HIVE-22366 (Vineet Garg, reviewed by Zoltan Haindrich)

This is an automated email from the ASF dual-hosted git repository.

vgarg pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git


The following commit(s) were added to refs/heads/master by this push:
     new 93c4411  HIVE-22783: Add test for HIVE-22366 (Vineet Garg, reviewed by Zoltan Haindrich)
93c4411 is described below

commit 93c4411aded816ca561ad2e751a286df66db99ff
Author: Vineet Garg <vg...@apache.org>
AuthorDate: Thu Feb 6 09:57:59 2020 -0800

    HIVE-22783: Add test for HIVE-22366 (Vineet Garg, reviewed by Zoltan Haindrich)
---
 .../hadoop/hive/ql/TestNumMetastoreCalls.java      | 138 +++++++
 .../hive/ql/TestNumMetastoreCallsObjectStore.java  | 413 +++++++++++++++++++++
 2 files changed, 551 insertions(+)

diff --git a/ql/src/test/org/apache/hadoop/hive/ql/TestNumMetastoreCalls.java b/ql/src/test/org/apache/hadoop/hive/ql/TestNumMetastoreCalls.java
new file mode 100644
index 0000000..6e755d5
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/TestNumMetastoreCalls.java
@@ -0,0 +1,138 @@
+/*
+ * 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;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.ql.session.SessionState;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+// this test is to ensure number of calls to metastore server by query compiler
+public class TestNumMetastoreCalls {
+  static HiveConf hConf = null;
+  static Driver driver = null;
+
+  @BeforeClass
+  public static void Setup() throws Exception {
+    hConf = new HiveConf(Driver.class);
+    driver = setUpImpl(hConf);
+    driver.run("create table t1(id1 int, name1 string)");
+    driver.run("create table t2(id2 int, id1 int, name2 string)");
+    driver.run("create database db1");
+    driver.run("create table db1.tdb1(id2 int, id1 int, name2 string)");
+    driver.run("create table tpart(id2 int, id1 int)"
+        + " partitioned by (name string)");
+    driver.run("alter table tpart add partition (name='p1')") ;
+    driver.run("alter table tpart add partition (name='p2')") ;
+  }
+
+  @AfterClass
+  public static void Teardown() throws Exception {
+    driver.run("drop table t1");
+    driver.run("drop table t2");
+    driver.run("drop table db1.tdb1");
+    driver.run("drop database db1 cascade");
+  }
+
+  private static Driver setUpImpl(HiveConf hiveConf) throws Exception {
+    hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_IN_TEST, true);
+    hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY, false);
+    hiveConf.setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER,
+            "org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory");
+    MetastoreConf.setBoolVar(hiveConf, MetastoreConf.ConfVars.HIVE_IN_TEST, true);
+    MetastoreConf.setVar(hiveConf, MetastoreConf.ConfVars.RAW_STORE_IMPL,
+        "org.apache.hadoop.hive.ql.TestNumMetastoreCallsObjectStore");
+    MetastoreConf.setBoolVar(hiveConf, MetastoreConf.ConfVars.SCHEDULED_QUERIES_ENABLED, false);
+    SessionState.start(hiveConf);
+    return new Driver(hiveConf);
+  }
+
+  // compiler should do 6 metastore calls for each table reference
+  // get table, get table col statistics
+  // pk, fk, unique, not null constraints
+  // for partitioned table there would be an extra call to get partitions
+  @Test
+  public void testSelectQuery() {
+    int numCallsBefore = TestNumMetastoreCallsObjectStore.getNumCalls();
+    int numCallsAfter = 0;
+
+    // simple select *
+    String query1 = "select * from t1";
+    int rc = driver.compile(query1, true);
+    assert(rc==0);
+    numCallsAfter = TestNumMetastoreCallsObjectStore.getNumCalls();
+    assert((numCallsAfter - numCallsBefore) == 6);
+
+    // single table
+    numCallsBefore = TestNumMetastoreCallsObjectStore.getNumCalls();
+    String query2 = "select count(distinct id1) from t1 group by name1";
+    rc = driver.compile(query2, true);
+    assert(rc==0);
+    numCallsAfter = TestNumMetastoreCallsObjectStore.getNumCalls();
+    assert((numCallsAfter - numCallsBefore) == 6);
+
+    // two different tables
+    numCallsBefore = TestNumMetastoreCallsObjectStore.getNumCalls();
+    String query3 = "select count(*) from t1 join t2 on t1.id1 = t2.id1 "
+        + "where t2.id2 > 0 group by t1.name1, t2.name2";
+    rc = driver.compile(query3, true);
+    assert(rc==0);
+    numCallsAfter = TestNumMetastoreCallsObjectStore.getNumCalls();
+    assert((numCallsAfter - numCallsBefore) == 12 );
+
+    //from different dbs
+    numCallsBefore = TestNumMetastoreCallsObjectStore.getNumCalls();
+    String query4 = "select count(*) from t1 join db1.tdb1 as t2 on t1.id1 = t2.id1 "
+        + "where t2.id2 > 0 group by t1.name1, t2.name2";
+    rc = driver.compile(query4, true);
+    assert(rc==0);
+    numCallsAfter = TestNumMetastoreCallsObjectStore.getNumCalls();
+    assert((numCallsAfter - numCallsBefore) == 12);
+
+    // three table join
+    numCallsBefore = TestNumMetastoreCallsObjectStore.getNumCalls();
+    String query5 = "select count(*) from t1 join db1.tdb1 as dbt2 on t1.id1 = dbt2.id1 "
+        + "join t2 on t1.id1 = t2.id1 "
+        + "where t2.id2 > 0 group by t1.name1, t2.name2";
+    rc = driver.compile(query5, true);
+    assert(rc==0);
+    numCallsAfter = TestNumMetastoreCallsObjectStore.getNumCalls();
+    assert((numCallsAfter - numCallsBefore) == 18);
+
+    // single partitioned table
+    numCallsBefore = TestNumMetastoreCallsObjectStore.getNumCalls();
+    String query6 = "select count(distinct id1) from tpart group by name";
+    rc = driver.compile(query6, true);
+    assert(rc==0);
+    numCallsAfter = TestNumMetastoreCallsObjectStore.getNumCalls();
+    assert((numCallsAfter - numCallsBefore) == 7);
+
+    // two different tables
+    numCallsBefore = TestNumMetastoreCallsObjectStore.getNumCalls();
+    String query7 = "select count(*) from t1 join tpart on t1.id1 = tpart.id1 "
+        + "where tpart.id2 > 0 group by t1.name1, tpart.name";
+    rc = driver.compile(query7, true);
+    assert(rc==0);
+    numCallsAfter = TestNumMetastoreCallsObjectStore.getNumCalls();
+    assert((numCallsAfter - numCallsBefore)  == 13);
+  }
+}
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/TestNumMetastoreCallsObjectStore.java b/ql/src/test/org/apache/hadoop/hive/ql/TestNumMetastoreCallsObjectStore.java
new file mode 100644
index 0000000..df72c42
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/TestNumMetastoreCallsObjectStore.java
@@ -0,0 +1,413 @@
+/*
+ * 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;
+
+import org.apache.hadoop.hive.common.TableName;
+import org.apache.hadoop.hive.metastore.*;
+import org.apache.hadoop.hive.metastore.api.AggrStats;
+import org.apache.hadoop.hive.metastore.api.Catalog;
+import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
+import org.apache.hadoop.hive.metastore.api.GetPartitionsFilterSpec;
+import org.apache.hadoop.hive.metastore.api.GetPartitionsProjectionSpec;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.PartitionValuesResponse;
+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;
+import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
+import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
+import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.TableMeta;
+import org.apache.hadoop.hive.metastore.api.UnknownDBException;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreServerUtils;
+import org.apache.thrift.TException;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+// this class is a test wrapper around ObjectStore overriding most of the get methods
+// used by compiler. This is used my TestNumMetastoreCalls to ensure the number of calls to
+// metastore
+public class TestNumMetastoreCallsObjectStore extends ObjectStore {
+
+  static Map<String, Integer> callMap = new HashMap<>();
+  static private int numCalls = 0;
+
+  static void incrementCall() {
+    numCalls++;
+  }
+
+  static int getNumCalls() {
+    return numCalls;
+  }
+
+  public TestNumMetastoreCallsObjectStore() {
+    super();
+  }
+
+
+  @Override public Catalog getCatalog(String catalogName)
+      throws NoSuchObjectException, MetaException {
+    incrementCall();
+    return super.getCatalog(catalogName);
+  }
+
+  @Override public List<String> getCatalogs() throws MetaException {
+    incrementCall();
+    return super.getCatalogs();
+  }
+
+  @Override public Database getDatabase(String catalogName, String name)
+      throws NoSuchObjectException {
+    incrementCall();
+    return super.getDatabase(catalogName, name);
+  }
+
+  @Override public List<String> getDatabases(String catName, String pattern) throws MetaException {
+    incrementCall();
+    return super.getDatabases(catName, pattern);
+  }
+
+  @Override public List<String> getAllDatabases(String catName) throws MetaException {
+    incrementCall();
+    return super.getAllDatabases(catName);
+  }
+
+  @Override public Table getTable(String catName, String dbName, String tableName,
+      String writeIdList) throws MetaException {
+    incrementCall();
+    return super.getTable(catName, dbName, tableName, writeIdList);
+  }
+
+  @Override public List<String> getTables(String catName, String dbName, String pattern)
+      throws MetaException {
+    incrementCall();
+    return super.getTables(catName, dbName, pattern);
+  }
+
+  @Override public List<String> getTables(String catName, String dbName, String pattern,
+      TableType tableType, int limit) throws MetaException {
+    incrementCall();
+    return super.getTables(catName, dbName, pattern, tableType, limit);
+  }
+
+  @Override public List<TableName> getTableNamesWithStats()
+      throws MetaException, NoSuchObjectException {
+    incrementCall();
+    return super.getTableNamesWithStats();
+  }
+
+  @Override public Map<String, List<String>> getPartitionColsWithStats(String catName,
+      String dbName, String tableName) throws MetaException, NoSuchObjectException {
+    incrementCall();
+    return super.getPartitionColsWithStats(catName, dbName, tableName);
+  }
+
+  @Override public List<TableName> getAllTableNamesForStats()
+      throws MetaException, NoSuchObjectException {
+    incrementCall();
+    return super.getAllTableNamesForStats();
+  }
+
+  @Override public List<Table> getAllMaterializedViewObjectsForRewriting(String catName)
+      throws MetaException {
+    incrementCall();
+    return super.getAllMaterializedViewObjectsForRewriting(catName);
+  }
+
+  @Override public List<String> getMaterializedViewsForRewriting(String catName, String dbName)
+      throws MetaException, NoSuchObjectException {
+    incrementCall();
+    return super.getMaterializedViewsForRewriting(catName, dbName);
+  }
+
+  @Override public int getDatabaseCount() throws MetaException {
+    incrementCall();
+    return super.getDatabaseCount();
+  }
+
+  @Override public int getPartitionCount() throws MetaException {
+    incrementCall();
+    return super.getPartitionCount();
+  }
+
+  @Override public int getTableCount() throws MetaException {
+    incrementCall();
+    return super.getTableCount();
+  }
+
+  @Override public List<TableMeta> getTableMeta(String catName, String dbNames, String tableNames,
+      List<String> tableTypes) throws MetaException {
+    incrementCall();
+    return super.getTableMeta(catName, dbNames, tableNames, tableTypes);
+  }
+
+  @Override public List<String> getAllTables(String catName, String dbName) throws MetaException {
+    incrementCall();
+    return super.getAllTables(catName, dbName);
+  }
+
+  @Override public List<Table> getTableObjectsByName(String catName, String db,
+      List<String> tbl_names) throws MetaException, UnknownDBException {
+    incrementCall();
+    return super.getTableObjectsByName(catName, db, tbl_names);
+  }
+
+  @Override public Partition getPartition(String catName, String dbName, String tableName,
+      List<String> part_vals) throws NoSuchObjectException, MetaException {
+    incrementCall();
+    return super.getPartition(catName, dbName, tableName, part_vals);
+  }
+
+  @Override public Partition getPartition(String catName, String dbName, String tableName,
+      List<String> part_vals, String validWriteIds) throws NoSuchObjectException, MetaException {
+    incrementCall();
+    return super.getPartition(catName, dbName, tableName, part_vals, validWriteIds);
+  }
+
+  @Override public List<Partition> getPartitions(String catName, String dbName, String tableName,
+      int maxParts) throws MetaException, NoSuchObjectException {
+    incrementCall();
+    return super.getPartitions(catName, dbName, tableName, maxParts);
+  }
+
+  @Override public Map<String, String> getPartitionLocations(String catName, String dbName,
+      String tblName, String baseLocationToNotShow, int max) {
+    incrementCall();
+    return super.getPartitionLocations(catName, dbName, tblName, baseLocationToNotShow, max);
+  }
+
+  @Override public List<Partition> getPartitionsWithAuth(String catName, String dbName,
+      String tblName, short max, String userName, List<String> groupNames)
+      throws MetaException, InvalidObjectException {
+    incrementCall();
+    return super.getPartitionsWithAuth(catName, dbName, tblName, max, userName, groupNames);
+  }
+
+  @Override public Partition getPartitionWithAuth(String catName, String dbName, String tblName,
+      List<String> partVals, String user_name, List<String> group_names)
+      throws NoSuchObjectException, MetaException, InvalidObjectException {
+    incrementCall();
+    return super.getPartitionWithAuth(catName, dbName, tblName, partVals, user_name, group_names);
+  }
+
+  @Override public List<String> listPartitionNames(String catName, String dbName, String tableName,
+      short max) throws MetaException {
+    incrementCall();
+    return super.listPartitionNames(catName, dbName, tableName, max);
+  }
+
+  @Override public PartitionValuesResponse listPartitionValues(String catName, String dbName,
+      String tableName, List<FieldSchema> cols, boolean applyDistinct, String filter,
+      boolean ascending, List<FieldSchema> order, long maxParts) throws MetaException {
+    incrementCall();
+    return super
+        .listPartitionValues(catName, dbName, tableName, cols, applyDistinct, filter, ascending,
+            order, maxParts);
+  }
+
+  @Override public List<Partition> listPartitionsPsWithAuth(String catName, String db_name,
+      String tbl_name, List<String> part_vals, short max_parts, String userName,
+      List<String> groupNames) throws MetaException, InvalidObjectException, NoSuchObjectException {
+    incrementCall();
+    return super
+        .listPartitionsPsWithAuth(catName, db_name, tbl_name, part_vals, max_parts, userName,
+            groupNames);
+  }
+
+  @Override public List<String> listPartitionNamesPs(String catName, String dbName,
+      String tableName, List<String> part_vals, short max_parts)
+      throws MetaException, NoSuchObjectException {
+    incrementCall();
+    return super.listPartitionNamesPs(catName, dbName, tableName, part_vals, max_parts);
+  }
+
+  @Override public List<Partition> getPartitionsByNames(String catName, String dbName,
+      String tblName, List<String> partNames) throws MetaException, NoSuchObjectException {
+    incrementCall();
+    return super.getPartitionsByNames(catName, dbName, tblName, partNames);
+  }
+
+  @Override public boolean getPartitionsByExpr(String catName, String dbName, String tblName,
+      byte[] expr, String defaultPartitionName, short maxParts, List<Partition> result)
+      throws TException {
+    incrementCall();
+    return super.getPartitionsByExpr(catName, dbName, tblName, expr, defaultPartitionName, maxParts,
+        result);
+  }
+
+  @Override protected boolean getPartitionsByExprInternal(String catName, String dbName,
+      String tblName, byte[] expr, String defaultPartitionName, short maxParts,
+      List<Partition> result, boolean allowSql, boolean allowJdo) throws TException {
+    incrementCall();
+    return super
+        .getPartitionsByExprInternal(catName, dbName, tblName, expr, defaultPartitionName, maxParts,
+            result, allowSql, allowJdo);
+  }
+
+  @Override public List<Partition> getPartitionsByFilter(String catName, String dbName,
+      String tblName, String filter, short maxParts) throws MetaException, NoSuchObjectException {
+    incrementCall();
+    return super.getPartitionsByFilter(catName, dbName, tblName, filter, maxParts);
+  }
+
+  @Override public int getNumPartitionsByFilter(String catName, String dbName, String tblName,
+      String filter) throws MetaException, NoSuchObjectException {
+    incrementCall();
+    return super.getNumPartitionsByFilter(catName, dbName, tblName, filter);
+  }
+
+  @Override public int getNumPartitionsByExpr(String catName, String dbName, String tblName,
+      byte[] expr) throws MetaException, NoSuchObjectException {
+    incrementCall();
+    return super.getNumPartitionsByExpr(catName, dbName, tblName, expr);
+  }
+
+  @Override protected List<Partition> getPartitionsByFilterInternal(String catName, String dbName,
+      String tblName, String filter, short maxParts, boolean allowSql, boolean allowJdo)
+      throws MetaException, NoSuchObjectException {
+    incrementCall();
+    return super.getPartitionsByFilterInternal(catName, dbName, tblName, filter, maxParts, allowSql,
+        allowJdo);
+  }
+
+  @Override public List<Partition> getPartitionSpecsByFilterAndProjection(Table table,
+      GetPartitionsProjectionSpec partitionsProjectSpec, GetPartitionsFilterSpec filterSpec)
+      throws MetaException, NoSuchObjectException {
+    incrementCall();
+    return super.getPartitionSpecsByFilterAndProjection(table, partitionsProjectSpec, filterSpec);
+  }
+
+  @Override public List<String> listTableNamesByFilter(String catName, String dbName, String filter,
+      short maxTables) throws MetaException {
+    incrementCall();
+    return super.listTableNamesByFilter(catName, dbName, filter, maxTables);
+  }
+
+  @Override public List<ColumnStatistics> getTableColumnStatistics(String catName, String dbName,
+      String tableName, List<String> colNames) throws MetaException, NoSuchObjectException {
+    incrementCall();
+    return super.getTableColumnStatistics(catName, dbName, tableName, colNames);
+  }
+
+  @Override public ColumnStatistics getTableColumnStatistics(String catName, String dbName,
+      String tableName, List<String> colNames, String engine)
+      throws MetaException, NoSuchObjectException {
+    incrementCall();
+    return super.getTableColumnStatistics(catName, dbName, tableName, colNames, engine);
+  }
+
+  @Override public ColumnStatistics getTableColumnStatistics(String catName, String dbName,
+      String tableName, List<String> colNames, String engine, String writeIdList)
+      throws MetaException, NoSuchObjectException {
+    incrementCall();
+    return super
+        .getTableColumnStatistics(catName, dbName, tableName, colNames, engine, writeIdList);
+  }
+
+  @Override public List<List<ColumnStatistics>> getPartitionColumnStatistics(String catName,
+      String dbName, String tableName, List<String> partNames, List<String> colNames)
+      throws MetaException, NoSuchObjectException {
+    incrementCall();
+    return super.getPartitionColumnStatistics(catName, dbName, tableName, partNames, colNames);
+  }
+
+  @Override public List<ColumnStatistics> getPartitionColumnStatistics(String catName,
+      String dbName, String tableName, List<String> partNames, List<String> colNames, String engine)
+      throws MetaException, NoSuchObjectException {
+    incrementCall();
+    return super
+        .getPartitionColumnStatistics(catName, dbName, tableName, partNames, colNames, engine);
+  }
+
+  @Override public List<ColumnStatistics> getPartitionColumnStatistics(String catName,
+      String dbName, String tableName, List<String> partNames, List<String> colNames, String engine,
+      String writeIdList) throws MetaException, NoSuchObjectException {
+    incrementCall();
+    return super
+        .getPartitionColumnStatistics(catName, dbName, tableName, partNames, colNames, engine,
+            writeIdList);
+  }
+
+  @Override public AggrStats get_aggr_stats_for(String catName, String dbName, String tblName,
+      List<String> partNames, List<String> colNames, String engine, String writeIdList)
+      throws MetaException, NoSuchObjectException {
+    incrementCall();
+    return super
+        .get_aggr_stats_for(catName, dbName, tblName, partNames, colNames, engine, writeIdList);
+  }
+
+  @Override public List<MetaStoreServerUtils.ColStatsObjWithSourceInfo> getPartitionColStatsForDatabase(
+      String catName, String dbName) throws MetaException, NoSuchObjectException {
+    incrementCall();
+    return super.getPartitionColStatsForDatabase(catName, dbName);
+  }
+
+  @Override public List<SQLPrimaryKey> getPrimaryKeys(String catName, String db_name,
+      String tbl_name) throws MetaException {
+    incrementCall();
+    return super.getPrimaryKeys(catName, db_name, tbl_name);
+  }
+
+  @Override public List<SQLForeignKey> getForeignKeys(String catName, String parent_db_name,
+      String parent_tbl_name, String foreign_db_name, String foreign_tbl_name)
+      throws MetaException {
+    incrementCall();
+    return super.getForeignKeys(catName, parent_db_name, parent_tbl_name, foreign_db_name,
+        foreign_tbl_name);
+  }
+
+  @Override public List<SQLUniqueConstraint> getUniqueConstraints(String catName, String db_name,
+      String tbl_name) throws MetaException {
+    incrementCall();
+    return super.getUniqueConstraints(catName, db_name, tbl_name);
+  }
+
+  @Override public List<SQLNotNullConstraint> getNotNullConstraints(String catName, String db_name,
+      String tbl_name) throws MetaException {
+    incrementCall();
+    return super.getNotNullConstraints(catName, db_name, tbl_name);
+  }
+
+  @Override public List<SQLDefaultConstraint> getDefaultConstraints(String catName, String db_name,
+      String tbl_name) throws MetaException {
+    incrementCall();
+    return super.getDefaultConstraints(catName, db_name, tbl_name);
+  }
+
+  @Override public List<SQLCheckConstraint> getCheckConstraints(String catName, String db_name,
+      String tbl_name) throws MetaException {
+    incrementCall();
+    return super.getCheckConstraints(catName, db_name, tbl_name);
+  }
+
+  @Override public List<RuntimeStat> getRuntimeStats(int maxEntries, int maxCreateTime)
+      throws MetaException {
+    incrementCall();
+    return super.getRuntimeStats(maxEntries, maxCreateTime);
+  }
+}