You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ga...@apache.org on 2017/12/07 17:54:04 UTC

[22/50] [abbrv] hive git commit: HIVE-17982 Move metastore specific itests

http://git-wip-us.apache.org/repos/asf/hive/blob/83cfbaf0/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestRetryingHMSHandler.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestRetryingHMSHandler.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestRetryingHMSHandler.java
deleted file mode 100644
index dfd80bc..0000000
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestRetryingHMSHandler.java
+++ /dev/null
@@ -1,123 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hive.metastore;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-
-import junit.framework.Assert;
-import junit.framework.TestCase;
-
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.api.Database;
-import org.apache.hadoop.hive.metastore.api.FieldSchema;
-import org.apache.hadoop.hive.metastore.api.Order;
-import org.apache.hadoop.hive.metastore.api.SerDeInfo;
-import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
-import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge;
-import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
-import org.apache.hadoop.hive.ql.io.HiveInputFormat;
-import org.apache.hadoop.hive.ql.io.HiveOutputFormat;
-import org.apache.hadoop.hive.serde.serdeConstants;
-import org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe;
-
-/**
- * TestRetryingHMSHandler. Test case for
- * {@link org.apache.hadoop.hive.metastore.RetryingHMSHandler}
- */
-public class TestRetryingHMSHandler extends TestCase {
-  private HiveConf hiveConf;
-  private HiveMetaStoreClient msc;
-
-  @Override
-  protected void setUp() throws Exception {
-
-    super.setUp();
-    System.setProperty("hive.metastore.pre.event.listeners",
-        AlternateFailurePreListener.class.getName());
-    int port = MetaStoreTestUtils.startMetaStoreWithRetry();
-    hiveConf = new HiveConf(this.getClass());
-    hiveConf.setVar(HiveConf.ConfVars.METASTOREURIS, "thrift://localhost:" + port);
-    hiveConf.setIntVar(HiveConf.ConfVars.METASTORETHRIFTCONNECTIONRETRIES, 3);
-    hiveConf.setIntVar(HiveConf.ConfVars.HMSHANDLERATTEMPTS, 2);
-    hiveConf.setTimeVar(HiveConf.ConfVars.HMSHANDLERINTERVAL, 0, TimeUnit.MILLISECONDS);
-    hiveConf.setBoolVar(HiveConf.ConfVars.HMSHANDLERFORCERELOADCONF, false);
-    msc = new HiveMetaStoreClient(hiveConf);
-  }
-
-  @Override
-  protected void tearDown() throws Exception {
-    super.tearDown();
-  }
-
-  // Create a database and a table in that database.  Because the AlternateFailurePreListener is
-  // being used each attempt to create something should require two calls by the RetryingHMSHandler
-  public void testRetryingHMSHandler() throws Exception {
-    String dbName = "hive4159";
-    String tblName = "tmptbl";
-
-    Database db = new Database();
-    db.setName(dbName);
-    msc.createDatabase(db);
-
-    Assert.assertEquals(2, AlternateFailurePreListener.getCallCount());
-
-    ArrayList<FieldSchema> cols = new ArrayList<FieldSchema>(2);
-    cols.add(new FieldSchema("c1", serdeConstants.STRING_TYPE_NAME, ""));
-    cols.add(new FieldSchema("c2", serdeConstants.INT_TYPE_NAME, ""));
-
-    Map<String, String> params = new HashMap<String, String>();
-    params.put("test_param_1", "Use this for comments etc");
-
-    Map<String, String> serdParams = new HashMap<String, String>();
-    serdParams.put(serdeConstants.SERIALIZATION_FORMAT, "1");
-
-    StorageDescriptor sd = new StorageDescriptor();
-
-    sd.setCols(cols);
-    sd.setCompressed(false);
-    sd.setNumBuckets(1);
-    sd.setParameters(params);
-    sd.setBucketCols(new ArrayList<String>(2));
-    sd.getBucketCols().add("name");
-    sd.setSerdeInfo(new SerDeInfo());
-    sd.getSerdeInfo().setName(tblName);
-    sd.getSerdeInfo().setParameters(serdParams);
-    sd.getSerdeInfo().getParameters()
-        .put(serdeConstants.SERIALIZATION_FORMAT, "1");
-    sd.getSerdeInfo().setSerializationLib(LazySimpleSerDe.class.getName());
-    sd.setInputFormat(HiveInputFormat.class.getName());
-    sd.setOutputFormat(HiveOutputFormat.class.getName());
-    sd.setSortCols(new ArrayList<Order>());
-
-    Table tbl = new Table();
-    tbl.setDbName(dbName);
-    tbl.setTableName(tblName);
-    tbl.setSd(sd);
-    tbl.setLastAccessTime(0);
-
-    msc.createTable(tbl);
-
-    Assert.assertEquals(4, AlternateFailurePreListener.getCallCount());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/83cfbaf0/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestSetUGIOnBothClientServer.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestSetUGIOnBothClientServer.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestSetUGIOnBothClientServer.java
deleted file mode 100644
index 98708a6..0000000
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestSetUGIOnBothClientServer.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hive.metastore;
-
-import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
-
-public class TestSetUGIOnBothClientServer extends TestRemoteHiveMetaStore{
-
-  public TestSetUGIOnBothClientServer() {
-    super();
-    isThriftClient = true;
-    // This will turn on setugi on both client and server processes of the test.
-    System.setProperty(ConfVars.METASTORE_EXECUTE_SET_UGI.varname, "true");
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/83cfbaf0/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestSetUGIOnOnlyClient.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestSetUGIOnOnlyClient.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestSetUGIOnOnlyClient.java
deleted file mode 100644
index 1a9abc9..0000000
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestSetUGIOnOnlyClient.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hive.metastore;
-
-import org.apache.hadoop.hive.conf.HiveConf;
-
-public class TestSetUGIOnOnlyClient extends TestRemoteHiveMetaStore{
-
-  @Override
-  protected HiveMetaStoreClient createClient() throws Exception {
-    hiveConf.setVar(HiveConf.ConfVars.METASTOREURIS, "thrift://localhost:" + port);
-    hiveConf.setBoolVar(HiveConf.ConfVars.METASTORE_EXECUTE_SET_UGI, true);
-    return new HiveMetaStoreClient(hiveConf);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/83cfbaf0/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestSetUGIOnOnlyServer.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestSetUGIOnOnlyServer.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestSetUGIOnOnlyServer.java
deleted file mode 100644
index b45fd01..0000000
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestSetUGIOnOnlyServer.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hive.metastore;
-
-import org.apache.hadoop.hive.conf.HiveConf;
-
-public class TestSetUGIOnOnlyServer extends TestSetUGIOnBothClientServer {
-
-  @Override
-  protected HiveMetaStoreClient createClient() throws Exception {
-    hiveConf.setVar(HiveConf.ConfVars.METASTOREURIS, "thrift://localhost:" + port);
-    hiveConf.setBoolVar(HiveConf.ConfVars.METASTORE_EXECUTE_SET_UGI, false);
-    return new HiveMetaStoreClient(hiveConf);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/83cfbaf0/standalone-metastore/pom.xml
----------------------------------------------------------------------
diff --git a/standalone-metastore/pom.xml b/standalone-metastore/pom.xml
index cce3282..d87863e 100644
--- a/standalone-metastore/pom.xml
+++ b/standalone-metastore/pom.xml
@@ -44,6 +44,7 @@
     <test.tmp.dir>${project.build.directory}/tmp</test.tmp.dir>
     <test.warehouse.dir>${project.build.directory}/warehouse</test.warehouse.dir>
     <test.warehouse.scheme>file://</test.warehouse.scheme>
+    <test.forkcount>1</test.forkcount>
 
     <!-- Plugin versions -->
     <ant.contrib.version>1.0b3</ant.contrib.version>
@@ -506,6 +507,7 @@
         <configuration>
           <redirectTestOutputToFile>true</redirectTestOutputToFile>
           <reuseForks>false</reuseForks>
+          <forkCount>${test.forkcount}</forkCount>
           <argLine>-Xmx2048m</argLine>
           <failIfNoTests>false</failIfNoTests>
           <systemPropertyVariables>

http://git-wip-us.apache.org/repos/asf/hive/blob/83cfbaf0/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/IndexBuilder.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/IndexBuilder.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/IndexBuilder.java
index 6c8b1d8..50fc186 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/IndexBuilder.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/IndexBuilder.java
@@ -73,6 +73,11 @@ public class IndexBuilder extends StorageDescriptorBuilder<IndexBuilder> {
     return this;
   }
 
+  public IndexBuilder addIndexParam(String key, String value) {
+    indexParams.put(key, value);
+    return this;
+  }
+
   public IndexBuilder setIndexName(String indexName) {
     this.indexName = indexName;
     return this;

http://git-wip-us.apache.org/repos/asf/hive/blob/83cfbaf0/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/PartitionBuilder.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/PartitionBuilder.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/PartitionBuilder.java
index 265625f..38e5a8f 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/PartitionBuilder.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/PartitionBuilder.java
@@ -53,9 +53,10 @@ public class PartitionBuilder extends StorageDescriptorBuilder<PartitionBuilder>
     return this;
   }
 
-  public PartitionBuilder setDbAndTableName(Table table) {
+  public PartitionBuilder fromTable(Table table) {
     this.dbName = table.getDbName();
     this.tableName = table.getTableName();
+    setCols(table.getSd().getCols());
     return this;
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/83cfbaf0/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/TableBuilder.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/TableBuilder.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/TableBuilder.java
index 1d457a6..69acf3c 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/TableBuilder.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/TableBuilder.java
@@ -17,8 +17,10 @@
  */
 package org.apache.hadoop.hive.metastore.client.builder;
 
+import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.Index;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
@@ -46,6 +48,8 @@ public class TableBuilder extends StorageDescriptorBuilder<TableBuilder> {
     tableParams = new HashMap<>();
     createTime = lastAccessTime = (int)(System.currentTimeMillis() / 1000);
     retention = 0;
+    partCols = new ArrayList<>();
+    type = TableType.MANAGED_TABLE.name();
     super.setChild(this);
   }
 
@@ -90,7 +94,6 @@ public class TableBuilder extends StorageDescriptorBuilder<TableBuilder> {
   }
 
   public TableBuilder addPartCol(String name, String type, String comment) {
-    if (partCols == null) partCols = new ArrayList<>();
     partCols.add(new FieldSchema(name, type, comment));
     return this;
   }
@@ -135,6 +138,13 @@ public class TableBuilder extends StorageDescriptorBuilder<TableBuilder> {
     return this;
   }
 
+  public TableBuilder fromIndex(Index index) {
+    dbName = index.getDbName();
+    tableName = index.getIndexTableName();
+    setCols(index.getSd().getCols());
+    return this;
+  }
+
   public Table build() throws MetaException {
     if (dbName == null || tableName == null) {
       throw new MetaException("You must set the database and table name");

http://git-wip-us.apache.org/repos/asf/hive/blob/83cfbaf0/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 b46cc38..57692d3 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
@@ -482,7 +482,7 @@ public class MetastoreConf {
         "hive.metastore.hbase.file.metadata.threads", 1,
         "Number of threads to use to read file metadata in background to cache it."),
     FILTER_HOOK("metastore.filter.hook", "hive.metastore.filter.hook",
-        "org.apache.hadoop.hive.metastore.DefaultMetaStoreFilterHookImpl",
+        org.apache.hadoop.hive.metastore.DefaultMetaStoreFilterHookImpl.class.getName(),
         "Metastore hook class for filtering the metadata read results. If hive.security.authorization.manager"
             + "is set to instance of HiveAuthorizerFactory, then this value is ignored."),
     FS_HANDLER_CLS("metastore.fs.handler.class", "hive.metastore.fs.handler.class",

http://git-wip-us.apache.org/repos/asf/hive/blob/83cfbaf0/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/FakeDerby.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/FakeDerby.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/FakeDerby.java
new file mode 100644
index 0000000..cc0bd77
--- /dev/null
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/FakeDerby.java
@@ -0,0 +1,404 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.metastore;
+
+import java.lang.Override;
+import java.sql.Array;
+import java.sql.Blob;
+import java.sql.CallableStatement;
+import java.sql.Clob;
+import java.sql.DatabaseMetaData;
+import java.sql.NClob;
+import java.sql.PreparedStatement;
+import java.sql.SQLClientInfoException;
+import java.sql.SQLException;
+import java.sql.SQLFeatureNotSupportedException;
+import java.sql.SQLWarning;
+import java.sql.SQLXML;
+import java.sql.Savepoint;
+import java.sql.Statement;
+import java.sql.Struct;
+import java.util.Map;
+import java.util.concurrent.Executor;
+import java.util.logging.Logger;
+import java.util.Properties;
+
+/**
+ * Fake derby driver - companion class to enable testing by TestObjectStoreInitRetry
+ */
+public class FakeDerby extends org.apache.derby.jdbc.EmbeddedDriver {
+
+  public class Connection implements java.sql.Connection {
+
+    private java.sql.Connection _baseConn;
+
+    public Connection(java.sql.Connection connection) {
+      TestObjectStoreInitRetry.debugTrace();
+      this._baseConn = connection;
+    }
+
+    @Override
+    public Statement createStatement() throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.createStatement();
+    }
+
+    @Override
+    public PreparedStatement prepareStatement(String sql) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.prepareStatement(sql);
+    }
+
+    @Override
+    public CallableStatement prepareCall(String sql) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.prepareCall(sql);
+    }
+
+    @Override
+    public String nativeSQL(String sql) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.nativeSQL(sql);
+    }
+
+    @Override
+    public void setAutoCommit(boolean autoCommit) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      TestObjectStoreInitRetry.misbehave();
+      _baseConn.setAutoCommit(autoCommit);
+    }
+
+    @Override
+    public boolean getAutoCommit() throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.getAutoCommit();
+    }
+
+    @Override
+    public void commit() throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      _baseConn.commit();
+    }
+
+    @Override
+    public void rollback() throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      _baseConn.rollback();
+    }
+
+    @Override
+    public void close() throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      _baseConn.close();
+    }
+
+    @Override
+    public boolean isClosed() throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.isClosed();
+    }
+
+    @Override
+    public DatabaseMetaData getMetaData() throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.getMetaData();
+    }
+
+    @Override
+    public void setReadOnly(boolean readOnly) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      _baseConn.setReadOnly(readOnly);
+    }
+
+    @Override
+    public boolean isReadOnly() throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.isReadOnly();
+    }
+
+    @Override
+    public void setCatalog(String catalog) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      _baseConn.setCatalog(catalog);
+    }
+
+    @Override
+    public String getCatalog() throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.getCatalog();
+    }
+
+    @Override
+    public void setTransactionIsolation(int level) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      _baseConn.setTransactionIsolation(level);
+    }
+
+    @Override
+    public int getTransactionIsolation() throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.getTransactionIsolation();
+    }
+
+    @Override
+    public SQLWarning getWarnings() throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.getWarnings();
+    }
+
+    @Override
+    public void clearWarnings() throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      _baseConn.clearWarnings();
+    }
+
+    @Override
+    public Statement createStatement(int resultSetType, int resultSetConcurrency) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.createStatement(resultSetType, resultSetConcurrency);
+    }
+
+    @Override
+    public PreparedStatement prepareStatement(String sql, int resultSetType, int resultSetConcurrency) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.prepareStatement(sql, resultSetType, resultSetConcurrency);
+    }
+
+    @Override
+    public CallableStatement prepareCall(String sql, int resultSetType, int resultSetConcurrency) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.prepareCall(sql, resultSetType, resultSetConcurrency);
+    }
+
+    @Override
+    public Map<String, Class<?>> getTypeMap() throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.getTypeMap();
+    }
+
+    @Override
+    public void setTypeMap(Map<String, Class<?>> map) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      _baseConn.setTypeMap(map);
+    }
+
+    @Override
+    public void setHoldability(int holdability) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      _baseConn.setHoldability(holdability);
+    }
+
+    @Override
+    public int getHoldability() throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.getHoldability();
+    }
+
+    @Override
+    public Savepoint setSavepoint() throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.setSavepoint();
+    }
+
+    @Override
+    public Savepoint setSavepoint(String name) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.setSavepoint(name);
+    }
+
+    @Override
+    public void rollback(Savepoint savepoint) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      _baseConn.rollback(savepoint);
+    }
+
+    @Override
+    public void releaseSavepoint(Savepoint savepoint) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      _baseConn.releaseSavepoint(savepoint);
+    }
+
+    @Override
+    public Statement createStatement(int resultSetType, int resultSetConcurrency, int resultSetHoldability) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.createStatement(resultSetType, resultSetConcurrency, resultSetHoldability);
+    }
+
+    @Override
+    public PreparedStatement prepareStatement(String sql, int resultSetType, int resultSetConcurrency, int resultSetHoldability) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.prepareStatement(sql, resultSetType, resultSetConcurrency, resultSetHoldability);
+    }
+
+    @Override
+    public CallableStatement prepareCall(String sql, int resultSetType, int resultSetConcurrency, int resultSetHoldability) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.prepareCall(sql, resultSetType, resultSetConcurrency, resultSetHoldability);
+    }
+
+    @Override
+    public PreparedStatement prepareStatement(String sql, int autoGeneratedKeys) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.prepareStatement(sql, autoGeneratedKeys);
+    }
+
+    @Override
+    public PreparedStatement prepareStatement(String sql, int[] columnIndexes) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.prepareStatement(sql, columnIndexes);
+    }
+
+    @Override
+    public PreparedStatement prepareStatement(String sql, String[] columnNames) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.prepareStatement(sql, columnNames);
+    }
+
+    @Override
+    public Clob createClob() throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.createClob();
+    }
+
+    @Override
+    public Blob createBlob() throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.createBlob();
+    }
+
+    @Override
+    public NClob createNClob() throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.createNClob();
+    }
+
+    @Override
+    public SQLXML createSQLXML() throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.createSQLXML();
+    }
+
+    @Override
+    public boolean isValid(int timeout) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.isValid(timeout);
+    }
+
+    @Override
+    public void setClientInfo(String name, String value) throws SQLClientInfoException {
+      TestObjectStoreInitRetry.debugTrace();
+      _baseConn.setClientInfo(name, value);
+    }
+
+    @Override
+    public void setClientInfo(Properties properties) throws SQLClientInfoException {
+      TestObjectStoreInitRetry.debugTrace();
+      _baseConn.setClientInfo(properties);
+    }
+
+    @Override
+    public String getClientInfo(String name) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.getClientInfo(name);
+    }
+
+    @Override
+    public Properties getClientInfo() throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.getClientInfo();
+    }
+
+    @Override
+    public Array createArrayOf(String typeName, Object[] elements) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.createArrayOf(typeName, elements);
+    }
+
+    @Override
+    public Struct createStruct(String typeName, Object[] attributes) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.createStruct(typeName, attributes);
+    }
+
+    @Override
+    public void setSchema(String schema) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      _baseConn.setSchema(schema);
+    }
+
+    @Override
+    public String getSchema() throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.getSchema();
+    }
+
+    @Override
+    public void abort(Executor executor) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      _baseConn.abort(executor);
+    }
+
+    @Override
+    public void setNetworkTimeout(Executor executor, int milliseconds) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      _baseConn.setNetworkTimeout(executor, milliseconds);
+    }
+
+    @Override
+    public int getNetworkTimeout() throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.getNetworkTimeout();
+    }
+
+    @Override
+    public <T> T unwrap(Class<T> iface) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.unwrap(iface);
+    }
+
+    @Override
+    public boolean isWrapperFor(Class<?> iface) throws SQLException {
+      TestObjectStoreInitRetry.debugTrace();
+      return _baseConn.isWrapperFor(iface);
+    }
+  }
+
+  public FakeDerby(){
+  }
+
+  @Override
+  public boolean acceptsURL(String url) throws SQLException {
+    url = url.replace("fderby","derby");
+    return super.acceptsURL(url);
+  }
+
+  @Override
+  public Connection connect(java.lang.String url, java.util.Properties info) throws SQLException {
+    TestObjectStoreInitRetry.misbehave();
+    url = url.replace("fderby","derby");
+    return new FakeDerby.Connection(super.connect(url, info));
+  }
+
+  @Override
+  public Logger getParentLogger() throws SQLFeatureNotSupportedException {
+    throw new SQLFeatureNotSupportedException(); // hope this is respected properly
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/83cfbaf0/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/MetaStoreTestUtils.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/MetaStoreTestUtils.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/MetaStoreTestUtils.java
index 380f3a1..e7146c4 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/MetaStoreTestUtils.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/MetaStoreTestUtils.java
@@ -26,6 +26,7 @@ import java.util.Map;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
 import org.apache.hadoop.hive.metastore.events.EventCleanerTask;
 import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge;
 import org.slf4j.Logger;
@@ -91,19 +92,22 @@ public class MetaStoreTestUtils {
 
   public static int startMetaStoreWithRetry(final HadoopThriftAuthBridge bridge, Configuration conf)
       throws Exception {
-    Exception metaStoreException = null;
-    int metaStorePort = 0;
+    int metaStorePort = findFreePort();
+    startMetaStoreWithRetry(metaStorePort, bridge, conf);
+    return metaStorePort;
+  }
 
+  public static void startMetaStoreWithRetry(int port, HadoopThriftAuthBridge bridge,
+                                             Configuration conf) throws Exception {
+    Exception metaStoreException = null;
     for (int tryCount = 0; tryCount < MetaStoreTestUtils.RETRY_COUNT; tryCount++) {
       try {
-        metaStorePort = MetaStoreTestUtils.findFreePort();
-        MetaStoreTestUtils.startMetaStore(metaStorePort, bridge, conf);
-        return metaStorePort;
+        MetaStoreTestUtils.startMetaStore(port, bridge, conf);
+        return;
       } catch (ConnectException ce) {
         metaStoreException = ce;
       }
     }
-
     throw metaStoreException;
   }
 
@@ -198,11 +202,20 @@ public class MetaStoreTestUtils {
   /**
    * Setup a configuration file for standalone mode.  There are a few config variables that have
    * defaults that require parts of Hive that aren't present in standalone mode.  This method
-   * sets them to something that will work without the rest of Hive.
+   * sets them to something that will work without the rest of Hive.  It only changes them if
+   * they have not already been set, to avoid clobbering intentional changes.
    * @param conf Configuration object
    */
   public static void setConfForStandloneMode(Configuration conf) {
-    MetastoreConf.setVar(conf, MetastoreConf.ConfVars.TASK_THREADS_ALWAYS,
-        EventCleanerTask.class.getName());
+    if (MetastoreConf.getVar(conf, ConfVars.TASK_THREADS_ALWAYS).equals(
+        ConfVars.TASK_THREADS_ALWAYS.getDefaultVal())) {
+      MetastoreConf.setVar(conf, ConfVars.TASK_THREADS_ALWAYS,
+          EventCleanerTask.class.getName());
+    }
+    if (MetastoreConf.getVar(conf, ConfVars.EXPRESSION_PROXY_CLASS).equals(
+        ConfVars.EXPRESSION_PROXY_CLASS.getDefaultVal())) {
+      MetastoreConf.setClass(conf, ConfVars.EXPRESSION_PROXY_CLASS,
+          DefaultPartitionExpressionProxy.class, PartitionExpressionProxy.class);
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/83cfbaf0/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestAdminUser.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestAdminUser.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestAdminUser.java
new file mode 100644
index 0000000..3b541d2
--- /dev/null
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestAdminUser.java
@@ -0,0 +1,46 @@
+/*
+ * 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.HiveMetaStore.HMSHandler;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.api.PrincipalType;
+import org.apache.hadoop.hive.metastore.api.Role;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestAdminUser {
+
+  @Test
+  public void testCreateAdminNAddUser() throws MetaException, NoSuchObjectException {
+    Configuration conf = MetastoreConf.newMetastoreConf();
+    MetastoreConf.setVar(conf, ConfVars.USERS_IN_ADMIN_ROLE, "adminuser");
+    MetaStoreTestUtils.setConfForStandloneMode(conf);
+    RawStore rawStore = new HMSHandler("testcreateroot", conf).getMS();
+    Role adminRole = rawStore.getRole(HiveMetaStore.ADMIN);
+    Assert.assertTrue(adminRole.getOwnerName().equals(HiveMetaStore.ADMIN));
+    Assert.assertEquals(rawStore.listPrincipalGlobalGrants(HiveMetaStore.ADMIN, PrincipalType.ROLE)
+     .get(0).getGrantInfo().getPrivilege(),"All");
+    Assert.assertEquals(rawStore.listRoles("adminuser", PrincipalType.USER).get(0).
+      getRoleName(),HiveMetaStore.ADMIN);
+ }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/83cfbaf0/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestEmbeddedHiveMetaStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestEmbeddedHiveMetaStore.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestEmbeddedHiveMetaStore.java
new file mode 100644
index 0000000..72758df
--- /dev/null
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestEmbeddedHiveMetaStore.java
@@ -0,0 +1,48 @@
+/*
+ * 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.util.StringUtils;
+import org.junit.After;
+import org.junit.Before;
+
+public class TestEmbeddedHiveMetaStore extends TestHiveMetaStore {
+
+  @Before
+  public void openWarehouse() throws Exception {
+    warehouse = new Warehouse(conf);
+    client = createClient();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    client.close();
+  }
+
+  @Override
+  protected HiveMetaStoreClient createClient() throws Exception {
+    try {
+      return new HiveMetaStoreClient(conf);
+    } catch (Throwable e) {
+      System.err.println("Unable to open the metastore");
+      System.err.println(StringUtils.stringifyException(e));
+      throw new Exception(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/83cfbaf0/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestFilterHooks.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestFilterHooks.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestFilterHooks.java
new file mode 100644
index 0000000..56afe33
--- /dev/null
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestFilterHooks.java
@@ -0,0 +1,303 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.fail;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.Index;
+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.PartitionSpec;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.client.builder.DatabaseBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.IndexBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.PartitionBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.TableBuilder;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
+import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TestFilterHooks {
+  private static final Logger LOG = LoggerFactory.getLogger(TestFilterHooks.class);
+
+  public static class DummyMetaStoreFilterHookImpl extends DefaultMetaStoreFilterHookImpl {
+    private static boolean blockResults = false;
+
+    public DummyMetaStoreFilterHookImpl(Configuration conf) {
+      super(conf);
+    }
+
+    @Override
+    public List<String> filterDatabases(List<String> dbList) throws MetaException  {
+      if (blockResults) {
+        return new ArrayList<>();
+      }
+      return super.filterDatabases(dbList);
+    }
+
+    @Override
+    public Database filterDatabase(Database dataBase) throws NoSuchObjectException {
+      if (blockResults) {
+        throw new NoSuchObjectException("Blocked access");
+      }
+      return super.filterDatabase(dataBase);
+    }
+
+    @Override
+    public List<String> filterTableNames(String dbName, List<String> tableList) throws MetaException {
+      if (blockResults) {
+        return new ArrayList<>();
+      }
+      return super.filterTableNames(dbName, tableList);
+    }
+
+    @Override
+    public Table filterTable(Table table) throws NoSuchObjectException {
+      if (blockResults) {
+        throw new NoSuchObjectException("Blocked access");
+      }
+      return super.filterTable(table);
+    }
+
+    @Override
+    public List<Table> filterTables(List<Table> tableList) throws MetaException {
+      if (blockResults) {
+        return new ArrayList<>();
+      }
+      return super.filterTables(tableList);
+    }
+
+    @Override
+    public List<Partition> filterPartitions(List<Partition> partitionList) throws MetaException {
+      if (blockResults) {
+        return new ArrayList<>();
+      }
+      return super.filterPartitions(partitionList);
+    }
+
+    @Override
+    public List<PartitionSpec> filterPartitionSpecs(
+        List<PartitionSpec> partitionSpecList) throws MetaException {
+      if (blockResults) {
+        return new ArrayList<>();
+      }
+      return super.filterPartitionSpecs(partitionSpecList);
+    }
+
+    @Override
+    public Partition filterPartition(Partition partition) throws NoSuchObjectException {
+      if (blockResults) {
+        throw new NoSuchObjectException("Blocked access");
+      }
+      return super.filterPartition(partition);
+    }
+
+    @Override
+    public List<String> filterPartitionNames(String dbName, String tblName,
+        List<String> partitionNames) throws MetaException {
+      if (blockResults) {
+        return new ArrayList<>();
+      }
+      return super.filterPartitionNames(dbName, tblName, partitionNames);
+    }
+
+    @Override
+    public Index filterIndex(Index index) throws NoSuchObjectException {
+      if (blockResults) {
+        throw new NoSuchObjectException("Blocked access");
+      }
+      return super.filterIndex(index);
+    }
+
+    @Override
+    public List<String> filterIndexNames(String dbName, String tblName,
+        List<String> indexList) throws MetaException {
+      if (blockResults) {
+        return new ArrayList<>();
+      }
+      return super.filterIndexNames(dbName, tblName, indexList);
+    }
+
+    @Override
+    public List<Index> filterIndexes(List<Index> indexeList) throws MetaException {
+      if (blockResults) {
+        return new ArrayList<>();
+      }
+      return super.filterIndexes(indexeList);
+    }
+  }
+
+  private static final String DBNAME1 = "testdb1";
+  private static final String DBNAME2 = "testdb2";
+  private static final String TAB1 = "tab1";
+  private static final String TAB2 = "tab2";
+  private static final String INDEX1 = "idx1";
+  private static Configuration conf;
+  private static HiveMetaStoreClient msc;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    DummyMetaStoreFilterHookImpl.blockResults = false;
+
+    conf = MetastoreConf.newMetastoreConf();
+    MetastoreConf.setLongVar(conf, ConfVars.THRIFT_CONNECTION_RETRIES, 3);
+    MetastoreConf.setBoolVar(conf, ConfVars.HIVE_SUPPORT_CONCURRENCY, false);
+    MetastoreConf.setClass(conf, ConfVars.FILTER_HOOK, DummyMetaStoreFilterHookImpl.class,
+        MetaStoreFilterHook.class);
+    MetaStoreTestUtils.setConfForStandloneMode(conf);
+    int port = MetaStoreTestUtils.findFreePort();
+    MetastoreConf.setVar(conf, ConfVars.THRIFT_URIS, "thrift://localhost:" + port);
+    MetaStoreTestUtils.startMetaStoreWithRetry(port, HadoopThriftAuthBridge.getBridge(), conf);
+
+    msc = new HiveMetaStoreClient(conf);
+
+    msc.dropDatabase(DBNAME1, true, true, true);
+    msc.dropDatabase(DBNAME2, true, true, true);
+    Database db1 = new DatabaseBuilder()
+        .setName(DBNAME1)
+        .build();
+    msc.createDatabase(db1);
+    Database db2 = new DatabaseBuilder()
+        .setName(DBNAME2)
+        .build();
+    msc.createDatabase(db2);
+    Table tab1 = new TableBuilder()
+        .setDbName(DBNAME1)
+        .setTableName(TAB1)
+        .addCol("id", "int")
+        .addCol("name", "string")
+        .build();
+    msc.createTable(tab1);
+    Table tab2 = new TableBuilder()
+        .setDbName(DBNAME1)
+        .setTableName(TAB2)
+        .addCol("id", "int")
+        .addPartCol("name", "string")
+        .build();
+    msc.createTable(tab2);
+    Partition part1 = new PartitionBuilder()
+        .fromTable(tab2)
+        .addValue("value1")
+        .build();
+    msc.add_partition(part1);
+    Partition part2 = new PartitionBuilder()
+        .fromTable(tab2)
+        .addValue("value2")
+        .build();
+    msc.add_partition(part2);
+    Index index = new IndexBuilder()
+        .setDbAndTableName(tab1)
+        .setIndexName(INDEX1)
+        .setDeferredRebuild(true)
+        .addCol("id", "int")
+        .build();
+    msc.createIndex(index, new TableBuilder().fromIndex(index).build());
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    msc.close();
+  }
+
+  @Test
+  public void testDefaultFilter() throws Exception {
+    assertNotNull(msc.getTable(DBNAME1, TAB1));
+    assertEquals(3, msc.getTables(DBNAME1, "*").size());
+    assertEquals(3, msc.getAllTables(DBNAME1).size());
+    assertEquals(1, msc.getTables(DBNAME1, TAB2).size());
+    assertEquals(0, msc.getAllTables(DBNAME2).size());
+
+    assertNotNull(msc.getDatabase(DBNAME1));
+    assertEquals(3, msc.getDatabases("*").size());
+    assertEquals(3, msc.getAllDatabases().size());
+    assertEquals(1, msc.getDatabases(DBNAME1).size());
+
+    assertNotNull(msc.getPartition(DBNAME1, TAB2, "name=value1"));
+    assertEquals(1, msc.getPartitionsByNames(DBNAME1, TAB2, Lists.newArrayList("name=value1")).size());
+
+    assertNotNull(msc.getIndex(DBNAME1, TAB1, INDEX1));
+  }
+
+  @Test
+  public void testDummyFilterForTables() throws Exception {
+    DummyMetaStoreFilterHookImpl.blockResults = true;
+    try {
+      msc.getTable(DBNAME1, TAB1);
+      fail("getTable() should fail with blocking mode");
+    } catch (NoSuchObjectException e) {
+      // Excepted
+    }
+    assertEquals(0, msc.getTables(DBNAME1, "*").size());
+    assertEquals(0, msc.getAllTables(DBNAME1).size());
+    assertEquals(0, msc.getTables(DBNAME1, TAB2).size());
+  }
+
+  @Test
+  public void testDummyFilterForDb() throws Exception {
+    DummyMetaStoreFilterHookImpl.blockResults = true;
+    try {
+      assertNotNull(msc.getDatabase(DBNAME1));
+      fail("getDatabase() should fail with blocking mode");
+    } catch (NoSuchObjectException e) {
+        // Excepted
+    }
+    assertEquals(0, msc.getDatabases("*").size());
+    assertEquals(0, msc.getAllDatabases().size());
+    assertEquals(0, msc.getDatabases(DBNAME1).size());
+  }
+
+  @Test
+  public void testDummyFilterForPartition() throws Exception {
+    DummyMetaStoreFilterHookImpl.blockResults = true;
+    try {
+      assertNotNull(msc.getPartition(DBNAME1, TAB2, "name=value1"));
+      fail("getPartition() should fail with blocking mode");
+    } catch (NoSuchObjectException e) {
+      // Excepted
+    }
+    assertEquals(0, msc.getPartitionsByNames(DBNAME1, TAB2,
+        Lists.newArrayList("name=value1")).size());
+  }
+
+  @Test
+  public void testDummyFilterForIndex() throws Exception {
+    DummyMetaStoreFilterHookImpl.blockResults = true;
+    try {
+      assertNotNull(msc.getIndex(DBNAME1, TAB1, INDEX1));
+      fail("getPartition() should fail with blocking mode");
+    } catch (NoSuchObjectException e) {
+      // Excepted
+    }
+  }
+
+}