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 2018/01/16 21:36:29 UTC

[1/7] hive git commit: HIVE-17982 Move metastore specific itests. This closes #279. (Alan Gates, reviewed by Peter Vary)

Repository: hive
Updated Branches:
  refs/heads/master fde503dca -> d9801d9c6


http://git-wip-us.apache.org/repos/asf/hive/blob/d9801d9c/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestPartitionNameWhitelistValidation.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestPartitionNameWhitelistValidation.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestPartitionNameWhitelistValidation.java
new file mode 100644
index 0000000..180a666
--- /dev/null
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestPartitionNameWhitelistValidation.java
@@ -0,0 +1,122 @@
+/*
+ * 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.*;
+
+import java.util.ArrayList;
+import java.util.List;
+
+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.junit.BeforeClass;
+import org.junit.Test;
+
+// Validate the metastore client call validatePartitionNameCharacters to ensure it throws
+// an exception if partition fields contain Unicode characters or commas
+
+public class TestPartitionNameWhitelistValidation {
+
+  private static final String partitionValidationPattern = "[\\x20-\\x7E&&[^,]]*";
+  private static Configuration conf;
+  private static HiveMetaStoreClient msc;
+
+  @BeforeClass
+  public static void setupBeforeClass() throws Exception {
+    System.setProperty(ConfVars.PARTITION_NAME_WHITELIST_PATTERN.toString(), partitionValidationPattern);
+    conf = MetastoreConf.newMetastoreConf();
+    MetaStoreTestUtils.setConfForStandloneMode(conf);
+    msc = new HiveMetaStoreClient(conf);
+  }
+
+  // Runs an instance of DisallowUnicodePreEventListener
+  // Returns whether or not it succeeded
+  private boolean runValidation(List<String> partVals) {
+    try {
+      msc.validatePartitionNameCharacters(partVals);
+    } catch (Exception e) {
+      return false;
+    }
+
+    return true;
+ }
+
+  // Sample data
+  private List<String> getPartValsWithUnicode() {
+    List<String> partVals = new ArrayList<>();
+    partVals.add("klâwen");
+    partVals.add("tägelîch");
+
+    return partVals;
+  }
+
+  private List<String> getPartValsWithCommas() {
+    List<String> partVals = new ArrayList<>();
+    partVals.add("a,b");
+    partVals.add("c,d,e,f");
+
+    return partVals;
+  }
+
+  private List<String> getPartValsWithValidCharacters() {
+    List<String> partVals = new ArrayList<>();
+    partVals.add("part1");
+    partVals.add("part2");
+
+    return partVals;
+  }
+
+  @Test
+  public void testAddPartitionWithCommas() {
+    assertFalse("Add a partition with commas in name",
+        runValidation(getPartValsWithCommas()));
+  }
+
+  @Test
+  public void testAddPartitionWithUnicode() {
+    assertFalse("Add a partition with unicode characters in name",
+        runValidation(getPartValsWithUnicode()));
+  }
+
+  @Test
+  public void testAddPartitionWithValidPartVal() {
+    assertTrue("Add a partition with unicode characters in name",
+        runValidation(getPartValsWithValidCharacters()));
+  }
+
+  @Test
+  public void testAppendPartitionWithUnicode() {
+    assertFalse("Append a partition with unicode characters in name",
+        runValidation(getPartValsWithUnicode()));
+  }
+
+  @Test
+  public void testAppendPartitionWithCommas() {
+    assertFalse("Append a partition with unicode characters in name",
+        runValidation(getPartValsWithCommas()));
+  }
+
+  @Test
+  public void testAppendPartitionWithValidCharacters() {
+    assertTrue("Append a partition with no unicode characters in name",
+        runValidation(getPartValsWithValidCharacters()));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/d9801d9c/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestRemoteHiveMetaStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestRemoteHiveMetaStore.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestRemoteHiveMetaStore.java
new file mode 100644
index 0000000..8976474
--- /dev/null
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestRemoteHiveMetaStore.java
@@ -0,0 +1,62 @@
+/*
+ * 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.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
+import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge;
+import org.junit.Assert;
+import org.junit.Before;
+
+
+public class TestRemoteHiveMetaStore extends TestHiveMetaStore {
+  private static boolean isServerStarted = false;
+  protected static int port;
+
+  public TestRemoteHiveMetaStore() {
+    super();
+    isThriftClient = true;
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    super.setUp();
+
+    if (isServerStarted) {
+      Assert.assertNotNull("Unable to connect to the MetaStore server", client);
+      MetastoreConf.setVar(conf, ConfVars.THRIFT_URIS, "thrift://localhost:" + port);
+      return;
+    }
+
+    port = MetaStoreTestUtils.startMetaStoreWithRetry(HadoopThriftAuthBridge.getBridge(),
+        conf);
+    System.out.println("Starting MetaStore Server on port " + port);
+    isServerStarted = true;
+
+    // This is default case with setugi off for both client and server
+    client = createClient();
+  }
+
+  @Override
+  protected HiveMetaStoreClient createClient() throws Exception {
+    MetastoreConf.setVar(conf, ConfVars.THRIFT_URIS, "thrift://localhost:" + port);
+    MetastoreConf.setBoolVar(conf, ConfVars.EXECUTE_SET_UGI, false);
+    return new HiveMetaStoreClient(conf);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/d9801d9c/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestRemoteHiveMetaStoreIpAddress.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestRemoteHiveMetaStoreIpAddress.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestRemoteHiveMetaStoreIpAddress.java
new file mode 100644
index 0000000..370cd28
--- /dev/null
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestRemoteHiveMetaStoreIpAddress.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.metastore;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.api.Database;
+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.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ *
+ * TestRemoteHiveMetaStoreIpAddress.
+ *
+ * Test which checks that the remote Hive metastore stores the proper IP address using
+ * IpAddressListener
+ */
+public class TestRemoteHiveMetaStoreIpAddress {
+  private static final Logger LOG = LoggerFactory.getLogger(TestRemoteHiveMetaStoreIpAddress.class);
+  private static Configuration conf;
+  private static HiveMetaStoreClient msc;
+
+  @Before
+  public void setUp() throws Exception {
+    conf = MetastoreConf.newMetastoreConf();
+
+
+    System.setProperty(ConfVars.EVENT_LISTENERS.toString(), IpAddressListener.class.getName());
+    MetaStoreTestUtils.setConfForStandloneMode(conf);
+    int port = MetaStoreTestUtils.startMetaStoreWithRetry(HadoopThriftAuthBridge.getBridge(), conf);
+    LOG.debug("Starting MetaStore Server on port " + port);
+    MetastoreConf.setVar(conf, ConfVars.THRIFT_URIS, "thrift://localhost:" + port);
+
+    msc = new HiveMetaStoreClient(conf);
+  }
+
+  @Test
+  public void testIpAddress() throws Exception {
+    Database db = new Database();
+    db.setName("testIpAddressIp");
+    msc.createDatabase(db);
+    msc.dropDatabase(db.getName());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/d9801d9c/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestRemoteUGIHiveMetaStoreIpAddress.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestRemoteUGIHiveMetaStoreIpAddress.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestRemoteUGIHiveMetaStoreIpAddress.java
new file mode 100644
index 0000000..92d2d0e
--- /dev/null
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestRemoteUGIHiveMetaStoreIpAddress.java
@@ -0,0 +1,28 @@
+/*
+ * 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.metastore.conf.MetastoreConf;
+
+public class TestRemoteUGIHiveMetaStoreIpAddress extends TestRemoteHiveMetaStoreIpAddress {
+  public TestRemoteUGIHiveMetaStoreIpAddress() {
+    System.setProperty(MetastoreConf.ConfVars.EXECUTE_SET_UGI.toString(), "true");
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/d9801d9c/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestRetryingHMSHandler.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestRetryingHMSHandler.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestRetryingHMSHandler.java
new file mode 100644
index 0000000..badcd60
--- /dev/null
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestRetryingHMSHandler.java
@@ -0,0 +1,81 @@
+/*
+ * 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.concurrent.TimeUnit;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.Table;
+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.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * TestRetryingHMSHandler. Test case for
+ * {@link org.apache.hadoop.hive.metastore.RetryingHMSHandler}
+ */
+public class TestRetryingHMSHandler {
+  private Configuration conf;
+  private HiveMetaStoreClient msc;
+
+  @Before
+  public void setUp() throws Exception {
+    System.setProperty("hive.metastore.pre.event.listeners",
+        AlternateFailurePreListener.class.getName());
+    conf = MetastoreConf.newMetastoreConf();
+    MetastoreConf.setLongVar(conf, ConfVars.THRIFT_CONNECTION_RETRIES, 3);
+    MetastoreConf.setLongVar(conf, ConfVars.HMSHANDLERATTEMPTS, 2);
+    MetastoreConf.setTimeVar(conf, ConfVars.HMSHANDLERINTERVAL, 0, TimeUnit.MILLISECONDS);
+    MetastoreConf.setBoolVar(conf, ConfVars.HMSHANDLERFORCERELOADCONF, false);
+    MetaStoreTestUtils.setConfForStandloneMode(conf);
+    int port = MetaStoreTestUtils.startMetaStoreWithRetry(HadoopThriftAuthBridge.getBridge(), conf);
+    MetastoreConf.setVar(conf, ConfVars.THRIFT_URIS, "thrift://localhost:" + port);
+    msc = new HiveMetaStoreClient(conf);
+  }
+
+  // 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
+  @Test
+  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());
+
+    Table tbl = new TableBuilder()
+        .setDbName(dbName)
+        .setTableName(tblName)
+        .addCol("c1", ColumnType.STRING_TYPE_NAME)
+        .build();
+
+    msc.createTable(tbl);
+
+    Assert.assertEquals(4, AlternateFailurePreListener.getCallCount());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/d9801d9c/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestSetUGIOnBothClientServer.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestSetUGIOnBothClientServer.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestSetUGIOnBothClientServer.java
new file mode 100644
index 0000000..e34d089
--- /dev/null
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestSetUGIOnBothClientServer.java
@@ -0,0 +1,31 @@
+/*
+ * 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.metastore.conf.MetastoreConf;
+
+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(MetastoreConf.ConfVars.EXECUTE_SET_UGI.toString(), "true");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/d9801d9c/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestSetUGIOnOnlyClient.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestSetUGIOnOnlyClient.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestSetUGIOnOnlyClient.java
new file mode 100644
index 0000000..beff656
--- /dev/null
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestSetUGIOnOnlyClient.java
@@ -0,0 +1,32 @@
+/*
+ * 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.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
+
+public class TestSetUGIOnOnlyClient extends TestRemoteHiveMetaStore{
+
+  @Override
+  protected HiveMetaStoreClient createClient() throws Exception {
+    MetastoreConf.setVar(conf, ConfVars.THRIFT_URIS, "thrift://localhost:" + port);
+    MetastoreConf.setBoolVar(conf, ConfVars.EXECUTE_SET_UGI, true);
+    return new HiveMetaStoreClient(conf);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/d9801d9c/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestSetUGIOnOnlyServer.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestSetUGIOnOnlyServer.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestSetUGIOnOnlyServer.java
new file mode 100644
index 0000000..bec5a55
--- /dev/null
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestSetUGIOnOnlyServer.java
@@ -0,0 +1,32 @@
+/*
+ * 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.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
+
+public class TestSetUGIOnOnlyServer extends TestSetUGIOnBothClientServer {
+
+  @Override
+  protected HiveMetaStoreClient createClient() throws Exception {
+    MetastoreConf.setVar(conf, ConfVars.THRIFT_URIS, "thrift://localhost:" + port);
+    MetastoreConf.setBoolVar(conf, ConfVars.EXECUTE_SET_UGI, false);
+    return new HiveMetaStoreClient(conf);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/d9801d9c/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/cache/TestCachedStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/cache/TestCachedStore.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/cache/TestCachedStore.java
index c0e84fc..b9a8f61 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/cache/TestCachedStore.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/cache/TestCachedStore.java
@@ -25,6 +25,7 @@ import java.util.Map;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.common.ndv.hll.HyperLogLog;
+import org.apache.hadoop.hive.metastore.MetaStoreTestUtils;
 import org.apache.hadoop.hive.metastore.ObjectStore;
 import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.TestObjectStore.MockPartitionExpressionProxy;
@@ -60,8 +61,7 @@ public class TestCachedStore {
   public void setUp() throws Exception {
     Configuration conf = MetastoreConf.newMetastoreConf();
     MetastoreConf.setBoolVar(conf, MetastoreConf.ConfVars.HIVE_IN_TEST, true);
-    MetastoreConf.setVar(conf, MetastoreConf.ConfVars.EXPRESSION_PROXY_CLASS,
-        MockPartitionExpressionProxy.class.getName());
+    MetaStoreTestUtils.setConfForStandloneMode(conf);
     objectStore = new ObjectStore();
     objectStore.setConf(conf);
     cachedStore = new CachedStore();


[2/7] hive git commit: HIVE-17982 Move metastore specific itests. This closes #279. (Alan Gates, reviewed by Peter Vary)

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/d9801d9c/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreTxns.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreTxns.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreTxns.java
new file mode 100644
index 0000000..d4cedb0
--- /dev/null
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreTxns.java
@@ -0,0 +1,264 @@
+/*
+ * 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.common.ValidTxnList;
+import org.apache.hadoop.hive.common.ValidReadTxnList;
+import org.apache.hadoop.hive.metastore.api.DataOperationType;
+import org.apache.hadoop.hive.metastore.api.HeartbeatTxnRangeResponse;
+import org.apache.hadoop.hive.metastore.api.LockResponse;
+import org.apache.hadoop.hive.metastore.api.LockState;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.txn.TxnDbUtil;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.List;
+
+/**
+ * Unit tests for {@link org.apache.hadoop.hive.metastore.HiveMetaStoreClient}.  For now this just has
+ * transaction and locking tests.  The goal here is not to test all
+ * functionality possible through the interface, as all permutations of DB
+ * operations should be tested in the appropriate DB handler classes.  The
+ * goal is to test that we can properly pass the messages through the thrift
+ * service.
+ *
+ * This is in the ql directory rather than the metastore directory because it
+ * required the hive-exec jar, and hive-exec jar already depends on
+ * hive-metastore jar, thus I can't make hive-metastore depend on hive-exec.
+ */
+public class TestHiveMetaStoreTxns {
+
+  private final Configuration conf = MetastoreConf.newMetastoreConf();
+  private IMetaStoreClient client;
+
+  @Test
+  public void testTxns() throws Exception {
+    List<Long> tids = client.openTxns("me", 3).getTxn_ids();
+    Assert.assertEquals(1L, (long) tids.get(0));
+    Assert.assertEquals(2L, (long) tids.get(1));
+    Assert.assertEquals(3L, (long) tids.get(2));
+    client.rollbackTxn(1);
+    client.commitTxn(2);
+    ValidTxnList validTxns = client.getValidTxns();
+    Assert.assertFalse(validTxns.isTxnValid(1));
+    Assert.assertTrue(validTxns.isTxnValid(2));
+    Assert.assertFalse(validTxns.isTxnValid(3));
+    Assert.assertFalse(validTxns.isTxnValid(4));
+  }
+
+  @Test
+  public void testOpenTxnNotExcluded() throws Exception {
+    List<Long> tids = client.openTxns("me", 3).getTxn_ids();
+    Assert.assertEquals(1L, (long) tids.get(0));
+    Assert.assertEquals(2L, (long) tids.get(1));
+    Assert.assertEquals(3L, (long) tids.get(2));
+    client.rollbackTxn(1);
+    client.commitTxn(2);
+    ValidTxnList validTxns = client.getValidTxns(3);
+    Assert.assertFalse(validTxns.isTxnValid(1));
+    Assert.assertTrue(validTxns.isTxnValid(2));
+    Assert.assertTrue(validTxns.isTxnValid(3));
+    Assert.assertFalse(validTxns.isTxnValid(4));
+  }
+
+  @Test
+  public void testTxnRange() throws Exception {
+    ValidTxnList validTxns = client.getValidTxns();
+    Assert.assertEquals(ValidTxnList.RangeResponse.NONE,
+        validTxns.isTxnRangeValid(1L, 3L));
+    List<Long> tids = client.openTxns("me", 5).getTxn_ids();
+
+    HeartbeatTxnRangeResponse rsp = client.heartbeatTxnRange(1, 5);
+    Assert.assertEquals(0, rsp.getNosuch().size());
+    Assert.assertEquals(0, rsp.getAborted().size());
+
+    client.rollbackTxn(1L);
+    client.commitTxn(2L);
+    client.commitTxn(3L);
+    client.commitTxn(4L);
+    validTxns = client.getValidTxns();
+    System.out.println("validTxns = " + validTxns);
+    Assert.assertEquals(ValidTxnList.RangeResponse.ALL,
+        validTxns.isTxnRangeValid(2L, 2L));
+    Assert.assertEquals(ValidTxnList.RangeResponse.ALL,
+        validTxns.isTxnRangeValid(2L, 3L));
+    Assert.assertEquals(ValidTxnList.RangeResponse.ALL,
+        validTxns.isTxnRangeValid(2L, 4L));
+    Assert.assertEquals(ValidTxnList.RangeResponse.ALL,
+        validTxns.isTxnRangeValid(3L, 4L));
+
+    Assert.assertEquals(ValidTxnList.RangeResponse.SOME,
+        validTxns.isTxnRangeValid(1L, 4L));
+    Assert.assertEquals(ValidTxnList.RangeResponse.SOME,
+        validTxns.isTxnRangeValid(2L, 5L));
+    Assert.assertEquals(ValidTxnList.RangeResponse.SOME,
+        validTxns.isTxnRangeValid(1L, 2L));
+    Assert.assertEquals(ValidTxnList.RangeResponse.SOME,
+        validTxns.isTxnRangeValid(4L, 5L));
+
+    Assert.assertEquals(ValidTxnList.RangeResponse.NONE,
+        validTxns.isTxnRangeValid(1L, 1L));
+    Assert.assertEquals(ValidTxnList.RangeResponse.NONE,
+        validTxns.isTxnRangeValid(5L, 10L));
+
+    validTxns = new ValidReadTxnList("10:5:4,5,6:");
+    Assert.assertEquals(ValidTxnList.RangeResponse.NONE,
+        validTxns.isTxnRangeValid(4,6));
+    Assert.assertEquals(ValidTxnList.RangeResponse.ALL,
+        validTxns.isTxnRangeValid(7, 10));
+    Assert.assertEquals(ValidTxnList.RangeResponse.SOME,
+        validTxns.isTxnRangeValid(7, 11));
+    Assert.assertEquals(ValidTxnList.RangeResponse.SOME,
+        validTxns.isTxnRangeValid(3, 6));
+    Assert.assertEquals(ValidTxnList.RangeResponse.SOME,
+        validTxns.isTxnRangeValid(4, 7));
+    Assert.assertEquals(ValidTxnList.RangeResponse.SOME,
+        validTxns.isTxnRangeValid(1, 12));
+    Assert.assertEquals(ValidTxnList.RangeResponse.ALL,
+        validTxns.isTxnRangeValid(1, 3));
+  }
+
+  @Test
+  public void testLocks() throws Exception {
+    LockRequestBuilder rqstBuilder = new LockRequestBuilder();
+    rqstBuilder.addLockComponent(new LockComponentBuilder()
+        .setDbName("mydb")
+        .setTableName("mytable")
+        .setPartitionName("mypartition")
+        .setExclusive()
+        .setOperationType(DataOperationType.NO_TXN)
+        .build());
+    rqstBuilder.addLockComponent(new LockComponentBuilder()
+        .setDbName("mydb")
+        .setTableName("yourtable")
+        .setSemiShared()
+        .setOperationType(DataOperationType.NO_TXN)
+        .build());
+    rqstBuilder.addLockComponent(new LockComponentBuilder()
+        .setDbName("yourdb")
+        .setOperationType(DataOperationType.NO_TXN)
+        .setShared()
+        .build());
+    rqstBuilder.setUser("fred");
+
+    LockResponse res = client.lock(rqstBuilder.build());
+    Assert.assertEquals(1L, res.getLockid());
+    Assert.assertEquals(LockState.ACQUIRED, res.getState());
+
+    res = client.checkLock(1);
+    Assert.assertEquals(1L, res.getLockid());
+    Assert.assertEquals(LockState.ACQUIRED, res.getState());
+
+    client.heartbeat(0, 1);
+
+    client.unlock(1);
+  }
+
+  @Test
+  public void testLocksWithTxn() throws Exception {
+    long txnid = client.openTxn("me");
+
+    LockRequestBuilder rqstBuilder = new LockRequestBuilder();
+    rqstBuilder.setTransactionId(txnid)
+      .addLockComponent(new LockComponentBuilder()
+        .setDbName("mydb")
+        .setTableName("mytable")
+        .setPartitionName("mypartition")
+        .setSemiShared()
+        .setOperationType(DataOperationType.UPDATE)
+        .build())
+      .addLockComponent(new LockComponentBuilder()
+        .setDbName("mydb")
+        .setTableName("yourtable")
+        .setSemiShared()
+        .setOperationType(DataOperationType.UPDATE)
+        .build())
+      .addLockComponent(new LockComponentBuilder()
+        .setDbName("yourdb")
+        .setShared()
+        .setOperationType(DataOperationType.SELECT)
+        .build())
+      .setUser("fred");
+
+    LockResponse res = client.lock(rqstBuilder.build());
+    Assert.assertEquals(1L, res.getLockid());
+    Assert.assertEquals(LockState.ACQUIRED, res.getState());
+
+    res = client.checkLock(1);
+    Assert.assertEquals(1L, res.getLockid());
+    Assert.assertEquals(LockState.ACQUIRED, res.getState());
+
+    client.heartbeat(txnid, 1);
+
+    client.commitTxn(txnid);
+  }
+
+  @Test
+  public void stringifyValidTxns() throws Exception {
+    // Test with just high water mark
+    ValidTxnList validTxns = new ValidReadTxnList("1:" + Long.MAX_VALUE + "::");
+    String asString = validTxns.toString();
+    Assert.assertEquals("1:" + Long.MAX_VALUE + "::", asString);
+    validTxns = new ValidReadTxnList(asString);
+    Assert.assertEquals(1, validTxns.getHighWatermark());
+    Assert.assertNotNull(validTxns.getInvalidTransactions());
+    Assert.assertEquals(0, validTxns.getInvalidTransactions().length);
+    asString = validTxns.toString();
+    Assert.assertEquals("1:" + Long.MAX_VALUE + "::", asString);
+    validTxns = new ValidReadTxnList(asString);
+    Assert.assertEquals(1, validTxns.getHighWatermark());
+    Assert.assertNotNull(validTxns.getInvalidTransactions());
+    Assert.assertEquals(0, validTxns.getInvalidTransactions().length);
+
+    // Test with open transactions
+    validTxns = new ValidReadTxnList("10:3:5:3");
+    asString = validTxns.toString();
+    if (!asString.equals("10:3:3:5") && !asString.equals("10:3:5:3")) {
+      Assert.fail("Unexpected string value " + asString);
+    }
+    validTxns = new ValidReadTxnList(asString);
+    Assert.assertEquals(10, validTxns.getHighWatermark());
+    Assert.assertNotNull(validTxns.getInvalidTransactions());
+    Assert.assertEquals(2, validTxns.getInvalidTransactions().length);
+    boolean sawThree = false, sawFive = false;
+    for (long tid : validTxns.getInvalidTransactions()) {
+      if (tid == 3)  sawThree = true;
+      else if (tid == 5) sawFive = true;
+      else  Assert.fail("Unexpected value " + tid);
+    }
+    Assert.assertTrue(sawThree);
+    Assert.assertTrue(sawFive);
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    MetaStoreTestUtils.setConfForStandloneMode(conf);
+    TxnDbUtil.setConfValues(conf);
+    TxnDbUtil.prepDb(conf);
+    client = new HiveMetaStoreClient(conf);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    TxnDbUtil.cleanDb(conf);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/d9801d9c/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreWithEnvironmentContext.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreWithEnvironmentContext.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreWithEnvironmentContext.java
new file mode 100644
index 0000000..fd75247
--- /dev/null
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreWithEnvironmentContext.java
@@ -0,0 +1,187 @@
+/*
+ * 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.List;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+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.events.AddPartitionEvent;
+import org.apache.hadoop.hive.metastore.events.AlterTableEvent;
+import org.apache.hadoop.hive.metastore.events.CreateDatabaseEvent;
+import org.apache.hadoop.hive.metastore.events.CreateTableEvent;
+import org.apache.hadoop.hive.metastore.events.DropDatabaseEvent;
+import org.apache.hadoop.hive.metastore.events.DropPartitionEvent;
+import org.apache.hadoop.hive.metastore.events.DropTableEvent;
+import org.apache.hadoop.hive.metastore.events.ListenerEvent;
+import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * TestHiveMetaStoreWithEnvironmentContext. Test case for _with_environment_context
+ * calls in {@link org.apache.hadoop.hive.metastore.HiveMetaStore}
+ */
+public class TestHiveMetaStoreWithEnvironmentContext {
+
+  private Configuration conf;
+  private HiveMetaStoreClient msc;
+  private EnvironmentContext envContext;
+  private final Database db = new Database();
+  private Table table;
+  private Partition partition;
+
+  private static final String dbName = "hive3252";
+  private static final String tblName = "tmptbl";
+  private static final String renamed = "tmptbl2";
+
+  @Before
+  public void setUp() throws Exception {
+    System.setProperty("hive.metastore.event.listeners",
+        DummyListener.class.getName());
+
+    conf = MetastoreConf.newMetastoreConf();
+    MetastoreConf.setLongVar(conf, ConfVars.THRIFT_CONNECTION_RETRIES, 3);
+    MetastoreConf.setBoolVar(conf, ConfVars.HIVE_SUPPORT_CONCURRENCY, false);
+    MetaStoreTestUtils.setConfForStandloneMode(conf);
+    int port = MetaStoreTestUtils.startMetaStoreWithRetry(HadoopThriftAuthBridge.getBridge(), conf);
+    MetastoreConf.setVar(conf, ConfVars.THRIFT_URIS, "thrift://localhost:" + port);
+    msc = new HiveMetaStoreClient(conf);
+
+    msc.dropDatabase(dbName, true, true);
+
+    Map<String, String> envProperties = new HashMap<>();
+    envProperties.put("hadoop.job.ugi", "test_user");
+    envContext = new EnvironmentContext(envProperties);
+
+    db.setName(dbName);
+
+    table = new TableBuilder()
+        .setDbName(dbName)
+        .setTableName(tblName)
+        .addTableParam("a", "string")
+        .addPartCol("b", "string")
+        .addCol("a", "string")
+        .addCol("b", "string")
+        .build();
+
+
+    partition = new PartitionBuilder()
+        .fromTable(table)
+        .addValue("2011")
+        .build();
+
+    DummyListener.notifyList.clear();
+  }
+
+  @Test
+  public void testEnvironmentContext() throws Exception {
+    int listSize = 0;
+
+    List<ListenerEvent> notifyList = DummyListener.notifyList;
+    assertEquals(notifyList.size(), listSize);
+    msc.createDatabase(db);
+    listSize++;
+    assertEquals(listSize, notifyList.size());
+    CreateDatabaseEvent dbEvent = (CreateDatabaseEvent)(notifyList.get(listSize - 1));
+    assert dbEvent.getStatus();
+
+    msc.createTable(table, envContext);
+    listSize++;
+    assertEquals(notifyList.size(), listSize);
+    CreateTableEvent tblEvent = (CreateTableEvent)(notifyList.get(listSize - 1));
+    assert tblEvent.getStatus();
+    assertEquals(envContext, tblEvent.getEnvironmentContext());
+
+    table = msc.getTable(dbName, tblName);
+
+    partition.getSd().setLocation(table.getSd().getLocation() + "/part1");
+    msc.add_partition(partition, envContext);
+    listSize++;
+    assertEquals(notifyList.size(), listSize);
+    AddPartitionEvent partEvent = (AddPartitionEvent)(notifyList.get(listSize-1));
+    assert partEvent.getStatus();
+    assertEquals(envContext, partEvent.getEnvironmentContext());
+
+    List<String> partVals = new ArrayList<>();
+    partVals.add("2012");
+    msc.appendPartition(dbName, tblName, partVals, envContext);
+    listSize++;
+    assertEquals(notifyList.size(), listSize);
+    AddPartitionEvent appendPartEvent = (AddPartitionEvent)(notifyList.get(listSize-1));
+    assert appendPartEvent.getStatus();
+    assertEquals(envContext, appendPartEvent.getEnvironmentContext());
+
+    table.setTableName(renamed);
+    msc.alter_table_with_environmentContext(dbName, tblName, table, envContext);
+    listSize++;
+    assertEquals(notifyList.size(), listSize);
+    AlterTableEvent alterTableEvent = (AlterTableEvent) notifyList.get(listSize-1);
+    assert alterTableEvent.getStatus();
+    assertEquals(envContext, alterTableEvent.getEnvironmentContext());
+
+    table.setTableName(tblName);
+    msc.alter_table_with_environmentContext(dbName, renamed, table, envContext);
+    listSize++;
+    assertEquals(notifyList.size(), listSize);
+
+    List<String> dropPartVals = new ArrayList<>();
+    dropPartVals.add("2011");
+    msc.dropPartition(dbName, tblName, dropPartVals, envContext);
+    listSize++;
+    assertEquals(notifyList.size(), listSize);
+    DropPartitionEvent dropPartEvent = (DropPartitionEvent)notifyList.get(listSize - 1);
+    assert dropPartEvent.getStatus();
+    assertEquals(envContext, dropPartEvent.getEnvironmentContext());
+
+    msc.dropPartition(dbName, tblName, "b=2012", true, envContext);
+    listSize++;
+    assertEquals(notifyList.size(), listSize);
+    DropPartitionEvent dropPartByNameEvent = (DropPartitionEvent)notifyList.get(listSize - 1);
+    assert dropPartByNameEvent.getStatus();
+    assertEquals(envContext, dropPartByNameEvent.getEnvironmentContext());
+
+    msc.dropTable(dbName, tblName, true, false, envContext);
+    listSize++;
+    assertEquals(notifyList.size(), listSize);
+    DropTableEvent dropTblEvent = (DropTableEvent)notifyList.get(listSize-1);
+    assert dropTblEvent.getStatus();
+    assertEquals(envContext, dropTblEvent.getEnvironmentContext());
+
+    msc.dropDatabase(dbName);
+    listSize++;
+    assertEquals(notifyList.size(), listSize);
+
+    DropDatabaseEvent dropDB = (DropDatabaseEvent)notifyList.get(listSize-1);
+    assert dropDB.getStatus();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/d9801d9c/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMarkPartition.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMarkPartition.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMarkPartition.java
new file mode 100644
index 0000000..6854a93
--- /dev/null
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMarkPartition.java
@@ -0,0 +1,117 @@
+/*
+ * 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.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.InvalidPartitionException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.PartitionEventType;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.UnknownTableException;
+import org.apache.hadoop.hive.metastore.client.builder.DatabaseBuilder;
+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.thrift.TException;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestMarkPartition {
+
+  protected Configuration conf;
+
+  @Before
+  public void setUp() throws Exception {
+
+    System.setProperty("hive.metastore.event.clean.freq", "1s");
+    System.setProperty("hive.metastore.event.expiry.duration", "2s");
+    conf = MetastoreConf.newMetastoreConf();
+    MetastoreConf.setBoolVar(conf, MetastoreConf.ConfVars.HIVE_SUPPORT_CONCURRENCY, false);
+    MetaStoreTestUtils.setConfForStandloneMode(conf);
+
+  }
+
+  @Test
+  public void testMarkingPartitionSet() throws TException, InterruptedException {
+    HiveMetaStoreClient msc = new HiveMetaStoreClient(conf);
+
+    final String dbName = "hive2215";
+    msc.dropDatabase(dbName, true, true, true);
+    Database db = new DatabaseBuilder()
+        .setName(dbName)
+        .build();
+    msc.createDatabase(db);
+
+    final String tableName = "tmptbl";
+    msc.dropTable(dbName, tableName, true, true);
+    Table table = new TableBuilder()
+        .setDbName(dbName)
+        .setTableName(tableName)
+        .addCol("a", "string")
+        .addPartCol("b", "string")
+        .build();
+    msc.createTable(table);
+
+    Partition part = new PartitionBuilder()
+        .fromTable(table)
+        .addValue("2011")
+        .build();
+    msc.add_partition(part);
+    Map<String,String> kvs = new HashMap<>();
+    kvs.put("b", "'2011'");
+    msc.markPartitionForEvent(dbName, tableName, kvs, PartitionEventType.LOAD_DONE);
+    Assert.assertTrue(msc.isPartitionMarkedForEvent(dbName, tableName, kvs, PartitionEventType.LOAD_DONE));
+    Thread.sleep(3000);
+    Assert.assertFalse(msc.isPartitionMarkedForEvent(dbName, tableName, kvs, PartitionEventType.LOAD_DONE));
+
+    kvs.put("b", "'2012'");
+    Assert.assertFalse(msc.isPartitionMarkedForEvent(dbName, tableName, kvs, PartitionEventType.LOAD_DONE));
+    try {
+      msc.markPartitionForEvent(dbName, "tmptbl2", kvs, PartitionEventType.LOAD_DONE);
+      Assert.fail("Expected UnknownTableException");
+    } catch (UnknownTableException e) {
+      // All good
+    } catch(Exception e){
+      Assert.fail("Expected UnknownTableException");
+    }
+    try{
+      msc.isPartitionMarkedForEvent(dbName, "tmptbl2", kvs, PartitionEventType.LOAD_DONE);
+      Assert.fail("Expected UnknownTableException");
+    } catch (UnknownTableException e) {
+      // All good
+    } catch(Exception e){
+      Assert.fail("Expected UnknownTableException, received " + e.getClass().getName());
+    }
+    kvs.put("a", "'2012'");
+    try {
+      msc.isPartitionMarkedForEvent(dbName, tableName, kvs, PartitionEventType.LOAD_DONE);
+      Assert.fail("Expected InvalidPartitionException");
+    } catch (InvalidPartitionException e) {
+      // All good
+    } catch(Exception e){
+      Assert.fail("Expected InvalidPartitionException, received " + e.getClass().getName());
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/d9801d9c/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMarkPartitionRemote.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMarkPartitionRemote.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMarkPartitionRemote.java
new file mode 100644
index 0000000..ac1cc4c
--- /dev/null
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMarkPartitionRemote.java
@@ -0,0 +1,36 @@
+/*
+ * 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.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
+import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge;
+import org.junit.Before;
+
+public class TestMarkPartitionRemote extends TestMarkPartition {
+
+  @Before
+  public void startServer() throws Exception {
+    int port = MetaStoreTestUtils.findFreePort();
+    MetastoreConf.setVar(conf, ConfVars.THRIFT_URIS, "thrift://localhost:" + port);
+    MetastoreConf.setLongVar(conf, ConfVars.THRIFT_CONNECTION_RETRIES, 3);
+    MetaStoreTestUtils.setConfForStandloneMode(conf);
+    MetaStoreTestUtils.startMetaStore(port, HadoopThriftAuthBridge.getBridge(), conf);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/d9801d9c/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEndFunctionListener.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEndFunctionListener.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEndFunctionListener.java
new file mode 100644
index 0000000..25e3a95
--- /dev/null
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEndFunctionListener.java
@@ -0,0 +1,145 @@
+/*
+ * 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.api.Database;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+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.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.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * TestMetaStoreEventListener. Test case for
+ * {@link org.apache.hadoop.hive.metastore.MetaStoreEndFunctionListener}
+ */
+public class TestMetaStoreEndFunctionListener {
+  private Configuration conf;
+  private HiveMetaStoreClient msc;
+
+  @Before
+  public void setUp() throws Exception {
+    System.setProperty("hive.metastore.event.listeners",
+        DummyListener.class.getName());
+    System.setProperty("hive.metastore.pre.event.listeners",
+        DummyPreListener.class.getName());
+    System.setProperty("hive.metastore.end.function.listeners",
+        DummyEndFunctionListener.class.getName());
+    conf = MetastoreConf.newMetastoreConf();
+    MetastoreConf.setLongVar(conf, ConfVars.THRIFT_CONNECTION_RETRIES, 3);
+    MetastoreConf.setBoolVar(conf, ConfVars.HIVE_SUPPORT_CONCURRENCY, false);
+    MetaStoreTestUtils.setConfForStandloneMode(conf);
+    int port = MetaStoreTestUtils.startMetaStoreWithRetry(HadoopThriftAuthBridge.getBridge(), conf);
+    MetastoreConf.setVar(conf, ConfVars.THRIFT_URIS, "thrift://localhost:" + port);
+    msc = new HiveMetaStoreClient(conf);
+  }
+
+  @Test
+  public void testEndFunctionListener() throws Exception {
+    /* Objective here is to ensure that when exceptions are thrown in HiveMetaStore in API methods
+     * they bubble up and are stored in the MetaStoreEndFunctionContext objects
+     */
+    String dbName = "hive3524";
+    String tblName = "tmptbl";
+    int listSize;
+
+    Database db = new DatabaseBuilder()
+        .setName(dbName)
+        .build();
+    msc.createDatabase(db);
+
+    try {
+      msc.getDatabase("UnknownDB");
+    } catch (Exception e) {
+      // All good
+    }
+    listSize = DummyEndFunctionListener.funcNameList.size();
+    String func_name = DummyEndFunctionListener.funcNameList.get(listSize-1);
+    MetaStoreEndFunctionContext context = DummyEndFunctionListener.contextList.get(listSize-1);
+    assertEquals(func_name,"get_database");
+    assertFalse(context.isSuccess());
+    Exception e = context.getException();
+    assertTrue((e!=null));
+    assertTrue((e instanceof NoSuchObjectException));
+    assertEquals(context.getInputTableName(), null);
+
+    String unknownTable = "UnknownTable";
+    Table table = new TableBuilder()
+        .setDbName(db)
+        .setTableName(tblName)
+        .addCol("a", "string")
+        .addPartCol("b", "string")
+        .build();
+    msc.createTable(table);
+    try {
+      msc.getTable(dbName, unknownTable);
+    } catch (Exception e1) {
+      // All good
+    }
+    listSize = DummyEndFunctionListener.funcNameList.size();
+    func_name = DummyEndFunctionListener.funcNameList.get(listSize-1);
+    context = DummyEndFunctionListener.contextList.get(listSize-1);
+    assertEquals(func_name,"get_table");
+    assertFalse(context.isSuccess());
+    e = context.getException();
+    assertTrue((e!=null));
+    assertTrue((e instanceof NoSuchObjectException));
+    assertEquals(context.getInputTableName(), unknownTable);
+
+    try {
+      msc.getPartition("hive3524", tblName, "b=2012");
+    } catch (Exception e2) {
+      // All good
+    }
+    listSize = DummyEndFunctionListener.funcNameList.size();
+    func_name = DummyEndFunctionListener.funcNameList.get(listSize-1);
+    context = DummyEndFunctionListener.contextList.get(listSize-1);
+    assertEquals(func_name,"get_partition_by_name");
+    assertFalse(context.isSuccess());
+    e = context.getException();
+    assertTrue((e!=null));
+    assertTrue((e instanceof NoSuchObjectException));
+    assertEquals(context.getInputTableName(), tblName);
+    try {
+      msc.dropTable(dbName, unknownTable);
+    } catch (Exception e4) {
+      // All good
+    }
+    listSize = DummyEndFunctionListener.funcNameList.size();
+    func_name = DummyEndFunctionListener.funcNameList.get(listSize-1);
+    context = DummyEndFunctionListener.contextList.get(listSize-1);
+    assertEquals(func_name,"get_table");
+    assertFalse(context.isSuccess());
+    e = context.getException();
+    assertTrue((e!=null));
+    assertTrue((e instanceof NoSuchObjectException));
+    assertEquals(context.getInputTableName(), "UnknownTable");
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/d9801d9c/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListener.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListener.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListener.java
new file mode 100644
index 0000000..1508ee5
--- /dev/null
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListener.java
@@ -0,0 +1,557 @@
+/*
+ * 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.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+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.Partition;
+import org.apache.hadoop.hive.metastore.api.PartitionEventType;
+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.events.AddIndexEvent;
+import org.apache.hadoop.hive.metastore.events.AddPartitionEvent;
+import org.apache.hadoop.hive.metastore.events.AlterIndexEvent;
+import org.apache.hadoop.hive.metastore.events.AlterPartitionEvent;
+import org.apache.hadoop.hive.metastore.events.AlterTableEvent;
+import org.apache.hadoop.hive.metastore.events.ConfigChangeEvent;
+import org.apache.hadoop.hive.metastore.events.CreateDatabaseEvent;
+import org.apache.hadoop.hive.metastore.events.CreateTableEvent;
+import org.apache.hadoop.hive.metastore.events.DropDatabaseEvent;
+import org.apache.hadoop.hive.metastore.events.DropIndexEvent;
+import org.apache.hadoop.hive.metastore.events.DropPartitionEvent;
+import org.apache.hadoop.hive.metastore.events.DropTableEvent;
+import org.apache.hadoop.hive.metastore.events.ListenerEvent;
+import org.apache.hadoop.hive.metastore.events.LoadPartitionDoneEvent;
+import org.apache.hadoop.hive.metastore.events.PreAddIndexEvent;
+import org.apache.hadoop.hive.metastore.events.PreAddPartitionEvent;
+import org.apache.hadoop.hive.metastore.events.PreAlterIndexEvent;
+import org.apache.hadoop.hive.metastore.events.PreAlterPartitionEvent;
+import org.apache.hadoop.hive.metastore.events.PreAlterTableEvent;
+import org.apache.hadoop.hive.metastore.events.PreCreateDatabaseEvent;
+import org.apache.hadoop.hive.metastore.events.PreCreateTableEvent;
+import org.apache.hadoop.hive.metastore.events.PreDropDatabaseEvent;
+import org.apache.hadoop.hive.metastore.events.PreDropIndexEvent;
+import org.apache.hadoop.hive.metastore.events.PreDropPartitionEvent;
+import org.apache.hadoop.hive.metastore.events.PreDropTableEvent;
+import org.apache.hadoop.hive.metastore.events.PreEventContext;
+import org.apache.hadoop.hive.metastore.events.PreLoadPartitionDoneEvent;
+import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotSame;
+import static org.junit.Assert.assertTrue;
+
+import com.google.common.collect.Lists;
+
+import junit.framework.TestCase;
+
+/**
+ * TestMetaStoreEventListener. Test case for
+ * {@link org.apache.hadoop.hive.metastore.MetaStoreEventListener} and
+ * {@link org.apache.hadoop.hive.metastore.MetaStorePreEventListener}
+ */
+public class TestMetaStoreEventListener {
+  private Configuration conf;
+  private HiveMetaStoreClient msc;
+
+  private static final String dbName = "hive2038";
+  private static final String tblName = "tmptbl";
+  private static final String renamed = "tmptbl2";
+  private static final String metaConfKey = "metastore.partition.name.whitelist.pattern";
+  private static final String metaConfVal = "";
+
+  @Before
+  public void setUp() throws Exception {
+    System.setProperty("hive.metastore.event.listeners",
+        DummyListener.class.getName());
+    System.setProperty("hive.metastore.pre.event.listeners",
+        DummyPreListener.class.getName());
+
+    conf = MetastoreConf.newMetastoreConf();
+
+    MetastoreConf.setVar(conf, ConfVars.PARTITION_NAME_WHITELIST_PATTERN, metaConfVal);
+    MetastoreConf.setLongVar(conf, ConfVars.THRIFT_CONNECTION_RETRIES, 3);
+    MetastoreConf.setBoolVar(conf, ConfVars.HIVE_SUPPORT_CONCURRENCY, false);
+    MetaStoreTestUtils.setConfForStandloneMode(conf);
+    int port = MetaStoreTestUtils.startMetaStoreWithRetry(HadoopThriftAuthBridge.getBridge(), conf);
+    MetastoreConf.setVar(conf, ConfVars.THRIFT_URIS, "thrift://localhost:" + port);
+
+    msc = new HiveMetaStoreClient(conf);
+
+    msc.dropDatabase(dbName, true, true, true);
+    DummyListener.notifyList.clear();
+    DummyPreListener.notifyList.clear();
+  }
+
+  private void validateCreateDb(Database expectedDb, Database actualDb) {
+    assertEquals(expectedDb.getName(), actualDb.getName());
+    assertEquals(expectedDb.getLocationUri(), actualDb.getLocationUri());
+  }
+
+  private void validateTable(Table expectedTable, Table actualTable) {
+    assertEquals(expectedTable.getTableName(), actualTable.getTableName());
+    assertEquals(expectedTable.getDbName(), actualTable.getDbName());
+    assertEquals(expectedTable.getSd().getLocation(), actualTable.getSd().getLocation());
+  }
+
+  private void validateCreateTable(Table expectedTable, Table actualTable) {
+    validateTable(expectedTable, actualTable);
+  }
+
+  private void validateAddPartition(Partition expectedPartition, Partition actualPartition) {
+    assertEquals(expectedPartition, actualPartition);
+  }
+
+  private void validateTableInAddPartition(Table expectedTable, Table actualTable) {
+    assertEquals(expectedTable, actualTable);
+  }
+
+  private void validatePartition(Partition expectedPartition, Partition actualPartition) {
+    assertEquals(expectedPartition.getValues(), actualPartition.getValues());
+    assertEquals(expectedPartition.getDbName(), actualPartition.getDbName());
+    assertEquals(expectedPartition.getTableName(), actualPartition.getTableName());
+  }
+
+  private void validateAlterPartition(Partition expectedOldPartition,
+      Partition expectedNewPartition, String actualOldPartitionDbName,
+      String actualOldPartitionTblName,List<String> actualOldPartitionValues,
+      Partition actualNewPartition) {
+    assertEquals(expectedOldPartition.getValues(), actualOldPartitionValues);
+    assertEquals(expectedOldPartition.getDbName(), actualOldPartitionDbName);
+    assertEquals(expectedOldPartition.getTableName(), actualOldPartitionTblName);
+
+    validatePartition(expectedNewPartition, actualNewPartition);
+  }
+
+  private void validateAlterTable(Table expectedOldTable, Table expectedNewTable,
+      Table actualOldTable, Table actualNewTable) {
+    validateTable(expectedOldTable, actualOldTable);
+    validateTable(expectedNewTable, actualNewTable);
+  }
+
+  private void validateAlterTableColumns(Table expectedOldTable, Table expectedNewTable,
+      Table actualOldTable, Table actualNewTable) {
+    validateAlterTable(expectedOldTable, expectedNewTable, actualOldTable, actualNewTable);
+
+    assertEquals(expectedOldTable.getSd().getCols(), actualOldTable.getSd().getCols());
+    assertEquals(expectedNewTable.getSd().getCols(), actualNewTable.getSd().getCols());
+  }
+
+  private void validateLoadPartitionDone(String expectedTableName,
+      Map<String,String> expectedPartitionName, String actualTableName,
+      Map<String,String> actualPartitionName) {
+    assertEquals(expectedPartitionName, actualPartitionName);
+    assertEquals(expectedTableName, actualTableName);
+  }
+
+  private void validateDropPartition(Iterator<Partition> expectedPartitions, Iterator<Partition> actualPartitions) {
+    while (expectedPartitions.hasNext()){
+      assertTrue(actualPartitions.hasNext());
+      validatePartition(expectedPartitions.next(), actualPartitions.next());
+    }
+    assertFalse(actualPartitions.hasNext());
+  }
+
+  private void validateTableInDropPartition(Table expectedTable, Table actualTable) {
+    validateTable(expectedTable, actualTable);
+  }
+
+  private void validateDropTable(Table expectedTable, Table actualTable) {
+    validateTable(expectedTable, actualTable);
+  }
+
+  private void validateDropDb(Database expectedDb, Database actualDb) {
+    assertEquals(expectedDb, actualDb);
+  }
+
+  private void validateIndex(Index expectedIndex, Index actualIndex) {
+    assertEquals(expectedIndex.getDbName(), actualIndex.getDbName());
+    assertEquals(expectedIndex.getIndexName(), actualIndex.getIndexName());
+    assertEquals(expectedIndex.getIndexHandlerClass(), actualIndex.getIndexHandlerClass());
+    assertEquals(expectedIndex.getOrigTableName(), actualIndex.getOrigTableName());
+    assertEquals(expectedIndex.getIndexTableName(), actualIndex.getIndexTableName());
+    assertEquals(expectedIndex.getSd().getLocation(), actualIndex.getSd().getLocation());
+  }
+
+  private void validateAddIndex(Index expectedIndex, Index actualIndex) {
+    validateIndex(expectedIndex, actualIndex);
+  }
+
+  private void validateAlterIndex(Index expectedOldIndex, Index actualOldIndex,
+      Index expectedNewIndex, Index actualNewIndex) {
+    validateIndex(expectedOldIndex, actualOldIndex);
+    validateIndex(expectedNewIndex, actualNewIndex);
+  }
+
+  private void validateDropIndex(Index expectedIndex, Index actualIndex) {
+    validateIndex(expectedIndex, actualIndex);
+  }
+
+  @Test
+  public void testListener() throws Exception {
+    int listSize = 0;
+
+    List<ListenerEvent> notifyList = DummyListener.notifyList;
+    List<PreEventContext> preNotifyList = DummyPreListener.notifyList;
+    assertEquals(notifyList.size(), listSize);
+    assertEquals(preNotifyList.size(), listSize);
+
+    Database db = new DatabaseBuilder()
+        .setName(dbName)
+        .build();
+    msc.createDatabase(db);
+    listSize++;
+    PreCreateDatabaseEvent preDbEvent = (PreCreateDatabaseEvent)(preNotifyList.get(preNotifyList.size() - 1));
+    db = msc.getDatabase(dbName);
+    assertEquals(listSize, notifyList.size());
+    assertEquals(listSize + 1, preNotifyList.size());
+    validateCreateDb(db, preDbEvent.getDatabase());
+
+    CreateDatabaseEvent dbEvent = (CreateDatabaseEvent)(notifyList.get(listSize - 1));
+    Assert.assertTrue(dbEvent.getStatus());
+    validateCreateDb(db, dbEvent.getDatabase());
+
+    Table table = new TableBuilder()
+        .setDbName(db)
+        .setTableName(tblName)
+        .addCol("a", "string")
+        .addPartCol("b", "string")
+        .build();
+    msc.createTable(table);
+    PreCreateTableEvent preTblEvent = (PreCreateTableEvent)(preNotifyList.get(preNotifyList.size() - 1));
+    listSize++;
+    Table tbl = msc.getTable(dbName, tblName);
+    validateCreateTable(tbl, preTblEvent.getTable());
+    assertEquals(notifyList.size(), listSize);
+
+    CreateTableEvent tblEvent = (CreateTableEvent)(notifyList.get(listSize - 1));
+    Assert.assertTrue(tblEvent.getStatus());
+    validateCreateTable(tbl, tblEvent.getTable());
+
+    String indexName = "tmptbl_i";
+    Index index = new IndexBuilder()
+        .setDbAndTableName(table)
+        .setIndexName(indexName)
+        .addCol("a", "string")
+        .setDeferredRebuild(true)
+        .addIndexParam("prop1", "val1")
+        .addIndexParam("prop2", "val2")
+        .build();
+    Table indexTable = new TableBuilder()
+        .fromIndex(index)
+        .build();
+    msc.createIndex(index, indexTable);
+    listSize += 2;  // creates index table internally
+    assertEquals(notifyList.size(), listSize);
+
+    AddIndexEvent addIndexEvent = (AddIndexEvent)notifyList.get(listSize - 1);
+    Assert.assertTrue(addIndexEvent.getStatus());
+    PreAddIndexEvent preAddIndexEvent = (PreAddIndexEvent)(preNotifyList.get(preNotifyList.size() - 2));
+
+    Index oldIndex = msc.getIndex(dbName, tblName, indexName);
+
+    validateAddIndex(oldIndex, addIndexEvent.getIndex());
+
+    validateAddIndex(oldIndex, preAddIndexEvent.getIndex());
+
+    Index alteredIndex = new Index(oldIndex);
+    alteredIndex.getParameters().put("prop3", "val3");
+    msc.alter_index(dbName, tblName, indexName, alteredIndex);
+    listSize++;
+    assertEquals(notifyList.size(), listSize);
+
+    Index newIndex = msc.getIndex(dbName, tblName, indexName);
+
+    AlterIndexEvent alterIndexEvent = (AlterIndexEvent) notifyList.get(listSize - 1);
+    Assert.assertTrue(alterIndexEvent.getStatus());
+    validateAlterIndex(oldIndex, alterIndexEvent.getOldIndex(),
+        newIndex, alterIndexEvent.getNewIndex());
+
+    PreAlterIndexEvent preAlterIndexEvent = (PreAlterIndexEvent) (preNotifyList.get(preNotifyList.size() - 1));
+    validateAlterIndex(oldIndex, preAlterIndexEvent.getOldIndex(),
+        newIndex, preAlterIndexEvent.getNewIndex());
+
+    msc.dropIndex(dbName, tblName, indexName, true);
+    listSize++;
+    assertEquals(notifyList.size(), listSize);
+
+    DropIndexEvent dropIndexEvent = (DropIndexEvent) notifyList.get(listSize - 1);
+    Assert.assertTrue(dropIndexEvent.getStatus());
+    validateDropIndex(newIndex, dropIndexEvent.getIndex());
+
+    PreDropIndexEvent preDropIndexEvent = (PreDropIndexEvent) (preNotifyList.get(preNotifyList.size() - 1));
+    validateDropIndex(newIndex, preDropIndexEvent.getIndex());
+
+    Partition part = new PartitionBuilder()
+        .fromTable(table)
+        .addValue("2011")
+        .build();
+    msc.add_partition(part);
+    listSize++;
+    assertEquals(notifyList.size(), listSize);
+    PreAddPartitionEvent prePartEvent = (PreAddPartitionEvent)(preNotifyList.get(preNotifyList.size() - 1));
+
+    AddPartitionEvent partEvent = (AddPartitionEvent)(notifyList.get(listSize-1));
+    Assert.assertTrue(partEvent.getStatus());
+    part = msc.getPartition("hive2038", "tmptbl", "b=2011");
+    Partition partAdded = partEvent.getPartitionIterator().next();
+    validateAddPartition(part, partAdded);
+    validateTableInAddPartition(tbl, partEvent.getTable());
+    validateAddPartition(part, prePartEvent.getPartitions().get(0));
+
+    // Test adding multiple partitions in a single partition-set, atomically.
+    int currentTime = (int)System.currentTimeMillis();
+    HiveMetaStoreClient hmsClient = new HiveMetaStoreClient(conf);
+    table = hmsClient.getTable(dbName, "tmptbl");
+    Partition partition1 = new Partition(Arrays.asList("20110101"), dbName, "tmptbl", currentTime,
+                                        currentTime, table.getSd(), table.getParameters());
+    Partition partition2 = new Partition(Arrays.asList("20110102"), dbName, "tmptbl", currentTime,
+                                        currentTime, table.getSd(), table.getParameters());
+    Partition partition3 = new Partition(Arrays.asList("20110103"), dbName, "tmptbl", currentTime,
+                                        currentTime, table.getSd(), table.getParameters());
+    hmsClient.add_partitions(Arrays.asList(partition1, partition2, partition3));
+    ++listSize;
+    AddPartitionEvent multiplePartitionEvent = (AddPartitionEvent)(notifyList.get(listSize-1));
+    assertEquals("Unexpected table value.", table, multiplePartitionEvent.getTable());
+    List<Partition> multiParts = Lists.newArrayList(multiplePartitionEvent.getPartitionIterator());
+    assertEquals("Unexpected number of partitions in event!", 3, multiParts.size());
+    assertEquals("Unexpected partition value.", partition1.getValues(), multiParts.get(0).getValues());
+    assertEquals("Unexpected partition value.", partition2.getValues(), multiParts.get(1).getValues());
+    assertEquals("Unexpected partition value.", partition3.getValues(), multiParts.get(2).getValues());
+
+    part.setLastAccessTime((int)(System.currentTimeMillis()/1000));
+    msc.alter_partition(dbName, tblName, part);
+    listSize++;
+    assertEquals(notifyList.size(), listSize);
+    PreAlterPartitionEvent preAlterPartEvent =
+        (PreAlterPartitionEvent)preNotifyList.get(preNotifyList.size() - 1);
+
+    //the partition did not change,
+    // so the new partition should be similar to the original partition
+    Partition origP = msc.getPartition(dbName, tblName, "b=2011");
+
+    AlterPartitionEvent alterPartEvent = (AlterPartitionEvent)notifyList.get(listSize - 1);
+    Assert.assertTrue(alterPartEvent.getStatus());
+    validateAlterPartition(origP, origP, alterPartEvent.getOldPartition().getDbName(),
+        alterPartEvent.getOldPartition().getTableName(),
+        alterPartEvent.getOldPartition().getValues(), alterPartEvent.getNewPartition());
+
+
+    validateAlterPartition(origP, origP, preAlterPartEvent.getDbName(),
+        preAlterPartEvent.getTableName(), preAlterPartEvent.getNewPartition().getValues(),
+        preAlterPartEvent.getNewPartition());
+
+    List<String> part_vals = new ArrayList<>();
+    part_vals.add("c=2012");
+    int preEventListSize;
+    preEventListSize = preNotifyList.size() + 1;
+    Partition newPart = msc.appendPartition(dbName, tblName, part_vals);
+
+    listSize++;
+    assertEquals(notifyList.size(), listSize);
+    assertEquals(preNotifyList.size(), preEventListSize);
+
+    AddPartitionEvent appendPartEvent =
+        (AddPartitionEvent)(notifyList.get(listSize-1));
+    Partition partAppended = appendPartEvent.getPartitionIterator().next();
+    validateAddPartition(newPart, partAppended);
+
+    PreAddPartitionEvent preAppendPartEvent =
+        (PreAddPartitionEvent)(preNotifyList.get(preNotifyList.size() - 1));
+    validateAddPartition(newPart, preAppendPartEvent.getPartitions().get(0));
+
+    Table renamedTable = new Table(table);
+    renamedTable.setTableName(renamed);
+    msc.alter_table(dbName, tblName, renamedTable);
+    listSize++;
+    assertEquals(notifyList.size(), listSize);
+    PreAlterTableEvent preAlterTableE = (PreAlterTableEvent) preNotifyList.get(preNotifyList.size() - 1);
+
+    renamedTable = msc.getTable(dbName, renamed);
+
+    AlterTableEvent alterTableE = (AlterTableEvent) notifyList.get(listSize-1);
+    Assert.assertTrue(alterTableE.getStatus());
+    validateAlterTable(tbl, renamedTable, alterTableE.getOldTable(), alterTableE.getNewTable());
+    validateAlterTable(tbl, renamedTable, preAlterTableE.getOldTable(),
+        preAlterTableE.getNewTable());
+
+    //change the table name back
+    table = new Table(renamedTable);
+    table.setTableName(tblName);
+    msc.alter_table(dbName, renamed, table);
+    listSize++;
+    assertEquals(notifyList.size(), listSize);
+
+    table = msc.getTable(dbName, tblName);
+    table.getSd().addToCols(new FieldSchema("c", "int", ""));
+    msc.alter_table(dbName, tblName, table);
+    listSize++;
+    assertEquals(notifyList.size(), listSize);
+    preAlterTableE = (PreAlterTableEvent) preNotifyList.get(preNotifyList.size() - 1);
+
+    Table altTable = msc.getTable(dbName, tblName);
+
+    alterTableE = (AlterTableEvent) notifyList.get(listSize-1);
+    Assert.assertTrue(alterTableE.getStatus());
+    validateAlterTableColumns(tbl, altTable, alterTableE.getOldTable(), alterTableE.getNewTable());
+    validateAlterTableColumns(tbl, altTable, preAlterTableE.getOldTable(),
+        preAlterTableE.getNewTable());
+
+    Map<String,String> kvs = new HashMap<>(1);
+    kvs.put("b", "2011");
+    msc.markPartitionForEvent("hive2038", "tmptbl", kvs, PartitionEventType.LOAD_DONE);
+    listSize++;
+    assertEquals(notifyList.size(), listSize);
+
+    LoadPartitionDoneEvent partMarkEvent = (LoadPartitionDoneEvent)notifyList.get(listSize - 1);
+    Assert.assertTrue(partMarkEvent.getStatus());
+    validateLoadPartitionDone("tmptbl", kvs, partMarkEvent.getTable().getTableName(),
+        partMarkEvent.getPartitionName());
+
+    PreLoadPartitionDoneEvent prePartMarkEvent =
+        (PreLoadPartitionDoneEvent)preNotifyList.get(preNotifyList.size() - 1);
+    validateLoadPartitionDone("tmptbl", kvs, prePartMarkEvent.getTableName(),
+        prePartMarkEvent.getPartitionName());
+
+    msc.dropPartition(dbName, tblName, Collections.singletonList("2011"));
+    listSize++;
+    assertEquals(notifyList.size(), listSize);
+    PreDropPartitionEvent preDropPart = (PreDropPartitionEvent) preNotifyList.get(preNotifyList
+        .size() - 1);
+
+    DropPartitionEvent dropPart = (DropPartitionEvent)notifyList.get(listSize - 1);
+    Assert.assertTrue(dropPart.getStatus());
+    validateDropPartition(Collections.singletonList(part).iterator(), dropPart.getPartitionIterator());
+    validateTableInDropPartition(tbl, dropPart.getTable());
+
+    validateDropPartition(Collections.singletonList(part).iterator(), preDropPart.getPartitionIterator());
+    validateTableInDropPartition(tbl, preDropPart.getTable());
+
+    msc.dropTable(dbName, tblName);
+    listSize++;
+    assertEquals(notifyList.size(), listSize);
+    PreDropTableEvent preDropTbl = (PreDropTableEvent)preNotifyList.get(preNotifyList.size() - 1);
+
+    DropTableEvent dropTbl = (DropTableEvent)notifyList.get(listSize-1);
+    Assert.assertTrue(dropTbl.getStatus());
+    validateDropTable(tbl, dropTbl.getTable());
+    validateDropTable(tbl, preDropTbl.getTable());
+
+    msc.dropDatabase(dbName);
+    listSize++;
+    assertEquals(notifyList.size(), listSize);
+    PreDropDatabaseEvent preDropDB = (PreDropDatabaseEvent)preNotifyList.get(preNotifyList.size() - 1);
+
+    DropDatabaseEvent dropDB = (DropDatabaseEvent)notifyList.get(listSize-1);
+    Assert.assertTrue(dropDB.getStatus());
+    validateDropDb(db, dropDB.getDatabase());
+    validateDropDb(db, preDropDB.getDatabase());
+
+    msc.setMetaConf("metastore.try.direct.sql", "false");
+    ConfigChangeEvent event = (ConfigChangeEvent) notifyList.get(notifyList.size() - 1);
+    assertEquals("metastore.try.direct.sql", event.getKey());
+    assertEquals("true", event.getOldValue());
+    assertEquals("false", event.getNewValue());
+  }
+
+  @Test
+  public void testMetaConfNotifyListenersClosingClient() throws Exception {
+    HiveMetaStoreClient closingClient = new HiveMetaStoreClient(conf, null);
+    closingClient.setMetaConf(metaConfKey, "[test pattern modified]");
+    ConfigChangeEvent event = (ConfigChangeEvent) DummyListener.getLastEvent();
+    assertEquals(event.getOldValue(), metaConfVal);
+    assertEquals(event.getNewValue(), "[test pattern modified]");
+    closingClient.close();
+
+    Thread.sleep(2 * 1000);
+
+    event = (ConfigChangeEvent) DummyListener.getLastEvent();
+    assertEquals(event.getOldValue(), "[test pattern modified]");
+    assertEquals(event.getNewValue(), metaConfVal);
+  }
+
+  @Test
+  public void testMetaConfNotifyListenersNonClosingClient() throws Exception {
+    HiveMetaStoreClient nonClosingClient = new HiveMetaStoreClient(conf, null);
+    nonClosingClient.setMetaConf(metaConfKey, "[test pattern modified]");
+    ConfigChangeEvent event = (ConfigChangeEvent) DummyListener.getLastEvent();
+    assertEquals(event.getOldValue(), metaConfVal);
+    assertEquals(event.getNewValue(), "[test pattern modified]");
+    // This should also trigger meta listener notification via TServerEventHandler#deleteContext
+    nonClosingClient.getTTransport().close();
+
+    Thread.sleep(2 * 1000);
+
+    event = (ConfigChangeEvent) DummyListener.getLastEvent();
+    assertEquals(event.getOldValue(), "[test pattern modified]");
+    assertEquals(event.getNewValue(), metaConfVal);
+  }
+
+  @Test
+  public void testMetaConfDuplicateNotification() throws Exception {
+    HiveMetaStoreClient closingClient = new HiveMetaStoreClient(conf, null);
+    closingClient.setMetaConf(metaConfKey, metaConfVal);
+    int beforeCloseNotificationEventCounts = DummyListener.notifyList.size();
+    closingClient.close();
+
+    Thread.sleep(2 * 1000);
+
+    int afterCloseNotificationEventCounts = DummyListener.notifyList.size();
+    // Setting key to same value, should not trigger configChange event during shutdown
+    assertEquals(beforeCloseNotificationEventCounts, afterCloseNotificationEventCounts);
+  }
+
+  @Test
+  public void testMetaConfSameHandler() throws Exception {
+    HiveMetaStoreClient closingClient = new HiveMetaStoreClient(conf, null);
+    closingClient.setMetaConf(metaConfKey, "[test pattern modified]");
+    ConfigChangeEvent event = (ConfigChangeEvent) DummyListener.getLastEvent();
+    int beforeCloseNotificationEventCounts = DummyListener.notifyList.size();
+    IHMSHandler beforeHandler = event.getIHMSHandler();
+    closingClient.close();
+
+    Thread.sleep(2 * 1000);
+    event = (ConfigChangeEvent) DummyListener.getLastEvent();
+    int afterCloseNotificationEventCounts = DummyListener.notifyList.size();
+    IHMSHandler afterHandler = event.getIHMSHandler();
+    // Meta-conf cleanup should trigger an event to listener
+    assertNotSame(beforeCloseNotificationEventCounts, afterCloseNotificationEventCounts);
+    // Both the handlers should be same
+    assertEquals(beforeHandler, afterHandler);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/d9801d9c/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListenerOnlyOnCommit.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListenerOnlyOnCommit.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListenerOnlyOnCommit.java
new file mode 100644
index 0000000..de729c7
--- /dev/null
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListenerOnlyOnCommit.java
@@ -0,0 +1,123 @@
+/*
+ * 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.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.Partition;
+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.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.events.ListenerEvent;
+import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import junit.framework.TestCase;
+
+/**
+ * Ensure that the status of MetaStore events depend on the RawStore's commit status.
+ */
+public class TestMetaStoreEventListenerOnlyOnCommit {
+
+  private Configuration conf;
+  private HiveMetaStoreClient msc;
+
+  @Before
+  public void setUp() throws Exception {
+    DummyRawStoreControlledCommit.setCommitSucceed(true);
+
+    System.setProperty(ConfVars.EVENT_LISTENERS.toString(), DummyListener.class.getName());
+    System.setProperty(ConfVars.RAW_STORE_IMPL.toString(),
+            DummyRawStoreControlledCommit.class.getName());
+
+    conf = MetastoreConf.newMetastoreConf();
+    MetastoreConf.setLongVar(conf, ConfVars.THRIFT_CONNECTION_RETRIES, 3);
+    MetastoreConf.setBoolVar(conf, ConfVars.HIVE_SUPPORT_CONCURRENCY, false);
+    MetaStoreTestUtils.setConfForStandloneMode(conf);
+    int port = MetaStoreTestUtils.startMetaStoreWithRetry(HadoopThriftAuthBridge.getBridge(), conf);
+    MetastoreConf.setVar(conf, ConfVars.THRIFT_URIS, "thrift://localhost:" + port);
+    msc = new HiveMetaStoreClient(conf);
+
+    DummyListener.notifyList.clear();
+  }
+
+  @Test
+  public void testEventStatus() throws Exception {
+    int listSize = 0;
+    List<ListenerEvent> notifyList = DummyListener.notifyList;
+    assertEquals(notifyList.size(), listSize);
+
+    String dbName = "tmpDb";
+    Database db = new DatabaseBuilder()
+        .setName(dbName)
+        .build();
+    msc.createDatabase(db);
+
+    listSize += 1;
+    notifyList = DummyListener.notifyList;
+    assertEquals(notifyList.size(), listSize);
+    assertTrue(DummyListener.getLastEvent().getStatus());
+
+    String tableName = "unittest_TestMetaStoreEventListenerOnlyOnCommit";
+    Table table = new TableBuilder()
+        .setDbName(db)
+        .setTableName(tableName)
+        .addCol("id", "int")
+        .addPartCol("ds", "string")
+        .build();
+    msc.createTable(table);
+    listSize += 1;
+    notifyList = DummyListener.notifyList;
+    assertEquals(notifyList.size(), listSize);
+    assertTrue(DummyListener.getLastEvent().getStatus());
+
+    Partition part = new PartitionBuilder()
+        .fromTable(table)
+        .addValue("foo1")
+        .build();
+    msc.add_partition(part);
+    listSize += 1;
+    notifyList = DummyListener.notifyList;
+    assertEquals(notifyList.size(), listSize);
+    assertTrue(DummyListener.getLastEvent().getStatus());
+
+    DummyRawStoreControlledCommit.setCommitSucceed(false);
+
+    part = new PartitionBuilder()
+        .fromTable(table)
+        .addValue("foo2")
+        .build();
+    msc.add_partition(part);
+    listSize += 1;
+    notifyList = DummyListener.notifyList;
+    assertEquals(notifyList.size(), listSize);
+    assertFalse(DummyListener.getLastEvent().getStatus());
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/d9801d9c/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListenerWithOldConf.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListenerWithOldConf.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListenerWithOldConf.java
new file mode 100644
index 0000000..82e39f1
--- /dev/null
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListenerWithOldConf.java
@@ -0,0 +1,178 @@
+/*
+ * 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 com.google.common.collect.Lists;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+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.Partition;
+import org.apache.hadoop.hive.metastore.api.PartitionEventType;
+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.events.AddIndexEvent;
+import org.apache.hadoop.hive.metastore.events.AddPartitionEvent;
+import org.apache.hadoop.hive.metastore.events.AlterIndexEvent;
+import org.apache.hadoop.hive.metastore.events.AlterPartitionEvent;
+import org.apache.hadoop.hive.metastore.events.AlterTableEvent;
+import org.apache.hadoop.hive.metastore.events.ConfigChangeEvent;
+import org.apache.hadoop.hive.metastore.events.CreateDatabaseEvent;
+import org.apache.hadoop.hive.metastore.events.CreateTableEvent;
+import org.apache.hadoop.hive.metastore.events.DropDatabaseEvent;
+import org.apache.hadoop.hive.metastore.events.DropIndexEvent;
+import org.apache.hadoop.hive.metastore.events.DropPartitionEvent;
+import org.apache.hadoop.hive.metastore.events.DropTableEvent;
+import org.apache.hadoop.hive.metastore.events.ListenerEvent;
+import org.apache.hadoop.hive.metastore.events.LoadPartitionDoneEvent;
+import org.apache.hadoop.hive.metastore.events.PreAddIndexEvent;
+import org.apache.hadoop.hive.metastore.events.PreAddPartitionEvent;
+import org.apache.hadoop.hive.metastore.events.PreAlterIndexEvent;
+import org.apache.hadoop.hive.metastore.events.PreAlterPartitionEvent;
+import org.apache.hadoop.hive.metastore.events.PreAlterTableEvent;
+import org.apache.hadoop.hive.metastore.events.PreCreateDatabaseEvent;
+import org.apache.hadoop.hive.metastore.events.PreCreateTableEvent;
+import org.apache.hadoop.hive.metastore.events.PreDropDatabaseEvent;
+import org.apache.hadoop.hive.metastore.events.PreDropIndexEvent;
+import org.apache.hadoop.hive.metastore.events.PreDropPartitionEvent;
+import org.apache.hadoop.hive.metastore.events.PreDropTableEvent;
+import org.apache.hadoop.hive.metastore.events.PreEventContext;
+import org.apache.hadoop.hive.metastore.events.PreLoadPartitionDoneEvent;
+import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotSame;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Mostly same tests as TestMetaStoreEventListener, but using old hive conf values instead of new
+ * metastore conf values.
+ */
+public class TestMetaStoreEventListenerWithOldConf {
+  private Configuration conf;
+
+  private static final String metaConfKey = "hive.metastore.partition.name.whitelist.pattern";
+  private static final String metaConfVal = "";
+
+  @Before
+  public void setUp() throws Exception {
+    System.setProperty("hive.metastore.event.listeners",
+        DummyListener.class.getName());
+    System.setProperty("hive.metastore.pre.event.listeners",
+        DummyPreListener.class.getName());
+
+    int port = MetaStoreTestUtils.findFreePort();
+    conf = MetastoreConf.newMetastoreConf();
+
+    MetastoreConf.setVar(conf, ConfVars.PARTITION_NAME_WHITELIST_PATTERN, metaConfVal);
+    MetastoreConf.setVar(conf, ConfVars.THRIFT_URIS, "thrift://localhost:" + port);
+    MetastoreConf.setLongVar(conf, ConfVars.THRIFT_CONNECTION_RETRIES, 3);
+    MetastoreConf.setBoolVar(conf, ConfVars.HIVE_SUPPORT_CONCURRENCY, false);
+    MetaStoreTestUtils.setConfForStandloneMode(conf);
+    MetaStoreTestUtils.startMetaStore(port, HadoopThriftAuthBridge.getBridge(), conf);
+
+    DummyListener.notifyList.clear();
+    DummyPreListener.notifyList.clear();
+  }
+
+  @Test
+  public void testMetaConfNotifyListenersClosingClient() throws Exception {
+    HiveMetaStoreClient closingClient = new HiveMetaStoreClient(conf, null);
+    closingClient.setMetaConf(metaConfKey, "[test pattern modified]");
+    ConfigChangeEvent event = (ConfigChangeEvent) DummyListener.getLastEvent();
+    assertEquals(event.getOldValue(), metaConfVal);
+    assertEquals(event.getNewValue(), "[test pattern modified]");
+    closingClient.close();
+
+    Thread.sleep(2 * 1000);
+
+    event = (ConfigChangeEvent) DummyListener.getLastEvent();
+    assertEquals(event.getOldValue(), "[test pattern modified]");
+    assertEquals(event.getNewValue(), metaConfVal);
+  }
+
+  @Test
+  public void testMetaConfNotifyListenersNonClosingClient() throws Exception {
+    HiveMetaStoreClient nonClosingClient = new HiveMetaStoreClient(conf, null);
+    nonClosingClient.setMetaConf(metaConfKey, "[test pattern modified]");
+    ConfigChangeEvent event = (ConfigChangeEvent) DummyListener.getLastEvent();
+    assertEquals(event.getOldValue(), metaConfVal);
+    assertEquals(event.getNewValue(), "[test pattern modified]");
+    // This should also trigger meta listener notification via TServerEventHandler#deleteContext
+    nonClosingClient.getTTransport().close();
+
+    Thread.sleep(2 * 1000);
+
+    event = (ConfigChangeEvent) DummyListener.getLastEvent();
+    assertEquals(event.getOldValue(), "[test pattern modified]");
+    assertEquals(event.getNewValue(), metaConfVal);
+  }
+
+  @Test
+  public void testMetaConfDuplicateNotification() throws Exception {
+    HiveMetaStoreClient closingClient = new HiveMetaStoreClient(conf, null);
+    closingClient.setMetaConf(metaConfKey, metaConfVal);
+    int beforeCloseNotificationEventCounts = DummyListener.notifyList.size();
+    closingClient.close();
+
+    Thread.sleep(2 * 1000);
+
+    int afterCloseNotificationEventCounts = DummyListener.notifyList.size();
+    // Setting key to same value, should not trigger configChange event during shutdown
+    assertEquals(beforeCloseNotificationEventCounts, afterCloseNotificationEventCounts);
+  }
+
+  @Test
+  public void testMetaConfSameHandler() throws Exception {
+    HiveMetaStoreClient closingClient = new HiveMetaStoreClient(conf, null);
+    closingClient.setMetaConf(metaConfKey, "[test pattern modified]");
+    ConfigChangeEvent event = (ConfigChangeEvent) DummyListener.getLastEvent();
+    int beforeCloseNotificationEventCounts = DummyListener.notifyList.size();
+    IHMSHandler beforeHandler = event.getHandler();
+    closingClient.close();
+
+    Thread.sleep(2 * 1000);
+    event = (ConfigChangeEvent) DummyListener.getLastEvent();
+    int afterCloseNotificationEventCounts = DummyListener.notifyList.size();
+    IHMSHandler afterHandler = event.getHandler();
+    // Meta-conf cleanup should trigger an event to listener
+    assertNotSame(beforeCloseNotificationEventCounts, afterCloseNotificationEventCounts);
+    // Both the handlers should be same
+    assertEquals(beforeHandler, afterHandler);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/d9801d9c/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreInitListener.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreInitListener.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreInitListener.java
new file mode 100644
index 0000000..f692b0a
--- /dev/null
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreInitListener.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.metastore;
+
+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.security.HadoopThriftAuthBridge;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * TestMetaStoreInitListener. Test case for
+ * {@link org.apache.hadoop.hive.metastore.MetaStoreInitListener}
+ */
+public class TestMetaStoreInitListener {
+  private Configuration conf;
+
+  @Before
+  public void setUp() throws Exception {
+    System.setProperty("hive.metastore.init.hooks", DummyMetaStoreInitListener.class.getName());
+    conf = MetastoreConf.newMetastoreConf();
+    MetastoreConf.setLongVar(conf, ConfVars.THRIFT_CONNECTION_RETRIES, 3);
+    MetastoreConf.setBoolVar(conf, ConfVars.HIVE_SUPPORT_CONCURRENCY, false);
+    MetaStoreTestUtils.setConfForStandloneMode(conf);
+    int port = MetaStoreTestUtils.startMetaStoreWithRetry(HadoopThriftAuthBridge.getBridge(), conf);
+    MetastoreConf.setVar(conf, ConfVars.THRIFT_URIS, "thrift://localhost:" + port);
+  }
+
+  @Test
+  public void testMetaStoreInitListener() throws Exception {
+    // DummyMataStoreInitListener's onInit will be called at HMSHandler
+    // initialization, and set this to true
+    Assert.assertTrue(DummyMetaStoreInitListener.wasCalled);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/d9801d9c/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreListenersError.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreListenersError.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreListenersError.java
new file mode 100644
index 0000000..3fdce48
--- /dev/null
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreListenersError.java
@@ -0,0 +1,94 @@
+/*
+ * 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.api.MetaException;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test for unwrapping InvocationTargetException, which is thrown from
+ * constructor of listener class
+ */
+public class TestMetaStoreListenersError {
+
+  @Test
+  public void testInitListenerException() throws Throwable {
+
+    System.setProperty("hive.metastore.init.hooks", ErrorInitListener.class.getName());
+    Configuration conf = MetastoreConf.newMetastoreConf();
+    MetaStoreTestUtils.setConfForStandloneMode(conf);
+    int port = MetaStoreTestUtils.findFreePort();
+    try {
+      HiveMetaStore.startMetaStore(port, HadoopThriftAuthBridge.getBridge(), conf);
+      Assert.fail();
+    } catch (Throwable throwable) {
+      Assert.assertEquals(MetaException.class, throwable.getClass());
+      Assert.assertEquals(
+          "Failed to instantiate listener named: " +
+              "org.apache.hadoop.hive.metastore.TestMetaStoreListenersError$ErrorInitListener, " +
+              "reason: java.lang.IllegalArgumentException: exception on constructor",
+          throwable.getMessage());
+    }
+  }
+
+  @Test
+  public void testEventListenerException() throws Throwable {
+
+    System.setProperty("hive.metastore.init.hooks", "");
+    System.setProperty("hive.metastore.event.listeners", ErrorEventListener.class.getName());
+    Configuration conf = MetastoreConf.newMetastoreConf();
+    MetaStoreTestUtils.setConfForStandloneMode(conf);
+    int port = MetaStoreTestUtils.findFreePort();
+    try {
+      HiveMetaStore.startMetaStore(port, HadoopThriftAuthBridge.getBridge(), conf);
+      Assert.fail();
+    } catch (Throwable throwable) {
+      Assert.assertEquals(MetaException.class, throwable.getClass());
+      Assert.assertEquals(
+          "Failed to instantiate listener named: " +
+              "org.apache.hadoop.hive.metastore.TestMetaStoreListenersError$ErrorEventListener, " +
+              "reason: java.lang.IllegalArgumentException: exception on constructor",
+          throwable.getMessage());
+    }
+  }
+
+  public static class ErrorInitListener extends MetaStoreInitListener {
+
+    public ErrorInitListener(Configuration config) {
+      super(config);
+      throw new IllegalArgumentException("exception on constructor");
+    }
+
+    public void onInit(MetaStoreInitContext context) throws MetaException {
+    }
+  }
+
+  public static class ErrorEventListener extends MetaStoreEventListener {
+
+    public ErrorEventListener(Configuration config) {
+      super(config);
+      throw new IllegalArgumentException("exception on constructor");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/d9801d9c/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStore.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStore.java
index 24ea62e..372dee6 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStore.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStore.java
@@ -129,8 +129,7 @@ public class TestObjectStore {
   @Before
   public void setUp() throws Exception {
     Configuration conf = MetastoreConf.newMetastoreConf();
-    MetastoreConf.setVar(conf, MetastoreConf.ConfVars.EXPRESSION_PROXY_CLASS,
-        MockPartitionExpressionProxy.class.getName());
+    MetaStoreTestUtils.setConfForStandloneMode(conf);
 
     objectStore = new ObjectStore();
     objectStore.setConf(conf);
@@ -462,8 +461,7 @@ public class TestObjectStore {
     String value1 = "another_value";
     Assume.assumeTrue(System.getProperty(key) == null);
     Configuration localConf = MetastoreConf.newMetastoreConf();
-    MetastoreConf.setVar(localConf, MetastoreConf.ConfVars.EXPRESSION_PROXY_CLASS,
-        MockPartitionExpressionProxy.class.getName());
+    MetaStoreTestUtils.setConfForStandloneMode(localConf);
     localConf.set(key, value);
     localConf.set(key1, value1);
     objectStore = new ObjectStore();
@@ -537,8 +535,7 @@ public class TestObjectStore {
             .debug(NUM_THREADS + " threads going to add notification"));
 
     Configuration conf = MetastoreConf.newMetastoreConf();
-    MetastoreConf.setVar(conf, MetastoreConf.ConfVars.EXPRESSION_PROXY_CLASS,
-        MockPartitionExpressionProxy.class.getName());
+    MetaStoreTestUtils.setConfForStandloneMode(conf);
     /*
        Below are the properties that need to be set based on what database this test is going to be run
      */

http://git-wip-us.apache.org/repos/asf/hive/blob/d9801d9c/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStoreInitRetry.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStoreInitRetry.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStoreInitRetry.java
new file mode 100644
index 0000000..b4e5a85
--- /dev/null
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStoreInitRetry.java
@@ -0,0 +1,132 @@
+/*
+ * 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 java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.concurrent.TimeUnit;
+
+import javax.jdo.JDOCanRetryException;
+
+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.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class TestObjectStoreInitRetry {
+  private static final Logger LOG = LoggerFactory.getLogger(TestObjectStoreInitRetry.class);
+
+  private static int injectConnectFailure = 0;
+
+  private static void setInjectConnectFailure(int x){
+    injectConnectFailure = x;
+  }
+
+  private static int getInjectConnectFailure(){
+    return injectConnectFailure;
+  }
+
+  private static void decrementInjectConnectFailure(){
+    injectConnectFailure--;
+  }
+
+  @BeforeClass
+  public static void oneTimeSetup() throws SQLException {
+    // dummy instantiation to make sure any static/ctor code blocks of that
+    // driver are loaded and ready to go.
+    DriverManager.registerDriver(new FakeDerby());
+  }
+
+  @AfterClass
+  public static void oneTimeTearDown() throws SQLException {
+    DriverManager.deregisterDriver(new FakeDerby());
+  }
+
+  static void misbehave() throws RuntimeException{
+    TestObjectStoreInitRetry.debugTrace();
+    if (TestObjectStoreInitRetry.getInjectConnectFailure() > 0){
+      TestObjectStoreInitRetry.decrementInjectConnectFailure();
+      RuntimeException re = new JDOCanRetryException();
+      LOG.debug("MISBEHAVE:" + TestObjectStoreInitRetry.getInjectConnectFailure(), re);
+      throw re;
+    }
+  }
+
+  // debug instrumenter - useful in finding which fns get called, and how often
+  static void debugTrace() {
+    if (LOG.isDebugEnabled()){
+      Exception e = new Exception();
+      LOG.debug("." + e.getStackTrace()[1].getLineNumber() + ":" + TestObjectStoreInitRetry.getInjectConnectFailure());
+    }
+  }
+
+  protected static Configuration conf;
+
+  @Test
+  public void testObjStoreRetry() throws Exception {
+    conf = MetastoreConf.newMetastoreConf();
+
+    MetastoreConf.setLongVar(conf, ConfVars.HMSHANDLERATTEMPTS, 4);
+    MetastoreConf.setTimeVar(conf, ConfVars.HMSHANDLERINTERVAL, 1, TimeUnit.SECONDS);
+    MetastoreConf.setVar(conf, ConfVars.CONNECTION_DRIVER,FakeDerby.class.getName());
+    MetastoreConf.setBoolVar(conf, ConfVars.TRY_DIRECT_SQL,true);
+    String jdbcUrl = MetastoreConf.getVar(conf, ConfVars.CONNECTURLKEY);
+    jdbcUrl = jdbcUrl.replace("derby","fderby");
+    MetastoreConf.setVar(conf, ConfVars.CONNECTURLKEY,jdbcUrl);
+    MetaStoreTestUtils.setConfForStandloneMode(conf);
+
+    FakeDerby fd = new FakeDerby();
+
+    ObjectStore objStore = new ObjectStore();
+
+    Exception savE = null;
+    try {
+      setInjectConnectFailure(5);
+      objStore.setConf(conf);
+      Assert.fail();
+    } catch (Exception e) {
+      LOG.info("Caught exception ", e);
+      savE = e;
+    }
+
+    /*
+     * A note on retries.
+     *
+     * We've configured a total of 4 attempts.
+     * 5 - 4 == 1 connect failure simulation count left after this.
+     */
+
+    assertEquals(1, getInjectConnectFailure());
+    assertNotNull(savE);
+
+    setInjectConnectFailure(0);
+    objStore.setConf(conf);
+    assertEquals(0, getInjectConnectFailure());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/d9801d9c/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestOldSchema.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestOldSchema.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestOldSchema.java
index bf8556d..6a44833 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestOldSchema.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestOldSchema.java
@@ -89,9 +89,8 @@ public class TestOldSchema {
   @Before
   public void setUp() throws Exception {
     Configuration conf = MetastoreConf.newMetastoreConf();
-    MetastoreConf.setClass(conf, MetastoreConf.ConfVars.EXPRESSION_PROXY_CLASS,
-        MockPartitionExpressionProxy.class, PartitionExpressionProxy.class);
     MetastoreConf.setBoolVar(conf, MetastoreConf.ConfVars.STATS_FETCH_BITVECTOR, false);
+    MetaStoreTestUtils.setConfForStandloneMode(conf);
 
     store = new ObjectStore();
     store.setConf(conf);


[3/7] hive git commit: HIVE-17982 Move metastore specific itests. This closes #279. (Alan Gates, reviewed by Peter Vary)

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/d9801d9c/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
new file mode 100644
index 0000000..2599ab1
--- /dev/null
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
@@ -0,0 +1,3071 @@
+/*
+ * 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.reflect.Field;
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.hive.metastore.client.builder.DatabaseBuilder;
+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.utils.FileUtils;
+import org.apache.hadoop.hive.metastore.utils.SecurityUtils;
+import org.datanucleus.api.jdo.JDOPersistenceManager;
+import org.datanucleus.api.jdo.JDOPersistenceManagerFactory;
+import org.junit.Assert;
+import org.junit.Before;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hive.metastore.api.AggrStats;
+import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
+import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData;
+import org.apache.hadoop.hive.metastore.api.ColumnStatisticsDesc;
+import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
+import org.apache.hadoop.hive.metastore.api.ConfigValSecurityException;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.DoubleColumnStatsData;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.Function;
+import org.apache.hadoop.hive.metastore.api.FunctionType;
+import org.apache.hadoop.hive.metastore.api.GetAllFunctionsResponse;
+import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
+import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
+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.PrincipalType;
+import org.apache.hadoop.hive.metastore.api.ResourceType;
+import org.apache.hadoop.hive.metastore.api.ResourceUri;
+import org.apache.hadoop.hive.metastore.api.SerDeInfo;
+import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
+import org.apache.hadoop.hive.metastore.api.StringColumnStatsData;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.Type;
+import org.apache.hadoop.hive.metastore.api.UnknownDBException;
+import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.thrift.TException;
+import org.junit.Test;
+
+import com.google.common.collect.Lists;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public abstract class TestHiveMetaStore {
+  private static final Logger LOG = LoggerFactory.getLogger(TestHiveMetaStore.class);
+  protected static HiveMetaStoreClient client;
+  protected static Configuration conf;
+  protected static Warehouse warehouse;
+  protected static boolean isThriftClient = false;
+
+  private static final String TEST_DB1_NAME = "testdb1";
+  private static final String TEST_DB2_NAME = "testdb2";
+
+  private static final int DEFAULT_LIMIT_PARTITION_REQUEST = 100;
+
+  protected abstract HiveMetaStoreClient createClient() throws Exception;
+
+  @Before
+  public void setUp() throws Exception {
+    conf = MetastoreConf.newMetastoreConf();
+    warehouse = new Warehouse(conf);
+
+    // set some values to use for getting conf. vars
+    MetastoreConf.setBoolVar(conf, ConfVars.METRICS_ENABLED, true);
+    conf.set("hive.key1", "value1");
+    conf.set("hive.key2", "http://www.example.com");
+    conf.set("hive.key3", "");
+    conf.set("hive.key4", "0");
+    conf.set("datanucleus.autoCreateTables", "false");
+
+    MetaStoreTestUtils.setConfForStandloneMode(conf);
+    MetastoreConf.setLongVar(conf, ConfVars.BATCH_RETRIEVE_MAX, 2);
+    MetastoreConf.setLongVar(conf, ConfVars.LIMIT_PARTITION_REQUEST, DEFAULT_LIMIT_PARTITION_REQUEST);
+    MetastoreConf.setVar(conf, ConfVars.STORAGE_SCHEMA_READER_IMPL, "no.such.class");
+  }
+
+  @Test
+  public void testNameMethods() {
+    Map<String, String> spec = new LinkedHashMap<>();
+    spec.put("ds", "2008-07-01 14:13:12");
+    spec.put("hr", "14");
+    List<String> vals = new ArrayList<>();
+    vals.addAll(spec.values());
+    String partName = "ds=2008-07-01 14%3A13%3A12/hr=14";
+
+    try {
+      List<String> testVals = client.partitionNameToVals(partName);
+      assertTrue("Values from name are incorrect", vals.equals(testVals));
+
+      Map<String, String> testSpec = client.partitionNameToSpec(partName);
+      assertTrue("Spec from name is incorrect", spec.equals(testSpec));
+
+      List<String> emptyVals = client.partitionNameToVals("");
+      assertTrue("Values should be empty", emptyVals.size() == 0);
+
+      Map<String, String> emptySpec =  client.partitionNameToSpec("");
+      assertTrue("Spec should be empty", emptySpec.size() == 0);
+    } catch (Exception e) {
+      fail();
+    }
+  }
+
+  /**
+   * tests create table and partition and tries to drop the table without
+   * droppping the partition
+   *
+   */
+  @Test
+  public void testPartition() throws Exception {
+    partitionTester(client, conf);
+  }
+
+  private static void partitionTester(HiveMetaStoreClient client, Configuration conf)
+    throws Exception {
+    try {
+      String dbName = "compdb";
+      String tblName = "comptbl";
+      String typeName = "Person";
+      List<String> vals = makeVals("2008-07-01 14:13:12", "14");
+      List<String> vals2 = makeVals("2008-07-01 14:13:12", "15");
+      List<String> vals3 = makeVals("2008-07-02 14:13:12", "15");
+      List<String> vals4 = makeVals("2008-07-03 14:13:12", "151");
+
+      client.dropTable(dbName, tblName);
+      silentDropDatabase(dbName);
+      Database db = new Database();
+      db.setName(dbName);
+      client.createDatabase(db);
+      db = client.getDatabase(dbName);
+      Path dbPath = new Path(db.getLocationUri());
+      FileSystem fs = FileSystem.get(dbPath.toUri(), conf);
+
+      client.dropType(typeName);
+      Type typ1 = new Type();
+      typ1.setName(typeName);
+      typ1.setFields(new ArrayList<>(2));
+      typ1.getFields().add(
+          new FieldSchema("name", ColumnType.STRING_TYPE_NAME, ""));
+      typ1.getFields().add(
+          new FieldSchema("income", ColumnType.INT_TYPE_NAME, ""));
+      client.createType(typ1);
+
+      List<String> skewedColValue = Collections.singletonList("1");
+      Table tbl = new TableBuilder()
+          .setDbName(dbName)
+          .setTableName(tblName)
+          .setCols(typ1.getFields())
+          .setNumBuckets(1)
+          .addBucketCol("name")
+          .addTableParam("test_param_1", "Use this for comments etc")
+          .addSerdeParam(ColumnType.SERIALIZATION_FORMAT, "1")
+          .addSkewedColName("name")
+          .setSkewedColValues(Collections.singletonList(skewedColValue))
+          .setSkewedColValueLocationMaps(Collections.singletonMap(skewedColValue, "location1"))
+          .addPartCol("ds", ColumnType.STRING_TYPE_NAME)
+          .addPartCol("hr", ColumnType.STRING_TYPE_NAME)
+          .build();
+
+      client.createTable(tbl);
+
+      if (isThriftClient) {
+        // the createTable() above does not update the location in the 'tbl'
+        // object when the client is a thrift client and the code below relies
+        // on the location being present in the 'tbl' object - so get the table
+        // from the metastore
+        tbl = client.getTable(dbName, tblName);
+      }
+
+      Partition part = makePartitionObject(dbName, tblName, vals, tbl, "/part1");
+      Partition part2 = makePartitionObject(dbName, tblName, vals2, tbl, "/part2");
+      Partition part3 = makePartitionObject(dbName, tblName, vals3, tbl, "/part3");
+      Partition part4 = makePartitionObject(dbName, tblName, vals4, tbl, "/part4");
+
+      // check if the partition exists (it shouldn't)
+      boolean exceptionThrown = false;
+      try {
+        Partition p = client.getPartition(dbName, tblName, vals);
+      } catch(Exception e) {
+        assertEquals("partition should not have existed",
+            NoSuchObjectException.class, e.getClass());
+        exceptionThrown = true;
+      }
+      assertTrue("getPartition() should have thrown NoSuchObjectException", exceptionThrown);
+      Partition retp = client.add_partition(part);
+      assertNotNull("Unable to create partition " + part, retp);
+      Partition retp2 = client.add_partition(part2);
+      assertNotNull("Unable to create partition " + part2, retp2);
+      Partition retp3 = client.add_partition(part3);
+      assertNotNull("Unable to create partition " + part3, retp3);
+      Partition retp4 = client.add_partition(part4);
+      assertNotNull("Unable to create partition " + part4, retp4);
+
+      Partition part_get = client.getPartition(dbName, tblName, part.getValues());
+      if(isThriftClient) {
+        // since we are using thrift, 'part' will not have the create time and
+        // last DDL time set since it does not get updated in the add_partition()
+        // call - likewise part2 and part3 - set it correctly so that equals check
+        // doesn't fail
+        adjust(client, part, dbName, tblName);
+        adjust(client, part2, dbName, tblName);
+        adjust(client, part3, dbName, tblName);
+      }
+      assertTrue("Partitions are not same", part.equals(part_get));
+
+      // check null cols schemas for a partition
+      List<String> vals6 = makeVals("2016-02-22 00:00:00", "16");
+      Partition part6 = makePartitionObject(dbName, tblName, vals6, tbl, "/part5");
+      part6.getSd().setCols(null);
+      LOG.info("Creating partition will null field schema");
+      client.add_partition(part6);
+      LOG.info("Listing all partitions for table " + dbName + "." + tblName);
+      final List<Partition> partitions = client.listPartitions(dbName, tblName, (short) -1);
+      boolean foundPart = false;
+      for (Partition p : partitions) {
+        if (p.getValues().equals(vals6)) {
+          assertNull(p.getSd().getCols());
+          LOG.info("Found partition " + p + " having null field schema");
+          foundPart = true;
+        }
+      }
+      assertTrue(foundPart);
+
+      String partName = "ds=" + FileUtils.escapePathName("2008-07-01 14:13:12") + "/hr=14";
+      String part2Name = "ds=" + FileUtils.escapePathName("2008-07-01 14:13:12") + "/hr=15";
+      String part3Name = "ds=" + FileUtils.escapePathName("2008-07-02 14:13:12") + "/hr=15";
+      String part4Name = "ds=" + FileUtils.escapePathName("2008-07-03 14:13:12") + "/hr=151";
+
+      part_get = client.getPartition(dbName, tblName, partName);
+      assertTrue("Partitions are not the same", part.equals(part_get));
+
+      // Test partition listing with a partial spec - ds is specified but hr is not
+      List<String> partialVals = new ArrayList<>();
+      partialVals.add(vals.get(0));
+      Set<Partition> parts = new HashSet<>();
+      parts.add(part);
+      parts.add(part2);
+
+      List<Partition> partial = client.listPartitions(dbName, tblName, partialVals,
+          (short) -1);
+      assertTrue("Should have returned 2 partitions", partial.size() == 2);
+      assertTrue("Not all parts returned", partial.containsAll(parts));
+
+      Set<String> partNames = new HashSet<>();
+      partNames.add(partName);
+      partNames.add(part2Name);
+      List<String> partialNames = client.listPartitionNames(dbName, tblName, partialVals,
+          (short) -1);
+      assertTrue("Should have returned 2 partition names", partialNames.size() == 2);
+      assertTrue("Not all part names returned", partialNames.containsAll(partNames));
+
+      partNames.add(part3Name);
+      partNames.add(part4Name);
+      partialVals.clear();
+      partialVals.add("");
+      partialNames = client.listPartitionNames(dbName, tblName, partialVals, (short) -1);
+      assertTrue("Should have returned 5 partition names", partialNames.size() == 5);
+      assertTrue("Not all part names returned", partialNames.containsAll(partNames));
+
+      // Test partition listing with a partial spec - hr is specified but ds is not
+      parts.clear();
+      parts.add(part2);
+      parts.add(part3);
+
+      partialVals.clear();
+      partialVals.add("");
+      partialVals.add(vals2.get(1));
+
+      partial = client.listPartitions(dbName, tblName, partialVals, (short) -1);
+      assertEquals("Should have returned 2 partitions", 2, partial.size());
+      assertTrue("Not all parts returned", partial.containsAll(parts));
+
+      partNames.clear();
+      partNames.add(part2Name);
+      partNames.add(part3Name);
+      partialNames = client.listPartitionNames(dbName, tblName, partialVals,
+          (short) -1);
+      assertEquals("Should have returned 2 partition names", 2, partialNames.size());
+      assertTrue("Not all part names returned", partialNames.containsAll(partNames));
+
+      // Verify escaped partition names don't return partitions
+      exceptionThrown = false;
+      try {
+        String badPartName = "ds=2008-07-01 14%3A13%3A12/hrs=14";
+        client.getPartition(dbName, tblName, badPartName);
+      } catch(NoSuchObjectException e) {
+        exceptionThrown = true;
+      }
+      assertTrue("Bad partition spec should have thrown an exception", exceptionThrown);
+
+      Path partPath = new Path(part.getSd().getLocation());
+
+
+      assertTrue(fs.exists(partPath));
+      client.dropPartition(dbName, tblName, part.getValues(), true);
+      assertFalse(fs.exists(partPath));
+
+      // Test append_partition_by_name
+      client.appendPartition(dbName, tblName, partName);
+      Partition part5 = client.getPartition(dbName, tblName, part.getValues());
+      assertTrue("Append partition by name failed", part5.getValues().equals(vals));
+      Path part5Path = new Path(part5.getSd().getLocation());
+      assertTrue(fs.exists(part5Path));
+
+      // Test drop_partition_by_name
+      assertTrue("Drop partition by name failed",
+          client.dropPartition(dbName, tblName, partName, true));
+      assertFalse(fs.exists(part5Path));
+
+      // add the partition again so that drop table with a partition can be
+      // tested
+      retp = client.add_partition(part);
+      assertNotNull("Unable to create partition " + part, retp);
+
+      // test add_partitions
+
+      List<String> mvals1 = makeVals("2008-07-04 14:13:12", "14641");
+      List<String> mvals2 = makeVals("2008-07-04 14:13:12", "14642");
+      List<String> mvals3 = makeVals("2008-07-04 14:13:12", "14643");
+      List<String> mvals4 = makeVals("2008-07-04 14:13:12", "14643"); // equal to 3
+      List<String> mvals5 = makeVals("2008-07-04 14:13:12", "14645");
+
+      Exception savedException;
+
+      // add_partitions(empty list) : ok, normal operation
+      client.add_partitions(new ArrayList<>());
+
+      // add_partitions(1,2,3) : ok, normal operation
+      Partition mpart1 = makePartitionObject(dbName, tblName, mvals1, tbl, "/mpart1");
+      Partition mpart2 = makePartitionObject(dbName, tblName, mvals2, tbl, "/mpart2");
+      Partition mpart3 = makePartitionObject(dbName, tblName, mvals3, tbl, "/mpart3");
+      client.add_partitions(Arrays.asList(mpart1,mpart2,mpart3));
+
+      if(isThriftClient) {
+        // do DDL time munging if thrift mode
+        adjust(client, mpart1, dbName, tblName);
+        adjust(client, mpart2, dbName, tblName);
+        adjust(client, mpart3, dbName, tblName);
+      }
+      verifyPartitionsPublished(client, dbName, tblName,
+          Arrays.asList(mvals1.get(0)),
+          Arrays.asList(mpart1,mpart2,mpart3));
+
+      Partition mpart4 = makePartitionObject(dbName, tblName, mvals4, tbl, "/mpart4");
+      Partition mpart5 = makePartitionObject(dbName, tblName, mvals5, tbl, "/mpart5");
+
+      // create dir for /mpart5
+      Path mp5Path = new Path(mpart5.getSd().getLocation());
+      warehouse.mkdirs(mp5Path);
+      assertTrue(fs.exists(mp5Path));
+
+      // add_partitions(5,4) : err = duplicate keyvals on mpart4
+      savedException = null;
+      try {
+        client.add_partitions(Arrays.asList(mpart5,mpart4));
+      } catch (Exception e) {
+        savedException = e;
+      } finally {
+        assertNotNull(savedException);
+      }
+
+      // check that /mpart4 does not exist, but /mpart5 still does.
+      assertTrue(fs.exists(mp5Path));
+      assertFalse(fs.exists(new Path(mpart4.getSd().getLocation())));
+
+      // add_partitions(5) : ok
+      client.add_partitions(Arrays.asList(mpart5));
+
+      if(isThriftClient) {
+        // do DDL time munging if thrift mode
+        adjust(client, mpart5, dbName, tblName);
+      }
+
+      verifyPartitionsPublished(client, dbName, tblName,
+          Arrays.asList(mvals1.get(0)),
+          Arrays.asList(mpart1,mpart2,mpart3,mpart5));
+
+      //// end add_partitions tests
+
+      client.dropTable(dbName, tblName);
+
+      client.dropType(typeName);
+
+      // recreate table as external, drop partition and it should
+      // still exist
+      tbl.setParameters(new HashMap<>());
+      tbl.getParameters().put("EXTERNAL", "TRUE");
+      client.createTable(tbl);
+      retp = client.add_partition(part);
+      assertTrue(fs.exists(partPath));
+      client.dropPartition(dbName, tblName, part.getValues(), true);
+      assertTrue(fs.exists(partPath));
+
+      for (String tableName : client.getTables(dbName, "*")) {
+        client.dropTable(dbName, tableName);
+      }
+
+      client.dropDatabase(dbName);
+
+    } catch (Exception e) {
+      System.err.println(StringUtils.stringifyException(e));
+      System.err.println("testPartition() failed.");
+      throw e;
+    }
+  }
+
+  private static void verifyPartitionsPublished(HiveMetaStoreClient client,
+      String dbName, String tblName, List<String> partialSpec,
+      List<Partition> expectedPartitions) throws TException {
+    // Test partition listing with a partial spec
+
+    List<Partition> mpartial = client.listPartitions(dbName, tblName, partialSpec,
+        (short) -1);
+    assertEquals("Should have returned "+expectedPartitions.size()+
+        " partitions, returned " + mpartial.size(),
+        expectedPartitions.size(), mpartial.size());
+    assertTrue("Not all parts returned", mpartial.containsAll(expectedPartitions));
+  }
+
+  private static List<String> makeVals(String ds, String id) {
+    List <String> vals4 = new ArrayList<>(2);
+    vals4 = new ArrayList<>(2);
+    vals4.add(ds);
+    vals4.add(id);
+    return vals4;
+  }
+
+  private static Partition makePartitionObject(String dbName, String tblName,
+      List<String> ptnVals, Table tbl, String ptnLocationSuffix) throws MetaException {
+    Partition part4 = new Partition();
+    part4.setDbName(dbName);
+    part4.setTableName(tblName);
+    part4.setValues(ptnVals);
+    part4.setParameters(new HashMap<>());
+    part4.setSd(tbl.getSd().deepCopy());
+    part4.getSd().setSerdeInfo(tbl.getSd().getSerdeInfo().deepCopy());
+    part4.getSd().setLocation(tbl.getSd().getLocation() + ptnLocationSuffix);
+    MetaStoreUtils.updatePartitionStatsFast(part4, warehouse, null);
+    return part4;
+  }
+
+  @Test
+  public void testListPartitions() throws Throwable {
+    // create a table with multiple partitions
+    String dbName = "compdb";
+    String tblName = "comptbl";
+    String typeName = "Person";
+
+    cleanUp(dbName, tblName, typeName);
+
+    List<List<String>> values = new ArrayList<>();
+    values.add(makeVals("2008-07-01 14:13:12", "14"));
+    values.add(makeVals("2008-07-01 14:13:12", "15"));
+    values.add(makeVals("2008-07-02 14:13:12", "15"));
+    values.add(makeVals("2008-07-03 14:13:12", "151"));
+
+    createMultiPartitionTableSchema(dbName, tblName, typeName, values);
+
+    List<Partition> partitions = client.listPartitions(dbName, tblName, (short)-1);
+    assertNotNull("should have returned partitions", partitions);
+    assertEquals(" should have returned " + values.size() +
+      " partitions", values.size(), partitions.size());
+
+    partitions = client.listPartitions(dbName, tblName, (short)(values.size()/2));
+
+    assertNotNull("should have returned partitions", partitions);
+    assertEquals(" should have returned " + values.size() / 2 +
+      " partitions",values.size() / 2, partitions.size());
+
+
+    partitions = client.listPartitions(dbName, tblName, (short) (values.size() * 2));
+
+    assertNotNull("should have returned partitions", partitions);
+    assertEquals(" should have returned " + values.size() +
+      " partitions",values.size(), partitions.size());
+
+    cleanUp(dbName, tblName, typeName);
+
+  }
+
+  @Test
+  public void testListPartitionsWihtLimitEnabled() throws Throwable {
+    // create a table with multiple partitions
+    String dbName = "compdb";
+    String tblName = "comptbl";
+    String typeName = "Person";
+
+    cleanUp(dbName, tblName, typeName);
+
+    // Create too many partitions, just enough to validate over limit requests
+    List<List<String>> values = new ArrayList<>();
+    for (int i=0; i<DEFAULT_LIMIT_PARTITION_REQUEST + 1; i++) {
+      values.add(makeVals("2008-07-01 14:13:12", Integer.toString(i)));
+    }
+
+    createMultiPartitionTableSchema(dbName, tblName, typeName, values);
+
+    List<Partition> partitions;
+    short maxParts;
+
+    // Requesting more partitions than allowed should throw an exception
+    try {
+      maxParts = -1;
+      partitions = client.listPartitions(dbName, tblName, maxParts);
+      fail("should have thrown MetaException about partition limit");
+    } catch (MetaException e) {
+      assertTrue(true);
+    }
+
+    // Requesting more partitions than allowed should throw an exception
+    try {
+      maxParts = DEFAULT_LIMIT_PARTITION_REQUEST + 1;
+      partitions = client.listPartitions(dbName, tblName, maxParts);
+      fail("should have thrown MetaException about partition limit");
+    } catch (MetaException e) {
+      assertTrue(true);
+    }
+
+    // Requesting less partitions than allowed should work
+    maxParts = DEFAULT_LIMIT_PARTITION_REQUEST / 2;
+    partitions = client.listPartitions(dbName, tblName, maxParts);
+    assertNotNull("should have returned partitions", partitions);
+    assertEquals(" should have returned 50 partitions", maxParts, partitions.size());
+  }
+
+  @Test
+  public void testAlterTableCascade() throws Throwable {
+    // create a table with multiple partitions
+    String dbName = "compdb";
+    String tblName = "comptbl";
+    String typeName = "Person";
+
+    cleanUp(dbName, tblName, typeName);
+
+    List<List<String>> values = new ArrayList<>();
+    values.add(makeVals("2008-07-01 14:13:12", "14"));
+    values.add(makeVals("2008-07-01 14:13:12", "15"));
+    values.add(makeVals("2008-07-02 14:13:12", "15"));
+    values.add(makeVals("2008-07-03 14:13:12", "151"));
+
+    createMultiPartitionTableSchema(dbName, tblName, typeName, values);
+    Table tbl = client.getTable(dbName, tblName);
+    List<FieldSchema> cols = tbl.getSd().getCols();
+    cols.add(new FieldSchema("new_col", ColumnType.STRING_TYPE_NAME, ""));
+    tbl.getSd().setCols(cols);
+    //add new column with cascade option
+    client.alter_table(dbName, tblName, tbl, true);
+    //
+    Table tbl2 = client.getTable(dbName, tblName);
+    assertEquals("Unexpected number of cols", 3, tbl2.getSd().getCols().size());
+    assertEquals("Unexpected column name", "new_col", tbl2.getSd().getCols().get(2).getName());
+    //get a partition
+    List<String> pvalues = new ArrayList<>(2);
+    pvalues.add("2008-07-01 14:13:12");
+    pvalues.add("14");
+    Partition partition = client.getPartition(dbName, tblName, pvalues);
+    assertEquals("Unexpected number of cols", 3, partition.getSd().getCols().size());
+    assertEquals("Unexpected column name", "new_col", partition.getSd().getCols().get(2).getName());
+
+    //add another column
+    cols = tbl.getSd().getCols();
+    cols.add(new FieldSchema("new_col2", ColumnType.STRING_TYPE_NAME, ""));
+    tbl.getSd().setCols(cols);
+    //add new column with no cascade option
+    client.alter_table(dbName, tblName, tbl, false);
+    tbl2 = client.getTable(dbName, tblName);
+    assertEquals("Unexpected number of cols", 4, tbl2.getSd().getCols().size());
+    assertEquals("Unexpected column name", "new_col2", tbl2.getSd().getCols().get(3).getName());
+    //get partition, this partition should not have the newly added column since cascade option
+    //was false
+    partition = client.getPartition(dbName, tblName, pvalues);
+    assertEquals("Unexpected number of cols", 3, partition.getSd().getCols().size());
+  }
+
+
+  @Test
+  public void testListPartitionNames() throws Throwable {
+    // create a table with multiple partitions
+    String dbName = "compdb";
+    String tblName = "comptbl";
+    String typeName = "Person";
+
+    cleanUp(dbName, tblName, typeName);
+
+    List<List<String>> values = new ArrayList<>();
+    values.add(makeVals("2008-07-01 14:13:12", "14"));
+    values.add(makeVals("2008-07-01 14:13:12", "15"));
+    values.add(makeVals("2008-07-02 14:13:12", "15"));
+    values.add(makeVals("2008-07-03 14:13:12", "151"));
+
+
+
+    createMultiPartitionTableSchema(dbName, tblName, typeName, values);
+
+    List<String> partitions = client.listPartitionNames(dbName, tblName, (short)-1);
+    assertNotNull("should have returned partitions", partitions);
+    assertEquals(" should have returned " + values.size() +
+      " partitions", values.size(), partitions.size());
+
+    partitions = client.listPartitionNames(dbName, tblName, (short)(values.size()/2));
+
+    assertNotNull("should have returned partitions", partitions);
+    assertEquals(" should have returned " + values.size() / 2 +
+      " partitions",values.size() / 2, partitions.size());
+
+
+    partitions = client.listPartitionNames(dbName, tblName, (short) (values.size() * 2));
+
+    assertNotNull("should have returned partitions", partitions);
+    assertEquals(" should have returned " + values.size() +
+      " partitions",values.size(), partitions.size());
+
+    cleanUp(dbName, tblName, typeName);
+
+  }
+
+
+  @Test
+  public void testDropTable() throws Throwable {
+    // create a table with multiple partitions
+    String dbName = "compdb";
+    String tblName = "comptbl";
+    String typeName = "Person";
+
+    cleanUp(dbName, tblName, typeName);
+
+    List<List<String>> values = new ArrayList<>();
+    values.add(makeVals("2008-07-01 14:13:12", "14"));
+    values.add(makeVals("2008-07-01 14:13:12", "15"));
+    values.add(makeVals("2008-07-02 14:13:12", "15"));
+    values.add(makeVals("2008-07-03 14:13:12", "151"));
+
+    createMultiPartitionTableSchema(dbName, tblName, typeName, values);
+
+    client.dropTable(dbName, tblName);
+    client.dropType(typeName);
+
+    boolean exceptionThrown = false;
+    try {
+      client.getTable(dbName, tblName);
+    } catch(Exception e) {
+      assertEquals("table should not have existed",
+          NoSuchObjectException.class, e.getClass());
+      exceptionThrown = true;
+    }
+    assertTrue("Table " + tblName + " should have been dropped ", exceptionThrown);
+
+  }
+
+  @Test
+  public void testAlterViewParititon() throws Throwable {
+    String dbName = "compdb";
+    String tblName = "comptbl";
+    String viewName = "compView";
+
+    client.dropTable(dbName, tblName);
+    silentDropDatabase(dbName);
+    Database db = new Database();
+    db.setName(dbName);
+    db.setDescription("Alter Partition Test database");
+    client.createDatabase(db);
+
+    Table tbl = new TableBuilder()
+        .setDbName(dbName)
+        .setTableName(tblName)
+        .addCol("name", ColumnType.STRING_TYPE_NAME)
+        .addCol("income", ColumnType.INT_TYPE_NAME)
+        .build();
+
+    client.createTable(tbl);
+
+    if (isThriftClient) {
+      // the createTable() above does not update the location in the 'tbl'
+      // object when the client is a thrift client and the code below relies
+      // on the location being present in the 'tbl' object - so get the table
+      // from the metastore
+      tbl = client.getTable(dbName, tblName);
+    }
+
+    ArrayList<FieldSchema> viewCols = new ArrayList<>(1);
+    viewCols.add(new FieldSchema("income", ColumnType.INT_TYPE_NAME, ""));
+
+    ArrayList<FieldSchema> viewPartitionCols = new ArrayList<>(1);
+    viewPartitionCols.add(new FieldSchema("name", ColumnType.STRING_TYPE_NAME, ""));
+
+    Table view = new Table();
+    view.setDbName(dbName);
+    view.setTableName(viewName);
+    view.setTableType(TableType.VIRTUAL_VIEW.name());
+    view.setPartitionKeys(viewPartitionCols);
+    view.setViewOriginalText("SELECT income, name FROM " + tblName);
+    view.setViewExpandedText("SELECT `" + tblName + "`.`income`, `" + tblName +
+        "`.`name` FROM `" + dbName + "`.`" + tblName + "`");
+    view.setRewriteEnabled(false);
+    StorageDescriptor viewSd = new StorageDescriptor();
+    view.setSd(viewSd);
+    viewSd.setCols(viewCols);
+    viewSd.setCompressed(false);
+    viewSd.setParameters(new HashMap<>());
+    viewSd.setSerdeInfo(new SerDeInfo());
+    viewSd.getSerdeInfo().setParameters(new HashMap<>());
+
+    client.createTable(view);
+
+    if (isThriftClient) {
+      // the createTable() above does not update the location in the 'tbl'
+      // object when the client is a thrift client and the code below relies
+      // on the location being present in the 'tbl' object - so get the table
+      // from the metastore
+      view = client.getTable(dbName, viewName);
+    }
+
+    List<String> vals = new ArrayList<>(1);
+    vals.add("abc");
+
+    Partition part = new Partition();
+    part.setDbName(dbName);
+    part.setTableName(viewName);
+    part.setValues(vals);
+    part.setParameters(new HashMap<>());
+
+    client.add_partition(part);
+
+    Partition part2 = client.getPartition(dbName, viewName, part.getValues());
+
+    part2.getParameters().put("a", "b");
+
+    client.alter_partition(dbName, viewName, part2, null);
+
+    Partition part3 = client.getPartition(dbName, viewName, part.getValues());
+    assertEquals("couldn't view alter partition", part3.getParameters().get(
+        "a"), "b");
+
+    client.dropTable(dbName, viewName);
+
+    client.dropTable(dbName, tblName);
+
+    client.dropDatabase(dbName);
+  }
+
+  @Test
+  public void testAlterPartition() throws Throwable {
+
+    try {
+      String dbName = "compdb";
+      String tblName = "comptbl";
+      List<String> vals = new ArrayList<>(2);
+      vals.add("2008-07-01");
+      vals.add("14");
+
+      client.dropTable(dbName, tblName);
+      silentDropDatabase(dbName);
+      Database db = new Database();
+      db.setName(dbName);
+      db.setDescription("Alter Partition Test database");
+      client.createDatabase(db);
+
+      Table tbl = new TableBuilder()
+          .setDbName(dbName)
+          .setTableName(tblName)
+          .addCol("name", ColumnType.STRING_TYPE_NAME)
+          .addCol("income", ColumnType.INT_TYPE_NAME)
+          .addTableParam("test_param_1", "Use this for comments etc")
+          .addBucketCol("name")
+          .addSerdeParam(ColumnType.SERIALIZATION_FORMAT, "1")
+          .addPartCol("ds", ColumnType.STRING_TYPE_NAME)
+          .addPartCol("hr", ColumnType.INT_TYPE_NAME)
+          .build();
+
+      client.createTable(tbl);
+
+      if (isThriftClient) {
+        // the createTable() above does not update the location in the 'tbl'
+        // object when the client is a thrift client and the code below relies
+        // on the location being present in the 'tbl' object - so get the table
+        // from the metastore
+        tbl = client.getTable(dbName, tblName);
+      }
+
+      Partition part = new Partition();
+      part.setDbName(dbName);
+      part.setTableName(tblName);
+      part.setValues(vals);
+      part.setParameters(new HashMap<>());
+      part.setSd(tbl.getSd());
+      part.getSd().setSerdeInfo(tbl.getSd().getSerdeInfo());
+      part.getSd().setLocation(tbl.getSd().getLocation() + "/part1");
+
+      client.add_partition(part);
+
+      Partition part2 = client.getPartition(dbName, tblName, part.getValues());
+
+      part2.getParameters().put("retention", "10");
+      part2.getSd().setNumBuckets(12);
+      part2.getSd().getSerdeInfo().getParameters().put("abc", "1");
+      client.alter_partition(dbName, tblName, part2, null);
+
+      Partition part3 = client.getPartition(dbName, tblName, part.getValues());
+      assertEquals("couldn't alter partition", part3.getParameters().get(
+          "retention"), "10");
+      assertEquals("couldn't alter partition", part3.getSd().getSerdeInfo()
+          .getParameters().get("abc"), "1");
+      assertEquals("couldn't alter partition", part3.getSd().getNumBuckets(),
+          12);
+
+      client.dropTable(dbName, tblName);
+
+      client.dropDatabase(dbName);
+    } catch (Exception e) {
+      System.err.println(StringUtils.stringifyException(e));
+      System.err.println("testPartition() failed.");
+      throw e;
+    }
+  }
+
+  @Test
+  public void testRenamePartition() throws Throwable {
+
+    try {
+      String dbName = "compdb1";
+      String tblName = "comptbl1";
+      List<String> vals = new ArrayList<>(2);
+      vals.add("2011-07-11");
+      vals.add("8");
+      String part_path = "/ds=2011-07-11/hr=8";
+      List<String> tmp_vals = new ArrayList<>(2);
+      tmp_vals.add("tmp_2011-07-11");
+      tmp_vals.add("-8");
+      String part2_path = "/ds=tmp_2011-07-11/hr=-8";
+
+      client.dropTable(dbName, tblName);
+      silentDropDatabase(dbName);
+      Database db = new Database();
+      db.setName(dbName);
+      db.setDescription("Rename Partition Test database");
+      client.createDatabase(db);
+
+      Table tbl = new TableBuilder()
+          .setDbName(dbName)
+          .setTableName(tblName)
+          .addCol("name", ColumnType.STRING_TYPE_NAME)
+          .addCol("income", ColumnType.INT_TYPE_NAME)
+          .addPartCol("ds", ColumnType.STRING_TYPE_NAME)
+          .addPartCol("hr", ColumnType.INT_TYPE_NAME)
+          .build();
+
+      client.createTable(tbl);
+
+      if (isThriftClient) {
+        // the createTable() above does not update the location in the 'tbl'
+        // object when the client is a thrift client and the code below relies
+        // on the location being present in the 'tbl' object - so get the table
+        // from the metastore
+        tbl = client.getTable(dbName, tblName);
+      }
+
+      Partition part = new Partition();
+      part.setDbName(dbName);
+      part.setTableName(tblName);
+      part.setValues(vals);
+      part.setParameters(new HashMap<>());
+      part.setSd(tbl.getSd().deepCopy());
+      part.getSd().setSerdeInfo(tbl.getSd().getSerdeInfo());
+      part.getSd().setLocation(tbl.getSd().getLocation() + "/part1");
+      part.getParameters().put("retention", "10");
+      part.getSd().setNumBuckets(12);
+      part.getSd().getSerdeInfo().getParameters().put("abc", "1");
+
+      client.add_partition(part);
+
+      part.setValues(tmp_vals);
+      client.renamePartition(dbName, tblName, vals, part);
+
+      boolean exceptionThrown = false;
+      try {
+        Partition p = client.getPartition(dbName, tblName, vals);
+      } catch(Exception e) {
+        assertEquals("partition should not have existed",
+            NoSuchObjectException.class, e.getClass());
+        exceptionThrown = true;
+      }
+      assertTrue("Expected NoSuchObjectException", exceptionThrown);
+
+      Partition part3 = client.getPartition(dbName, tblName, tmp_vals);
+      assertEquals("couldn't rename partition", part3.getParameters().get(
+          "retention"), "10");
+      assertEquals("couldn't rename partition", part3.getSd().getSerdeInfo()
+          .getParameters().get("abc"), "1");
+      assertEquals("couldn't rename partition", part3.getSd().getNumBuckets(),
+          12);
+      assertEquals("new partition sd matches", part3.getSd().getLocation(),
+          tbl.getSd().getLocation() + part2_path);
+
+      part.setValues(vals);
+      client.renamePartition(dbName, tblName, tmp_vals, part);
+
+      exceptionThrown = false;
+      try {
+        Partition p = client.getPartition(dbName, tblName, tmp_vals);
+      } catch(Exception e) {
+        assertEquals("partition should not have existed",
+            NoSuchObjectException.class, e.getClass());
+        exceptionThrown = true;
+      }
+      assertTrue("Expected NoSuchObjectException", exceptionThrown);
+
+      part3 = client.getPartition(dbName, tblName, vals);
+      assertEquals("couldn't rename partition", part3.getParameters().get(
+          "retention"), "10");
+      assertEquals("couldn't rename partition", part3.getSd().getSerdeInfo()
+          .getParameters().get("abc"), "1");
+      assertEquals("couldn't rename partition", part3.getSd().getNumBuckets(),
+          12);
+      assertEquals("new partition sd matches", part3.getSd().getLocation(),
+          tbl.getSd().getLocation() + part_path);
+
+      client.dropTable(dbName, tblName);
+
+      client.dropDatabase(dbName);
+    } catch (Exception e) {
+      System.err.println(StringUtils.stringifyException(e));
+      System.err.println("testRenamePartition() failed.");
+      throw e;
+    }
+  }
+
+  @Test
+  public void testDatabase() throws Throwable {
+    try {
+      // clear up any existing databases
+      silentDropDatabase(TEST_DB1_NAME);
+      silentDropDatabase(TEST_DB2_NAME);
+
+      Database db = new DatabaseBuilder()
+          .setName(TEST_DB1_NAME)
+          .setOwnerName(SecurityUtils.getUser())
+          .build();
+      Assert.assertEquals(SecurityUtils.getUser(), db.getOwnerName());
+      client.createDatabase(db);
+
+      db = client.getDatabase(TEST_DB1_NAME);
+
+      assertEquals("name of returned db is different from that of inserted db",
+          TEST_DB1_NAME, db.getName());
+      assertEquals("location of the returned db is different from that of inserted db",
+          warehouse.getDatabasePath(db).toString(), db.getLocationUri());
+      assertEquals(db.getOwnerName(), SecurityUtils.getUser());
+      assertEquals(db.getOwnerType(), PrincipalType.USER);
+      Database db2 = new Database();
+      db2.setName(TEST_DB2_NAME);
+      client.createDatabase(db2);
+
+      db2 = client.getDatabase(TEST_DB2_NAME);
+
+      assertEquals("name of returned db is different from that of inserted db",
+          TEST_DB2_NAME, db2.getName());
+      assertEquals("location of the returned db is different from that of inserted db",
+          warehouse.getDatabasePath(db2).toString(), db2.getLocationUri());
+
+      List<String> dbs = client.getDatabases(".*");
+
+      assertTrue("first database is not " + TEST_DB1_NAME, dbs.contains(TEST_DB1_NAME));
+      assertTrue("second database is not " + TEST_DB2_NAME, dbs.contains(TEST_DB2_NAME));
+
+      client.dropDatabase(TEST_DB1_NAME);
+      client.dropDatabase(TEST_DB2_NAME);
+      silentDropDatabase(TEST_DB1_NAME);
+      silentDropDatabase(TEST_DB2_NAME);
+    } catch (Throwable e) {
+      System.err.println(StringUtils.stringifyException(e));
+      System.err.println("testDatabase() failed.");
+      throw e;
+    }
+  }
+
+  @Test
+  public void testDatabaseLocationWithPermissionProblems() throws Exception {
+
+    // Note: The following test will fail if you are running this test as root. Setting
+    // permission to '0' on the database folder will not preclude root from being able
+    // to create the necessary files.
+
+    if (System.getProperty("user.name").equals("root")) {
+      System.err.println("Skipping test because you are running as root!");
+      return;
+    }
+
+    silentDropDatabase(TEST_DB1_NAME);
+
+    Database db = new Database();
+    db.setName(TEST_DB1_NAME);
+    String dbLocation =
+      MetastoreConf.getVar(conf, ConfVars.WAREHOUSE) + "/test/_testDB_create_";
+    FileSystem fs = FileSystem.get(new Path(dbLocation).toUri(), conf);
+    fs.mkdirs(
+              new Path(MetastoreConf.getVar(conf, ConfVars.WAREHOUSE) + "/test"),
+              new FsPermission((short) 0));
+    db.setLocationUri(dbLocation);
+
+
+    boolean createFailed = false;
+    try {
+      client.createDatabase(db);
+    } catch (MetaException cantCreateDB) {
+      createFailed = true;
+    } finally {
+      // Cleanup
+      if (!createFailed) {
+        try {
+          client.dropDatabase(TEST_DB1_NAME);
+        } catch(Exception e) {
+          System.err.println("Failed to remove database in cleanup: " + e.getMessage());
+        }
+      }
+
+      fs.setPermission(new Path(MetastoreConf.getVar(conf, ConfVars.WAREHOUSE) + "/test"),
+                       new FsPermission((short) 755));
+      fs.delete(new Path(MetastoreConf.getVar(conf, ConfVars.WAREHOUSE) + "/test"), true);
+    }
+
+    assertTrue("Database creation succeeded even with permission problem", createFailed);
+  }
+
+  @Test
+  public void testDatabaseLocation() throws Throwable {
+    try {
+      // clear up any existing databases
+      silentDropDatabase(TEST_DB1_NAME);
+
+      Database db = new Database();
+      db.setName(TEST_DB1_NAME);
+      String dbLocation =
+          MetastoreConf.getVar(conf, ConfVars.WAREHOUSE) + "/_testDB_create_";
+      db.setLocationUri(dbLocation);
+      client.createDatabase(db);
+
+      db = client.getDatabase(TEST_DB1_NAME);
+
+      assertEquals("name of returned db is different from that of inserted db",
+          TEST_DB1_NAME, db.getName());
+      assertEquals("location of the returned db is different from that of inserted db",
+          warehouse.getDnsPath(new Path(dbLocation)).toString(), db.getLocationUri());
+
+      client.dropDatabase(TEST_DB1_NAME);
+      silentDropDatabase(TEST_DB1_NAME);
+
+      boolean objectNotExist = false;
+      try {
+        client.getDatabase(TEST_DB1_NAME);
+      } catch (NoSuchObjectException e) {
+        objectNotExist = true;
+      }
+      assertTrue("Database " + TEST_DB1_NAME + " exists ", objectNotExist);
+
+      db = new Database();
+      db.setName(TEST_DB1_NAME);
+      dbLocation =
+          MetastoreConf.getVar(conf, ConfVars.WAREHOUSE) + "/_testDB_file_";
+      FileSystem fs = FileSystem.get(new Path(dbLocation).toUri(), conf);
+      fs.createNewFile(new Path(dbLocation));
+      fs.deleteOnExit(new Path(dbLocation));
+      db.setLocationUri(dbLocation);
+
+      boolean createFailed = false;
+      try {
+        client.createDatabase(db);
+      } catch (MetaException cantCreateDB) {
+        System.err.println(cantCreateDB.getMessage());
+        createFailed = true;
+      }
+      assertTrue("Database creation succeeded even location exists and is a file", createFailed);
+
+      objectNotExist = false;
+      try {
+        client.getDatabase(TEST_DB1_NAME);
+      } catch (NoSuchObjectException e) {
+        objectNotExist = true;
+      }
+      assertTrue("Database " + TEST_DB1_NAME + " exists when location is specified and is a file",
+          objectNotExist);
+
+    } catch (Throwable e) {
+      System.err.println(StringUtils.stringifyException(e));
+      System.err.println("testDatabaseLocation() failed.");
+      throw e;
+    }
+  }
+
+
+  @Test
+  public void testSimpleTypeApi() throws Exception {
+    try {
+      client.dropType(ColumnType.INT_TYPE_NAME);
+
+      Type typ1 = new Type();
+      typ1.setName(ColumnType.INT_TYPE_NAME);
+      boolean ret = client.createType(typ1);
+      assertTrue("Unable to create type", ret);
+
+      Type typ1_2 = client.getType(ColumnType.INT_TYPE_NAME);
+      assertNotNull(typ1_2);
+      assertEquals(typ1.getName(), typ1_2.getName());
+
+      ret = client.dropType(ColumnType.INT_TYPE_NAME);
+      assertTrue("unable to drop type integer", ret);
+
+      boolean exceptionThrown = false;
+      try {
+        client.getType(ColumnType.INT_TYPE_NAME);
+      } catch (NoSuchObjectException e) {
+        exceptionThrown = true;
+      }
+      assertTrue("Expected NoSuchObjectException", exceptionThrown);
+    } catch (Exception e) {
+      System.err.println(StringUtils.stringifyException(e));
+      System.err.println("testSimpleTypeApi() failed.");
+      throw e;
+    }
+  }
+
+  // TODO:pc need to enhance this with complex fields and getType_all function
+  @Test
+  public void testComplexTypeApi() throws Exception {
+    try {
+      client.dropType("Person");
+
+      Type typ1 = new Type();
+      typ1.setName("Person");
+      typ1.setFields(new ArrayList<>(2));
+      typ1.getFields().add(
+          new FieldSchema("name", ColumnType.STRING_TYPE_NAME, ""));
+      typ1.getFields().add(
+          new FieldSchema("income", ColumnType.INT_TYPE_NAME, ""));
+      boolean ret = client.createType(typ1);
+      assertTrue("Unable to create type", ret);
+
+      Type typ1_2 = client.getType("Person");
+      assertNotNull("type Person not found", typ1_2);
+      assertEquals(typ1.getName(), typ1_2.getName());
+      assertEquals(typ1.getFields().size(), typ1_2.getFields().size());
+      assertEquals(typ1.getFields().get(0), typ1_2.getFields().get(0));
+      assertEquals(typ1.getFields().get(1), typ1_2.getFields().get(1));
+
+      client.dropType("Family");
+
+      Type fam = new Type();
+      fam.setName("Family");
+      fam.setFields(new ArrayList<>(2));
+      fam.getFields().add(
+          new FieldSchema("name", ColumnType.STRING_TYPE_NAME, ""));
+      fam.getFields().add(
+          new FieldSchema("members",
+              ColumnType.getListType(typ1.getName()), ""));
+
+      ret = client.createType(fam);
+      assertTrue("Unable to create type " + fam.getName(), ret);
+
+      Type fam2 = client.getType("Family");
+      assertNotNull("type Person not found", fam2);
+      assertEquals(fam.getName(), fam2.getName());
+      assertEquals(fam.getFields().size(), fam2.getFields().size());
+      assertEquals(fam.getFields().get(0), fam2.getFields().get(0));
+      assertEquals(fam.getFields().get(1), fam2.getFields().get(1));
+
+      ret = client.dropType("Family");
+      assertTrue("unable to drop type Family", ret);
+
+      ret = client.dropType("Person");
+      assertTrue("unable to drop type Person", ret);
+
+      boolean exceptionThrown = false;
+      try {
+        client.getType("Person");
+      } catch (NoSuchObjectException e) {
+        exceptionThrown = true;
+      }
+      assertTrue("Expected NoSuchObjectException", exceptionThrown);
+    } catch (Exception e) {
+      System.err.println(StringUtils.stringifyException(e));
+      System.err.println("testComplexTypeApi() failed.");
+      throw e;
+    }
+  }
+
+  @Test
+  public void testSimpleTable() throws Exception {
+    try {
+      String dbName = "simpdb";
+      String tblName = "simptbl";
+      String tblName2 = "simptbl2";
+      String typeName = "Person";
+
+      client.dropTable(dbName, tblName);
+      silentDropDatabase(dbName);
+
+      Database db = new Database();
+      db.setName(dbName);
+      client.createDatabase(db);
+
+      client.dropType(typeName);
+      Type typ1 = new Type();
+      typ1.setName(typeName);
+      typ1.setFields(new ArrayList<>(2));
+      typ1.getFields().add(
+          new FieldSchema("name", ColumnType.STRING_TYPE_NAME, ""));
+      typ1.getFields().add(
+          new FieldSchema("income", ColumnType.INT_TYPE_NAME, ""));
+      client.createType(typ1);
+
+      Table tbl = new TableBuilder()
+          .setDbName(dbName)
+          .setTableName(tblName)
+          .setCols(typ1.getFields())
+          .setNumBuckets(1)
+          .addBucketCol("name")
+          .addStorageDescriptorParam("test_param_1", "Use this for comments etc")
+          .build();
+
+      client.createTable(tbl);
+
+      if (isThriftClient) {
+        // the createTable() above does not update the location in the 'tbl'
+        // object when the client is a thrift client and the code below relies
+        // on the location being present in the 'tbl' object - so get the table
+        // from the metastore
+        tbl = client.getTable(dbName, tblName);
+      }
+
+      Table tbl2 = client.getTable(dbName, tblName);
+      assertNotNull(tbl2);
+      assertEquals(tbl2.getDbName(), dbName);
+      assertEquals(tbl2.getTableName(), tblName);
+      assertEquals(tbl2.getSd().getCols().size(), typ1.getFields().size());
+      assertEquals(tbl2.getSd().isCompressed(), false);
+      assertEquals(tbl2.getSd().getNumBuckets(), 1);
+      assertEquals(tbl2.getSd().getLocation(), tbl.getSd().getLocation());
+      assertNotNull(tbl2.getSd().getSerdeInfo());
+      tbl.getSd().getSerdeInfo().setParameters(new HashMap<>());
+      tbl.getSd().getSerdeInfo().getParameters().put(ColumnType.SERIALIZATION_FORMAT, "1");
+
+      tbl2.setTableName(tblName2);
+      tbl2.setParameters(new HashMap<>());
+      tbl2.getParameters().put("EXTERNAL", "TRUE");
+      tbl2.getSd().setLocation(tbl.getSd().getLocation() + "-2");
+
+      List<FieldSchema> fieldSchemas = client.getFields(dbName, tblName);
+      assertNotNull(fieldSchemas);
+      assertEquals(fieldSchemas.size(), tbl.getSd().getCols().size());
+      for (FieldSchema fs : tbl.getSd().getCols()) {
+        assertTrue(fieldSchemas.contains(fs));
+      }
+
+      List<FieldSchema> fieldSchemasFull = client.getSchema(dbName, tblName);
+      assertNotNull(fieldSchemasFull);
+      assertEquals(fieldSchemasFull.size(), tbl.getSd().getCols().size()
+          + tbl.getPartitionKeys().size());
+      for (FieldSchema fs : tbl.getSd().getCols()) {
+        assertTrue(fieldSchemasFull.contains(fs));
+      }
+      for (FieldSchema fs : tbl.getPartitionKeys()) {
+        assertTrue(fieldSchemasFull.contains(fs));
+      }
+
+      client.createTable(tbl2);
+      if (isThriftClient) {
+        tbl2 = client.getTable(tbl2.getDbName(), tbl2.getTableName());
+      }
+
+      Table tbl3 = client.getTable(dbName, tblName2);
+      assertNotNull(tbl3);
+      assertEquals(tbl3.getDbName(), dbName);
+      assertEquals(tbl3.getTableName(), tblName2);
+      assertEquals(tbl3.getSd().getCols().size(), typ1.getFields().size());
+      assertEquals(tbl3.getSd().isCompressed(), false);
+      assertEquals(tbl3.getSd().getNumBuckets(), 1);
+      assertEquals(tbl3.getSd().getLocation(), tbl2.getSd().getLocation());
+      assertEquals(tbl3.getParameters(), tbl2.getParameters());
+
+      fieldSchemas = client.getFields(dbName, tblName2);
+      assertNotNull(fieldSchemas);
+      assertEquals(fieldSchemas.size(), tbl2.getSd().getCols().size());
+      for (FieldSchema fs : tbl2.getSd().getCols()) {
+        assertTrue(fieldSchemas.contains(fs));
+      }
+
+      fieldSchemasFull = client.getSchema(dbName, tblName2);
+      assertNotNull(fieldSchemasFull);
+      assertEquals(fieldSchemasFull.size(), tbl2.getSd().getCols().size()
+          + tbl2.getPartitionKeys().size());
+      for (FieldSchema fs : tbl2.getSd().getCols()) {
+        assertTrue(fieldSchemasFull.contains(fs));
+      }
+      for (FieldSchema fs : tbl2.getPartitionKeys()) {
+        assertTrue(fieldSchemasFull.contains(fs));
+      }
+
+      assertEquals("Use this for comments etc", tbl2.getSd().getParameters()
+          .get("test_param_1"));
+      assertEquals("name", tbl2.getSd().getBucketCols().get(0));
+      assertTrue("Partition key list is not empty",
+          (tbl2.getPartitionKeys() == null)
+              || (tbl2.getPartitionKeys().size() == 0));
+
+      //test get_table_objects_by_name functionality
+      ArrayList<String> tableNames = new ArrayList<>();
+      tableNames.add(tblName2);
+      tableNames.add(tblName);
+      tableNames.add(tblName2);
+      List<Table> foundTables = client.getTableObjectsByName(dbName, tableNames);
+
+      assertEquals(2, foundTables.size());
+      for (Table t: foundTables) {
+        if (t.getTableName().equals(tblName2)) {
+          assertEquals(t.getSd().getLocation(), tbl2.getSd().getLocation());
+        } else {
+          assertEquals(t.getTableName(), tblName);
+          assertEquals(t.getSd().getLocation(), tbl.getSd().getLocation());
+        }
+        assertEquals(t.getSd().getCols().size(), typ1.getFields().size());
+        assertEquals(t.getSd().isCompressed(), false);
+        assertEquals(foundTables.get(0).getSd().getNumBuckets(), 1);
+        assertNotNull(t.getSd().getSerdeInfo());
+        assertEquals(t.getDbName(), dbName);
+      }
+
+      tableNames.add(1, "table_that_doesnt_exist");
+      foundTables = client.getTableObjectsByName(dbName, tableNames);
+      assertEquals(foundTables.size(), 2);
+
+      InvalidOperationException ioe = null;
+      try {
+        foundTables = client.getTableObjectsByName(dbName, null);
+      } catch (InvalidOperationException e) {
+        ioe = e;
+      }
+      assertNotNull(ioe);
+      assertTrue("Table not found", ioe.getMessage().contains("null tables"));
+
+      UnknownDBException udbe = null;
+      try {
+        foundTables = client.getTableObjectsByName("db_that_doesnt_exist", tableNames);
+      } catch (UnknownDBException e) {
+        udbe = e;
+      }
+      assertNotNull(udbe);
+      assertTrue("DB not found", udbe.getMessage().contains("not find database db_that_doesnt_exist"));
+
+      udbe = null;
+      try {
+        foundTables = client.getTableObjectsByName("", tableNames);
+      } catch (UnknownDBException e) {
+        udbe = e;
+      }
+      assertNotNull(udbe);
+      assertTrue("DB not found", udbe.getMessage().contains("is null or empty"));
+
+      FileSystem fs = FileSystem.get((new Path(tbl.getSd().getLocation())).toUri(), conf);
+      client.dropTable(dbName, tblName);
+      assertFalse(fs.exists(new Path(tbl.getSd().getLocation())));
+
+      client.dropTable(dbName, tblName2);
+      assertTrue(fs.exists(new Path(tbl2.getSd().getLocation())));
+
+      client.dropType(typeName);
+      client.dropDatabase(dbName);
+    } catch (Exception e) {
+      System.err.println(StringUtils.stringifyException(e));
+      System.err.println("testSimpleTable() failed.");
+      throw e;
+    }
+  }
+
+  // Tests that in the absence of stats for partitions, and/or absence of columns
+  // to get stats for, the metastore does not break. See HIVE-12083 for motivation.
+  @Test
+  public void testStatsFastTrivial() throws Throwable {
+    String dbName = "tstatsfast";
+    String tblName = "t1";
+    String tblOwner = "statstester";
+    String typeName = "Person";
+    int lastAccessed = 12083;
+
+    cleanUp(dbName,tblName,typeName);
+
+    List<List<String>> values = new ArrayList<>();
+    values.add(makeVals("2008-07-01 14:13:12", "14"));
+    values.add(makeVals("2008-07-01 14:13:12", "15"));
+    values.add(makeVals("2008-07-02 14:13:12", "15"));
+    values.add(makeVals("2008-07-03 14:13:12", "151"));
+
+    createMultiPartitionTableSchema(dbName, tblName, typeName, values);
+
+    List<String> emptyColNames = new ArrayList<>();
+    List<String> emptyPartNames = new ArrayList<>();
+
+    List<String> colNames = new ArrayList<>();
+    colNames.add("name");
+    colNames.add("income");
+    List<String> partNames = client.listPartitionNames(dbName,tblName,(short)-1);
+
+    assertEquals(0,emptyColNames.size());
+    assertEquals(0,emptyPartNames.size());
+    assertEquals(2,colNames.size());
+    assertEquals(4,partNames.size());
+
+    // Test for both colNames and partNames being empty:
+    AggrStats aggrStatsEmpty = client.getAggrColStatsFor(dbName,tblName,emptyColNames,emptyPartNames);
+    assertNotNull(aggrStatsEmpty); // short-circuited on client-side, verifying that it's an empty object, not null
+    assertEquals(0,aggrStatsEmpty.getPartsFound());
+    assertNotNull(aggrStatsEmpty.getColStats());
+    assert(aggrStatsEmpty.getColStats().isEmpty());
+
+    // Test for only colNames being empty
+    AggrStats aggrStatsOnlyParts = client.getAggrColStatsFor(dbName,tblName,emptyColNames,partNames);
+    assertNotNull(aggrStatsOnlyParts); // short-circuited on client-side, verifying that it's an empty object, not null
+    assertEquals(0,aggrStatsOnlyParts.getPartsFound());
+    assertNotNull(aggrStatsOnlyParts.getColStats());
+    assert(aggrStatsOnlyParts.getColStats().isEmpty());
+
+    // Test for only partNames being empty
+    AggrStats aggrStatsOnlyCols = client.getAggrColStatsFor(dbName,tblName,colNames,emptyPartNames);
+    assertNotNull(aggrStatsOnlyCols); // short-circuited on client-side, verifying that it's an empty object, not null
+    assertEquals(0,aggrStatsOnlyCols.getPartsFound());
+    assertNotNull(aggrStatsOnlyCols.getColStats());
+    assert(aggrStatsOnlyCols.getColStats().isEmpty());
+
+    // Test for valid values for both.
+    AggrStats aggrStatsFull = client.getAggrColStatsFor(dbName,tblName,colNames,partNames);
+    assertNotNull(aggrStatsFull);
+    assertEquals(0,aggrStatsFull.getPartsFound()); // would still be empty, because no stats are actually populated.
+    assertNotNull(aggrStatsFull.getColStats());
+    assert(aggrStatsFull.getColStats().isEmpty());
+
+  }
+
+  @Test
+  public void testColumnStatistics() throws Throwable {
+
+    String dbName = "columnstatstestdb";
+    String tblName = "tbl";
+    String typeName = "Person";
+    String tblOwner = "testowner";
+    int lastAccessed = 6796;
+
+    try {
+      cleanUp(dbName, tblName, typeName);
+      Database db = new Database();
+      db.setName(dbName);
+      client.createDatabase(db);
+      createTableForTestFilter(dbName,tblName, tblOwner, lastAccessed, true);
+
+      // Create a ColumnStatistics Obj
+      String[] colName = new String[]{"income", "name"};
+      double lowValue = 50000.21;
+      double highValue = 1200000.4525;
+      long numNulls = 3;
+      long numDVs = 22;
+      double avgColLen = 50.30;
+      long maxColLen = 102;
+      String[] colType = new String[] {"double", "string"};
+      boolean isTblLevel = true;
+      String partName = null;
+      List<ColumnStatisticsObj> statsObjs = new ArrayList<>();
+
+      ColumnStatisticsDesc statsDesc = new ColumnStatisticsDesc();
+      statsDesc.setDbName(dbName);
+      statsDesc.setTableName(tblName);
+      statsDesc.setIsTblLevel(isTblLevel);
+      statsDesc.setPartName(partName);
+
+      ColumnStatisticsObj statsObj = new ColumnStatisticsObj();
+      statsObj.setColName(colName[0]);
+      statsObj.setColType(colType[0]);
+
+      ColumnStatisticsData statsData = new ColumnStatisticsData();
+      DoubleColumnStatsData numericStats = new DoubleColumnStatsData();
+      statsData.setDoubleStats(numericStats);
+
+      statsData.getDoubleStats().setHighValue(highValue);
+      statsData.getDoubleStats().setLowValue(lowValue);
+      statsData.getDoubleStats().setNumDVs(numDVs);
+      statsData.getDoubleStats().setNumNulls(numNulls);
+
+      statsObj.setStatsData(statsData);
+      statsObjs.add(statsObj);
+
+      statsObj = new ColumnStatisticsObj();
+      statsObj.setColName(colName[1]);
+      statsObj.setColType(colType[1]);
+
+      statsData = new ColumnStatisticsData();
+      StringColumnStatsData stringStats = new StringColumnStatsData();
+      statsData.setStringStats(stringStats);
+      statsData.getStringStats().setAvgColLen(avgColLen);
+      statsData.getStringStats().setMaxColLen(maxColLen);
+      statsData.getStringStats().setNumDVs(numDVs);
+      statsData.getStringStats().setNumNulls(numNulls);
+
+      statsObj.setStatsData(statsData);
+      statsObjs.add(statsObj);
+
+      ColumnStatistics colStats = new ColumnStatistics();
+      colStats.setStatsDesc(statsDesc);
+      colStats.setStatsObj(statsObjs);
+
+      // write stats objs persistently
+      client.updateTableColumnStatistics(colStats);
+
+      // retrieve the stats obj that was just written
+      ColumnStatisticsObj colStats2 = client.getTableColumnStatistics(
+          dbName, tblName, Lists.newArrayList(colName[0])).get(0);
+
+     // compare stats obj to ensure what we get is what we wrote
+      assertNotNull(colStats2);
+      assertEquals(colStats2.getColName(), colName[0]);
+      assertEquals(colStats2.getStatsData().getDoubleStats().getLowValue(), lowValue, 0.01);
+      assertEquals(colStats2.getStatsData().getDoubleStats().getHighValue(), highValue, 0.01);
+      assertEquals(colStats2.getStatsData().getDoubleStats().getNumNulls(), numNulls);
+      assertEquals(colStats2.getStatsData().getDoubleStats().getNumDVs(), numDVs);
+
+      // test delete column stats; if no col name is passed all column stats associated with the
+      // table is deleted
+      boolean status = client.deleteTableColumnStatistics(dbName, tblName, null);
+      assertTrue(status);
+      // try to query stats for a column for which stats doesn't exist
+      assertTrue(client.getTableColumnStatistics(
+          dbName, tblName, Lists.newArrayList(colName[1])).isEmpty());
+
+      colStats.setStatsDesc(statsDesc);
+      colStats.setStatsObj(statsObjs);
+
+      // update table level column stats
+      client.updateTableColumnStatistics(colStats);
+
+      // query column stats for column whose stats were updated in the previous call
+      colStats2 = client.getTableColumnStatistics(
+          dbName, tblName, Lists.newArrayList(colName[0])).get(0);
+
+      // partition level column statistics test
+      // create a table with multiple partitions
+      cleanUp(dbName, tblName, typeName);
+
+      List<List<String>> values = new ArrayList<>();
+      values.add(makeVals("2008-07-01 14:13:12", "14"));
+      values.add(makeVals("2008-07-01 14:13:12", "15"));
+      values.add(makeVals("2008-07-02 14:13:12", "15"));
+      values.add(makeVals("2008-07-03 14:13:12", "151"));
+
+      createMultiPartitionTableSchema(dbName, tblName, typeName, values);
+
+      List<String> partitions = client.listPartitionNames(dbName, tblName, (short)-1);
+
+      partName = partitions.get(0);
+      isTblLevel = false;
+
+      // create a new columnstatistics desc to represent partition level column stats
+      statsDesc = new ColumnStatisticsDesc();
+      statsDesc.setDbName(dbName);
+      statsDesc.setTableName(tblName);
+      statsDesc.setPartName(partName);
+      statsDesc.setIsTblLevel(isTblLevel);
+
+      colStats = new ColumnStatistics();
+      colStats.setStatsDesc(statsDesc);
+      colStats.setStatsObj(statsObjs);
+
+     client.updatePartitionColumnStatistics(colStats);
+
+     colStats2 = client.getPartitionColumnStatistics(dbName, tblName,
+         Lists.newArrayList(partName), Lists.newArrayList(colName[1])).get(partName).get(0);
+
+     // compare stats obj to ensure what we get is what we wrote
+     assertNotNull(colStats2);
+     assertEquals(colStats.getStatsDesc().getPartName(), partName);
+     assertEquals(colStats2.getColName(), colName[1]);
+     assertEquals(colStats2.getStatsData().getStringStats().getMaxColLen(), maxColLen);
+     assertEquals(colStats2.getStatsData().getStringStats().getAvgColLen(), avgColLen, 0.01);
+     assertEquals(colStats2.getStatsData().getStringStats().getNumNulls(), numNulls);
+     assertEquals(colStats2.getStatsData().getStringStats().getNumDVs(), numDVs);
+
+     // test stats deletion at partition level
+     client.deletePartitionColumnStatistics(dbName, tblName, partName, colName[1]);
+
+     colStats2 = client.getPartitionColumnStatistics(dbName, tblName,
+         Lists.newArrayList(partName), Lists.newArrayList(colName[0])).get(partName).get(0);
+
+     // test get stats on a column for which stats doesn't exist
+     assertTrue(client.getPartitionColumnStatistics(dbName, tblName,
+           Lists.newArrayList(partName), Lists.newArrayList(colName[1])).isEmpty());
+    } catch (Exception e) {
+      System.err.println(StringUtils.stringifyException(e));
+      System.err.println("testColumnStatistics() failed.");
+      throw e;
+    } finally {
+      cleanUp(dbName, tblName, typeName);
+    }
+  }
+
+  @Test(expected = MetaException.class)
+  public void testGetSchemaWithNoClassDefFoundError() throws TException {
+    String dbName = "testDb";
+    String tblName = "testTable";
+
+    client.dropTable(dbName, tblName);
+    silentDropDatabase(dbName);
+
+    Database db = new Database();
+    db.setName(dbName);
+    client.createDatabase(db);
+
+    Table tbl = new TableBuilder()
+        .setDbName(dbName)
+        .setTableName(tblName)
+        .addCol("name", ColumnType.STRING_TYPE_NAME, "")
+        .setSerdeLib("no.such.class")
+        .build();
+    client.createTable(tbl);
+
+    client.getSchema(dbName, tblName);
+  }
+
+  @Test
+  public void testAlterTable() throws Exception {
+    String dbName = "alterdb";
+    String invTblName = "alter-tbl";
+    String tblName = "altertbl";
+
+    try {
+      client.dropTable(dbName, tblName);
+      silentDropDatabase(dbName);
+
+      Database db = new Database();
+      db.setName(dbName);
+      client.createDatabase(db);
+
+      ArrayList<FieldSchema> invCols = new ArrayList<>(2);
+      invCols.add(new FieldSchema("n-ame", ColumnType.STRING_TYPE_NAME, ""));
+      invCols.add(new FieldSchema("in.come", ColumnType.INT_TYPE_NAME, ""));
+
+      Table tbl = new TableBuilder()
+          .setDbName(dbName)
+          .setTableName(invTblName)
+          .setCols(invCols)
+          .build();
+
+      boolean failed = false;
+      try {
+        client.createTable(tbl);
+      } catch (InvalidObjectException ex) {
+        failed = true;
+      }
+      if (!failed) {
+        assertTrue("Able to create table with invalid name: " + invTblName,
+            false);
+      }
+
+      // create an invalid table which has wrong column type
+      ArrayList<FieldSchema> invColsInvType = new ArrayList<>(2);
+      invColsInvType.add(new FieldSchema("name", ColumnType.STRING_TYPE_NAME, ""));
+      invColsInvType.add(new FieldSchema("income", "xyz", ""));
+      tbl.setTableName(tblName);
+      tbl.getSd().setCols(invColsInvType);
+      boolean failChecker = false;
+      try {
+        client.createTable(tbl);
+      } catch (InvalidObjectException ex) {
+        failChecker = true;
+      }
+      if (!failChecker) {
+        assertTrue("Able to create table with invalid column type: " + invTblName,
+            false);
+      }
+
+      ArrayList<FieldSchema> cols = new ArrayList<>(2);
+      cols.add(new FieldSchema("name", ColumnType.STRING_TYPE_NAME, ""));
+      cols.add(new FieldSchema("income", ColumnType.INT_TYPE_NAME, ""));
+
+      // create a valid table
+      tbl.setTableName(tblName);
+      tbl.getSd().setCols(cols);
+      client.createTable(tbl);
+
+      if (isThriftClient) {
+        tbl = client.getTable(tbl.getDbName(), tbl.getTableName());
+      }
+
+      // now try to invalid alter table
+      Table tbl2 = client.getTable(dbName, tblName);
+      failed = false;
+      try {
+        tbl2.setTableName(invTblName);
+        tbl2.getSd().setCols(invCols);
+        client.alter_table(dbName, tblName, tbl2);
+      } catch (InvalidOperationException ex) {
+        failed = true;
+      }
+      if (!failed) {
+        assertTrue("Able to rename table with invalid name: " + invTblName,
+            false);
+      }
+
+      //try an invalid alter table with partition key name
+      Table tbl_pk = client.getTable(tbl.getDbName(), tbl.getTableName());
+      List<FieldSchema> partitionKeys = tbl_pk.getPartitionKeys();
+      for (FieldSchema fs : partitionKeys) {
+        fs.setName("invalid_to_change_name");
+        fs.setComment("can_change_comment");
+      }
+      tbl_pk.setPartitionKeys(partitionKeys);
+      try {
+        client.alter_table(dbName, tblName, tbl_pk);
+      } catch (InvalidOperationException ex) {
+        failed = true;
+      }
+      assertTrue("Should not have succeeded in altering partition key name", failed);
+
+      //try a valid alter table partition key comment
+      failed = false;
+      tbl_pk = client.getTable(tbl.getDbName(), tbl.getTableName());
+      partitionKeys = tbl_pk.getPartitionKeys();
+      for (FieldSchema fs : partitionKeys) {
+        fs.setComment("can_change_comment");
+      }
+      tbl_pk.setPartitionKeys(partitionKeys);
+      try {
+        client.alter_table(dbName, tblName, tbl_pk);
+      } catch (InvalidOperationException ex) {
+        failed = true;
+      }
+      assertFalse("Should not have failed alter table partition comment", failed);
+      Table newT = client.getTable(tbl.getDbName(), tbl.getTableName());
+      assertEquals(partitionKeys, newT.getPartitionKeys());
+
+      // try a valid alter table
+      tbl2.setTableName(tblName + "_renamed");
+      tbl2.getSd().setCols(cols);
+      tbl2.getSd().setNumBuckets(32);
+      client.alter_table(dbName, tblName, tbl2);
+      Table tbl3 = client.getTable(dbName, tbl2.getTableName());
+      assertEquals("Alter table didn't succeed. Num buckets is different ",
+          tbl2.getSd().getNumBuckets(), tbl3.getSd().getNumBuckets());
+      // check that data has moved
+      FileSystem fs = FileSystem.get((new Path(tbl.getSd().getLocation())).toUri(), conf);
+      assertFalse("old table location still exists", fs.exists(new Path(tbl
+          .getSd().getLocation())));
+      assertTrue("data did not move to new location", fs.exists(new Path(tbl3
+          .getSd().getLocation())));
+
+      if (!isThriftClient) {
+        assertEquals("alter table didn't move data correct location", tbl3
+            .getSd().getLocation(), tbl2.getSd().getLocation());
+      }
+
+      // alter table with invalid column type
+      tbl_pk.getSd().setCols(invColsInvType);
+      failed = false;
+      try {
+        client.alter_table(dbName, tbl2.getTableName(), tbl_pk);
+      } catch (InvalidOperationException ex) {
+        failed = true;
+      }
+      assertTrue("Should not have succeeded in altering column", failed);
+
+    } catch (Exception e) {
+      System.err.println(StringUtils.stringifyException(e));
+      System.err.println("testSimpleTable() failed.");
+      throw e;
+    } finally {
+      silentDropDatabase(dbName);
+    }
+  }
+
+  @Test
+  public void testComplexTable() throws Exception {
+
+    String dbName = "compdb";
+    String tblName = "comptbl";
+    String typeName = "Person";
+
+    try {
+      client.dropTable(dbName, tblName);
+      silentDropDatabase(dbName);
+      Database db = new Database();
+      db.setName(dbName);
+      client.createDatabase(db);
+
+      client.dropType(typeName);
+      Type typ1 = new Type();
+      typ1.setName(typeName);
+      typ1.setFields(new ArrayList<>(2));
+      typ1.getFields().add(
+          new FieldSchema("name", ColumnType.STRING_TYPE_NAME, ""));
+      typ1.getFields().add(
+          new FieldSchema("income", ColumnType.INT_TYPE_NAME, ""));
+      client.createType(typ1);
+
+      Table tbl = new TableBuilder()
+          .setDbName(dbName)
+          .setTableName(tblName)
+          .setCols(typ1.getFields())
+          .addPartCol("ds", ColumnType.DATE_TYPE_NAME)
+          .addPartCol("hr", ColumnType.INT_TYPE_NAME)
+          .setNumBuckets(1)
+          .addBucketCol("name")
+          .addStorageDescriptorParam("test_param_1","Use this for comments etc")
+          .build();
+
+      client.createTable(tbl);
+
+      Table tbl2 = client.getTable(dbName, tblName);
+      assertEquals(tbl2.getDbName(), dbName);
+      assertEquals(tbl2.getTableName(), tblName);
+      assertEquals(tbl2.getSd().getCols().size(), typ1.getFields().size());
+      assertFalse(tbl2.getSd().isCompressed());
+      assertFalse(tbl2.getSd().isStoredAsSubDirectories());
+      assertEquals(tbl2.getSd().getNumBuckets(), 1);
+
+      assertEquals("Use this for comments etc", tbl2.getSd().getParameters()
+          .get("test_param_1"));
+      assertEquals("name", tbl2.getSd().getBucketCols().get(0));
+
+      assertNotNull(tbl2.getPartitionKeys());
+      assertEquals(2, tbl2.getPartitionKeys().size());
+      assertEquals(ColumnType.DATE_TYPE_NAME, tbl2.getPartitionKeys().get(0)
+          .getType());
+      assertEquals(ColumnType.INT_TYPE_NAME, tbl2.getPartitionKeys().get(1)
+          .getType());
+      assertEquals("ds", tbl2.getPartitionKeys().get(0).getName());
+      assertEquals("hr", tbl2.getPartitionKeys().get(1).getName());
+
+      List<FieldSchema> fieldSchemas = client.getFields(dbName, tblName);
+      assertNotNull(fieldSchemas);
+      assertEquals(fieldSchemas.size(), tbl.getSd().getCols().size());
+      for (FieldSchema fs : tbl.getSd().getCols()) {
+        assertTrue(fieldSchemas.contains(fs));
+      }
+
+      List<FieldSchema> fieldSchemasFull = client.getSchema(dbName, tblName);
+      assertNotNull(fieldSchemasFull);
+      assertEquals(fieldSchemasFull.size(), tbl.getSd().getCols().size()
+          + tbl.getPartitionKeys().size());
+      for (FieldSchema fs : tbl.getSd().getCols()) {
+        assertTrue(fieldSchemasFull.contains(fs));
+      }
+      for (FieldSchema fs : tbl.getPartitionKeys()) {
+        assertTrue(fieldSchemasFull.contains(fs));
+      }
+    } catch (Exception e) {
+      System.err.println(StringUtils.stringifyException(e));
+      System.err.println("testComplexTable() failed.");
+      throw e;
+    } finally {
+      client.dropTable(dbName, tblName);
+      boolean ret = client.dropType(typeName);
+      assertTrue("Unable to drop type " + typeName, ret);
+      client.dropDatabase(dbName);
+    }
+  }
+
+  @Test
+  public void testTableDatabase() throws Exception {
+    String dbName = "testDb";
+    String tblName_1 = "testTbl_1";
+    String tblName_2 = "testTbl_2";
+
+    try {
+      silentDropDatabase(dbName);
+
+      Database db = new Database();
+      db.setName(dbName);
+      String dbLocation =
+          MetastoreConf.getVar(conf, ConfVars.WAREHOUSE) + "_testDB_table_create_";
+      db.setLocationUri(dbLocation);
+      client.createDatabase(db);
+      db = client.getDatabase(dbName);
+
+      Table tbl = new TableBuilder()
+          .setDbName(dbName)
+          .setTableName(tblName_1)
+          .addCol("name", ColumnType.STRING_TYPE_NAME)
+          .addCol("income", ColumnType.INT_TYPE_NAME)
+          .build();
+
+      client.createTable(tbl);
+      tbl = client.getTable(dbName, tblName_1);
+
+      Path path = new Path(tbl.getSd().getLocation());
+      System.err.println("Table's location " + path + ", Database's location " + db.getLocationUri());
+      assertEquals("Table location is not a subset of the database location",
+          path.getParent().toString(), db.getLocationUri());
+
+    } catch (Exception e) {
+      System.err.println(StringUtils.stringifyException(e));
+      System.err.println("testTableDatabase() failed.");
+      throw e;
+    } finally {
+      silentDropDatabase(dbName);
+    }
+  }
+
+
+  @Test
+  public void testGetConfigValue() {
+
+    String val = "value";
+
+    if (!isThriftClient) {
+      try {
+        assertEquals(client.getConfigValue("hive.key1", val), "value1");
+        assertEquals(client.getConfigValue("hive.key2", val), "http://www.example.com");
+        assertEquals(client.getConfigValue("hive.key3", val), "");
+        assertEquals(client.getConfigValue("hive.key4", val), "0");
+        assertEquals(client.getConfigValue("hive.key5", val), val);
+        assertEquals(client.getConfigValue(null, val), val);
+      } catch (TException e) {
+        e.printStackTrace();
+        fail();
+      }
+    }
+
+    boolean threwException = false;
+    try {
+      // Attempting to get the password should throw an exception
+      client.getConfigValue("javax.jdo.option.ConnectionPassword", "password");
+    } catch (ConfigValSecurityException e) {
+      threwException = true;
+    } catch (TException e) {
+      e.printStackTrace();
+      fail();
+    }
+    assert (threwException);
+  }
+
+  private static void adjust(HiveMetaStoreClient client, Partition part,
+      String dbName, String tblName) throws TException {
+    Partition part_get = client.getPartition(dbName, tblName, part.getValues());
+    part.setCreateTime(part_get.getCreateTime());
+    part.putToParameters(org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.DDL_TIME, Long.toString(part_get.getCreateTime()));
+  }
+
+  private static void silentDropDatabase(String dbName) throws TException {
+    try {
+      for (String tableName : client.getTables(dbName, "*")) {
+        client.dropTable(dbName, tableName);
+      }
+      client.dropDatabase(dbName);
+    } catch (NoSuchObjectException|InvalidOperationException e) {
+      // NOP
+    }
+  }
+
+  /**
+   * Tests for list partition by filter functionality.
+   */
+
+  @Test
+  public void testPartitionFilter() throws Exception {
+    String dbName = "filterdb";
+    String tblName = "filtertbl";
+
+    silentDropDatabase(dbName);
+
+    Database db = new Database();
+    db.setName(dbName);
+    client.createDatabase(db);
+
+    Table tbl = new TableBuilder()
+        .setDbName(dbName)
+        .setTableName(tblName)
+        .addCol("c1", ColumnType.STRING_TYPE_NAME)
+        .addCol("c2", ColumnType.INT_TYPE_NAME)
+        .addPartCol("p1", ColumnType.STRING_TYPE_NAME)
+        .addPartCol("p2", ColumnType.STRING_TYPE_NAME)
+        .addPartCol("p3", ColumnType.INT_TYPE_NAME)
+        .build();
+    client.createTable(tbl);
+
+    tbl = client.getTable(dbName, tblName);
+
+    add_partition(client, tbl, Lists.newArrayList("p11", "p21", "31"), "part1");
+    add_partition(client, tbl, Lists.newArrayList("p11", "p22", "32"), "part2");
+    add_partition(client, tbl, Lists.newArrayList("p12", "p21", "31"), "part3");
+    add_partition(client, tbl, Lists.newArrayList("p12", "p23", "32"), "part4");
+    add_partition(client, tbl, Lists.newArrayList("p13", "p24", "31"), "part5");
+    add_partition(client, tbl, Lists.newArrayList("p13", "p25", "-33"), "part6");
+
+    // Test equals operator for strings and integers.
+    checkFilter(client, dbName, tblName, "p1 = \"p11\"", 2);
+    checkFilter(client, dbName, tblName, "p1 = \"p12\"", 2);
+    checkFilter(client, dbName, tblName, "p2 = \"p21\"", 2);
+    checkFilter(client, dbName, tblName, "p2 = \"p23\"", 1);
+    checkFilter(client, dbName, tblName, "p3 = 31", 3);
+    checkFilter(client, dbName, tblName, "p3 = 33", 0);
+    checkFilter(client, dbName, tblName, "p3 = -33", 1);
+    checkFilter(client, dbName, tblName, "p1 = \"p11\" and p2=\"p22\"", 1);
+    checkFilter(client, dbName, tblName, "p1 = \"p11\" or p2=\"p23\"", 3);
+    checkFilter(client, dbName, tblName, "p1 = \"p11\" or p1=\"p12\"", 4);
+    checkFilter(client, dbName, tblName, "p1 = \"p11\" or p1=\"p12\"", 4);
+    checkFilter(client, dbName, tblName, "p1 = \"p11\" or p1=\"p12\"", 4);
+    checkFilter(client, dbName, tblName, "p1 = \"p11\" and p3 = 31", 1);
+    checkFilter(client, dbName, tblName, "p3 = -33 or p1 = \"p12\"", 3);
+
+    // Test not-equals operator for strings and integers.
+    checkFilter(client, dbName, tblName, "p1 != \"p11\"", 4);
+    checkFilter(client, dbName, tblName, "p2 != \"p23\"", 5);
+    checkFilter(client, dbName, tblName, "p2 != \"p33\"", 6);
+    checkFilter(client, dbName, tblName, "p3 != 32", 4);
+    checkFilter(client, dbName, tblName, "p3 != 8589934592", 6);
+    checkFilter(client, dbName, tblName, "p1 != \"p11\" and p1 != \"p12\"", 2);
+    checkFilter(client, dbName, tblName, "p1 != \"p11\" and p2 != \"p22\"", 4);
+    checkFilter(client, dbName, tblName, "p1 != \"p11\" or p2 != \"p22\"", 5);
+    checkFilter(client, dbName, tblName, "p1 != \"p12\" and p2 != \"p25\"", 3);
+    checkFilter(client, dbName, tblName, "p1 != \"p12\" or p2 != \"p25\"", 6);
+    checkFilter(client, dbName, tblName, "p3 != -33 or p1 != \"p13\"", 5);
+    checkFilter(client, dbName, tblName, "p1 != \"p11\" and p3 = 31", 2);
+    checkFilter(client, dbName, tblName, "p3 != 31 and p1 = \"p12\"", 1);
+
+    // Test reverse order.
+    checkFilter(client, dbName, tblName, "31 != p3 and p1 = \"p12\"", 1);
+    checkFilter(client, dbName, tblName, "\"p23\" = p2", 1);
+
+    // Test and/or more...
+    checkFilter(client, dbName, tblName,
+        "p1 = \"p11\" or (p1=\"p12\" and p2=\"p21\")", 3);
+    checkFilter(client, dbName, tblName,
+       "p1 = \"p11\" or (p1=\"p12\" and p2=\"p21\") Or " +
+       "(p1=\"p13\" aNd p2=\"p24\")", 4);
+    //test for and or precedence
+    checkFilter(client, dbName, tblName,
+       "p1=\"p12\" and (p2=\"p27\" Or p2=\"p21\")", 1);
+    checkFilter(client, dbName, tblName,
+       "p1=\"p12\" and p2=\"p27\" Or p2=\"p21\"", 2);
+
+    // Test gt/lt/lte/gte/like for strings.
+    checkFilter(client, dbName, tblName, "p1 > \"p12\"", 2);
+    checkFilter(client, dbName, tblName, "p1 >= \"p12\"", 4);
+    checkFilter(client, dbName, tblName, "p1 < \"p12\"", 2);
+    checkFilter(client, dbName, tblName, "p1 <= \"p12\"", 4);
+    checkFilter(client, dbName, tblName, "p1 like \"p1.*\"", 6);
+    checkFilter(client, dbName, tblName, "p2 like \"p.*3\"", 1);
+
+    // Test gt/lt/lte/gte for numbers.
+    checkFilter(client, dbName, tblName, "p3 < 0", 1);
+    checkFilter(client, dbName, tblName, "p3 >= -33", 6);
+    checkFilter(client, dbName, tblName, "p3 > -33", 5);
+    checkFilter(client, dbName, tblName, "p3 > 31 and p3 < 32", 0);
+    checkFilter(client, dbName, tblName, "p3 > 31 or p3 < 31", 3);
+    checkFilter(client, dbName, tblName, "p3 > 30 or p3 < 30", 6);
+    checkFilter(client, dbName, tblName, "p3 >= 31 or p3 < -32", 6);
+    checkFilter(client, dbName, tblName, "p3 >= 32", 2);
+    checkFilter(client, dbName, tblName, "p3 > 32", 0);
+
+    // Test between
+    checkFilter(client, dbName, tblName, "p1 between \"p11\" and \"p12\"", 4);
+    checkFilter(client, dbName, tblName, "p1 not between \"p11\" and \"p12\"", 2);
+    checkFilter(client, dbName, tblName, "p3 not between 0 and 2", 6);
+    checkFilter(client, dbName, tblName, "p3 between 31 and 32", 5);
+    checkFilter(client, dbName, tblName, "p3 between 32 and 31", 0);
+    checkFilter(client, dbName, tblName, "p3 between -32 and 34 and p3 not between 31 and 32", 0);
+    checkFilter(client, dbName, tblName, "p3 between 1 and 3 or p3 not between 1 and 3", 6);
+    checkFilter(client, dbName, tblName,
+        "p3 between 31 and 32 and p1 between \"p12\" and \"p14\"", 3);
+
+    //Test for setting the maximum partition count
+    List<Partition> partitions = client.listPartitionsByFilter(dbName,
+        tblName, "p1 >= \"p12\"", (short) 2);
+    assertEquals("User specified row limit for partitions",
+        2, partitions.size());
+
+    //Negative tests
+    Exception me = null;
+    try {
+      client.listPartitionsByFilter(dbName,
+          tblName, "p3 >= \"p12\"", (short) -1);
+    } catch(MetaException e) {
+      me = e;
+    }
+    assertNotNull(me);
+    assertTrue("Filter on int partition key", me.getMessage().contains(
+          "Filtering is supported only on partition keys of type string"));
+
+    me = null;
+    try {
+      client.listPartitionsByFilter(dbName,
+          tblName, "c1 >= \"p12\"", (short) -1);
+    } catch(MetaException e) {
+      me = e;
+    }
+    assertNotNull(me);
+    assertTrue("Filter on invalid key", me.getMessage().contains(
+          "<c1> is not a partitioning key for the table"));
+
+    me = null;
+    try {
+      client.listPartitionsByFilter(dbName,
+          tblName, "c1 >= ", (short) -1);
+    } catch(MetaException e) {
+      me = e;
+    }
+    assertNotNull(me);
+    assertTrue("Invalid filter string", me.getMessage().contains(
+          "Error parsing partition filter"));
+
+    me = null;
+    try {
+      client.listPartitionsByFilter("invDBName",
+          "invTableName", "p1 = \"p11\"", (short) -1);
+    } catch(NoSuchObjectException e) {
+      me = e;
+    }
+    assertNotNull(me);
+    assertTrue("NoSuchObject exception", me.getMessage().contains(
+          "invDBName.invTableName table not found"));
+
+    client.dropTable(dbName, tblName);
+    client.dropDatabase(dbName);
+  }
+
+
+  /**
+   * Test filtering on table with single partition
+   */
+  @Test
+  public void testFilterSinglePartition() throws Exception {
+      String dbName = "filterdb";
+      String tblName = "filtertbl";
+
+      List<String> vals = new ArrayList<>(1);
+      vals.add("p11");
+      List <String> vals2 = new ArrayList<>(1);
+      vals2.add("p12");
+      List <String> vals3 = new ArrayList<>(1);
+      vals3.add("p13");
+
+      silentDropDatabase(dbName);
+
+      Database db = new Database();
+      db.setName(dbName);
+      client.createDatabase(db);
+
+      Table tbl = new TableBuilder()
+          .setDbName(dbName)
+          .setTableName(tblName)
+          .addCol("c1", ColumnType.STRING_TYPE_NAME)
+          .addCol("c2", ColumnType.INT_TYPE_NAME)
+          .addPartCol("p1", ColumnType.STRING_TYPE_NAME)
+          .build();
+      client.createTable(tbl);
+
+      tbl = client.getTable(dbName, tblName);
+
+      add_partition(client, tbl, vals, "part1");
+      add_partition(client, tbl, vals2, "part2");
+      add_partition(client, tbl, vals3, "part3");
+
+      checkFilter(client, dbName, tblName, "p1 = \"p12\"", 1);
+      checkFilter(client, dbName, tblName, "p1 < \"p12\"", 1);
+      checkFilter(client, dbName, tblName, "p1 > \"p12\"", 1);
+      checkFilter(client, dbName, tblName, "p1 >= \"p12\"", 2);
+      checkFilter(client, dbName, tblName, "p1 <= \"p12\"", 2);
+      checkFilter(client, dbName, tblName, "p1 <> \"p12\"", 2);
+      checkFilter(client, dbName, tblName, "p1 like \"p1.*\"", 3);
+      checkFilter(client, dbName, tblName, "p1 like \"p.*2\"", 1);
+
+      client.dropTable(dbName, tblName);
+      client.dropDatabase(dbName);
+  }
+
+  /**
+   * Test filtering based on the value of the last partition
+   */
+  @Test
+  public void testFilterLastPartition() throws Exception {
+      String dbName = "filterdb";
+      String tblName = "filtertbl";
+
+      List<String> vals = new ArrayList<>(2);
+      vals.add("p11");
+      vals.add("p21");
+      List <String> vals2 = new ArrayList<>(2);
+      vals2.add("p11");
+      vals2.add("p22");
+      List <String> vals3 = new ArrayList<>(2);
+      vals3.add("p12");
+      vals3.add("p21");
+
+      cleanUp(dbName, tblName, null);
+
+      createDb(dbName);
+
+      Table tbl = new TableBuilder()
+          .setDbName(dbName)
+          .setTableName(tblName)
+          .addCol("c1", ColumnType.STRING_TYPE_NAME)
+          .addCol("c2", ColumnType.INT_TYPE_NAME)
+          .addPartCol("p1", ColumnType.STRING_TYPE_NAME)
+          .addPartCol("p2", ColumnType.STRING_TYPE_NAME)
+          .build();
+
+      client.createTable(tbl);
+      tbl = client.getTable(dbName, tblName);
+
+      add_partition(client, tbl, vals, "part1");
+      add_partition(client, tbl, vals2, "part2");
+      add_partition(client, tbl, vals3, "part3");
+
+      checkFilter(client, dbN

<TRUNCATED>

[6/7] hive git commit: HIVE-17982 Move metastore specific itests. This closes #279. (Alan Gates, reviewed by Peter Vary)

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/d9801d9c/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
deleted file mode 100644
index 0aa1d4e..0000000
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
+++ /dev/null
@@ -1,3514 +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.lang.reflect.Field;
-import java.io.IOException;
-import java.sql.Connection;
-import java.sql.PreparedStatement;
-import java.sql.SQLException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Set;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-
-import junit.framework.TestCase;
-
-import org.datanucleus.api.jdo.JDOPersistenceManager;
-import org.datanucleus.api.jdo.JDOPersistenceManagerFactory;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hive.common.FileUtils;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
-import org.apache.hadoop.hive.metastore.api.AggrStats;
-import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
-import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
-import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData;
-import org.apache.hadoop.hive.metastore.api.ColumnStatisticsDesc;
-import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
-import org.apache.hadoop.hive.metastore.api.ConfigValSecurityException;
-import org.apache.hadoop.hive.metastore.api.Database;
-import org.apache.hadoop.hive.metastore.api.DoubleColumnStatsData;
-import org.apache.hadoop.hive.metastore.api.FieldSchema;
-import org.apache.hadoop.hive.metastore.api.Function;
-import org.apache.hadoop.hive.metastore.api.FunctionType;
-import org.apache.hadoop.hive.metastore.api.GetAllFunctionsResponse;
-import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
-import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
-import org.apache.hadoop.hive.metastore.api.MetaException;
-import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
-import org.apache.hadoop.hive.metastore.api.Order;
-import org.apache.hadoop.hive.metastore.api.Partition;
-import org.apache.hadoop.hive.metastore.api.PrincipalType;
-import org.apache.hadoop.hive.metastore.api.ResourceType;
-import org.apache.hadoop.hive.metastore.api.ResourceUri;
-import org.apache.hadoop.hive.metastore.api.SerDeInfo;
-import org.apache.hadoop.hive.metastore.api.SkewedInfo;
-import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
-import org.apache.hadoop.hive.metastore.api.StringColumnStatsData;
-import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.hadoop.hive.metastore.api.Type;
-import org.apache.hadoop.hive.metastore.api.UnknownDBException;
-import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
-import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
-import org.apache.hadoop.hive.ql.exec.Utilities;
-import org.apache.hadoop.hive.ql.io.HiveInputFormat;
-import org.apache.hadoop.hive.ql.io.HiveOutputFormat;
-import org.apache.hadoop.hive.ql.session.SessionState;
-import org.apache.hadoop.hive.serde.serdeConstants;
-import org.apache.hadoop.hive.serde2.SerDeException;
-import org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe;
-import org.apache.hadoop.util.StringUtils;
-import org.apache.thrift.TException;
-import org.junit.Assert;
-import org.junit.Test;
-
-import com.google.common.collect.Lists;
-
-public abstract class TestHiveMetaStore extends TestCase {
-  private static final Logger LOG = LoggerFactory.getLogger(TestHiveMetaStore.class);
-  protected static HiveMetaStoreClient client;
-  protected static HiveConf hiveConf;
-  protected static Warehouse warehouse;
-  protected static boolean isThriftClient = false;
-
-  private static final String TEST_DB1_NAME = "testdb1";
-  private static final String TEST_DB2_NAME = "testdb2";
-
-  private static final int DEFAULT_LIMIT_PARTITION_REQUEST = 100;
-
-  protected abstract HiveMetaStoreClient createClient() throws Exception;
-
-  @Override
-  protected void setUp() throws Exception {
-    hiveConf = new HiveConf(this.getClass());
-    warehouse = new Warehouse(hiveConf);
-
-    // set some values to use for getting conf. vars
-    hiveConf.set("hive.metastore.metrics.enabled","true");
-    hiveConf.set("hive.key1", "value1");
-    hiveConf.set("hive.key2", "http://www.example.com");
-    hiveConf.set("hive.key3", "");
-    hiveConf.set("hive.key4", "0");
-    hiveConf.set("datanucleus.autoCreateTables", "false");
-
-    hiveConf.setIntVar(ConfVars.METASTORE_BATCH_RETRIEVE_MAX, 2);
-    hiveConf.setIntVar(ConfVars.METASTORE_LIMIT_PARTITION_REQUEST, DEFAULT_LIMIT_PARTITION_REQUEST);
-  }
-
-  public void testNameMethods() {
-    Map<String, String> spec = new LinkedHashMap<String, String>();
-    spec.put("ds", "2008-07-01 14:13:12");
-    spec.put("hr", "14");
-    List<String> vals = new ArrayList<String>();
-    for(String v : spec.values()) {
-      vals.add(v);
-    }
-    String partName = "ds=2008-07-01 14%3A13%3A12/hr=14";
-
-    try {
-      List<String> testVals = client.partitionNameToVals(partName);
-      assertTrue("Values from name are incorrect", vals.equals(testVals));
-
-      Map<String, String> testSpec = client.partitionNameToSpec(partName);
-      assertTrue("Spec from name is incorrect", spec.equals(testSpec));
-
-      List<String> emptyVals = client.partitionNameToVals("");
-      assertTrue("Values should be empty", emptyVals.size() == 0);
-
-      Map<String, String> emptySpec =  client.partitionNameToSpec("");
-      assertTrue("Spec should be empty", emptySpec.size() == 0);
-    } catch (Exception e) {
-      assert(false);
-    }
-  }
-
-  /**
-   * tests create table and partition and tries to drop the table without
-   * droppping the partition
-   *
-   * @throws Exception
-   */
-  public void testPartition() throws Exception {
-    partitionTester(client, hiveConf);
-  }
-
-  public static void partitionTester(HiveMetaStoreClient client, HiveConf hiveConf)
-    throws Exception {
-    try {
-      String dbName = "compdb";
-      String tblName = "comptbl";
-      String typeName = "Person";
-      List<String> vals = makeVals("2008-07-01 14:13:12", "14");
-      List<String> vals2 = makeVals("2008-07-01 14:13:12", "15");
-      List<String> vals3 = makeVals("2008-07-02 14:13:12", "15");
-      List<String> vals4 = makeVals("2008-07-03 14:13:12", "151");
-
-      client.dropTable(dbName, tblName);
-      silentDropDatabase(dbName);
-      Database db = new Database();
-      db.setName(dbName);
-      client.createDatabase(db);
-      db = client.getDatabase(dbName);
-      Path dbPath = new Path(db.getLocationUri());
-      FileSystem fs = FileSystem.get(dbPath.toUri(), hiveConf);
-
-      client.dropType(typeName);
-      Type typ1 = new Type();
-      typ1.setName(typeName);
-      typ1.setFields(new ArrayList<FieldSchema>(2));
-      typ1.getFields().add(
-          new FieldSchema("name", serdeConstants.STRING_TYPE_NAME, ""));
-      typ1.getFields().add(
-          new FieldSchema("income", serdeConstants.INT_TYPE_NAME, ""));
-      client.createType(typ1);
-
-      Table tbl = new Table();
-      tbl.setDbName(dbName);
-      tbl.setTableName(tblName);
-      StorageDescriptor sd = new StorageDescriptor();
-      tbl.setSd(sd);
-      sd.setCols(typ1.getFields());
-      sd.setCompressed(false);
-      sd.setNumBuckets(1);
-      sd.setParameters(new HashMap<String, String>());
-      sd.getParameters().put("test_param_1", "Use this for comments etc");
-      sd.setBucketCols(new ArrayList<String>(2));
-      sd.getBucketCols().add("name");
-      sd.setSerdeInfo(new SerDeInfo());
-      sd.getSerdeInfo().setName(tbl.getTableName());
-      sd.getSerdeInfo().setParameters(new HashMap<String, String>());
-      sd.getSerdeInfo().getParameters()
-          .put(serdeConstants.SERIALIZATION_FORMAT, "1");
-      sd.setSortCols(new ArrayList<Order>());
-      sd.setStoredAsSubDirectories(false);
-      sd.getSerdeInfo().setSerializationLib(LazySimpleSerDe.class.getName());
-      sd.setInputFormat(HiveInputFormat.class.getName());
-      sd.setOutputFormat(HiveOutputFormat.class.getName());
-
-      //skewed information
-      SkewedInfo skewInfor = new SkewedInfo();
-      skewInfor.setSkewedColNames(Arrays.asList("name"));
-      List<String> skv = Arrays.asList("1");
-      skewInfor.setSkewedColValues(Arrays.asList(skv));
-      Map<List<String>, String> scvlm = new HashMap<List<String>, String>();
-      scvlm.put(skv, "location1");
-      skewInfor.setSkewedColValueLocationMaps(scvlm);
-      sd.setSkewedInfo(skewInfor);
-
-      tbl.setPartitionKeys(new ArrayList<FieldSchema>(2));
-      tbl.getPartitionKeys().add(
-          new FieldSchema("ds", serdeConstants.STRING_TYPE_NAME, ""));
-      tbl.getPartitionKeys().add(
-          new FieldSchema("hr", serdeConstants.STRING_TYPE_NAME, ""));
-
-      client.createTable(tbl);
-
-      if (isThriftClient) {
-        // the createTable() above does not update the location in the 'tbl'
-        // object when the client is a thrift client and the code below relies
-        // on the location being present in the 'tbl' object - so get the table
-        // from the metastore
-        tbl = client.getTable(dbName, tblName);
-      }
-
-      Partition part = makePartitionObject(dbName, tblName, vals, tbl, "/part1");
-      Partition part2 = makePartitionObject(dbName, tblName, vals2, tbl, "/part2");
-      Partition part3 = makePartitionObject(dbName, tblName, vals3, tbl, "/part3");
-      Partition part4 = makePartitionObject(dbName, tblName, vals4, tbl, "/part4");
-
-      // check if the partition exists (it shouldn't)
-      boolean exceptionThrown = false;
-      try {
-        Partition p = client.getPartition(dbName, tblName, vals);
-      } catch(Exception e) {
-        assertEquals("partition should not have existed",
-            NoSuchObjectException.class, e.getClass());
-        exceptionThrown = true;
-      }
-      assertTrue("getPartition() should have thrown NoSuchObjectException", exceptionThrown);
-      Partition retp = client.add_partition(part);
-      assertNotNull("Unable to create partition " + part, retp);
-      Partition retp2 = client.add_partition(part2);
-      assertNotNull("Unable to create partition " + part2, retp2);
-      Partition retp3 = client.add_partition(part3);
-      assertNotNull("Unable to create partition " + part3, retp3);
-      Partition retp4 = client.add_partition(part4);
-      assertNotNull("Unable to create partition " + part4, retp4);
-
-      Partition part_get = client.getPartition(dbName, tblName, part.getValues());
-      if(isThriftClient) {
-        // since we are using thrift, 'part' will not have the create time and
-        // last DDL time set since it does not get updated in the add_partition()
-        // call - likewise part2 and part3 - set it correctly so that equals check
-        // doesn't fail
-        adjust(client, part, dbName, tblName);
-        adjust(client, part2, dbName, tblName);
-        adjust(client, part3, dbName, tblName);
-      }
-      assertTrue("Partitions are not same", part.equals(part_get));
-
-      // check null cols schemas for a partition
-      List<String> vals6 = makeVals("2016-02-22 00:00:00", "16");
-      Partition part6 = makePartitionObject(dbName, tblName, vals6, tbl, "/part5");
-      part6.getSd().setCols(null);
-      LOG.info("Creating partition will null field schema");
-      client.add_partition(part6);
-      LOG.info("Listing all partitions for table " + dbName + "." + tblName);
-      final List<Partition> partitions = client.listPartitions(dbName, tblName, (short) -1);
-      boolean foundPart = false;
-      for (Partition p : partitions) {
-        if (p.getValues().equals(vals6)) {
-          assertNull(p.getSd().getCols());
-          LOG.info("Found partition " + p + " having null field schema");
-          foundPart = true;
-        }
-      }
-      assertTrue(foundPart);
-
-      String partName = "ds=" + FileUtils.escapePathName("2008-07-01 14:13:12") + "/hr=14";
-      String part2Name = "ds=" + FileUtils.escapePathName("2008-07-01 14:13:12") + "/hr=15";
-      String part3Name = "ds=" + FileUtils.escapePathName("2008-07-02 14:13:12") + "/hr=15";
-      String part4Name = "ds=" + FileUtils.escapePathName("2008-07-03 14:13:12") + "/hr=151";
-
-      part_get = client.getPartition(dbName, tblName, partName);
-      assertTrue("Partitions are not the same", part.equals(part_get));
-
-      // Test partition listing with a partial spec - ds is specified but hr is not
-      List<String> partialVals = new ArrayList<String>();
-      partialVals.add(vals.get(0));
-      Set<Partition> parts = new HashSet<Partition>();
-      parts.add(part);
-      parts.add(part2);
-
-      List<Partition> partial = client.listPartitions(dbName, tblName, partialVals,
-          (short) -1);
-      assertTrue("Should have returned 2 partitions", partial.size() == 2);
-      assertTrue("Not all parts returned", partial.containsAll(parts));
-
-      Set<String> partNames = new HashSet<String>();
-      partNames.add(partName);
-      partNames.add(part2Name);
-      List<String> partialNames = client.listPartitionNames(dbName, tblName, partialVals,
-          (short) -1);
-      assertTrue("Should have returned 2 partition names", partialNames.size() == 2);
-      assertTrue("Not all part names returned", partialNames.containsAll(partNames));
-
-      partNames.add(part3Name);
-      partNames.add(part4Name);
-      partialVals.clear();
-      partialVals.add("");
-      partialNames = client.listPartitionNames(dbName, tblName, partialVals, (short) -1);
-      assertTrue("Should have returned 5 partition names", partialNames.size() == 5);
-      assertTrue("Not all part names returned", partialNames.containsAll(partNames));
-
-      // Test partition listing with a partial spec - hr is specified but ds is not
-      parts.clear();
-      parts.add(part2);
-      parts.add(part3);
-
-      partialVals.clear();
-      partialVals.add("");
-      partialVals.add(vals2.get(1));
-
-      partial = client.listPartitions(dbName, tblName, partialVals, (short) -1);
-      assertEquals("Should have returned 2 partitions", 2, partial.size());
-      assertTrue("Not all parts returned", partial.containsAll(parts));
-
-      partNames.clear();
-      partNames.add(part2Name);
-      partNames.add(part3Name);
-      partialNames = client.listPartitionNames(dbName, tblName, partialVals,
-          (short) -1);
-      assertEquals("Should have returned 2 partition names", 2, partialNames.size());
-      assertTrue("Not all part names returned", partialNames.containsAll(partNames));
-
-      // Verify escaped partition names don't return partitions
-      exceptionThrown = false;
-      try {
-        String badPartName = "ds=2008-07-01 14%3A13%3A12/hrs=14";
-        client.getPartition(dbName, tblName, badPartName);
-      } catch(NoSuchObjectException e) {
-        exceptionThrown = true;
-      }
-      assertTrue("Bad partition spec should have thrown an exception", exceptionThrown);
-
-      Path partPath = new Path(part.getSd().getLocation());
-
-
-      assertTrue(fs.exists(partPath));
-      client.dropPartition(dbName, tblName, part.getValues(), true);
-      assertFalse(fs.exists(partPath));
-
-      // Test append_partition_by_name
-      client.appendPartition(dbName, tblName, partName);
-      Partition part5 = client.getPartition(dbName, tblName, part.getValues());
-      assertTrue("Append partition by name failed", part5.getValues().equals(vals));;
-      Path part5Path = new Path(part5.getSd().getLocation());
-      assertTrue(fs.exists(part5Path));
-
-      // Test drop_partition_by_name
-      assertTrue("Drop partition by name failed",
-          client.dropPartition(dbName, tblName, partName, true));
-      assertFalse(fs.exists(part5Path));
-
-      // add the partition again so that drop table with a partition can be
-      // tested
-      retp = client.add_partition(part);
-      assertNotNull("Unable to create partition " + part, retp);
-
-      // test add_partitions
-
-      List<String> mvals1 = makeVals("2008-07-04 14:13:12", "14641");
-      List<String> mvals2 = makeVals("2008-07-04 14:13:12", "14642");
-      List<String> mvals3 = makeVals("2008-07-04 14:13:12", "14643");
-      List<String> mvals4 = makeVals("2008-07-04 14:13:12", "14643"); // equal to 3
-      List<String> mvals5 = makeVals("2008-07-04 14:13:12", "14645");
-
-      Exception savedException;
-
-      // add_partitions(empty list) : ok, normal operation
-      client.add_partitions(new ArrayList<Partition>());
-
-      // add_partitions(1,2,3) : ok, normal operation
-      Partition mpart1 = makePartitionObject(dbName, tblName, mvals1, tbl, "/mpart1");
-      Partition mpart2 = makePartitionObject(dbName, tblName, mvals2, tbl, "/mpart2");
-      Partition mpart3 = makePartitionObject(dbName, tblName, mvals3, tbl, "/mpart3");
-      client.add_partitions(Arrays.asList(mpart1,mpart2,mpart3));
-
-      if(isThriftClient) {
-        // do DDL time munging if thrift mode
-        adjust(client, mpart1, dbName, tblName);
-        adjust(client, mpart2, dbName, tblName);
-        adjust(client, mpart3, dbName, tblName);
-      }
-      verifyPartitionsPublished(client, dbName, tblName,
-          Arrays.asList(mvals1.get(0)),
-          Arrays.asList(mpart1,mpart2,mpart3));
-
-      Partition mpart4 = makePartitionObject(dbName, tblName, mvals4, tbl, "/mpart4");
-      Partition mpart5 = makePartitionObject(dbName, tblName, mvals5, tbl, "/mpart5");
-
-      // create dir for /mpart5
-      Path mp5Path = new Path(mpart5.getSd().getLocation());
-      warehouse.mkdirs(mp5Path);
-      assertTrue(fs.exists(mp5Path));
-
-      // add_partitions(5,4) : err = duplicate keyvals on mpart4
-      savedException = null;
-      try {
-        client.add_partitions(Arrays.asList(mpart5,mpart4));
-      } catch (Exception e) {
-        savedException = e;
-      } finally {
-        assertNotNull(savedException);
-      }
-
-      // check that /mpart4 does not exist, but /mpart5 still does.
-      assertTrue(fs.exists(mp5Path));
-      assertFalse(fs.exists(new Path(mpart4.getSd().getLocation())));
-
-      // add_partitions(5) : ok
-      client.add_partitions(Arrays.asList(mpart5));
-
-      if(isThriftClient) {
-        // do DDL time munging if thrift mode
-        adjust(client, mpart5, dbName, tblName);
-      }
-
-      verifyPartitionsPublished(client, dbName, tblName,
-          Arrays.asList(mvals1.get(0)),
-          Arrays.asList(mpart1,mpart2,mpart3,mpart5));
-
-      //// end add_partitions tests
-
-      client.dropTable(dbName, tblName);
-
-      client.dropType(typeName);
-
-      // recreate table as external, drop partition and it should
-      // still exist
-      tbl.setParameters(new HashMap<String, String>());
-      tbl.getParameters().put("EXTERNAL", "TRUE");
-      client.createTable(tbl);
-      retp = client.add_partition(part);
-      assertTrue(fs.exists(partPath));
-      client.dropPartition(dbName, tblName, part.getValues(), true);
-      assertTrue(fs.exists(partPath));
-
-      for (String tableName : client.getTables(dbName, "*")) {
-        client.dropTable(dbName, tableName);
-      }
-
-      client.dropDatabase(dbName);
-
-    } catch (Exception e) {
-      System.err.println(StringUtils.stringifyException(e));
-      System.err.println("testPartition() failed.");
-      throw e;
-    }
-  }
-
-  private static void verifyPartitionsPublished(HiveMetaStoreClient client,
-      String dbName, String tblName, List<String> partialSpec,
-      List<Partition> expectedPartitions)
-          throws NoSuchObjectException, MetaException, TException {
-    // Test partition listing with a partial spec
-
-    List<Partition> mpartial = client.listPartitions(dbName, tblName, partialSpec,
-        (short) -1);
-    assertEquals("Should have returned "+expectedPartitions.size()+
-        " partitions, returned " + mpartial.size(),
-        expectedPartitions.size(), mpartial.size());
-    assertTrue("Not all parts returned", mpartial.containsAll(expectedPartitions));
-  }
-
-  private static List<String> makeVals(String ds, String id) {
-    List <String> vals4 = new ArrayList<String>(2);
-    vals4 = new ArrayList<String>(2);
-    vals4.add(ds);
-    vals4.add(id);
-    return vals4;
-  }
-
-  private static Partition makePartitionObject(String dbName, String tblName,
-      List<String> ptnVals, Table tbl, String ptnLocationSuffix) throws MetaException {
-    Partition part4 = new Partition();
-    part4.setDbName(dbName);
-    part4.setTableName(tblName);
-    part4.setValues(ptnVals);
-    part4.setParameters(new HashMap<String, String>());
-    part4.setSd(tbl.getSd().deepCopy());
-    part4.getSd().setSerdeInfo(tbl.getSd().getSerdeInfo().deepCopy());
-    part4.getSd().setLocation(tbl.getSd().getLocation() + ptnLocationSuffix);
-    MetaStoreUtils.updatePartitionStatsFast(part4, warehouse, null);
-    return part4;
-  }
-
-  public void testListPartitions() throws Throwable {
-    // create a table with multiple partitions
-    String dbName = "compdb";
-    String tblName = "comptbl";
-    String typeName = "Person";
-
-    cleanUp(dbName, tblName, typeName);
-
-    List<List<String>> values = new ArrayList<List<String>>();
-    values.add(makeVals("2008-07-01 14:13:12", "14"));
-    values.add(makeVals("2008-07-01 14:13:12", "15"));
-    values.add(makeVals("2008-07-02 14:13:12", "15"));
-    values.add(makeVals("2008-07-03 14:13:12", "151"));
-
-    createMultiPartitionTableSchema(dbName, tblName, typeName, values);
-
-    List<Partition> partitions = client.listPartitions(dbName, tblName, (short)-1);
-    assertNotNull("should have returned partitions", partitions);
-    assertEquals(" should have returned " + values.size() +
-      " partitions", values.size(), partitions.size());
-
-    partitions = client.listPartitions(dbName, tblName, (short)(values.size()/2));
-
-    assertNotNull("should have returned partitions", partitions);
-    assertEquals(" should have returned " + values.size() / 2 +
-      " partitions",values.size() / 2, partitions.size());
-
-
-    partitions = client.listPartitions(dbName, tblName, (short) (values.size() * 2));
-
-    assertNotNull("should have returned partitions", partitions);
-    assertEquals(" should have returned " + values.size() +
-      " partitions",values.size(), partitions.size());
-
-    cleanUp(dbName, tblName, typeName);
-
-  }
-
-  public void testListPartitionsWihtLimitEnabled() throws Throwable {
-    // create a table with multiple partitions
-    String dbName = "compdb";
-    String tblName = "comptbl";
-    String typeName = "Person";
-
-    cleanUp(dbName, tblName, typeName);
-
-    // Create too many partitions, just enough to validate over limit requests
-    List<List<String>> values = new ArrayList<List<String>>();
-    for (int i=0; i<DEFAULT_LIMIT_PARTITION_REQUEST + 1; i++) {
-      values.add(makeVals("2008-07-01 14:13:12", Integer.toString(i)));
-    }
-
-    createMultiPartitionTableSchema(dbName, tblName, typeName, values);
-
-    List<Partition> partitions;
-    short maxParts;
-
-    // Requesting more partitions than allowed should throw an exception
-    try {
-      maxParts = -1;
-      partitions = client.listPartitions(dbName, tblName, maxParts);
-      fail("should have thrown MetaException about partition limit");
-    } catch (MetaException e) {
-      assertTrue(true);
-    }
-
-    // Requesting more partitions than allowed should throw an exception
-    try {
-      maxParts = DEFAULT_LIMIT_PARTITION_REQUEST + 1;
-      partitions = client.listPartitions(dbName, tblName, maxParts);
-      fail("should have thrown MetaException about partition limit");
-    } catch (MetaException e) {
-      assertTrue(true);
-    }
-
-    // Requesting less partitions than allowed should work
-    maxParts = DEFAULT_LIMIT_PARTITION_REQUEST / 2;
-    partitions = client.listPartitions(dbName, tblName, maxParts);
-    assertNotNull("should have returned partitions", partitions);
-    assertEquals(" should have returned 50 partitions", maxParts, partitions.size());
-  }
-
-  public void testAlterTableCascade() throws Throwable {
-    // create a table with multiple partitions
-    String dbName = "compdb";
-    String tblName = "comptbl";
-    String typeName = "Person";
-
-    cleanUp(dbName, tblName, typeName);
-
-    List<List<String>> values = new ArrayList<List<String>>();
-    values.add(makeVals("2008-07-01 14:13:12", "14"));
-    values.add(makeVals("2008-07-01 14:13:12", "15"));
-    values.add(makeVals("2008-07-02 14:13:12", "15"));
-    values.add(makeVals("2008-07-03 14:13:12", "151"));
-
-    createMultiPartitionTableSchema(dbName, tblName, typeName, values);
-    Table tbl = client.getTable(dbName, tblName);
-    List<FieldSchema> cols = tbl.getSd().getCols();
-    cols.add(new FieldSchema("new_col", serdeConstants.STRING_TYPE_NAME, ""));
-    tbl.getSd().setCols(cols);
-    //add new column with cascade option
-    client.alter_table(dbName, tblName, tbl, true);
-    //
-    Table tbl2 = client.getTable(dbName, tblName);
-    Assert.assertEquals("Unexpected number of cols", 3, tbl2.getSd().getCols().size());
-    Assert.assertEquals("Unexpected column name", "new_col", tbl2.getSd().getCols().get(2).getName());
-    //get a partition
-    List<String> pvalues = new ArrayList<>(2);
-    pvalues.add("2008-07-01 14:13:12");
-    pvalues.add("14");
-    Partition partition = client.getPartition(dbName, tblName, pvalues);
-    Assert.assertEquals("Unexpected number of cols", 3, partition.getSd().getCols().size());
-    Assert.assertEquals("Unexpected column name", "new_col", partition.getSd().getCols().get(2).getName());
-
-    //add another column
-    cols = tbl.getSd().getCols();
-    cols.add(new FieldSchema("new_col2", serdeConstants.STRING_TYPE_NAME, ""));
-    tbl.getSd().setCols(cols);
-    //add new column with no cascade option
-    client.alter_table(dbName, tblName, tbl, false);
-    tbl2 = client.getTable(dbName, tblName);
-    Assert.assertEquals("Unexpected number of cols", 4, tbl2.getSd().getCols().size());
-    Assert.assertEquals("Unexpected column name", "new_col2", tbl2.getSd().getCols().get(3).getName());
-    //get partition, this partition should not have the newly added column since cascade option
-    //was false
-    partition = client.getPartition(dbName, tblName, pvalues);
-    Assert.assertEquals("Unexpected number of cols", 3, partition.getSd().getCols().size());  
-  }
-
-
-  public void testListPartitionNames() throws Throwable {
-    // create a table with multiple partitions
-    String dbName = "compdb";
-    String tblName = "comptbl";
-    String typeName = "Person";
-
-    cleanUp(dbName, tblName, typeName);
-
-    List<List<String>> values = new ArrayList<List<String>>();
-    values.add(makeVals("2008-07-01 14:13:12", "14"));
-    values.add(makeVals("2008-07-01 14:13:12", "15"));
-    values.add(makeVals("2008-07-02 14:13:12", "15"));
-    values.add(makeVals("2008-07-03 14:13:12", "151"));
-
-
-
-    createMultiPartitionTableSchema(dbName, tblName, typeName, values);
-
-    List<String> partitions = client.listPartitionNames(dbName, tblName, (short)-1);
-    assertNotNull("should have returned partitions", partitions);
-    assertEquals(" should have returned " + values.size() +
-      " partitions", values.size(), partitions.size());
-
-    partitions = client.listPartitionNames(dbName, tblName, (short)(values.size()/2));
-
-    assertNotNull("should have returned partitions", partitions);
-    assertEquals(" should have returned " + values.size() / 2 +
-      " partitions",values.size() / 2, partitions.size());
-
-
-    partitions = client.listPartitionNames(dbName, tblName, (short) (values.size() * 2));
-
-    assertNotNull("should have returned partitions", partitions);
-    assertEquals(" should have returned " + values.size() +
-      " partitions",values.size(), partitions.size());
-
-    cleanUp(dbName, tblName, typeName);
-
-  }
-
-
-  public void testDropTable() throws Throwable {
-    // create a table with multiple partitions
-    String dbName = "compdb";
-    String tblName = "comptbl";
-    String typeName = "Person";
-
-    cleanUp(dbName, tblName, typeName);
-
-    List<List<String>> values = new ArrayList<List<String>>();
-    values.add(makeVals("2008-07-01 14:13:12", "14"));
-    values.add(makeVals("2008-07-01 14:13:12", "15"));
-    values.add(makeVals("2008-07-02 14:13:12", "15"));
-    values.add(makeVals("2008-07-03 14:13:12", "151"));
-
-    createMultiPartitionTableSchema(dbName, tblName, typeName, values);
-
-    client.dropTable(dbName, tblName);
-    client.dropType(typeName);
-
-    boolean exceptionThrown = false;
-    try {
-      client.getTable(dbName, tblName);
-    } catch(Exception e) {
-      assertEquals("table should not have existed",
-          NoSuchObjectException.class, e.getClass());
-      exceptionThrown = true;
-    }
-    assertTrue("Table " + tblName + " should have been dropped ", exceptionThrown);
-
-  }
-
-  public void testAlterViewParititon() throws Throwable {
-    String dbName = "compdb";
-    String tblName = "comptbl";
-    String viewName = "compView";
-
-    client.dropTable(dbName, tblName);
-    silentDropDatabase(dbName);
-    Database db = new Database();
-    db.setName(dbName);
-    db.setDescription("Alter Partition Test database");
-    client.createDatabase(db);
-
-    ArrayList<FieldSchema> cols = new ArrayList<FieldSchema>(2);
-    cols.add(new FieldSchema("name", serdeConstants.STRING_TYPE_NAME, ""));
-    cols.add(new FieldSchema("income", serdeConstants.INT_TYPE_NAME, ""));
-
-    Table tbl = new Table();
-    tbl.setDbName(dbName);
-    tbl.setTableName(tblName);
-    StorageDescriptor sd = new StorageDescriptor();
-    tbl.setSd(sd);
-    sd.setCols(cols);
-    sd.setCompressed(false);
-    sd.setParameters(new HashMap<String, String>());
-    sd.setSerdeInfo(new SerDeInfo());
-    sd.getSerdeInfo().setName(tbl.getTableName());
-    sd.getSerdeInfo().setParameters(new HashMap<String, String>());
-    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>());
-
-    client.createTable(tbl);
-
-    if (isThriftClient) {
-      // the createTable() above does not update the location in the 'tbl'
-      // object when the client is a thrift client and the code below relies
-      // on the location being present in the 'tbl' object - so get the table
-      // from the metastore
-      tbl = client.getTable(dbName, tblName);
-    }
-
-    ArrayList<FieldSchema> viewCols = new ArrayList<FieldSchema>(1);
-    viewCols.add(new FieldSchema("income", serdeConstants.INT_TYPE_NAME, ""));
-
-    ArrayList<FieldSchema> viewPartitionCols = new ArrayList<FieldSchema>(1);
-    viewPartitionCols.add(new FieldSchema("name", serdeConstants.STRING_TYPE_NAME, ""));
-
-    Table view = new Table();
-    view.setDbName(dbName);
-    view.setTableName(viewName);
-    view.setTableType(TableType.VIRTUAL_VIEW.name());
-    view.setPartitionKeys(viewPartitionCols);
-    view.setViewOriginalText("SELECT income, name FROM " + tblName);
-    view.setViewExpandedText("SELECT `" + tblName + "`.`income`, `" + tblName +
-        "`.`name` FROM `" + dbName + "`.`" + tblName + "`");
-    view.setRewriteEnabled(false);
-    StorageDescriptor viewSd = new StorageDescriptor();
-    view.setSd(viewSd);
-    viewSd.setCols(viewCols);
-    viewSd.setCompressed(false);
-    viewSd.setParameters(new HashMap<String, String>());
-    viewSd.setSerdeInfo(new SerDeInfo());
-    viewSd.getSerdeInfo().setParameters(new HashMap<String, String>());
-
-    client.createTable(view);
-
-    if (isThriftClient) {
-      // the createTable() above does not update the location in the 'tbl'
-      // object when the client is a thrift client and the code below relies
-      // on the location being present in the 'tbl' object - so get the table
-      // from the metastore
-      view = client.getTable(dbName, viewName);
-    }
-
-    List<String> vals = new ArrayList<String>(1);
-    vals.add("abc");
-
-    Partition part = new Partition();
-    part.setDbName(dbName);
-    part.setTableName(viewName);
-    part.setValues(vals);
-    part.setParameters(new HashMap<String, String>());
-
-    client.add_partition(part);
-
-    Partition part2 = client.getPartition(dbName, viewName, part.getValues());
-
-    part2.getParameters().put("a", "b");
-
-    client.alter_partition(dbName, viewName, part2, null);
-
-    Partition part3 = client.getPartition(dbName, viewName, part.getValues());
-    assertEquals("couldn't view alter partition", part3.getParameters().get(
-        "a"), "b");
-
-    client.dropTable(dbName, viewName);
-
-    client.dropTable(dbName, tblName);
-
-    client.dropDatabase(dbName);
-  }
-
-  public void testAlterPartition() throws Throwable {
-
-    try {
-      String dbName = "compdb";
-      String tblName = "comptbl";
-      List<String> vals = new ArrayList<String>(2);
-      vals.add("2008-07-01");
-      vals.add("14");
-
-      client.dropTable(dbName, tblName);
-      silentDropDatabase(dbName);
-      Database db = new Database();
-      db.setName(dbName);
-      db.setDescription("Alter Partition Test database");
-      client.createDatabase(db);
-
-      ArrayList<FieldSchema> cols = new ArrayList<FieldSchema>(2);
-      cols.add(new FieldSchema("name", serdeConstants.STRING_TYPE_NAME, ""));
-      cols.add(new FieldSchema("income", serdeConstants.INT_TYPE_NAME, ""));
-
-      Table tbl = new Table();
-      tbl.setDbName(dbName);
-      tbl.setTableName(tblName);
-      StorageDescriptor sd = new StorageDescriptor();
-      tbl.setSd(sd);
-      sd.setCols(cols);
-      sd.setCompressed(false);
-      sd.setNumBuckets(1);
-      sd.setParameters(new HashMap<String, String>());
-      sd.getParameters().put("test_param_1", "Use this for comments etc");
-      sd.setBucketCols(new ArrayList<String>(2));
-      sd.getBucketCols().add("name");
-      sd.setSerdeInfo(new SerDeInfo());
-      sd.getSerdeInfo().setName(tbl.getTableName());
-      sd.getSerdeInfo().setParameters(new HashMap<String, String>());
-      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>());
-
-      tbl.setPartitionKeys(new ArrayList<FieldSchema>(2));
-      tbl.getPartitionKeys().add(
-          new FieldSchema("ds", serdeConstants.STRING_TYPE_NAME, ""));
-      tbl.getPartitionKeys().add(
-          new FieldSchema("hr", serdeConstants.INT_TYPE_NAME, ""));
-
-      client.createTable(tbl);
-
-      if (isThriftClient) {
-        // the createTable() above does not update the location in the 'tbl'
-        // object when the client is a thrift client and the code below relies
-        // on the location being present in the 'tbl' object - so get the table
-        // from the metastore
-        tbl = client.getTable(dbName, tblName);
-      }
-
-      Partition part = new Partition();
-      part.setDbName(dbName);
-      part.setTableName(tblName);
-      part.setValues(vals);
-      part.setParameters(new HashMap<String, String>());
-      part.setSd(tbl.getSd());
-      part.getSd().setSerdeInfo(tbl.getSd().getSerdeInfo());
-      part.getSd().setLocation(tbl.getSd().getLocation() + "/part1");
-
-      client.add_partition(part);
-
-      Partition part2 = client.getPartition(dbName, tblName, part.getValues());
-
-      part2.getParameters().put("retention", "10");
-      part2.getSd().setNumBuckets(12);
-      part2.getSd().getSerdeInfo().getParameters().put("abc", "1");
-      client.alter_partition(dbName, tblName, part2, null);
-
-      Partition part3 = client.getPartition(dbName, tblName, part.getValues());
-      assertEquals("couldn't alter partition", part3.getParameters().get(
-          "retention"), "10");
-      assertEquals("couldn't alter partition", part3.getSd().getSerdeInfo()
-          .getParameters().get("abc"), "1");
-      assertEquals("couldn't alter partition", part3.getSd().getNumBuckets(),
-          12);
-
-      client.dropTable(dbName, tblName);
-
-      client.dropDatabase(dbName);
-    } catch (Exception e) {
-      System.err.println(StringUtils.stringifyException(e));
-      System.err.println("testPartition() failed.");
-      throw e;
-    }
-  }
-
-  public void testRenamePartition() throws Throwable {
-
-    try {
-      String dbName = "compdb1";
-      String tblName = "comptbl1";
-      List<String> vals = new ArrayList<String>(2);
-      vals.add("2011-07-11");
-      vals.add("8");
-      String part_path = "/ds=2011-07-11/hr=8";
-      List<String> tmp_vals = new ArrayList<String>(2);
-      tmp_vals.add("tmp_2011-07-11");
-      tmp_vals.add("-8");
-      String part2_path = "/ds=tmp_2011-07-11/hr=-8";
-
-      client.dropTable(dbName, tblName);
-      silentDropDatabase(dbName);
-      Database db = new Database();
-      db.setName(dbName);
-      db.setDescription("Rename Partition Test database");
-      client.createDatabase(db);
-
-      ArrayList<FieldSchema> cols = new ArrayList<FieldSchema>(2);
-      cols.add(new FieldSchema("name", serdeConstants.STRING_TYPE_NAME, ""));
-      cols.add(new FieldSchema("income", serdeConstants.INT_TYPE_NAME, ""));
-
-      Table tbl = new Table();
-      tbl.setDbName(dbName);
-      tbl.setTableName(tblName);
-      StorageDescriptor sd = new StorageDescriptor();
-      tbl.setSd(sd);
-      sd.setCols(cols);
-      sd.setCompressed(false);
-      sd.setNumBuckets(1);
-      sd.setParameters(new HashMap<String, String>());
-      sd.getParameters().put("test_param_1", "Use this for comments etc");
-      sd.setBucketCols(new ArrayList<String>(2));
-      sd.getBucketCols().add("name");
-      sd.setSerdeInfo(new SerDeInfo());
-      sd.getSerdeInfo().setName(tbl.getTableName());
-      sd.getSerdeInfo().setParameters(new HashMap<String, String>());
-      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>());
-
-      tbl.setPartitionKeys(new ArrayList<FieldSchema>(2));
-      tbl.getPartitionKeys().add(
-          new FieldSchema("ds", serdeConstants.STRING_TYPE_NAME, ""));
-      tbl.getPartitionKeys().add(
-          new FieldSchema("hr", serdeConstants.INT_TYPE_NAME, ""));
-
-      client.createTable(tbl);
-
-      if (isThriftClient) {
-        // the createTable() above does not update the location in the 'tbl'
-        // object when the client is a thrift client and the code below relies
-        // on the location being present in the 'tbl' object - so get the table
-        // from the metastore
-        tbl = client.getTable(dbName, tblName);
-      }
-
-      Partition part = new Partition();
-      part.setDbName(dbName);
-      part.setTableName(tblName);
-      part.setValues(vals);
-      part.setParameters(new HashMap<String, String>());
-      part.setSd(tbl.getSd().deepCopy());
-      part.getSd().setSerdeInfo(tbl.getSd().getSerdeInfo());
-      part.getSd().setLocation(tbl.getSd().getLocation() + "/part1");
-      part.getParameters().put("retention", "10");
-      part.getSd().setNumBuckets(12);
-      part.getSd().getSerdeInfo().getParameters().put("abc", "1");
-
-      client.add_partition(part);
-
-      part.setValues(tmp_vals);
-      client.renamePartition(dbName, tblName, vals, part);
-
-      boolean exceptionThrown = false;
-      try {
-        Partition p = client.getPartition(dbName, tblName, vals);
-      } catch(Exception e) {
-        assertEquals("partition should not have existed",
-            NoSuchObjectException.class, e.getClass());
-        exceptionThrown = true;
-      }
-      assertTrue("Expected NoSuchObjectException", exceptionThrown);
-
-      Partition part3 = client.getPartition(dbName, tblName, tmp_vals);
-      assertEquals("couldn't rename partition", part3.getParameters().get(
-          "retention"), "10");
-      assertEquals("couldn't rename partition", part3.getSd().getSerdeInfo()
-          .getParameters().get("abc"), "1");
-      assertEquals("couldn't rename partition", part3.getSd().getNumBuckets(),
-          12);
-      assertEquals("new partition sd matches", part3.getSd().getLocation(),
-          tbl.getSd().getLocation() + part2_path);
-
-      part.setValues(vals);
-      client.renamePartition(dbName, tblName, tmp_vals, part);
-
-      exceptionThrown = false;
-      try {
-        Partition p = client.getPartition(dbName, tblName, tmp_vals);
-      } catch(Exception e) {
-        assertEquals("partition should not have existed",
-            NoSuchObjectException.class, e.getClass());
-        exceptionThrown = true;
-      }
-      assertTrue("Expected NoSuchObjectException", exceptionThrown);
-
-      part3 = client.getPartition(dbName, tblName, vals);
-      assertEquals("couldn't rename partition", part3.getParameters().get(
-          "retention"), "10");
-      assertEquals("couldn't rename partition", part3.getSd().getSerdeInfo()
-          .getParameters().get("abc"), "1");
-      assertEquals("couldn't rename partition", part3.getSd().getNumBuckets(),
-          12);
-      assertEquals("new partition sd matches", part3.getSd().getLocation(),
-          tbl.getSd().getLocation() + part_path);
-
-      client.dropTable(dbName, tblName);
-
-      client.dropDatabase(dbName);
-    } catch (Exception e) {
-      System.err.println(StringUtils.stringifyException(e));
-      System.err.println("testRenamePartition() failed.");
-      throw e;
-    }
-  }
-
-  public void testDatabase() throws Throwable {
-    try {
-      // clear up any existing databases
-      silentDropDatabase(TEST_DB1_NAME);
-      silentDropDatabase(TEST_DB2_NAME);
-
-      Database db = new Database();
-      db.setName(TEST_DB1_NAME);
-      db.setOwnerName(SessionState.getUserFromAuthenticator());
-      db.setOwnerType(PrincipalType.USER);
-      client.createDatabase(db);
-
-      db = client.getDatabase(TEST_DB1_NAME);
-
-      assertEquals("name of returned db is different from that of inserted db",
-          TEST_DB1_NAME, db.getName());
-      assertEquals("location of the returned db is different from that of inserted db",
-          warehouse.getDatabasePath(db).toString(), db.getLocationUri());
-      assertEquals(db.getOwnerName(), SessionState.getUserFromAuthenticator());
-      assertEquals(db.getOwnerType(), PrincipalType.USER);
-      Database db2 = new Database();
-      db2.setName(TEST_DB2_NAME);
-      client.createDatabase(db2);
-
-      db2 = client.getDatabase(TEST_DB2_NAME);
-
-      assertEquals("name of returned db is different from that of inserted db",
-          TEST_DB2_NAME, db2.getName());
-      assertEquals("location of the returned db is different from that of inserted db",
-          warehouse.getDatabasePath(db2).toString(), db2.getLocationUri());
-
-      List<String> dbs = client.getDatabases(".*");
-
-      assertTrue("first database is not " + TEST_DB1_NAME, dbs.contains(TEST_DB1_NAME));
-      assertTrue("second database is not " + TEST_DB2_NAME, dbs.contains(TEST_DB2_NAME));
-
-      client.dropDatabase(TEST_DB1_NAME);
-      client.dropDatabase(TEST_DB2_NAME);
-      silentDropDatabase(TEST_DB1_NAME);
-      silentDropDatabase(TEST_DB2_NAME);
-    } catch (Throwable e) {
-      System.err.println(StringUtils.stringifyException(e));
-      System.err.println("testDatabase() failed.");
-      throw e;
-    }
-  }
-
-  public void testDatabaseLocationWithPermissionProblems() throws Exception {
-
-    // Note: The following test will fail if you are running this test as root. Setting
-    // permission to '0' on the database folder will not preclude root from being able
-    // to create the necessary files.
-
-    if (System.getProperty("user.name").equals("root")) {
-      System.err.println("Skipping test because you are running as root!");
-      return;
-    }
-
-    silentDropDatabase(TEST_DB1_NAME);
-
-    Database db = new Database();
-    db.setName(TEST_DB1_NAME);
-    String dbLocation =
-      HiveConf.getVar(hiveConf, HiveConf.ConfVars.METASTOREWAREHOUSE) + "/test/_testDB_create_";
-    FileSystem fs = FileSystem.get(new Path(dbLocation).toUri(), hiveConf);
-    fs.mkdirs(
-              new Path(HiveConf.getVar(hiveConf, HiveConf.ConfVars.METASTOREWAREHOUSE) + "/test"),
-              new FsPermission((short) 0));
-    db.setLocationUri(dbLocation);
-
-
-    boolean createFailed = false;
-    try {
-      client.createDatabase(db);
-    } catch (MetaException cantCreateDB) {
-      createFailed = true;
-    } finally {
-      // Cleanup
-      if (!createFailed) {
-        try {
-          client.dropDatabase(TEST_DB1_NAME);
-        } catch(Exception e) {
-          System.err.println("Failed to remove database in cleanup: " + e.getMessage());
-        }
-      }
-
-      fs.setPermission(new Path(HiveConf.getVar(hiveConf, HiveConf.ConfVars.METASTOREWAREHOUSE) + "/test"),
-                       new FsPermission((short) 755));
-      fs.delete(new Path(HiveConf.getVar(hiveConf, HiveConf.ConfVars.METASTOREWAREHOUSE) + "/test"), true);
-    }
-
-    assertTrue("Database creation succeeded even with permission problem", createFailed);
-  }
-
-  public void testDatabaseLocation() throws Throwable {
-    try {
-      // clear up any existing databases
-      silentDropDatabase(TEST_DB1_NAME);
-
-      Database db = new Database();
-      db.setName(TEST_DB1_NAME);
-      String dbLocation =
-          HiveConf.getVar(hiveConf, HiveConf.ConfVars.METASTOREWAREHOUSE) + "/_testDB_create_";
-      db.setLocationUri(dbLocation);
-      client.createDatabase(db);
-
-      db = client.getDatabase(TEST_DB1_NAME);
-
-      assertEquals("name of returned db is different from that of inserted db",
-          TEST_DB1_NAME, db.getName());
-      assertEquals("location of the returned db is different from that of inserted db",
-          warehouse.getDnsPath(new Path(dbLocation)).toString(), db.getLocationUri());
-
-      client.dropDatabase(TEST_DB1_NAME);
-      silentDropDatabase(TEST_DB1_NAME);
-
-      boolean objectNotExist = false;
-      try {
-        client.getDatabase(TEST_DB1_NAME);
-      } catch (NoSuchObjectException e) {
-        objectNotExist = true;
-      }
-      assertTrue("Database " + TEST_DB1_NAME + " exists ", objectNotExist);
-
-      db = new Database();
-      db.setName(TEST_DB1_NAME);
-      dbLocation =
-          HiveConf.getVar(hiveConf, HiveConf.ConfVars.METASTOREWAREHOUSE) + "/_testDB_file_";
-      FileSystem fs = FileSystem.get(new Path(dbLocation).toUri(), hiveConf);
-      fs.createNewFile(new Path(dbLocation));
-      fs.deleteOnExit(new Path(dbLocation));
-      db.setLocationUri(dbLocation);
-
-      boolean createFailed = false;
-      try {
-        client.createDatabase(db);
-      } catch (MetaException cantCreateDB) {
-        System.err.println(cantCreateDB.getMessage());
-        createFailed = true;
-      }
-      assertTrue("Database creation succeeded even location exists and is a file", createFailed);
-
-      objectNotExist = false;
-      try {
-        client.getDatabase(TEST_DB1_NAME);
-      } catch (NoSuchObjectException e) {
-        objectNotExist = true;
-      }
-      assertTrue("Database " + TEST_DB1_NAME + " exists when location is specified and is a file",
-          objectNotExist);
-
-    } catch (Throwable e) {
-      System.err.println(StringUtils.stringifyException(e));
-      System.err.println("testDatabaseLocation() failed.");
-      throw e;
-    }
-  }
-
-
-  public void testSimpleTypeApi() throws Exception {
-    try {
-      client.dropType(serdeConstants.INT_TYPE_NAME);
-
-      Type typ1 = new Type();
-      typ1.setName(serdeConstants.INT_TYPE_NAME);
-      boolean ret = client.createType(typ1);
-      assertTrue("Unable to create type", ret);
-
-      Type typ1_2 = client.getType(serdeConstants.INT_TYPE_NAME);
-      assertNotNull(typ1_2);
-      assertEquals(typ1.getName(), typ1_2.getName());
-
-      ret = client.dropType(serdeConstants.INT_TYPE_NAME);
-      assertTrue("unable to drop type integer", ret);
-
-      boolean exceptionThrown = false;
-      try {
-        client.getType(serdeConstants.INT_TYPE_NAME);
-      } catch (NoSuchObjectException e) {
-        exceptionThrown = true;
-      }
-      assertTrue("Expected NoSuchObjectException", exceptionThrown);
-    } catch (Exception e) {
-      System.err.println(StringUtils.stringifyException(e));
-      System.err.println("testSimpleTypeApi() failed.");
-      throw e;
-    }
-  }
-
-  // TODO:pc need to enhance this with complex fields and getType_all function
-  public void testComplexTypeApi() throws Exception {
-    try {
-      client.dropType("Person");
-
-      Type typ1 = new Type();
-      typ1.setName("Person");
-      typ1.setFields(new ArrayList<FieldSchema>(2));
-      typ1.getFields().add(
-          new FieldSchema("name", serdeConstants.STRING_TYPE_NAME, ""));
-      typ1.getFields().add(
-          new FieldSchema("income", serdeConstants.INT_TYPE_NAME, ""));
-      boolean ret = client.createType(typ1);
-      assertTrue("Unable to create type", ret);
-
-      Type typ1_2 = client.getType("Person");
-      assertNotNull("type Person not found", typ1_2);
-      assertEquals(typ1.getName(), typ1_2.getName());
-      assertEquals(typ1.getFields().size(), typ1_2.getFields().size());
-      assertEquals(typ1.getFields().get(0), typ1_2.getFields().get(0));
-      assertEquals(typ1.getFields().get(1), typ1_2.getFields().get(1));
-
-      client.dropType("Family");
-
-      Type fam = new Type();
-      fam.setName("Family");
-      fam.setFields(new ArrayList<FieldSchema>(2));
-      fam.getFields().add(
-          new FieldSchema("name", serdeConstants.STRING_TYPE_NAME, ""));
-      fam.getFields().add(
-          new FieldSchema("members",
-              ColumnType.getListType(typ1.getName()), ""));
-
-      ret = client.createType(fam);
-      assertTrue("Unable to create type " + fam.getName(), ret);
-
-      Type fam2 = client.getType("Family");
-      assertNotNull("type Person not found", fam2);
-      assertEquals(fam.getName(), fam2.getName());
-      assertEquals(fam.getFields().size(), fam2.getFields().size());
-      assertEquals(fam.getFields().get(0), fam2.getFields().get(0));
-      assertEquals(fam.getFields().get(1), fam2.getFields().get(1));
-
-      ret = client.dropType("Family");
-      assertTrue("unable to drop type Family", ret);
-
-      ret = client.dropType("Person");
-      assertTrue("unable to drop type Person", ret);
-
-      boolean exceptionThrown = false;
-      try {
-        client.getType("Person");
-      } catch (NoSuchObjectException e) {
-        exceptionThrown = true;
-      }
-      assertTrue("Expected NoSuchObjectException", exceptionThrown);
-    } catch (Exception e) {
-      System.err.println(StringUtils.stringifyException(e));
-      System.err.println("testComplexTypeApi() failed.");
-      throw e;
-    }
-  }
-
-  public void testSimpleTable() throws Exception {
-    try {
-      String dbName = "simpdb";
-      String tblName = "simptbl";
-      String tblName2 = "simptbl2";
-      String typeName = "Person";
-
-      client.dropTable(dbName, tblName);
-      silentDropDatabase(dbName);
-
-      Database db = new Database();
-      db.setName(dbName);
-      client.createDatabase(db);
-
-      client.dropType(typeName);
-      Type typ1 = new Type();
-      typ1.setName(typeName);
-      typ1.setFields(new ArrayList<FieldSchema>(2));
-      typ1.getFields().add(
-          new FieldSchema("name", serdeConstants.STRING_TYPE_NAME, ""));
-      typ1.getFields().add(
-          new FieldSchema("income", serdeConstants.INT_TYPE_NAME, ""));
-      client.createType(typ1);
-
-      Table tbl = new Table();
-      tbl.setDbName(dbName);
-      tbl.setTableName(tblName);
-      StorageDescriptor sd = new StorageDescriptor();
-      tbl.setSd(sd);
-      sd.setCols(typ1.getFields());
-      sd.setCompressed(false);
-      sd.setNumBuckets(1);
-      sd.setParameters(new HashMap<String, String>());
-      sd.getParameters().put("test_param_1", "Use this for comments etc");
-      sd.setBucketCols(new ArrayList<String>(2));
-      sd.getBucketCols().add("name");
-      sd.setSerdeInfo(new SerDeInfo());
-      sd.getSerdeInfo().setName(tbl.getTableName());
-      sd.getSerdeInfo().setParameters(new HashMap<String, String>());
-      sd.getSerdeInfo().getParameters().put(
-          org.apache.hadoop.hive.serde.serdeConstants.SERIALIZATION_FORMAT, "1");
-      sd.getSerdeInfo().setSerializationLib(
-          org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe.class.getName());
-      sd.setInputFormat(HiveInputFormat.class.getName());
-      sd.setInputFormat(HiveOutputFormat.class.getName());
-
-      tbl.setPartitionKeys(new ArrayList<FieldSchema>());
-
-      client.createTable(tbl);
-
-      if (isThriftClient) {
-        // the createTable() above does not update the location in the 'tbl'
-        // object when the client is a thrift client and the code below relies
-        // on the location being present in the 'tbl' object - so get the table
-        // from the metastore
-        tbl = client.getTable(dbName, tblName);
-      }
-
-      Table tbl2 = client.getTable(dbName, tblName);
-      assertNotNull(tbl2);
-      assertEquals(tbl2.getDbName(), dbName);
-      assertEquals(tbl2.getTableName(), tblName);
-      assertEquals(tbl2.getSd().getCols().size(), typ1.getFields().size());
-      assertEquals(tbl2.getSd().isCompressed(), false);
-      assertEquals(tbl2.getSd().getNumBuckets(), 1);
-      assertEquals(tbl2.getSd().getLocation(), tbl.getSd().getLocation());
-      assertNotNull(tbl2.getSd().getSerdeInfo());
-      sd.getSerdeInfo().setParameters(new HashMap<String, String>());
-      sd.getSerdeInfo().getParameters().put(
-          org.apache.hadoop.hive.serde.serdeConstants.SERIALIZATION_FORMAT, "1");
-
-      tbl2.setTableName(tblName2);
-      tbl2.setParameters(new HashMap<String, String>());
-      tbl2.getParameters().put("EXTERNAL", "TRUE");
-      tbl2.getSd().setLocation(tbl.getSd().getLocation() + "-2");
-
-      List<FieldSchema> fieldSchemas = client.getFields(dbName, tblName);
-      assertNotNull(fieldSchemas);
-      assertEquals(fieldSchemas.size(), tbl.getSd().getCols().size());
-      for (FieldSchema fs : tbl.getSd().getCols()) {
-        assertTrue(fieldSchemas.contains(fs));
-      }
-
-      List<FieldSchema> fieldSchemasFull = client.getSchema(dbName, tblName);
-      assertNotNull(fieldSchemasFull);
-      assertEquals(fieldSchemasFull.size(), tbl.getSd().getCols().size()
-          + tbl.getPartitionKeys().size());
-      for (FieldSchema fs : tbl.getSd().getCols()) {
-        assertTrue(fieldSchemasFull.contains(fs));
-      }
-      for (FieldSchema fs : tbl.getPartitionKeys()) {
-        assertTrue(fieldSchemasFull.contains(fs));
-      }
-
-      client.createTable(tbl2);
-      if (isThriftClient) {
-        tbl2 = client.getTable(tbl2.getDbName(), tbl2.getTableName());
-      }
-
-      Table tbl3 = client.getTable(dbName, tblName2);
-      assertNotNull(tbl3);
-      assertEquals(tbl3.getDbName(), dbName);
-      assertEquals(tbl3.getTableName(), tblName2);
-      assertEquals(tbl3.getSd().getCols().size(), typ1.getFields().size());
-      assertEquals(tbl3.getSd().isCompressed(), false);
-      assertEquals(tbl3.getSd().getNumBuckets(), 1);
-      assertEquals(tbl3.getSd().getLocation(), tbl2.getSd().getLocation());
-      assertEquals(tbl3.getParameters(), tbl2.getParameters());
-
-      fieldSchemas = client.getFields(dbName, tblName2);
-      assertNotNull(fieldSchemas);
-      assertEquals(fieldSchemas.size(), tbl2.getSd().getCols().size());
-      for (FieldSchema fs : tbl2.getSd().getCols()) {
-        assertTrue(fieldSchemas.contains(fs));
-      }
-
-      fieldSchemasFull = client.getSchema(dbName, tblName2);
-      assertNotNull(fieldSchemasFull);
-      assertEquals(fieldSchemasFull.size(), tbl2.getSd().getCols().size()
-          + tbl2.getPartitionKeys().size());
-      for (FieldSchema fs : tbl2.getSd().getCols()) {
-        assertTrue(fieldSchemasFull.contains(fs));
-      }
-      for (FieldSchema fs : tbl2.getPartitionKeys()) {
-        assertTrue(fieldSchemasFull.contains(fs));
-      }
-
-      assertEquals("Use this for comments etc", tbl2.getSd().getParameters()
-          .get("test_param_1"));
-      assertEquals("name", tbl2.getSd().getBucketCols().get(0));
-      assertTrue("Partition key list is not empty",
-          (tbl2.getPartitionKeys() == null)
-              || (tbl2.getPartitionKeys().size() == 0));
-
-      //test get_table_objects_by_name functionality
-      ArrayList<String> tableNames = new ArrayList<String>();
-      tableNames.add(tblName2);
-      tableNames.add(tblName);
-      tableNames.add(tblName2);
-      List<Table> foundTables = client.getTableObjectsByName(dbName, tableNames);
-
-      assertEquals(2, foundTables.size());
-      for (Table t: foundTables) {
-        if (t.getTableName().equals(tblName2)) {
-          assertEquals(t.getSd().getLocation(), tbl2.getSd().getLocation());
-        } else {
-          assertEquals(t.getTableName(), tblName);
-          assertEquals(t.getSd().getLocation(), tbl.getSd().getLocation());
-        }
-        assertEquals(t.getSd().getCols().size(), typ1.getFields().size());
-        assertEquals(t.getSd().isCompressed(), false);
-        assertEquals(foundTables.get(0).getSd().getNumBuckets(), 1);
-        assertNotNull(t.getSd().getSerdeInfo());
-        assertEquals(t.getDbName(), dbName);
-      }
-
-      tableNames.add(1, "table_that_doesnt_exist");
-      foundTables = client.getTableObjectsByName(dbName, tableNames);
-      assertEquals(foundTables.size(), 2);
-
-      InvalidOperationException ioe = null;
-      try {
-        foundTables = client.getTableObjectsByName(dbName, null);
-      } catch (InvalidOperationException e) {
-        ioe = e;
-      }
-      assertNotNull(ioe);
-      assertTrue("Table not found", ioe.getMessage().contains("null tables"));
-
-      UnknownDBException udbe = null;
-      try {
-        foundTables = client.getTableObjectsByName("db_that_doesnt_exist", tableNames);
-      } catch (UnknownDBException e) {
-        udbe = e;
-      }
-      assertNotNull(udbe);
-      assertTrue("DB not found", udbe.getMessage().contains("not find database db_that_doesnt_exist"));
-
-      udbe = null;
-      try {
-        foundTables = client.getTableObjectsByName("", tableNames);
-      } catch (UnknownDBException e) {
-        udbe = e;
-      }
-      assertNotNull(udbe);
-      assertTrue("DB not found", udbe.getMessage().contains("is null or empty"));
-
-      FileSystem fs = FileSystem.get((new Path(tbl.getSd().getLocation())).toUri(), hiveConf);
-      client.dropTable(dbName, tblName);
-      assertFalse(fs.exists(new Path(tbl.getSd().getLocation())));
-
-      client.dropTable(dbName, tblName2);
-      assertTrue(fs.exists(new Path(tbl2.getSd().getLocation())));
-
-      client.dropType(typeName);
-      client.dropDatabase(dbName);
-    } catch (Exception e) {
-      System.err.println(StringUtils.stringifyException(e));
-      System.err.println("testSimpleTable() failed.");
-      throw e;
-    }
-  }
-
-  // Tests that in the absence of stats for partitions, and/or absence of columns
-  // to get stats for, the metastore does not break. See HIVE-12083 for motivation.
-  public void testStatsFastTrivial() throws Throwable {
-    String dbName = "tstatsfast";
-    String tblName = "t1";
-    String tblOwner = "statstester";
-    String typeName = "Person";
-    int lastAccessed = 12083;
-
-    cleanUp(dbName,tblName,typeName);
-
-    List<List<String>> values = new ArrayList<List<String>>();
-    values.add(makeVals("2008-07-01 14:13:12", "14"));
-    values.add(makeVals("2008-07-01 14:13:12", "15"));
-    values.add(makeVals("2008-07-02 14:13:12", "15"));
-    values.add(makeVals("2008-07-03 14:13:12", "151"));
-
-    createMultiPartitionTableSchema(dbName, tblName, typeName, values);
-
-    List<String> emptyColNames = new ArrayList<String>();
-    List<String> emptyPartNames = new ArrayList<String>();
-
-    List<String> colNames = new ArrayList<String>();
-    colNames.add("name");
-    colNames.add("income");
-    List<String> partNames = client.listPartitionNames(dbName,tblName,(short)-1);
-
-    assertEquals(0,emptyColNames.size());
-    assertEquals(0,emptyPartNames.size());
-    assertEquals(2,colNames.size());
-    assertEquals(4,partNames.size());
-
-    // Test for both colNames and partNames being empty:
-    AggrStats aggrStatsEmpty = client.getAggrColStatsFor(dbName,tblName,emptyColNames,emptyPartNames);
-    assertNotNull(aggrStatsEmpty); // short-circuited on client-side, verifying that it's an empty object, not null
-    assertEquals(0,aggrStatsEmpty.getPartsFound());
-    assertNotNull(aggrStatsEmpty.getColStats());
-    assert(aggrStatsEmpty.getColStats().isEmpty());
-
-    // Test for only colNames being empty
-    AggrStats aggrStatsOnlyParts = client.getAggrColStatsFor(dbName,tblName,emptyColNames,partNames);
-    assertNotNull(aggrStatsOnlyParts); // short-circuited on client-side, verifying that it's an empty object, not null
-    assertEquals(0,aggrStatsOnlyParts.getPartsFound());
-    assertNotNull(aggrStatsOnlyParts.getColStats());
-    assert(aggrStatsOnlyParts.getColStats().isEmpty());
-
-    // Test for only partNames being empty
-    AggrStats aggrStatsOnlyCols = client.getAggrColStatsFor(dbName,tblName,colNames,emptyPartNames);
-    assertNotNull(aggrStatsOnlyCols); // short-circuited on client-side, verifying that it's an empty object, not null
-    assertEquals(0,aggrStatsOnlyCols.getPartsFound());
-    assertNotNull(aggrStatsOnlyCols.getColStats());
-    assert(aggrStatsOnlyCols.getColStats().isEmpty());
-
-    // Test for valid values for both.
-    AggrStats aggrStatsFull = client.getAggrColStatsFor(dbName,tblName,colNames,partNames);
-    assertNotNull(aggrStatsFull);
-    assertEquals(0,aggrStatsFull.getPartsFound()); // would still be empty, because no stats are actually populated.
-    assertNotNull(aggrStatsFull.getColStats());
-    assert(aggrStatsFull.getColStats().isEmpty());
-
-  }
-
-  public void testColumnStatistics() throws Throwable {
-
-    String dbName = "columnstatstestdb";
-    String tblName = "tbl";
-    String typeName = "Person";
-    String tblOwner = "testowner";
-    int lastAccessed = 6796;
-
-    try {
-      cleanUp(dbName, tblName, typeName);
-      Database db = new Database();
-      db.setName(dbName);
-      client.createDatabase(db);
-      createTableForTestFilter(dbName,tblName, tblOwner, lastAccessed, true);
-
-      // Create a ColumnStatistics Obj
-      String[] colName = new String[]{"income", "name"};
-      double lowValue = 50000.21;
-      double highValue = 1200000.4525;
-      long numNulls = 3;
-      long numDVs = 22;
-      double avgColLen = 50.30;
-      long maxColLen = 102;
-      String[] colType = new String[] {"double", "string"};
-      boolean isTblLevel = true;
-      String partName = null;
-      List<ColumnStatisticsObj> statsObjs = new ArrayList<ColumnStatisticsObj>();
-
-      ColumnStatisticsDesc statsDesc = new ColumnStatisticsDesc();
-      statsDesc.setDbName(dbName);
-      statsDesc.setTableName(tblName);
-      statsDesc.setIsTblLevel(isTblLevel);
-      statsDesc.setPartName(partName);
-
-      ColumnStatisticsObj statsObj = new ColumnStatisticsObj();
-      statsObj.setColName(colName[0]);
-      statsObj.setColType(colType[0]);
-
-      ColumnStatisticsData statsData = new ColumnStatisticsData();
-      DoubleColumnStatsData numericStats = new DoubleColumnStatsData();
-      statsData.setDoubleStats(numericStats);
-
-      statsData.getDoubleStats().setHighValue(highValue);
-      statsData.getDoubleStats().setLowValue(lowValue);
-      statsData.getDoubleStats().setNumDVs(numDVs);
-      statsData.getDoubleStats().setNumNulls(numNulls);
-
-      statsObj.setStatsData(statsData);
-      statsObjs.add(statsObj);
-
-      statsObj = new ColumnStatisticsObj();
-      statsObj.setColName(colName[1]);
-      statsObj.setColType(colType[1]);
-
-      statsData = new ColumnStatisticsData();
-      StringColumnStatsData stringStats = new StringColumnStatsData();
-      statsData.setStringStats(stringStats);
-      statsData.getStringStats().setAvgColLen(avgColLen);
-      statsData.getStringStats().setMaxColLen(maxColLen);
-      statsData.getStringStats().setNumDVs(numDVs);
-      statsData.getStringStats().setNumNulls(numNulls);
-
-      statsObj.setStatsData(statsData);
-      statsObjs.add(statsObj);
-
-      ColumnStatistics colStats = new ColumnStatistics();
-      colStats.setStatsDesc(statsDesc);
-      colStats.setStatsObj(statsObjs);
-
-      // write stats objs persistently
-      client.updateTableColumnStatistics(colStats);
-
-      // retrieve the stats obj that was just written
-      ColumnStatisticsObj colStats2 = client.getTableColumnStatistics(
-          dbName, tblName, Lists.newArrayList(colName[0])).get(0);
-
-     // compare stats obj to ensure what we get is what we wrote
-      assertNotNull(colStats2);
-      assertEquals(colStats2.getColName(), colName[0]);
-      assertEquals(colStats2.getStatsData().getDoubleStats().getLowValue(), lowValue);
-      assertEquals(colStats2.getStatsData().getDoubleStats().getHighValue(), highValue);
-      assertEquals(colStats2.getStatsData().getDoubleStats().getNumNulls(), numNulls);
-      assertEquals(colStats2.getStatsData().getDoubleStats().getNumDVs(), numDVs);
-
-      // test delete column stats; if no col name is passed all column stats associated with the
-      // table is deleted
-      boolean status = client.deleteTableColumnStatistics(dbName, tblName, null);
-      assertTrue(status);
-      // try to query stats for a column for which stats doesn't exist
-      assertTrue(client.getTableColumnStatistics(
-          dbName, tblName, Lists.newArrayList(colName[1])).isEmpty());
-
-      colStats.setStatsDesc(statsDesc);
-      colStats.setStatsObj(statsObjs);
-
-      // update table level column stats
-      client.updateTableColumnStatistics(colStats);
-
-      // query column stats for column whose stats were updated in the previous call
-      colStats2 = client.getTableColumnStatistics(
-          dbName, tblName, Lists.newArrayList(colName[0])).get(0);
-
-      // partition level column statistics test
-      // create a table with multiple partitions
-      cleanUp(dbName, tblName, typeName);
-
-      List<List<String>> values = new ArrayList<List<String>>();
-      values.add(makeVals("2008-07-01 14:13:12", "14"));
-      values.add(makeVals("2008-07-01 14:13:12", "15"));
-      values.add(makeVals("2008-07-02 14:13:12", "15"));
-      values.add(makeVals("2008-07-03 14:13:12", "151"));
-
-      createMultiPartitionTableSchema(dbName, tblName, typeName, values);
-
-      List<String> partitions = client.listPartitionNames(dbName, tblName, (short)-1);
-
-      partName = partitions.get(0);
-      isTblLevel = false;
-
-      // create a new columnstatistics desc to represent partition level column stats
-      statsDesc = new ColumnStatisticsDesc();
-      statsDesc.setDbName(dbName);
-      statsDesc.setTableName(tblName);
-      statsDesc.setPartName(partName);
-      statsDesc.setIsTblLevel(isTblLevel);
-
-      colStats = new ColumnStatistics();
-      colStats.setStatsDesc(statsDesc);
-      colStats.setStatsObj(statsObjs);
-
-     client.updatePartitionColumnStatistics(colStats);
-
-     colStats2 = client.getPartitionColumnStatistics(dbName, tblName,
-         Lists.newArrayList(partName), Lists.newArrayList(colName[1])).get(partName).get(0);
-
-     // compare stats obj to ensure what we get is what we wrote
-     assertNotNull(colStats2);
-     assertEquals(colStats.getStatsDesc().getPartName(), partName);
-     assertEquals(colStats2.getColName(), colName[1]);
-     assertEquals(colStats2.getStatsData().getStringStats().getMaxColLen(), maxColLen);
-     assertEquals(colStats2.getStatsData().getStringStats().getAvgColLen(), avgColLen);
-     assertEquals(colStats2.getStatsData().getStringStats().getNumNulls(), numNulls);
-     assertEquals(colStats2.getStatsData().getStringStats().getNumDVs(), numDVs);
-
-     // test stats deletion at partition level
-     client.deletePartitionColumnStatistics(dbName, tblName, partName, colName[1]);
-
-     colStats2 = client.getPartitionColumnStatistics(dbName, tblName,
-         Lists.newArrayList(partName), Lists.newArrayList(colName[0])).get(partName).get(0);
-
-     // test get stats on a column for which stats doesn't exist
-     assertTrue(client.getPartitionColumnStatistics(dbName, tblName,
-           Lists.newArrayList(partName), Lists.newArrayList(colName[1])).isEmpty());
-    } catch (Exception e) {
-      System.err.println(StringUtils.stringifyException(e));
-      System.err.println("testColumnStatistics() failed.");
-      throw e;
-    } finally {
-      cleanUp(dbName, tblName, typeName);
-    }
-  }
-
-  static class ClassNotFoundSerde extends LazySimpleSerDe {
-
-    public ClassNotFoundSerde() throws Exception {
-    }
-
-    @Override
-    public void initialize(Configuration job, Properties tbl) throws SerDeException {
-      super.initialize(job, tbl);
-      throw new NoClassDefFoundError();
-    }
-
-  }
-
-  public void testGetSchemaWithNoClassDefFoundError() throws Exception {
-    try {
-      String dbName = "testDb";
-      String tblName = "testTable";
-
-      client.dropTable(dbName, tblName);
-      silentDropDatabase(dbName);
-
-      Database db = new Database();
-      db.setName(dbName);
-      client.createDatabase(db);
-
-      Table tbl = new Table();
-      tbl.setDbName(dbName);
-      tbl.setTableName(tblName);
-
-      ArrayList<FieldSchema> cols = new ArrayList<FieldSchema>(1);
-      cols.add(new FieldSchema("name", serdeConstants.STRING_TYPE_NAME, ""));
-
-      StorageDescriptor sd = new StorageDescriptor();
-      tbl.setSd(sd);
-      sd.setCols(cols);
-      SerDeInfo serdeInfo = new SerDeInfo();
-      sd.setSerdeInfo(serdeInfo);
-      serdeInfo.setSerializationLib(ClassNotFoundSerde.class.getName());
-
-      client.createTable(tbl);
-
-      Boolean MetaExceptionCaught = false;
-      try {
-        client.getSchema(dbName, tblName);
-      } catch (MetaException me) {
-        MetaExceptionCaught = true;
-      }
-      assertTrue("MetaException is expected to be caught for throwing NoClassDefFoundError", MetaExceptionCaught);
-    } catch (Throwable e) {
-      System.err.println(StringUtils.stringifyException(e));
-      System.err.println("testGetSchemaWithNoClassDefFoundError() failed.");
-      throw e;
-    }
-  }
-
-  public void testAlterTable() throws Exception {
-    String dbName = "alterdb";
-    String invTblName = "alter-tbl";
-    String tblName = "altertbl";
-
-    try {
-      client.dropTable(dbName, tblName);
-      silentDropDatabase(dbName);
-
-      Database db = new Database();
-      db.setName(dbName);
-      client.createDatabase(db);
-
-      ArrayList<FieldSchema> invCols = new ArrayList<FieldSchema>(2);
-      invCols.add(new FieldSchema("n-ame", serdeConstants.STRING_TYPE_NAME, ""));
-      invCols.add(new FieldSchema("in.come", serdeConstants.INT_TYPE_NAME, ""));
-
-      Table tbl = new Table();
-      tbl.setDbName(dbName);
-      tbl.setTableName(invTblName);
-      StorageDescriptor sd = new StorageDescriptor();
-      tbl.setSd(sd);
-      sd.setCols(invCols);
-      sd.setCompressed(false);
-      sd.setNumBuckets(1);
-      sd.setParameters(new HashMap<String, String>());
-      sd.getParameters().put("test_param_1", "Use this for comments etc");
-      sd.setBucketCols(new ArrayList<String>(2));
-      sd.getBucketCols().add("name");
-      sd.setSerdeInfo(new SerDeInfo());
-      sd.getSerdeInfo().setName(tbl.getTableName());
-      sd.getSerdeInfo().setParameters(new HashMap<String, String>());
-      sd.getSerdeInfo().getParameters().put(
-          org.apache.hadoop.hive.serde.serdeConstants.SERIALIZATION_FORMAT, "1");
-      sd.getSerdeInfo().setSerializationLib(LazySimpleSerDe.class.getName());
-      sd.setInputFormat(HiveInputFormat.class.getName());
-      sd.setOutputFormat(HiveOutputFormat.class.getName());
-      
-      boolean failed = false;
-      try {
-        client.createTable(tbl);
-      } catch (InvalidObjectException ex) {
-        failed = true;
-      }
-      if (!failed) {
-        assertTrue("Able to create table with invalid name: " + invTblName,
-            false);
-      }
-
-      // create an invalid table which has wrong column type
-      ArrayList<FieldSchema> invColsInvType = new ArrayList<FieldSchema>(2);
-      invColsInvType.add(new FieldSchema("name", serdeConstants.STRING_TYPE_NAME, ""));
-      invColsInvType.add(new FieldSchema("income", "xyz", ""));
-      tbl.setTableName(tblName);
-      tbl.getSd().setCols(invColsInvType);
-      boolean failChecker = false;
-      try {
-        client.createTable(tbl);
-      } catch (InvalidObjectException ex) {
-        failChecker = true;
-      }
-      if (!failChecker) {
-        assertTrue("Able to create table with invalid column type: " + invTblName,
-            false);
-      }
-
-      ArrayList<FieldSchema> cols = new ArrayList<FieldSchema>(2);
-      cols.add(new FieldSchema("name", serdeConstants.STRING_TYPE_NAME, ""));
-      cols.add(new FieldSchema("income", serdeConstants.INT_TYPE_NAME, ""));
-
-      // create a valid table
-      tbl.setTableName(tblName);
-      tbl.getSd().setCols(cols);
-      client.createTable(tbl);
-
-      if (isThriftClient) {
-        tbl = client.getTable(tbl.getDbName(), tbl.getTableName());
-      }
-
-      // now try to invalid alter table
-      Table tbl2 = client.getTable(dbName, tblName);
-      failed = false;
-      try {
-        tbl2.setTableName(invTblName);
-        tbl2.getSd().setCols(invCols);
-        client.alter_table(dbName, tblName, tbl2);
-      } catch (InvalidOperationException ex) {
-        failed = true;
-      }
-      if (!failed) {
-        assertTrue("Able to rename table with invalid name: " + invTblName,
-            false);
-      }
-
-      //try an invalid alter table with partition key name
-      Table tbl_pk = client.getTable(tbl.getDbName(), tbl.getTableName());
-      List<FieldSchema> partitionKeys = tbl_pk.getPartitionKeys();
-      for (FieldSchema fs : partitionKeys) {
-        fs.setName("invalid_to_change_name");
-        fs.setComment("can_change_comment");
-      }
-      tbl_pk.setPartitionKeys(partitionKeys);
-      try {
-        client.alter_table(dbName, tblName, tbl_pk);
-      } catch (InvalidOperationException ex) {
-        failed = true;
-      }
-      assertTrue("Should not have succeeded in altering partition key name", failed);
-
-      //try a valid alter table partition key comment
-      failed = false;
-      tbl_pk = client.getTable(tbl.getDbName(), tbl.getTableName());
-      partitionKeys = tbl_pk.getPartitionKeys();
-      for (FieldSchema fs : partitionKeys) {
-        fs.setComment("can_change_comment");
-      }
-      tbl_pk.setPartitionKeys(partitionKeys);
-      try {
-        client.alter_table(dbName, tblName, tbl_pk);
-      } catch (InvalidOperationException ex) {
-        failed = true;
-      }
-      assertFalse("Should not have failed alter table partition comment", failed);
-      Table newT = client.getTable(tbl.getDbName(), tbl.getTableName());
-      assertEquals(partitionKeys, newT.getPartitionKeys());
-
-      // try a valid alter table
-      tbl2.setTableName(tblName + "_renamed");
-      tbl2.getSd().setCols(cols);
-      tbl2.getSd().setNumBuckets(32);
-      client.alter_table(dbName, tblName, tbl2);
-      Table tbl3 = client.getTable(dbName, tbl2.getTableName());
-      assertEquals("Alter table didn't succeed. Num buckets is different ",
-          tbl2.getSd().getNumBuckets(), tbl3.getSd().getNumBuckets());
-      // check that data has moved
-      FileSystem fs = FileSystem.get((new Path(tbl.getSd().getLocation())).toUri(), hiveConf);
-      assertFalse("old table location still exists", fs.exists(new Path(tbl
-          .getSd().getLocation())));
-      assertTrue("data did not move to new location", fs.exists(new Path(tbl3
-          .getSd().getLocation())));
-
-      if (!isThriftClient) {
-        assertEquals("alter table didn't move data correct location", tbl3
-            .getSd().getLocation(), tbl2.getSd().getLocation());
-      }
-
-      // alter table with invalid column type
-      tbl_pk.getSd().setCols(invColsInvType);
-      failed = false;
-      try {
-        client.alter_table(dbName, tbl2.getTableName(), tbl_pk);
-      } catch (InvalidOperationException ex) {
-        failed = true;
-      }
-      assertTrue("Should not have succeeded in altering column", failed);
-
-    } catch (Exception e) {
-      System.err.println(StringUtils.stringifyException(e));
-      System.err.println("testSimpleTable() failed.");
-      throw e;
-    } finally {
-      silentDropDatabase(dbName);
-    }
-  }
-
-  public void testComplexTable() throws Exception {
-
-    String dbName = "compdb";
-    String tblName = "comptbl";
-    String typeName = "Person";
-
-    try {
-      client.dropTable(dbName, tblName);
-      silentDropDatabase(dbName);
-      Database db = new Database();
-      db.setName(dbName);
-      client.createDatabase(db);
-
-      client.dropType(typeName);
-      Type typ1 = new Type();
-      typ1.setName(typeName);
-      typ1.setFields(new ArrayList<FieldSchema>(2));
-      typ1.getFields().add(
-          new FieldSchema("name", serdeConstants.STRING_TYPE_NAME, ""));
-      typ1.getFields().add(
-          new FieldSchema("income", serdeConstants.INT_TYPE_NAME, ""));
-      client.createType(typ1);
-
-      Table tbl = new Table();
-      tbl.setDbName(dbName);
-      tbl.setTableName(tblName);
-      StorageDescriptor sd = new StorageDescriptor();
-      tbl.setSd(sd);
-      sd.setCols(typ1.getFields());
-      sd.setCompressed(false);
-      sd.setNumBuckets(1);
-      sd.setParameters(new HashMap<String, String>());
-      sd.getParameters().put("test_param_1", "Use this for comments etc");
-      sd.setBucketCols(new ArrayList<String>(2));
-      sd.getBucketCols().add("name");
-      sd.setSerdeInfo(new SerDeInfo());
-      sd.getSerdeInfo().setName(tbl.getTableName());
-      sd.getSerdeInfo().setParameters(new HashMap<String, String>());
-      sd.getSerdeInfo().getParameters().put(
-          org.apache.hadoop.hive.serde.serdeConstants.SERIALIZATION_FORMAT, "9");
-      sd.getSerdeInfo().setSerializationLib(
-          org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe.class.getName());
-      sd.setInputFormat(HiveInputFormat.class.getName());
-      sd.setOutputFormat(HiveOutputFormat.class.getName());
-      
-      tbl.setPartitionKeys(new ArrayList<FieldSchema>(2));
-      tbl.getPartitionKeys().add(
-          new FieldSchema("ds",
-              org.apache.hadoop.hive.serde.serdeConstants.DATE_TYPE_NAME, ""));
-      tbl.getPartitionKeys().add(
-          new FieldSchema("hr",
-              org.apache.hadoop.hive.serde.serdeConstants.INT_TYPE_NAME, ""));
-
-      client.createTable(tbl);
-
-      Table tbl2 = client.getTable(dbName, tblName);
-      assertEquals(tbl2.getDbName(), dbName);
-      assertEquals(tbl2.getTableName(), tblName);
-      assertEquals(tbl2.getSd().getCols().size(), typ1.getFields().size());
-      assertFalse(tbl2.getSd().isCompressed());
-      assertFalse(tbl2.getSd().isStoredAsSubDirectories());
-      assertEquals(tbl2.getSd().getNumBuckets(), 1);
-
-      assertEquals("Use this for comments etc", tbl2.getSd().getParameters()
-          .get("test_param_1"));
-      assertEquals("name", tbl2.getSd().getBucketCols().get(0));
-
-      assertNotNull(tbl2.getPartitionKeys());
-      assertEquals(2, tbl2.getPartitionKeys().size());
-      assertEquals(serdeConstants.DATE_TYPE_NAME, tbl2.getPartitionKeys().get(0)
-          .getType());
-      assertEquals(serdeConstants.INT_TYPE_NAME, tbl2.getPartitionKeys().get(1)
-          .getType());
-      assertEquals("ds", tbl2.getPartitionKeys().get(0).getName());
-      assertEquals("hr", tbl2.getPartitionKeys().get(1).getName());
-
-      List<FieldSchema> fieldSchemas = client.getFields(dbName, tblName);
-      assertNotNull(fieldSchemas);
-      assertEquals(fieldSchemas.size(), tbl.getSd().getCols().size());
-      for (FieldSchema fs : tbl.getSd().getCols()) {
-        assertTrue(fieldSchemas.contains(fs));
-      }
-
-      List<FieldSchema> fieldSchemasFull = client.getSchema(dbName, tblName);
-      assertNotNull(fieldSchemasFull);
-      assertEquals(fieldSchemasFull.size(), tbl.getSd().getCols().size()
-          + tbl.getPartitionKeys().size());
-      for (FieldSchema fs : tbl.getSd().getCols()) {
-        assertTrue(fieldSchemasFull.contains(fs));
-      }
-      for (FieldSchema fs : tbl.getPartitionKeys()) {
-        assertTrue(fieldSchemasFull.contains(fs));
-      }
-    } catch (Exception e) {
-      System.err.println(StringUtils.stringifyException(e));
-      System.err.println("testComplexTable() failed.");
-      throw e;
-    } finally {
-      client.dropTable(dbName, tblName);
-      boolean ret = client.dropType(typeName);
-      assertTrue("Unable to drop type " + typeName, ret);
-      client.dropDatabase(dbName);
-    }
-  }
-
-  public void testTableDatabase() throws Exception {
-    String dbName = "testDb";
-    String tblName_1 = "testTbl_1";
-    String tblName_2 = "testTbl_2";
-
-    try {
-      silentDropDatabase(dbName);
-
-      Database db = new Database();
-      db.setName(dbName);
-      String dbLocation =
-          HiveConf.getVar(hiveConf, HiveConf.ConfVars.METASTOREWAREHOUSE) + "_testDB_table_create_";
-      db.setLocationUri(dbLocation);
-      client.createDatabase(db);
-      db = client.getDatabase(dbName);
-
-      Table tbl = new Table();
-      tbl.setDbName(dbName);
-      tbl.setTableName(tblName_1);
-
-      ArrayList<FieldSchema> cols = new ArrayList<FieldSchema>(2);
-      cols.add(new FieldSchema("name", serdeConstants.STRING_TYPE_NAME, ""));
-      cols.add(new FieldSchema("income", serdeConstants.INT_TYPE_NAME, ""));
-
-      StorageDescriptor sd = new StorageDescriptor();
-      sd.setSerdeInfo(new SerDeInfo());
-      sd.getSerdeInfo().setName(tbl.getTableName());
-      sd.getSerdeInfo().setParameters(new HashMap<String, String>());
-      sd.setParameters(new HashMap<String, String>());
-      sd.getSerdeInfo().getParameters().put(
-          org.apache.hadoop.hive.serde.serdeConstants.SERIALIZATION_FORMAT, "9");
-      sd.getSerdeInfo().setSerializationLib(
-          org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe.class.getName());
-      sd.setInputFormat(HiveInputFormat.class.getName());
-      sd.setOutputFormat(HiveOutputFormat.class.getName());
-
-      tbl.setSd(sd);
-      tbl.getSd().setCols(cols);
-      client.createTable(tbl);
-      tbl = client.getTable(dbName, tblName_1);
-
-      Path path = new Path(tbl.getSd().getLocation());
-      System.err.println("Table's location " + path + ", Database's location " + db.getLocationUri());
-      assertEquals("Table location is not a subset of the database location",
-          path.getParent().toString(), db.getLocationUri());
-
-    } catch (Exception e) {
-      System.err.println(StringUtils.stringifyException(e));
-      System.err.println("testTableDatabase() failed.");
-      throw e;
-    } finally {
-      silentDropDatabase(dbName);
-    }
-  }
-
-
-  public void testGetConfigValue() {
-
-    String val = "value";
-
-    if (!isThriftClient) {
-      try {
-        assertEquals(client.getConfigValue("hive.key1", val), "value1");
-        assertEquals(client.getConfigValue("hive.key2", val), "http://www.example.com");
-        assertEquals(client.getConfigValue("hive.key3", val), "");
-        assertEquals(client.getConfigValue("hive.key4", val), "0");
-        assertEquals(client.getConfigValue("hive.key5", val), val);
-        assertEquals(client.getConfigValue(null, val), val);
-      } catch (ConfigValSecurityException e) {
-        e.printStackTrace();
-        assert (false);
-      } catch (TException e) {
-        e.printStackTrace();
-        assert (false);
-      }
-    }
-
-    boolean threwException = false;
-    try {
-      // Attempting to get the password should throw an exception
-      client.getConfigValue("javax.jdo.option.ConnectionPassword", "password");
-    } catch (ConfigValSecurityException e) {
-      threwException = true;
-    } catch (TException e) {
-      e.printStackTrace();
-      assert (false);
-    }
-    assert (threwException);
-  }
-
-  private static void adjust(HiveMetaStoreClient client, Partition part,
-      String dbName, String tblName)
-  throws NoSuchObjectException, MetaException, TException {
-    Partition part_get = client.getPartition(dbName, tblName, part.getValues());
-    part.setCreateTime(part_get.getCreateTime());
-    part.putToParameters(org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.DDL_TIME, Long.toString(part_get.getCreateTime()));
-  }
-
-  private static void silentDropDatabase(String dbName) throws MetaException, TException {
-    try {
-      for (String tableName : client.getTables(dbName, "*")) {
-        client.dropTable(dbName, tableName);
-      }
-      client.dropDatabase(dbName);
-    } catch (NoSuchObjectException e) {
-    } catch (InvalidOperationException e) {
-    }
-  }
-
-  /**
-   * Tests for list partition by filter functionality.
-   * @throws Exception
-   */
-
-  public void testPartitionFilter() throws Exception {
-    String dbName = "filterdb";
-    String tblName = "filtertbl";
-
-    silentDropDatabase(dbName);
-
-    Database db = new Database();
-    db.setName(dbName);
-    client.createDatabase(db);
-
-    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, ""));
-
-    ArrayList<FieldSchema> partCols = new ArrayList<FieldSchema>(3);
-    partCols.add(new FieldSchema("p1", serdeConstants.STRING_TYPE_NAME, ""));
-    partCols.add(new FieldSchema("p2", serdeConstants.STRING_TYPE_NAME, ""));
-    partCols.add(new FieldSchema("p3", serdeConstants.INT_TYPE_NAME, ""));
-
-    Table tbl = new Table();
-    tbl.setDbName(dbName);
-    tbl.setTableName(tblName);
-    StorageDescriptor sd = new StorageDescriptor();
-    tbl.setSd(sd);
-    sd.setCols(cols);
-    sd.setCompressed(false);
-    sd.setNumBuckets(1);
-    sd.setParameters(new HashMap<String, String>());
-    sd.setBucketCols(new ArrayList<String>());
-    sd.setSerdeInfo(new SerDeInfo());
-  

<TRUNCATED>

[7/7] hive git commit: HIVE-17982 Move metastore specific itests. This closes #279. (Alan Gates, reviewed by Peter Vary)

Posted by ga...@apache.org.
HIVE-17982 Move metastore specific itests.  This closes #279.  (Alan Gates, reviewed by Peter Vary)


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

Branch: refs/heads/master
Commit: d9801d9c6c406d5871147b80bc2e0359c3dbd085
Parents: fde503d
Author: Alan Gates <ga...@hortonworks.com>
Authored: Tue Jan 16 13:36:45 2018 -0800
Committer: Alan Gates <ga...@hortonworks.com>
Committed: Tue Jan 16 13:36:45 2018 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hive/metastore/FakeDerby.java |  424 ---
 .../hive/metastore/TestAcidTableSetup.java      |  245 ++
 .../hadoop/hive/metastore/TestAdminUser.java    |   45 -
 .../metastore/TestEmbeddedHiveMetaStore.java    |   54 -
 .../hadoop/hive/metastore/TestFilterHooks.java  |  280 --
 .../hive/metastore/TestHiveMetaStore.java       | 3514 ------------------
 .../hive/metastore/TestHiveMetaStoreTxns.java   |  270 --
 ...TestHiveMetaStoreWithEnvironmentContext.java |  219 --
 .../hive/metastore/TestMarkPartition.java       |  108 -
 .../hive/metastore/TestMarkPartitionRemote.java |   32 -
 .../TestMetaStoreEndFunctionListener.java       |  142 -
 .../metastore/TestMetaStoreEventListener.java   |  526 ---
 .../TestMetaStoreEventListenerOnlyOnCommit.java |  103 -
 .../metastore/TestMetaStoreInitListener.java    |   67 -
 .../metastore/TestMetaStoreListenersError.java  |   85 -
 .../metastore/TestObjectStoreInitRetry.java     |  127 -
 .../TestPartitionNameWhitelistValidation.java   |  123 -
 .../hive/metastore/TestRemoteHiveMetaStore.java |   60 -
 .../TestRemoteHiveMetaStoreIpAddress.java       |   80 -
 .../TestRemoteUGIHiveMetaStoreIpAddress.java    |   28 -
 .../hive/metastore/TestRetryingHMSHandler.java  |  123 -
 .../metastore/TestSetUGIOnBothClientServer.java |   31 -
 .../hive/metastore/TestSetUGIOnOnlyClient.java  |   31 -
 .../hive/metastore/TestSetUGIOnOnlyServer.java  |   31 -
 standalone-metastore/pom.xml                    |    2 +
 .../metastore/client/builder/IndexBuilder.java  |    5 +
 .../client/builder/PartitionBuilder.java        |    3 +-
 .../metastore/client/builder/TableBuilder.java  |   12 +-
 .../hive/metastore/conf/MetastoreConf.java      |    2 +-
 .../apache/hadoop/hive/metastore/FakeDerby.java |  404 ++
 .../hive/metastore/MetaStoreTestUtils.java      |   32 +-
 .../hadoop/hive/metastore/TestAdminUser.java    |   46 +
 .../metastore/TestEmbeddedHiveMetaStore.java    |   48 +
 .../hadoop/hive/metastore/TestFilterHooks.java  |  302 ++
 .../hive/metastore/TestHiveMetaStore.java       | 3071 +++++++++++++++
 .../hive/metastore/TestHiveMetaStoreTxns.java   |  264 ++
 ...TestHiveMetaStoreWithEnvironmentContext.java |  187 +
 .../hive/metastore/TestMarkPartition.java       |  117 +
 .../hive/metastore/TestMarkPartitionRemote.java |   36 +
 .../TestMetaStoreEndFunctionListener.java       |  145 +
 .../metastore/TestMetaStoreEventListener.java   |  557 +++
 .../TestMetaStoreEventListenerOnlyOnCommit.java |  123 +
 .../TestMetaStoreEventListenerWithOldConf.java  |  178 +
 .../metastore/TestMetaStoreInitListener.java    |   54 +
 .../metastore/TestMetaStoreListenersError.java  |   94 +
 .../hadoop/hive/metastore/TestObjectStore.java  |    9 +-
 .../metastore/TestObjectStoreInitRetry.java     |  132 +
 .../hadoop/hive/metastore/TestOldSchema.java    |    3 +-
 .../TestPartitionNameWhitelistValidation.java   |  122 +
 .../hive/metastore/TestRemoteHiveMetaStore.java |   62 +
 .../TestRemoteHiveMetaStoreIpAddress.java       |   64 +
 .../TestRemoteUGIHiveMetaStoreIpAddress.java    |   28 +
 .../hive/metastore/TestRetryingHMSHandler.java  |   81 +
 .../metastore/TestSetUGIOnBothClientServer.java |   31 +
 .../hive/metastore/TestSetUGIOnOnlyClient.java  |   32 +
 .../hive/metastore/TestSetUGIOnOnlyServer.java  |   32 +
 .../hive/metastore/cache/TestCachedStore.java   |    4 +-
 57 files changed, 6505 insertions(+), 6525 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/d9801d9c/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/FakeDerby.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/FakeDerby.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/FakeDerby.java
deleted file mode 100644
index 51be504..0000000
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/FakeDerby.java
+++ /dev/null
@@ -1,424 +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.lang.Exception;
-import java.lang.Override;
-import java.lang.RuntimeException;
-import java.lang.StackTraceElement;
-import java.sql.Array;
-import java.sql.Blob;
-import java.sql.CallableStatement;
-import java.sql.Clob;
-import java.sql.Connection;
-import java.sql.DatabaseMetaData;
-import java.sql.DriverManager;
-import java.sql.DriverPropertyInfo;
-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;
-
-import javax.jdo.JDOCanRetryException;
-
-import junit.framework.TestCase;
-import org.junit.Test;
-
-import org.apache.derby.jdbc.EmbeddedDriver;
-
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
-import org.apache.hadoop.hive.metastore.ObjectStore;
-
-import org.apache.hadoop.hive.metastore.TestObjectStoreInitRetry;
-
-
-/**
- * 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/d9801d9c/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestAcidTableSetup.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestAcidTableSetup.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestAcidTableSetup.java
new file mode 100644
index 0000000..62bd94a
--- /dev/null
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestAcidTableSetup.java
@@ -0,0 +1,245 @@
+/*
+ * 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.List;
+import java.util.Map;
+
+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.junit.Before;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.Type;
+import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
+import org.apache.thrift.TException;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class TestAcidTableSetup {
+  private static final Logger LOG = LoggerFactory.getLogger(TestHiveMetaStore.class);
+  protected static HiveMetaStoreClient client;
+  protected static Configuration conf;
+
+  @Before
+  public void setUp() throws Exception {
+    conf = MetastoreConf.newMetastoreConf();
+
+    MetastoreConf.setClass(conf, ConfVars.EXPRESSION_PROXY_CLASS,
+        DefaultPartitionExpressionProxy.class, PartitionExpressionProxy.class);
+    client = new HiveMetaStoreClient(conf);
+  }
+
+  @Test
+  public void testTransactionalValidation() throws Throwable {
+    String dbName = "acidDb";
+    silentDropDatabase(dbName);
+    Database db = new Database();
+    db.setName(dbName);
+    client.createDatabase(db);
+    String tblName = "acidTable";
+    Map<String, String> fields = new HashMap<>();
+    fields.put("name", ColumnType.STRING_TYPE_NAME);
+    fields.put("income", ColumnType.INT_TYPE_NAME);
+
+    Type type = createType("Person1", fields);
+
+    Map<String, String> params = new HashMap<>();
+    params.put("transactional", "");
+
+    /// CREATE TABLE scenarios
+
+    // Fail - No "transactional" property is specified
+    try {
+      Table t = new TableBuilder()
+          .setDbName(dbName)
+          .setTableName(tblName)
+          .setTableParams(params)
+          .setCols(type.getFields())
+          .build();
+      client.createTable(t);
+      fail("Expected exception");
+    } catch (MetaException e) {
+      assertEquals("'transactional' property of TBLPROPERTIES may only have value 'true': acidDb.acidTable",
+          e.getMessage());
+    }
+
+    // Fail - "transactional" property is set to an invalid value
+    try {
+      params.clear();
+      params.put("transactional", "foobar");
+      Table t = new TableBuilder()
+          .setDbName(dbName)
+          .setTableName(tblName)
+          .setTableParams(params)
+          .setCols(type.getFields())
+          .build();
+      client.createTable(t);
+      fail("Expected exception");
+    } catch (MetaException e) {
+      assertEquals("'transactional' property of TBLPROPERTIES may only have value 'true': acidDb.acidTable",
+          e.getMessage());
+    }
+
+    // Fail - "transactional" is set to true, but the table is not bucketed
+    try {
+      params.clear();
+      params.put("transactional", "true");
+      Table t = new TableBuilder()
+          .setDbName(dbName)
+          .setTableName(tblName)
+          .setTableParams(params)
+          .setCols(type.getFields())
+          .build();
+      client.createTable(t);
+      fail("Expected exception");
+    } catch (MetaException e) {
+      assertEquals("The table must be stored using an ACID compliant format (such as ORC): acidDb.acidTable",
+          e.getMessage());
+    }
+
+    List<String> bucketCols = new ArrayList<>();
+    bucketCols.add("income");
+    // Fail - "transactional" is set to true, and the table is bucketed, but doesn't use ORC
+    try {
+      params.clear();
+      params.put("transactional", "true");
+      Table t = new TableBuilder()
+          .setDbName(dbName)
+          .setTableName(tblName)
+          .setTableParams(params)
+          .setCols(type.getFields())
+          .setBucketCols(bucketCols)
+          .build();
+      client.createTable(t);
+      fail("Expected exception");
+    } catch (MetaException e) {
+      assertEquals("The table must be stored using an ACID compliant format (such as ORC): acidDb.acidTable",
+          e.getMessage());
+    }
+
+    // Succeed - "transactional" is set to true, and the table is bucketed, and uses ORC
+    params.clear();
+    params.put("transactional", "true");
+    Table t = new TableBuilder()
+        .setDbName(dbName)
+        .setTableName(tblName)
+        .setTableParams(params)
+        .setCols(type.getFields())
+        .setBucketCols(bucketCols)
+        .setInputFormat("org.apache.hadoop.hive.ql.io.orc.OrcInputFormat")
+        .setOutputFormat("org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat")
+        .build();
+    client.createTable(t);
+    assertTrue("CREATE TABLE should succeed",
+        "true".equals(t.getParameters().get(hive_metastoreConstants.TABLE_IS_TRANSACTIONAL)));
+
+    /// ALTER TABLE scenarios
+
+    // Fail - trying to set "transactional" to "false" is not allowed
+    try {
+      params.clear();
+      params.put("transactional", "false");
+      t = new Table();
+      t.setParameters(params);
+      t.setDbName(dbName);
+      t.setTableName(tblName);
+      client.alter_table(dbName, tblName, t);
+      fail("Expected exception");
+    } catch (MetaException e) {
+      assertEquals("TBLPROPERTIES with 'transactional'='true' cannot be unset: acidDb.acidTable", e.getMessage());
+    }
+
+    // Fail - trying to set "transactional" to "true" but doesn't satisfy bucketing and Input/OutputFormat requirement
+    try {
+      tblName += "1";
+      params.clear();
+      t = new TableBuilder()
+          .setDbName(dbName)
+          .setTableName(tblName)
+          .setCols(type.getFields())
+          .setInputFormat("org.apache.hadoop.mapred.FileInputFormat")
+          .build();
+      client.createTable(t);
+      params.put("transactional", "true");
+      t.setParameters(params);
+      client.alter_table(dbName, tblName, t);
+      fail("Expected exception");
+    } catch (MetaException e) {
+      assertEquals("The table must be stored using an ACID compliant format (such as ORC): acidDb.acidTable1",
+          e.getMessage());
+    }
+
+    // Succeed - trying to set "transactional" to "true", and satisfies bucketing and Input/OutputFormat requirement
+    tblName += "2";
+    params.clear();
+    t = new TableBuilder()
+        .setDbName(dbName)
+        .setTableName(tblName)
+        .setCols(type.getFields())
+        .setNumBuckets(1)
+        .setBucketCols(bucketCols)
+        .setInputFormat("org.apache.hadoop.hive.ql.io.orc.OrcInputFormat")
+        .setOutputFormat("org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat")
+        .build();
+    client.createTable(t);
+    params.put("transactional", "true");
+    t.setParameters(params);
+    client.alter_table(dbName, tblName, t);
+    assertTrue("ALTER TABLE should succeed",
+        "true".equals(t.getParameters().get(hive_metastoreConstants.TABLE_IS_TRANSACTIONAL)));
+  }
+
+  private static void silentDropDatabase(String dbName) throws TException {
+    try {
+      for (String tableName : client.getTables(dbName, "*")) {
+        client.dropTable(dbName, tableName);
+      }
+      client.dropDatabase(dbName);
+    } catch (NoSuchObjectException|InvalidOperationException e) {
+      // NOP
+    }
+  }
+
+  private Type createType(String typeName, Map<String, String> fields) throws Throwable {
+    Type typ1 = new Type();
+    typ1.setName(typeName);
+    typ1.setFields(new ArrayList<>(fields.size()));
+    for(String fieldName : fields.keySet()) {
+      typ1.getFields().add(
+          new FieldSchema(fieldName, fields.get(fieldName), ""));
+    }
+    client.createType(typ1);
+    return typ1;
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/d9801d9c/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestAdminUser.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestAdminUser.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestAdminUser.java
deleted file mode 100644
index e9dabee..0000000
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestAdminUser.java
+++ /dev/null
@@ -1,45 +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.io.IOException;
-
-import junit.framework.TestCase;
-
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
-import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
-import org.apache.hadoop.hive.metastore.api.PrincipalType;
-import org.apache.hadoop.hive.metastore.api.Role;
-import org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory;
-
-public class TestAdminUser extends TestCase{
-
- public void testCreateAdminNAddUser() throws IOException, Throwable {
-   HiveConf conf = new HiveConf();
-   conf.setVar(ConfVars.USERS_IN_ADMIN_ROLE, "adminuser");
-   conf.setVar(ConfVars.HIVE_AUTHORIZATION_MANAGER,SQLStdHiveAuthorizerFactory.class.getName());
-   RawStore rawStore = new HMSHandler("testcreateroot", conf).getMS();
-   Role adminRole = rawStore.getRole(HiveMetaStore.ADMIN);
-   assertTrue(adminRole.getOwnerName().equals(HiveMetaStore.ADMIN));
-   assertEquals(rawStore.listPrincipalGlobalGrants(HiveMetaStore.ADMIN, PrincipalType.ROLE)
-    .get(0).getGrantInfo().getPrivilege(),"All");
-   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/d9801d9c/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestEmbeddedHiveMetaStore.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestEmbeddedHiveMetaStore.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestEmbeddedHiveMetaStore.java
deleted file mode 100644
index 462768d..0000000
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestEmbeddedHiveMetaStore.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hive.metastore;
-
-import org.apache.hadoop.util.StringUtils;
-
-public class TestEmbeddedHiveMetaStore extends TestHiveMetaStore {
-
-  @Override
-  protected void setUp() throws Exception {
-    super.setUp();
-    warehouse = new Warehouse(hiveConf);
-    client = createClient();
-  }
-
-  @Override
-  protected void tearDown() throws Exception {
-    try {
-      super.tearDown();
-      client.close();
-    } catch (Throwable e) {
-      System.err.println("Unable to close metastore");
-      System.err.println(StringUtils.stringifyException(e));
-      throw new Exception(e);
-    }
-  }
-
-  @Override
-  protected HiveMetaStoreClient createClient() throws Exception {
-    try {
-      return new HiveMetaStoreClient(hiveConf);
-    } 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/d9801d9c/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestFilterHooks.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestFilterHooks.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestFilterHooks.java
deleted file mode 100644
index 2382582..0000000
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestFilterHooks.java
+++ /dev/null
@@ -1,280 +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 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.UtilsForTest;
-import org.apache.hadoop.hive.cli.CliSessionState;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
-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.ql.DriverFactory;
-import org.apache.hadoop.hive.ql.IDriver;
-import org.apache.hadoop.hive.ql.session.SessionState;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.collect.Lists;
-
-public class TestFilterHooks {
-  private static final Logger LOG = LoggerFactory.getLogger(TestFilterHooks.class);
-
-  public static class DummyMetaStoreFilterHookImpl extends DefaultMetaStoreFilterHookImpl {
-    public 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<String>();
-      }
-      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<String>();
-      }
-      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<Table>();
-      }
-      return super.filterTables(tableList);
-    }
-
-    @Override
-    public List<Partition> filterPartitions(List<Partition> partitionList) throws MetaException {
-      if (blockResults) {
-        return new ArrayList<Partition>();
-      }
-      return super.filterPartitions(partitionList);
-    }
-
-    @Override
-    public List<PartitionSpec> filterPartitionSpecs(
-        List<PartitionSpec> partitionSpecList) throws MetaException {
-      if (blockResults) {
-        return new ArrayList<PartitionSpec>();
-      }
-      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<String>();
-      }
-      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<String>();
-      }
-      return super.filterIndexNames(dbName, tblName, indexList);
-    }
-
-    @Override
-    public List<Index> filterIndexes(List<Index> indexeList) throws MetaException {
-      if (blockResults) {
-        return new ArrayList<Index>();
-      }
-      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 HiveConf hiveConf;
-  private static HiveMetaStoreClient msc;
-  private static IDriver driver;
-
-  @BeforeClass
-  public static void setUp() throws Exception {
-    DummyMetaStoreFilterHookImpl.blockResults = false;
-
-    hiveConf = new HiveConf(TestFilterHooks.class);
-    hiveConf.setIntVar(HiveConf.ConfVars.METASTORETHRIFTCONNECTIONRETRIES, 3);
-    hiveConf.set(HiveConf.ConfVars.PREEXECHOOKS.varname, "");
-    hiveConf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, "");
-    hiveConf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false");
-    hiveConf.setVar(ConfVars.METASTORE_FILTER_HOOK, DummyMetaStoreFilterHookImpl.class.getName());
-    UtilsForTest.setNewDerbyDbLocation(hiveConf, TestFilterHooks.class.getSimpleName());
-    int port = MetaStoreTestUtils.startMetaStoreWithRetry(hiveConf);
-    hiveConf.setVar(HiveConf.ConfVars.METASTOREURIS, "thrift://localhost:" + port);
-
-    SessionState.start(new CliSessionState(hiveConf));
-    msc = new HiveMetaStoreClient(hiveConf);
-    driver = DriverFactory.newDriver(hiveConf);
-
-    driver.run("drop database if exists " + DBNAME1  + " cascade");
-    driver.run("drop database if exists " + DBNAME2  + " cascade");
-    driver.run("create database " + DBNAME1);
-    driver.run("create database " + DBNAME2);
-    driver.run("use " + DBNAME1);
-    driver.run("create table " + DBNAME1 + "." + TAB1 + " (id int, name string)");
-    driver.run("create table " + TAB2 + " (id int) partitioned by (name string)");
-    driver.run("ALTER TABLE " + TAB2 + " ADD PARTITION (name='value1')");
-    driver.run("ALTER TABLE " + TAB2 + " ADD PARTITION (name='value2')");
-    driver.run("CREATE INDEX " + INDEX1 + " on table " + TAB1 + "(id) AS 'COMPACT' WITH DEFERRED REBUILD");
-  }
-
-  @AfterClass
-  public static void tearDown() throws Exception {
-    DummyMetaStoreFilterHookImpl.blockResults = false;
-    driver.run("drop database if exists " + DBNAME1  + " cascade");
-    driver.run("drop database if exists " + DBNAME2  + " cascade");
-    driver.close();
-    driver.destroy();
-    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
-    }
-  }
-
-}


[5/7] hive git commit: HIVE-17982 Move metastore specific itests. This closes #279. (Alan Gates, reviewed by Peter Vary)

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/d9801d9c/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreTxns.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreTxns.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreTxns.java
deleted file mode 100644
index a19cc86..0000000
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreTxns.java
+++ /dev/null
@@ -1,270 +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 junit.framework.Assert;
-
-import org.apache.hadoop.hive.common.ValidTxnList;
-import org.apache.hadoop.hive.common.ValidReadTxnList;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.api.DataOperationType;
-import org.apache.hadoop.hive.metastore.api.HeartbeatTxnRangeResponse;
-import org.apache.hadoop.hive.metastore.api.LockResponse;
-import org.apache.hadoop.hive.metastore.api.LockState;
-import org.apache.hadoop.hive.metastore.txn.TxnDbUtil;
-import org.apache.log4j.Level;
-import org.apache.log4j.LogManager;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.util.List;
-
-/**
- * Unit tests for {@link org.apache.hadoop.hive.metastore.HiveMetaStoreClient}.  For now this just has
- * transaction and locking tests.  The goal here is not to test all
- * functionality possible through the interface, as all permutations of DB
- * operations should be tested in the appropriate DB handler classes.  The
- * goal is to test that we can properly pass the messages through the thrift
- * service.
- *
- * This is in the ql directory rather than the metastore directory because it
- * required the hive-exec jar, and hive-exec jar already depends on
- * hive-metastore jar, thus I can't make hive-metastore depend on hive-exec.
- */
-public class TestHiveMetaStoreTxns {
-
-  private final HiveConf conf = new HiveConf();
-  private IMetaStoreClient client;
-
-  public TestHiveMetaStoreTxns() throws Exception {
-    TxnDbUtil.setConfValues(conf);
-    LogManager.getRootLogger().setLevel(Level.DEBUG);
-    tearDown();
-  }
-
-  @Test
-  public void testTxns() throws Exception {
-    List<Long> tids = client.openTxns("me", 3).getTxn_ids();
-    Assert.assertEquals(1L, (long) tids.get(0));
-    Assert.assertEquals(2L, (long) tids.get(1));
-    Assert.assertEquals(3L, (long) tids.get(2));
-    client.rollbackTxn(1);
-    client.commitTxn(2);
-    ValidTxnList validTxns = client.getValidTxns();
-    Assert.assertFalse(validTxns.isTxnValid(1));
-    Assert.assertTrue(validTxns.isTxnValid(2));
-    Assert.assertFalse(validTxns.isTxnValid(3));
-    Assert.assertFalse(validTxns.isTxnValid(4));
-  }
-
-  @Test
-  public void testOpenTxnNotExcluded() throws Exception {
-    List<Long> tids = client.openTxns("me", 3).getTxn_ids();
-    Assert.assertEquals(1L, (long) tids.get(0));
-    Assert.assertEquals(2L, (long) tids.get(1));
-    Assert.assertEquals(3L, (long) tids.get(2));
-    client.rollbackTxn(1);
-    client.commitTxn(2);
-    ValidTxnList validTxns = client.getValidTxns(3);
-    Assert.assertFalse(validTxns.isTxnValid(1));
-    Assert.assertTrue(validTxns.isTxnValid(2));
-    Assert.assertTrue(validTxns.isTxnValid(3));
-    Assert.assertFalse(validTxns.isTxnValid(4));
-  }
-
-  @Test
-  public void testTxnRange() throws Exception {
-    ValidTxnList validTxns = client.getValidTxns();
-    Assert.assertEquals(ValidTxnList.RangeResponse.NONE,
-        validTxns.isTxnRangeValid(1L, 3L));
-    List<Long> tids = client.openTxns("me", 5).getTxn_ids();
-
-    HeartbeatTxnRangeResponse rsp = client.heartbeatTxnRange(1, 5);
-    Assert.assertEquals(0, rsp.getNosuch().size());
-    Assert.assertEquals(0, rsp.getAborted().size());
-
-    client.rollbackTxn(1L);
-    client.commitTxn(2L);
-    client.commitTxn(3L);
-    client.commitTxn(4L);
-    validTxns = client.getValidTxns();
-    System.out.println("validTxns = " + validTxns);
-    Assert.assertEquals(ValidTxnList.RangeResponse.ALL,
-        validTxns.isTxnRangeValid(2L, 2L));
-    Assert.assertEquals(ValidTxnList.RangeResponse.ALL,
-        validTxns.isTxnRangeValid(2L, 3L));
-    Assert.assertEquals(ValidTxnList.RangeResponse.ALL,
-        validTxns.isTxnRangeValid(2L, 4L));
-    Assert.assertEquals(ValidTxnList.RangeResponse.ALL,
-        validTxns.isTxnRangeValid(3L, 4L));
-
-    Assert.assertEquals(ValidTxnList.RangeResponse.SOME,
-        validTxns.isTxnRangeValid(1L, 4L));
-    Assert.assertEquals(ValidTxnList.RangeResponse.SOME,
-        validTxns.isTxnRangeValid(2L, 5L));
-    Assert.assertEquals(ValidTxnList.RangeResponse.SOME,
-        validTxns.isTxnRangeValid(1L, 2L));
-    Assert.assertEquals(ValidTxnList.RangeResponse.SOME,
-        validTxns.isTxnRangeValid(4L, 5L));
-
-    Assert.assertEquals(ValidTxnList.RangeResponse.NONE,
-        validTxns.isTxnRangeValid(1L, 1L));
-    Assert.assertEquals(ValidTxnList.RangeResponse.NONE,
-        validTxns.isTxnRangeValid(5L, 10L));
-
-    validTxns = new ValidReadTxnList("10:5:4,5,6:");
-    Assert.assertEquals(ValidTxnList.RangeResponse.NONE,
-        validTxns.isTxnRangeValid(4,6));
-    Assert.assertEquals(ValidTxnList.RangeResponse.ALL,
-        validTxns.isTxnRangeValid(7, 10));
-    Assert.assertEquals(ValidTxnList.RangeResponse.SOME,
-        validTxns.isTxnRangeValid(7, 11));
-    Assert.assertEquals(ValidTxnList.RangeResponse.SOME,
-        validTxns.isTxnRangeValid(3, 6));
-    Assert.assertEquals(ValidTxnList.RangeResponse.SOME,
-        validTxns.isTxnRangeValid(4, 7));
-    Assert.assertEquals(ValidTxnList.RangeResponse.SOME,
-        validTxns.isTxnRangeValid(1, 12));
-    Assert.assertEquals(ValidTxnList.RangeResponse.ALL,
-        validTxns.isTxnRangeValid(1, 3));
-  }
-
-  @Test
-  public void testLocks() throws Exception {
-    LockRequestBuilder rqstBuilder = new LockRequestBuilder();
-    rqstBuilder.addLockComponent(new LockComponentBuilder()
-        .setDbName("mydb")
-        .setTableName("mytable")
-        .setPartitionName("mypartition")
-        .setExclusive()
-        .setOperationType(DataOperationType.NO_TXN)
-        .build());
-    rqstBuilder.addLockComponent(new LockComponentBuilder()
-        .setDbName("mydb")
-        .setTableName("yourtable")
-        .setSemiShared()
-        .setOperationType(DataOperationType.NO_TXN)
-        .build());
-    rqstBuilder.addLockComponent(new LockComponentBuilder()
-        .setDbName("yourdb")
-        .setOperationType(DataOperationType.NO_TXN)
-        .setShared()
-        .build());
-    rqstBuilder.setUser("fred");
-
-    LockResponse res = client.lock(rqstBuilder.build());
-    Assert.assertEquals(1L, res.getLockid());
-    Assert.assertEquals(LockState.ACQUIRED, res.getState());
-
-    res = client.checkLock(1);
-    Assert.assertEquals(1L, res.getLockid());
-    Assert.assertEquals(LockState.ACQUIRED, res.getState());
-
-    client.heartbeat(0, 1);
-
-    client.unlock(1);
-  }
-
-  @Test
-  public void testLocksWithTxn() throws Exception {
-    long txnid = client.openTxn("me");
-
-    LockRequestBuilder rqstBuilder = new LockRequestBuilder();
-    rqstBuilder.setTransactionId(txnid)
-      .addLockComponent(new LockComponentBuilder()
-        .setDbName("mydb")
-        .setTableName("mytable")
-        .setPartitionName("mypartition")
-        .setSemiShared()
-        .setOperationType(DataOperationType.UPDATE)
-        .build())
-      .addLockComponent(new LockComponentBuilder()
-        .setDbName("mydb")
-        .setTableName("yourtable")
-        .setSemiShared()
-        .setOperationType(DataOperationType.UPDATE)
-        .build())
-      .addLockComponent(new LockComponentBuilder()
-        .setDbName("yourdb")
-        .setShared()
-        .setOperationType(DataOperationType.SELECT)
-        .build())
-      .setUser("fred");
-
-    LockResponse res = client.lock(rqstBuilder.build());
-    Assert.assertEquals(1L, res.getLockid());
-    Assert.assertEquals(LockState.ACQUIRED, res.getState());
-
-    res = client.checkLock(1);
-    Assert.assertEquals(1L, res.getLockid());
-    Assert.assertEquals(LockState.ACQUIRED, res.getState());
-
-    client.heartbeat(txnid, 1);
-
-    client.commitTxn(txnid);
-  }
-
-  @Test
-  public void stringifyValidTxns() throws Exception {
-    // Test with just high water mark
-    ValidTxnList validTxns = new ValidReadTxnList("1:" + Long.MAX_VALUE + "::");
-    String asString = validTxns.toString();
-    Assert.assertEquals("1:" + Long.MAX_VALUE + "::", asString);
-    validTxns = new ValidReadTxnList(asString);
-    Assert.assertEquals(1, validTxns.getHighWatermark());
-    Assert.assertNotNull(validTxns.getInvalidTransactions());
-    Assert.assertEquals(0, validTxns.getInvalidTransactions().length);
-    asString = validTxns.toString();
-    Assert.assertEquals("1:" + Long.MAX_VALUE + "::", asString);
-    validTxns = new ValidReadTxnList(asString);
-    Assert.assertEquals(1, validTxns.getHighWatermark());
-    Assert.assertNotNull(validTxns.getInvalidTransactions());
-    Assert.assertEquals(0, validTxns.getInvalidTransactions().length);
-
-    // Test with open transactions
-    validTxns = new ValidReadTxnList("10:3:5:3");
-    asString = validTxns.toString();
-    if (!asString.equals("10:3:3:5") && !asString.equals("10:3:5:3")) {
-      Assert.fail("Unexpected string value " + asString);
-    }
-    validTxns = new ValidReadTxnList(asString);
-    Assert.assertEquals(10, validTxns.getHighWatermark());
-    Assert.assertNotNull(validTxns.getInvalidTransactions());
-    Assert.assertEquals(2, validTxns.getInvalidTransactions().length);
-    boolean sawThree = false, sawFive = false;
-    for (long tid : validTxns.getInvalidTransactions()) {
-      if (tid == 3)  sawThree = true;
-      else if (tid == 5) sawFive = true;
-      else  Assert.fail("Unexpected value " + tid);
-    }
-    Assert.assertTrue(sawThree);
-    Assert.assertTrue(sawFive);
-  }
-
-  @Before
-  public void setUp() throws Exception {
-    TxnDbUtil.prepDb(conf);
-    client = new HiveMetaStoreClient(conf);
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    TxnDbUtil.cleanDb(conf);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/d9801d9c/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreWithEnvironmentContext.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreWithEnvironmentContext.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreWithEnvironmentContext.java
deleted file mode 100644
index c29a34d..0000000
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreWithEnvironmentContext.java
+++ /dev/null
@@ -1,219 +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.List;
-import java.util.Map;
-
-import junit.framework.TestCase;
-
-import org.apache.hadoop.hive.cli.CliSessionState;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.api.Database;
-import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
-import org.apache.hadoop.hive.metastore.api.FieldSchema;
-import org.apache.hadoop.hive.metastore.api.Partition;
-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.events.AddPartitionEvent;
-import org.apache.hadoop.hive.metastore.events.AlterTableEvent;
-import org.apache.hadoop.hive.metastore.events.CreateDatabaseEvent;
-import org.apache.hadoop.hive.metastore.events.CreateTableEvent;
-import org.apache.hadoop.hive.metastore.events.DropDatabaseEvent;
-import org.apache.hadoop.hive.metastore.events.DropPartitionEvent;
-import org.apache.hadoop.hive.metastore.events.DropTableEvent;
-import org.apache.hadoop.hive.metastore.events.ListenerEvent;
-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.ql.session.SessionState;
-import org.apache.hadoop.hive.serde.serdeConstants;
-import org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe;
-
-/**
- * TestHiveMetaStoreWithEnvironmentContext. Test case for _with_environment_context
- * calls in {@link org.apache.hadoop.hive.metastore.HiveMetaStore}
- */
-public class TestHiveMetaStoreWithEnvironmentContext extends TestCase {
-
-  private HiveConf hiveConf;
-  private HiveMetaStoreClient msc;
-  private EnvironmentContext envContext;
-  private final Database db = new Database();
-  private Table table = new Table();
-  private final Partition partition = new Partition();
-
-  private static final String dbName = "hive3252";
-  private static final String tblName = "tmptbl";
-  private static final String renamed = "tmptbl2";
-
-  @Override
-  protected void setUp() throws Exception {
-    super.setUp();
-
-    System.setProperty("hive.metastore.event.listeners",
-        DummyListener.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.set(HiveConf.ConfVars.PREEXECHOOKS.varname, "");
-    hiveConf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, "");
-    hiveConf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false");
-    SessionState.start(new CliSessionState(hiveConf));
-    msc = new HiveMetaStoreClient(hiveConf);
-
-    msc.dropDatabase(dbName, true, true);
-
-    Map<String, String> envProperties = new HashMap<String, String>();
-    envProperties.put("hadoop.job.ugi", "test_user");
-    envContext = new EnvironmentContext(envProperties);
-
-    db.setName(dbName);
-
-    Map<String, String> tableParams = new HashMap<String, String>();
-    tableParams.put("a", "string");
-    List<FieldSchema> partitionKeys = new ArrayList<FieldSchema>();
-    partitionKeys.add(new FieldSchema("b", "string", ""));
-
-    List<FieldSchema> cols = new ArrayList<FieldSchema>();
-    cols.add(new FieldSchema("a", "string", ""));
-    cols.add(new FieldSchema("b", "string", ""));
-    StorageDescriptor sd = new StorageDescriptor();
-    sd.setCols(cols);
-    sd.setCompressed(false);
-    sd.setParameters(tableParams);
-    sd.setSerdeInfo(new SerDeInfo());
-    sd.getSerdeInfo().setName(tblName);
-    sd.getSerdeInfo().setParameters(new HashMap<String, String>());
-    sd.getSerdeInfo().getParameters().put(serdeConstants.SERIALIZATION_FORMAT, "1");
-    sd.getSerdeInfo().setSerializationLib(LazySimpleSerDe.class.getName());
-    sd.setInputFormat(HiveInputFormat.class.getName());
-    sd.setOutputFormat(HiveOutputFormat.class.getName());
-
-    table.setDbName(dbName);
-    table.setTableName(tblName);
-    table.setParameters(tableParams);
-    table.setPartitionKeys(partitionKeys);
-    table.setSd(sd);
-
-    List<String> partValues = new ArrayList<String>();
-    partValues.add("2011");
-    partition.setDbName(dbName);
-    partition.setTableName(tblName);
-    partition.setValues(partValues);
-    partition.setSd(table.getSd().deepCopy());
-    partition.getSd().setSerdeInfo(table.getSd().getSerdeInfo().deepCopy());
-
-    DummyListener.notifyList.clear();
-  }
-
-  @Override
-  protected void tearDown() throws Exception {
-    super.tearDown();
-  }
-
-  public void testEnvironmentContext() throws Exception {
-    int listSize = 0;
-
-    List<ListenerEvent> notifyList = DummyListener.notifyList;
-    assertEquals(notifyList.size(), listSize);
-    msc.createDatabase(db);
-    listSize++;
-    assertEquals(listSize, notifyList.size());
-    CreateDatabaseEvent dbEvent = (CreateDatabaseEvent)(notifyList.get(listSize - 1));
-    assert dbEvent.getStatus();
-
-    msc.createTable(table, envContext);
-    listSize++;
-    assertEquals(notifyList.size(), listSize);
-    CreateTableEvent tblEvent = (CreateTableEvent)(notifyList.get(listSize - 1));
-    assert tblEvent.getStatus();
-    assertEquals(envContext, tblEvent.getEnvironmentContext());
-
-    table = msc.getTable(dbName, tblName);
-
-    partition.getSd().setLocation(table.getSd().getLocation() + "/part1");
-    msc.add_partition(partition, envContext);
-    listSize++;
-    assertEquals(notifyList.size(), listSize);
-    AddPartitionEvent partEvent = (AddPartitionEvent)(notifyList.get(listSize-1));
-    assert partEvent.getStatus();
-    assertEquals(envContext, partEvent.getEnvironmentContext());
-
-    List<String> partVals = new ArrayList<String>();
-    partVals.add("2012");
-    msc.appendPartition(dbName, tblName, partVals, envContext);
-    listSize++;
-    assertEquals(notifyList.size(), listSize);
-    AddPartitionEvent appendPartEvent = (AddPartitionEvent)(notifyList.get(listSize-1));
-    assert appendPartEvent.getStatus();
-    assertEquals(envContext, appendPartEvent.getEnvironmentContext());
-
-    table.setTableName(renamed);
-    msc.alter_table_with_environmentContext(dbName, tblName, table, envContext);
-    listSize++;
-    assertEquals(notifyList.size(), listSize);
-    AlterTableEvent alterTableEvent = (AlterTableEvent) notifyList.get(listSize-1);
-    assert alterTableEvent.getStatus();
-    assertEquals(envContext, alterTableEvent.getEnvironmentContext());
-
-    table.setTableName(tblName);
-    msc.alter_table_with_environmentContext(dbName, renamed, table, envContext);
-    listSize++;
-    assertEquals(notifyList.size(), listSize);
-
-    List<String> dropPartVals = new ArrayList<String>();
-    dropPartVals.add("2011");
-    msc.dropPartition(dbName, tblName, dropPartVals, envContext);
-    listSize++;
-    assertEquals(notifyList.size(), listSize);
-    DropPartitionEvent dropPartEvent = (DropPartitionEvent)notifyList.get(listSize - 1);
-    assert dropPartEvent.getStatus();
-    assertEquals(envContext, dropPartEvent.getEnvironmentContext());
-
-    msc.dropPartition(dbName, tblName, "b=2012", true, envContext);
-    listSize++;
-    assertEquals(notifyList.size(), listSize);
-    DropPartitionEvent dropPartByNameEvent = (DropPartitionEvent)notifyList.get(listSize - 1);
-    assert dropPartByNameEvent.getStatus();
-    assertEquals(envContext, dropPartByNameEvent.getEnvironmentContext());
-
-    msc.dropTable(dbName, tblName, true, false, envContext);
-    listSize++;
-    assertEquals(notifyList.size(), listSize);
-    DropTableEvent dropTblEvent = (DropTableEvent)notifyList.get(listSize-1);
-    assert dropTblEvent.getStatus();
-    assertEquals(envContext, dropTblEvent.getEnvironmentContext());
-
-    msc.dropDatabase(dbName);
-    listSize++;
-    assertEquals(notifyList.size(), listSize);
-
-    DropDatabaseEvent dropDB = (DropDatabaseEvent)notifyList.get(listSize-1);
-    assert dropDB.getStatus();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/d9801d9c/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMarkPartition.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMarkPartition.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMarkPartition.java
deleted file mode 100644
index 7b3a896..0000000
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMarkPartition.java
+++ /dev/null
@@ -1,108 +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.HashMap;
-import java.util.Map;
-
-import junit.framework.TestCase;
-
-import org.apache.hadoop.hive.cli.CliSessionState;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
-import org.apache.hadoop.hive.metastore.api.InvalidPartitionException;
-import org.apache.hadoop.hive.metastore.api.MetaException;
-import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
-import org.apache.hadoop.hive.metastore.api.PartitionEventType;
-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.ql.CommandNeedRetryException;
-import org.apache.hadoop.hive.ql.DriverFactory;
-import org.apache.hadoop.hive.ql.IDriver;
-import org.apache.hadoop.hive.ql.session.SessionState;
-import org.apache.thrift.TException;
-
-public class TestMarkPartition extends TestCase{
-
-  protected HiveConf hiveConf;
-  private IDriver driver;
-
-  @Override
-  protected void setUp() throws Exception {
-
-    super.setUp();
-    System.setProperty("hive.metastore.event.clean.freq", "2");
-    System.setProperty("hive.metastore.event.expiry.duration", "5");
-    hiveConf = new HiveConf(this.getClass());
-    hiveConf.set(HiveConf.ConfVars.PREEXECHOOKS.varname, "");
-    hiveConf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, "");
-    hiveConf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false");
-    SessionState.start(new CliSessionState(hiveConf));
-
-  }
-
-  public void testMarkingPartitionSet() throws CommandNeedRetryException, MetaException,
-  TException, NoSuchObjectException, UnknownDBException, UnknownTableException,
-  InvalidPartitionException, UnknownPartitionException, InterruptedException {
-    HiveMetaStoreClient msc = new HiveMetaStoreClient(hiveConf);
-    driver = DriverFactory.newDriver(hiveConf);
-    driver.run("drop database if exists hive2215 cascade");
-    driver.run("create database hive2215");
-    driver.run("use hive2215");
-    driver.run("drop table if exists tmptbl");
-    driver.run("create table tmptbl (a string) partitioned by (b string)");
-    driver.run("alter table tmptbl add partition (b='2011')");
-    Map<String,String> kvs = new HashMap<String, String>();
-    kvs.put("b", "'2011'");
-    msc.markPartitionForEvent("hive2215", "tmptbl", kvs, PartitionEventType.LOAD_DONE);
-    assert msc.isPartitionMarkedForEvent("hive2215", "tmptbl", kvs, PartitionEventType.LOAD_DONE);
-    Thread.sleep(10000);
-    assert !msc.isPartitionMarkedForEvent("hive2215", "tmptbl", kvs, PartitionEventType.LOAD_DONE);
-
-    kvs.put("b", "'2012'");
-    assert !msc.isPartitionMarkedForEvent("hive2215", "tmptbl", kvs, PartitionEventType.LOAD_DONE);
-    try{
-      msc.markPartitionForEvent("hive2215", "tmptbl2", kvs, PartitionEventType.LOAD_DONE);
-      assert false;
-    } catch(Exception e){
-      assert e instanceof UnknownTableException;
-    }
-    try{
-      msc.isPartitionMarkedForEvent("hive2215", "tmptbl2", kvs, PartitionEventType.LOAD_DONE);
-      assert false;
-    } catch(Exception e){
-      assert e instanceof UnknownTableException;
-    }
-    kvs.put("a", "'2012'");
-    try{
-      msc.isPartitionMarkedForEvent("hive2215", "tmptbl", kvs, PartitionEventType.LOAD_DONE);
-      assert false;
-    } catch(Exception e){
-      assert e instanceof InvalidPartitionException;
-    }
-  }
-
-  @Override
-  protected void tearDown() throws Exception {
-    driver.run("drop database if exists hive2215 cascade");
-    super.tearDown();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/d9801d9c/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMarkPartitionRemote.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMarkPartitionRemote.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMarkPartitionRemote.java
deleted file mode 100644
index c541193..0000000
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMarkPartitionRemote.java
+++ /dev/null
@@ -1,32 +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;
-import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
-
-public class TestMarkPartitionRemote extends TestMarkPartition {
-
-  @Override
-  protected void setUp() throws Exception {
-    super.setUp();
-    hiveConf.setVar(HiveConf.ConfVars.METASTOREURIS, "thrift://localhost:" + MetaStoreTestUtils.startMetaStore());
-    hiveConf.setIntVar(HiveConf.ConfVars.METASTORETHRIFTCONNECTIONRETRIES, 3);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/d9801d9c/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEndFunctionListener.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEndFunctionListener.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEndFunctionListener.java
deleted file mode 100644
index 1ca18b9..0000000
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEndFunctionListener.java
+++ /dev/null
@@ -1,142 +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.cli.CliSessionState;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
-import org.apache.hadoop.hive.ql.DriverFactory;
-import org.apache.hadoop.hive.ql.IDriver;
-import org.apache.hadoop.hive.ql.session.SessionState;
-
-import junit.framework.TestCase;
-
-/**
- * TestMetaStoreEventListener. Test case for
- * {@link org.apache.hadoop.hive.metastore.MetaStoreEndFunctionListener}
- */
-public class TestMetaStoreEndFunctionListener extends TestCase {
-  private HiveConf hiveConf;
-  private HiveMetaStoreClient msc;
-  private IDriver driver;
-
-  @Override
-  protected void setUp() throws Exception {
-
-    super.setUp();
-    System.setProperty("hive.metastore.event.listeners",
-        DummyListener.class.getName());
-    System.setProperty("hive.metastore.pre.event.listeners",
-        DummyPreListener.class.getName());
-    System.setProperty("hive.metastore.end.function.listeners",
-        DummyEndFunctionListener.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.set(HiveConf.ConfVars.PREEXECHOOKS.varname, "");
-    hiveConf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, "");
-    hiveConf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false");
-    SessionState.start(new CliSessionState(hiveConf));
-    msc = new HiveMetaStoreClient(hiveConf);
-    driver = DriverFactory.newDriver(hiveConf);
-  }
-
-  @Override
-  protected void tearDown() throws Exception {
-    super.tearDown();
-  }
-
-  public void testEndFunctionListener() throws Exception {
-    /* Objective here is to ensure that when exceptions are thrown in HiveMetaStore in API methods
-     * they bubble up and are stored in the MetaStoreEndFunctionContext objects
-     */
-    String dbName = "hive3524";
-    String tblName = "tmptbl";
-    int listSize = 0;
-
-    driver.run("create database " + dbName);
-
-    try {
-      msc.getDatabase("UnknownDB");
-    }
-    catch (Exception e) {
-    }
-    listSize = DummyEndFunctionListener.funcNameList.size();
-    String func_name = DummyEndFunctionListener.funcNameList.get(listSize-1);
-    MetaStoreEndFunctionContext context = DummyEndFunctionListener.contextList.get(listSize-1);
-    assertEquals(func_name,"get_database");
-    assertFalse(context.isSuccess());
-    Exception e = context.getException();
-    assertTrue((e!=null));
-    assertTrue((e instanceof NoSuchObjectException));
-    assertEquals(context.getInputTableName(), null);
-
-    driver.run("use " + dbName);
-    driver.run(String.format("create table %s (a string) partitioned by (b string)", tblName));
-    String tableName = "Unknown";
-    try {
-      msc.getTable(dbName, tableName);
-    }
-    catch (Exception e1) {
-    }
-    listSize = DummyEndFunctionListener.funcNameList.size();
-    func_name = DummyEndFunctionListener.funcNameList.get(listSize-1);
-    context = DummyEndFunctionListener.contextList.get(listSize-1);
-    assertEquals(func_name,"get_table");
-    assertFalse(context.isSuccess());
-    e = context.getException();
-    assertTrue((e!=null));
-    assertTrue((e instanceof NoSuchObjectException));
-    assertEquals(context.getInputTableName(), tableName);
-
-    try {
-      msc.getPartition("hive3524", tblName, "b=2012");
-    }
-    catch (Exception e2) {
-    }
-    listSize = DummyEndFunctionListener.funcNameList.size();
-    func_name = DummyEndFunctionListener.funcNameList.get(listSize-1);
-    context = DummyEndFunctionListener.contextList.get(listSize-1);
-    assertEquals(func_name,"get_partition_by_name");
-    assertFalse(context.isSuccess());
-    e = context.getException();
-    assertTrue((e!=null));
-    assertTrue((e instanceof NoSuchObjectException));
-    assertEquals(context.getInputTableName(), tblName);
-    try {
-      driver.run("drop table Unknown");
-    }
-    catch (Exception e4) {
-    }
-    listSize = DummyEndFunctionListener.funcNameList.size();
-    func_name = DummyEndFunctionListener.funcNameList.get(listSize-1);
-    context = DummyEndFunctionListener.contextList.get(listSize-1);
-    assertEquals(func_name,"get_table");
-    assertFalse(context.isSuccess());
-    e = context.getException();
-    assertTrue((e!=null));
-    assertTrue((e instanceof NoSuchObjectException));
-    assertEquals(context.getInputTableName(), "Unknown");
-
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/d9801d9c/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListener.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListener.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListener.java
deleted file mode 100644
index 358e5d1..0000000
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListener.java
+++ /dev/null
@@ -1,526 +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.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.hadoop.hive.cli.CliSessionState;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.api.Database;
-import org.apache.hadoop.hive.metastore.api.Index;
-import org.apache.hadoop.hive.metastore.api.Partition;
-import org.apache.hadoop.hive.metastore.api.PartitionEventType;
-import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.hadoop.hive.metastore.events.AddIndexEvent;
-import org.apache.hadoop.hive.metastore.events.AddPartitionEvent;
-import org.apache.hadoop.hive.metastore.events.AlterIndexEvent;
-import org.apache.hadoop.hive.metastore.events.AlterPartitionEvent;
-import org.apache.hadoop.hive.metastore.events.AlterTableEvent;
-import org.apache.hadoop.hive.metastore.events.ConfigChangeEvent;
-import org.apache.hadoop.hive.metastore.events.CreateDatabaseEvent;
-import org.apache.hadoop.hive.metastore.events.CreateTableEvent;
-import org.apache.hadoop.hive.metastore.events.DropDatabaseEvent;
-import org.apache.hadoop.hive.metastore.events.DropIndexEvent;
-import org.apache.hadoop.hive.metastore.events.DropPartitionEvent;
-import org.apache.hadoop.hive.metastore.events.DropTableEvent;
-import org.apache.hadoop.hive.metastore.events.ListenerEvent;
-import org.apache.hadoop.hive.metastore.events.LoadPartitionDoneEvent;
-import org.apache.hadoop.hive.metastore.events.PreAddIndexEvent;
-import org.apache.hadoop.hive.metastore.events.PreAddPartitionEvent;
-import org.apache.hadoop.hive.metastore.events.PreAlterIndexEvent;
-import org.apache.hadoop.hive.metastore.events.PreAlterPartitionEvent;
-import org.apache.hadoop.hive.metastore.events.PreAlterTableEvent;
-import org.apache.hadoop.hive.metastore.events.PreCreateDatabaseEvent;
-import org.apache.hadoop.hive.metastore.events.PreCreateTableEvent;
-import org.apache.hadoop.hive.metastore.events.PreDropDatabaseEvent;
-import org.apache.hadoop.hive.metastore.events.PreDropIndexEvent;
-import org.apache.hadoop.hive.metastore.events.PreDropPartitionEvent;
-import org.apache.hadoop.hive.metastore.events.PreDropTableEvent;
-import org.apache.hadoop.hive.metastore.events.PreEventContext;
-import org.apache.hadoop.hive.metastore.events.PreLoadPartitionDoneEvent;
-import org.apache.hadoop.hive.ql.DriverFactory;
-import org.apache.hadoop.hive.ql.IDriver;
-import org.apache.hadoop.hive.ql.processors.SetProcessor;
-import org.apache.hadoop.hive.ql.session.SessionState;
-
-import com.google.common.collect.Lists;
-
-import junit.framework.TestCase;
-
-/**
- * TestMetaStoreEventListener. Test case for
- * {@link org.apache.hadoop.hive.metastore.MetaStoreEventListener} and
- * {@link org.apache.hadoop.hive.metastore.MetaStorePreEventListener}
- */
-public class TestMetaStoreEventListener extends TestCase {
-  private HiveConf hiveConf;
-  private HiveMetaStoreClient msc;
-  private IDriver driver;
-
-  private static final String dbName = "hive2038";
-  private static final String tblName = "tmptbl";
-  private static final String renamed = "tmptbl2";
-  private static final String metaConfKey = "hive.metastore.partition.name.whitelist.pattern";
-  private static final String metaConfVal = "";
-
-  @Override
-  protected void setUp() throws Exception {
-
-    super.setUp();
-
-    System.setProperty("hive.metastore.event.listeners",
-        DummyListener.class.getName());
-    System.setProperty("hive.metastore.pre.event.listeners",
-        DummyPreListener.class.getName());
-
-    hiveConf = new HiveConf(this.getClass());
-
-    hiveConf.setVar(HiveConf.ConfVars.METASTORE_PARTITION_NAME_WHITELIST_PATTERN, metaConfVal);
-    int port = MetaStoreTestUtils.startMetaStoreWithRetry(hiveConf);
-
-    hiveConf.setVar(HiveConf.ConfVars.METASTOREURIS, "thrift://localhost:" + port);
-    hiveConf.setIntVar(HiveConf.ConfVars.METASTORETHRIFTCONNECTIONRETRIES, 3);
-    hiveConf.set(HiveConf.ConfVars.PREEXECHOOKS.varname, "");
-    hiveConf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, "");
-    hiveConf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false");
-    hiveConf.set(HiveConf.ConfVars.HIVE_TXN_MANAGER.varname,
-        "org.apache.hadoop.hive.ql.lockmgr.DummyTxnManager");
-    SessionState.start(new CliSessionState(hiveConf));
-    msc = new HiveMetaStoreClient(hiveConf);
-    driver = DriverFactory.newDriver(hiveConf);
-
-    driver.run("drop database if exists " + dbName + " cascade");
-
-    DummyListener.notifyList.clear();
-    DummyPreListener.notifyList.clear();
-  }
-
-  @Override
-  protected void tearDown() throws Exception {
-    super.tearDown();
-  }
-
-  private void validateCreateDb(Database expectedDb, Database actualDb) {
-    assertEquals(expectedDb.getName(), actualDb.getName());
-    assertEquals(expectedDb.getLocationUri(), actualDb.getLocationUri());
-  }
-
-  private void validateTable(Table expectedTable, Table actualTable) {
-    assertEquals(expectedTable.getTableName(), actualTable.getTableName());
-    assertEquals(expectedTable.getDbName(), actualTable.getDbName());
-    assertEquals(expectedTable.getSd().getLocation(), actualTable.getSd().getLocation());
-  }
-
-  private void validateCreateTable(Table expectedTable, Table actualTable) {
-    validateTable(expectedTable, actualTable);
-  }
-
-  private void validateAddPartition(Partition expectedPartition, Partition actualPartition) {
-    assertEquals(expectedPartition, actualPartition);
-  }
-
-  private void validateTableInAddPartition(Table expectedTable, Table actualTable) {
-    assertEquals(expectedTable, actualTable);
-  }
-
-  private void validatePartition(Partition expectedPartition, Partition actualPartition) {
-    assertEquals(expectedPartition.getValues(), actualPartition.getValues());
-    assertEquals(expectedPartition.getDbName(), actualPartition.getDbName());
-    assertEquals(expectedPartition.getTableName(), actualPartition.getTableName());
-  }
-
-  private void validateAlterPartition(Partition expectedOldPartition,
-      Partition expectedNewPartition, String actualOldPartitionDbName,
-      String actualOldPartitionTblName,List<String> actualOldPartitionValues,
-      Partition actualNewPartition) {
-    assertEquals(expectedOldPartition.getValues(), actualOldPartitionValues);
-    assertEquals(expectedOldPartition.getDbName(), actualOldPartitionDbName);
-    assertEquals(expectedOldPartition.getTableName(), actualOldPartitionTblName);
-
-    validatePartition(expectedNewPartition, actualNewPartition);
-  }
-
-  private void validateAlterTable(Table expectedOldTable, Table expectedNewTable,
-      Table actualOldTable, Table actualNewTable) {
-    validateTable(expectedOldTable, actualOldTable);
-    validateTable(expectedNewTable, actualNewTable);
-  }
-
-  private void validateAlterTableColumns(Table expectedOldTable, Table expectedNewTable,
-      Table actualOldTable, Table actualNewTable) {
-    validateAlterTable(expectedOldTable, expectedNewTable, actualOldTable, actualNewTable);
-
-    assertEquals(expectedOldTable.getSd().getCols(), actualOldTable.getSd().getCols());
-    assertEquals(expectedNewTable.getSd().getCols(), actualNewTable.getSd().getCols());
-  }
-
-  private void validateLoadPartitionDone(String expectedTableName,
-      Map<String,String> expectedPartitionName, String actualTableName,
-      Map<String,String> actualPartitionName) {
-    assertEquals(expectedPartitionName, actualPartitionName);
-    assertEquals(expectedTableName, actualTableName);
-  }
-
-  private void validateDropPartition(Iterator<Partition> expectedPartitions, Iterator<Partition> actualPartitions) {
-    while (expectedPartitions.hasNext()){
-      assertTrue(actualPartitions.hasNext());
-      validatePartition(expectedPartitions.next(), actualPartitions.next());
-    }
-    assertFalse(actualPartitions.hasNext());
-  }
-
-  private void validateTableInDropPartition(Table expectedTable, Table actualTable) {
-    validateTable(expectedTable, actualTable);
-  }
-
-  private void validateDropTable(Table expectedTable, Table actualTable) {
-    validateTable(expectedTable, actualTable);
-  }
-
-  private void validateDropDb(Database expectedDb, Database actualDb) {
-    assertEquals(expectedDb, actualDb);
-  }
-
-  private void validateIndex(Index expectedIndex, Index actualIndex) {
-    assertEquals(expectedIndex.getDbName(), actualIndex.getDbName());
-    assertEquals(expectedIndex.getIndexName(), actualIndex.getIndexName());
-    assertEquals(expectedIndex.getIndexHandlerClass(), actualIndex.getIndexHandlerClass());
-    assertEquals(expectedIndex.getOrigTableName(), actualIndex.getOrigTableName());
-    assertEquals(expectedIndex.getIndexTableName(), actualIndex.getIndexTableName());
-    assertEquals(expectedIndex.getSd().getLocation(), actualIndex.getSd().getLocation());
-  }
-
-  private void validateAddIndex(Index expectedIndex, Index actualIndex) {
-    validateIndex(expectedIndex, actualIndex);
-  }
-
-  private void validateAlterIndex(Index expectedOldIndex, Index actualOldIndex,
-      Index expectedNewIndex, Index actualNewIndex) {
-    validateIndex(expectedOldIndex, actualOldIndex);
-    validateIndex(expectedNewIndex, actualNewIndex);
-  }
-
-  private void validateDropIndex(Index expectedIndex, Index actualIndex) {
-    validateIndex(expectedIndex, actualIndex);
-  }
-
-  public void testListener() throws Exception {
-    int listSize = 0;
-
-    List<ListenerEvent> notifyList = DummyListener.notifyList;
-    List<PreEventContext> preNotifyList = DummyPreListener.notifyList;
-    assertEquals(notifyList.size(), listSize);
-    assertEquals(preNotifyList.size(), listSize);
-
-    driver.run("create database " + dbName);
-    listSize++;
-    PreCreateDatabaseEvent preDbEvent = (PreCreateDatabaseEvent)(preNotifyList.get(preNotifyList.size() - 1));
-    Database db = msc.getDatabase(dbName);
-    assertEquals(listSize, notifyList.size());
-    assertEquals(listSize + 1, preNotifyList.size());
-    validateCreateDb(db, preDbEvent.getDatabase());
-
-    CreateDatabaseEvent dbEvent = (CreateDatabaseEvent)(notifyList.get(listSize - 1));
-    assert dbEvent.getStatus();
-    validateCreateDb(db, dbEvent.getDatabase());
-
-
-    driver.run("use " + dbName);
-    driver.run(String.format("create table %s (a string) partitioned by (b string)", tblName));
-    PreCreateTableEvent preTblEvent = (PreCreateTableEvent)(preNotifyList.get(preNotifyList.size() - 1));
-    listSize++;
-    Table tbl = msc.getTable(dbName, tblName);
-    validateCreateTable(tbl, preTblEvent.getTable());
-    assertEquals(notifyList.size(), listSize);
-
-    CreateTableEvent tblEvent = (CreateTableEvent)(notifyList.get(listSize - 1));
-    assert tblEvent.getStatus();
-    validateCreateTable(tbl, tblEvent.getTable());
-
-    driver.run("create index tmptbl_i on table tmptbl(a) as 'compact' " +
-        "WITH DEFERRED REBUILD IDXPROPERTIES ('prop1'='val1', 'prop2'='val2')");
-    listSize += 2;  // creates index table internally
-    assertEquals(notifyList.size(), listSize);
-
-    AddIndexEvent addIndexEvent = (AddIndexEvent)notifyList.get(listSize - 1);
-    assert addIndexEvent.getStatus();
-    PreAddIndexEvent preAddIndexEvent = (PreAddIndexEvent)(preNotifyList.get(preNotifyList.size() - 3));
-
-    Index oldIndex = msc.getIndex(dbName, "tmptbl", "tmptbl_i");
-
-    validateAddIndex(oldIndex, addIndexEvent.getIndex());
-
-    validateAddIndex(oldIndex, preAddIndexEvent.getIndex());
-
-    driver.run("alter index tmptbl_i on tmptbl set IDXPROPERTIES " +
-        "('prop1'='val1_new', 'prop3'='val3')");
-    listSize++;
-    assertEquals(notifyList.size(), listSize);
-
-    Index newIndex = msc.getIndex(dbName, "tmptbl", "tmptbl_i");
-
-    AlterIndexEvent alterIndexEvent = (AlterIndexEvent) notifyList.get(listSize - 1);
-    assert alterIndexEvent.getStatus();
-    validateAlterIndex(oldIndex, alterIndexEvent.getOldIndex(),
-        newIndex, alterIndexEvent.getNewIndex());
-
-    PreAlterIndexEvent preAlterIndexEvent = (PreAlterIndexEvent) (preNotifyList.get(preNotifyList.size() - 1));
-    validateAlterIndex(oldIndex, preAlterIndexEvent.getOldIndex(),
-        newIndex, preAlterIndexEvent.getNewIndex());
-
-    driver.run("drop index tmptbl_i on tmptbl");
-    listSize++;
-    assertEquals(notifyList.size(), listSize);
-
-    DropIndexEvent dropIndexEvent = (DropIndexEvent) notifyList.get(listSize - 1);
-    assert dropIndexEvent.getStatus();
-    validateDropIndex(newIndex, dropIndexEvent.getIndex());
-
-    PreDropIndexEvent preDropIndexEvent = (PreDropIndexEvent) (preNotifyList.get(preNotifyList.size() - 1));
-    validateDropIndex(newIndex, preDropIndexEvent.getIndex());
-
-    driver.run("alter table tmptbl add partition (b='2011')");
-    listSize++;
-    assertEquals(notifyList.size(), listSize);
-    PreAddPartitionEvent prePartEvent = (PreAddPartitionEvent)(preNotifyList.get(preNotifyList.size() - 1));
-
-    AddPartitionEvent partEvent = (AddPartitionEvent)(notifyList.get(listSize-1));
-    assert partEvent.getStatus();
-    Partition part = msc.getPartition("hive2038", "tmptbl", "b=2011");
-    Partition partAdded = partEvent.getPartitionIterator().next();
-    validateAddPartition(part, partAdded);
-    validateTableInAddPartition(tbl, partEvent.getTable());
-    validateAddPartition(part, prePartEvent.getPartitions().get(0));
-
-    // Test adding multiple partitions in a single partition-set, atomically.
-    int currentTime = (int)System.currentTimeMillis();
-    HiveMetaStoreClient hmsClient = new HiveMetaStoreClient(hiveConf);
-    Table table = hmsClient.getTable(dbName, "tmptbl");
-    Partition partition1 = new Partition(Arrays.asList("20110101"), dbName, "tmptbl", currentTime,
-                                        currentTime, table.getSd(), table.getParameters());
-    Partition partition2 = new Partition(Arrays.asList("20110102"), dbName, "tmptbl", currentTime,
-                                        currentTime, table.getSd(), table.getParameters());
-    Partition partition3 = new Partition(Arrays.asList("20110103"), dbName, "tmptbl", currentTime,
-                                        currentTime, table.getSd(), table.getParameters());
-    hmsClient.add_partitions(Arrays.asList(partition1, partition2, partition3));
-    ++listSize;
-    AddPartitionEvent multiplePartitionEvent = (AddPartitionEvent)(notifyList.get(listSize-1));
-    assertEquals("Unexpected table value.", table, multiplePartitionEvent.getTable());
-    List<Partition> multiParts = Lists.newArrayList(multiplePartitionEvent.getPartitionIterator());
-    assertEquals("Unexpected number of partitions in event!", 3, multiParts.size());
-    assertEquals("Unexpected partition value.", partition1.getValues(), multiParts.get(0).getValues());
-    assertEquals("Unexpected partition value.", partition2.getValues(), multiParts.get(1).getValues());
-    assertEquals("Unexpected partition value.", partition3.getValues(), multiParts.get(2).getValues());
-
-    driver.run(String.format("alter table %s touch partition (%s)", tblName, "b='2011'"));
-    listSize++;
-    assertEquals(notifyList.size(), listSize);
-    PreAlterPartitionEvent preAlterPartEvent =
-        (PreAlterPartitionEvent)preNotifyList.get(preNotifyList.size() - 1);
-
-    //the partition did not change,
-    // so the new partition should be similar to the original partition
-    Partition origP = msc.getPartition(dbName, tblName, "b=2011");
-
-    AlterPartitionEvent alterPartEvent = (AlterPartitionEvent)notifyList.get(listSize - 1);
-    assert alterPartEvent.getStatus();
-    validateAlterPartition(origP, origP, alterPartEvent.getOldPartition().getDbName(),
-        alterPartEvent.getOldPartition().getTableName(),
-        alterPartEvent.getOldPartition().getValues(), alterPartEvent.getNewPartition());
-
-
-    validateAlterPartition(origP, origP, preAlterPartEvent.getDbName(),
-        preAlterPartEvent.getTableName(), preAlterPartEvent.getNewPartition().getValues(),
-        preAlterPartEvent.getNewPartition());
-
-    List<String> part_vals = new ArrayList<String>();
-    part_vals.add("c=2012");
-    int preEventListSize;
-    preEventListSize = preNotifyList.size() + 1;
-    Partition newPart = msc.appendPartition(dbName, tblName, part_vals);
-
-    listSize++;
-    assertEquals(notifyList.size(), listSize);
-    assertEquals(preNotifyList.size(), preEventListSize);
-
-    AddPartitionEvent appendPartEvent =
-        (AddPartitionEvent)(notifyList.get(listSize-1));
-    Partition partAppended = appendPartEvent.getPartitionIterator().next();
-    validateAddPartition(newPart, partAppended);
-
-    PreAddPartitionEvent preAppendPartEvent =
-        (PreAddPartitionEvent)(preNotifyList.get(preNotifyList.size() - 1));
-    validateAddPartition(newPart, preAppendPartEvent.getPartitions().get(0));
-
-    driver.run(String.format("alter table %s rename to %s", tblName, renamed));
-    listSize++;
-    assertEquals(notifyList.size(), listSize);
-    PreAlterTableEvent preAlterTableE = (PreAlterTableEvent) preNotifyList.get(preNotifyList.size() - 1);
-
-    Table renamedTable = msc.getTable(dbName, renamed);
-
-    AlterTableEvent alterTableE = (AlterTableEvent) notifyList.get(listSize-1);
-    assert alterTableE.getStatus();
-    validateAlterTable(tbl, renamedTable, alterTableE.getOldTable(), alterTableE.getNewTable());
-    validateAlterTable(tbl, renamedTable, preAlterTableE.getOldTable(),
-        preAlterTableE.getNewTable());
-
-    //change the table name back
-    driver.run(String.format("alter table %s rename to %s", renamed, tblName));
-    listSize++;
-    assertEquals(notifyList.size(), listSize);
-
-    driver.run(String.format("alter table %s ADD COLUMNS (c int)", tblName));
-    listSize++;
-    assertEquals(notifyList.size(), listSize);
-    preAlterTableE = (PreAlterTableEvent) preNotifyList.get(preNotifyList.size() - 1);
-
-    Table altTable = msc.getTable(dbName, tblName);
-
-    alterTableE = (AlterTableEvent) notifyList.get(listSize-1);
-    assert alterTableE.getStatus();
-    validateAlterTableColumns(tbl, altTable, alterTableE.getOldTable(), alterTableE.getNewTable());
-    validateAlterTableColumns(tbl, altTable, preAlterTableE.getOldTable(),
-        preAlterTableE.getNewTable());
-
-    Map<String,String> kvs = new HashMap<String, String>(1);
-    kvs.put("b", "2011");
-    msc.markPartitionForEvent("hive2038", "tmptbl", kvs, PartitionEventType.LOAD_DONE);
-    listSize++;
-    assertEquals(notifyList.size(), listSize);
-
-    LoadPartitionDoneEvent partMarkEvent = (LoadPartitionDoneEvent)notifyList.get(listSize - 1);
-    assert partMarkEvent.getStatus();
-    validateLoadPartitionDone("tmptbl", kvs, partMarkEvent.getTable().getTableName(),
-        partMarkEvent.getPartitionName());
-
-    PreLoadPartitionDoneEvent prePartMarkEvent =
-        (PreLoadPartitionDoneEvent)preNotifyList.get(preNotifyList.size() - 1);
-    validateLoadPartitionDone("tmptbl", kvs, prePartMarkEvent.getTableName(),
-        prePartMarkEvent.getPartitionName());
-
-    driver.run(String.format("alter table %s drop partition (b='2011')", tblName));
-    listSize++;
-    assertEquals(notifyList.size(), listSize);
-    PreDropPartitionEvent preDropPart = (PreDropPartitionEvent) preNotifyList.get(preNotifyList
-        .size() - 1);
-
-    DropPartitionEvent dropPart = (DropPartitionEvent)notifyList.get(listSize - 1);
-    assert dropPart.getStatus();
-    validateDropPartition(Collections.singletonList(part).iterator(), dropPart.getPartitionIterator());
-    validateTableInDropPartition(tbl, dropPart.getTable());
-
-    validateDropPartition(Collections.singletonList(part).iterator(), preDropPart.getPartitionIterator());
-    validateTableInDropPartition(tbl, preDropPart.getTable());
-
-    driver.run("drop table " + tblName);
-    listSize++;
-    assertEquals(notifyList.size(), listSize);
-    PreDropTableEvent preDropTbl = (PreDropTableEvent)preNotifyList.get(preNotifyList.size() - 1);
-
-    DropTableEvent dropTbl = (DropTableEvent)notifyList.get(listSize-1);
-    assert dropTbl.getStatus();
-    validateDropTable(tbl, dropTbl.getTable());
-    validateDropTable(tbl, preDropTbl.getTable());
-
-    driver.run("drop database " + dbName);
-    listSize++;
-    assertEquals(notifyList.size(), listSize);
-    PreDropDatabaseEvent preDropDB = (PreDropDatabaseEvent)preNotifyList.get(preNotifyList.size() - 1);
-
-    DropDatabaseEvent dropDB = (DropDatabaseEvent)notifyList.get(listSize-1);
-    assert dropDB.getStatus();
-    validateDropDb(db, dropDB.getDatabase());
-    validateDropDb(db, preDropDB.getDatabase());
-
-    SetProcessor.setVariable("metaconf:hive.metastore.try.direct.sql", "false");
-    ConfigChangeEvent event = (ConfigChangeEvent) notifyList.get(notifyList.size() - 1);
-    assertEquals("hive.metastore.try.direct.sql", event.getKey());
-    assertEquals("true", event.getOldValue());
-    assertEquals("false", event.getNewValue());
-  }
-
-  public void testMetaConfNotifyListenersClosingClient() throws Exception {
-    HiveMetaStoreClient closingClient = new HiveMetaStoreClient(hiveConf, null);
-    closingClient.setMetaConf(metaConfKey, "[test pattern modified]");
-    ConfigChangeEvent event = (ConfigChangeEvent) DummyListener.getLastEvent();
-    assertEquals(event.getOldValue(), metaConfVal);
-    assertEquals(event.getNewValue(), "[test pattern modified]");
-    closingClient.close();
-
-    Thread.sleep(5 * 1000);
-
-    event = (ConfigChangeEvent) DummyListener.getLastEvent();
-    assertEquals(event.getOldValue(), "[test pattern modified]");
-    assertEquals(event.getNewValue(), metaConfVal);
-  }
-
-  public void testMetaConfNotifyListenersNonClosingClient() throws Exception {
-    HiveMetaStoreClient nonClosingClient = new HiveMetaStoreClient(hiveConf, null);
-    nonClosingClient.setMetaConf(metaConfKey, "[test pattern modified]");
-    ConfigChangeEvent event = (ConfigChangeEvent) DummyListener.getLastEvent();
-    assertEquals(event.getOldValue(), metaConfVal);
-    assertEquals(event.getNewValue(), "[test pattern modified]");
-    // This should also trigger meta listener notification via TServerEventHandler#deleteContext
-    nonClosingClient.getTTransport().close();
-
-    Thread.sleep(5 * 1000);
-
-    event = (ConfigChangeEvent) DummyListener.getLastEvent();
-    assertEquals(event.getOldValue(), "[test pattern modified]");
-    assertEquals(event.getNewValue(), metaConfVal);
-  }
-
-  public void testMetaConfDuplicateNotification() throws Exception {
-    HiveMetaStoreClient closingClient = new HiveMetaStoreClient(hiveConf, null);
-    closingClient.setMetaConf(metaConfKey, metaConfVal);
-    int beforeCloseNotificationEventCounts = DummyListener.notifyList.size();
-    closingClient.close();
-
-    Thread.sleep(5 * 1000);
-
-    int afterCloseNotificationEventCounts = DummyListener.notifyList.size();
-    // Setting key to same value, should not trigger configChange event during shutdown
-    assertEquals(beforeCloseNotificationEventCounts, afterCloseNotificationEventCounts);
-  }
-
-  public void testMetaConfSameHandler() throws Exception {
-    HiveMetaStoreClient closingClient = new HiveMetaStoreClient(hiveConf, null);
-    closingClient.setMetaConf(metaConfKey, "[test pattern modified]");
-    ConfigChangeEvent event = (ConfigChangeEvent) DummyListener.getLastEvent();
-    int beforeCloseNotificationEventCounts = DummyListener.notifyList.size();
-    IHMSHandler beforeHandler = event.getIHMSHandler();
-    closingClient.close();
-
-    Thread.sleep(5 * 1000);
-    event = (ConfigChangeEvent) DummyListener.getLastEvent();
-    int afterCloseNotificationEventCounts = DummyListener.notifyList.size();
-    IHMSHandler afterHandler = event.getIHMSHandler();
-    // Meta-conf cleanup should trigger an event to listener
-    assertNotSame(beforeCloseNotificationEventCounts, afterCloseNotificationEventCounts);
-    // Both the handlers should be same
-    assertEquals(beforeHandler, afterHandler);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/d9801d9c/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListenerOnlyOnCommit.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListenerOnlyOnCommit.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListenerOnlyOnCommit.java
deleted file mode 100644
index cc2c5f9..0000000
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListenerOnlyOnCommit.java
+++ /dev/null
@@ -1,103 +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.List;
-
-import org.apache.hadoop.hive.cli.CliSessionState;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.events.ListenerEvent;
-import org.apache.hadoop.hive.ql.DriverFactory;
-import org.apache.hadoop.hive.ql.IDriver;
-import org.apache.hadoop.hive.ql.session.SessionState;
-
-import junit.framework.TestCase;
-
-/**
- * Ensure that the status of MetaStore events depend on the RawStore's commit status.
- */
-public class TestMetaStoreEventListenerOnlyOnCommit extends TestCase {
-
-  private HiveConf hiveConf;
-  private HiveMetaStoreClient msc;
-  private IDriver driver;
-
-  @Override
-  protected void setUp() throws Exception {
-
-    super.setUp();
-
-    DummyRawStoreControlledCommit.setCommitSucceed(true);
-
-    System.setProperty(HiveConf.ConfVars.METASTORE_EVENT_LISTENERS.varname,
-            DummyListener.class.getName());
-    System.setProperty(HiveConf.ConfVars.METASTORE_RAW_STORE_IMPL.varname,
-            DummyRawStoreControlledCommit.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.set(HiveConf.ConfVars.PREEXECHOOKS.varname, "");
-    hiveConf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, "");
-    hiveConf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false");
-    SessionState.start(new CliSessionState(hiveConf));
-    msc = new HiveMetaStoreClient(hiveConf);
-    driver = DriverFactory.newDriver(hiveConf);
-
-    DummyListener.notifyList.clear();
-  }
-
-  public void testEventStatus() throws Exception {
-    int listSize = 0;
-    List<ListenerEvent> notifyList = DummyListener.notifyList;
-    assertEquals(notifyList.size(), listSize);
-
-    driver.run("CREATE DATABASE tmpDb");
-    listSize += 1;
-    notifyList = DummyListener.notifyList;
-    assertEquals(notifyList.size(), listSize);
-    assertTrue(DummyListener.getLastEvent().getStatus());
-
-    driver.run("CREATE TABLE unittest_TestMetaStoreEventListenerOnlyOnCommit (id INT) " +
-                "PARTITIONED BY (ds STRING)");
-    listSize += 1;
-    notifyList = DummyListener.notifyList;
-    assertEquals(notifyList.size(), listSize);
-    assertTrue(DummyListener.getLastEvent().getStatus());
-
-    driver.run("ALTER TABLE unittest_TestMetaStoreEventListenerOnlyOnCommit " +
-                "ADD PARTITION(ds='foo1')");
-    listSize += 1;
-    notifyList = DummyListener.notifyList;
-    assertEquals(notifyList.size(), listSize);
-    assertTrue(DummyListener.getLastEvent().getStatus());
-
-    DummyRawStoreControlledCommit.setCommitSucceed(false);
-
-    driver.run("ALTER TABLE unittest_TestMetaStoreEventListenerOnlyOnCommit " +
-                "ADD PARTITION(ds='foo2')");
-    listSize += 1;
-    notifyList = DummyListener.notifyList;
-    assertEquals(notifyList.size(), listSize);
-    assertFalse(DummyListener.getLastEvent().getStatus());
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/d9801d9c/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreInitListener.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreInitListener.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreInitListener.java
deleted file mode 100644
index 025cc40..0000000
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreInitListener.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hive.metastore;
-
-import org.apache.hadoop.hive.cli.CliSessionState;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.ql.DriverFactory;
-import org.apache.hadoop.hive.ql.IDriver;
-import org.apache.hadoop.hive.ql.session.SessionState;
-
-import junit.framework.TestCase;
-
-/**
- * TestMetaStoreInitListener. Test case for
- * {@link org.apache.hadoop.hive.metastore.MetaStoreInitListener}
- */
-public class TestMetaStoreInitListener extends TestCase {
-  private HiveConf hiveConf;
-  private HiveMetaStoreClient msc;
-  private IDriver driver;
-
-  @Override
-  protected void setUp() throws Exception {
-
-    super.setUp();
-    System.setProperty("hive.metastore.init.hooks",
-        DummyMetaStoreInitListener.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.set(HiveConf.ConfVars.PREEXECHOOKS.varname, "");
-    hiveConf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, "");
-    hiveConf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false");
-    SessionState.start(new CliSessionState(hiveConf));
-    msc = new HiveMetaStoreClient(hiveConf);
-    driver = DriverFactory.newDriver(hiveConf);
-  }
-
-  @Override
-  protected void tearDown() throws Exception {
-    super.tearDown();
-  }
-
-  public void testMetaStoreInitListener() throws Exception {
-    // DummyMataStoreInitListener's onInit will be called at HMSHandler
-    // initialization, and set this to true
-    assertTrue(DummyMetaStoreInitListener.wasCalled);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/d9801d9c/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreListenersError.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreListenersError.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreListenersError.java
deleted file mode 100644
index 99b67bb..0000000
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreListenersError.java
+++ /dev/null
@@ -1,85 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hive.metastore;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.metastore.api.MetaException;
-import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge;
-
-import junit.framework.Assert;
-import junit.framework.TestCase;
-
-/**
- * Test for unwrapping InvocationTargetException, which is thrown from
- * constructor of listener class
- */
-public class TestMetaStoreListenersError extends TestCase {
-
-  public void testInitListenerException() throws Throwable {
-
-    System.setProperty("hive.metastore.init.hooks", ErrorInitListener.class.getName());
-    int port = MetaStoreTestUtils.findFreePort();
-    try {
-      HiveMetaStore.startMetaStore(port, HadoopThriftAuthBridge.getBridge());
-    } catch (Throwable throwable) {
-      Assert.assertEquals(MetaException.class, throwable.getClass());
-      Assert.assertEquals(
-          "Failed to instantiate listener named: " +
-              "org.apache.hadoop.hive.metastore.TestMetaStoreListenersError$ErrorInitListener, " +
-              "reason: java.lang.IllegalArgumentException: exception on constructor",
-          throwable.getMessage());
-    }
-  }
-
-  public void testEventListenerException() throws Throwable {
-
-    System.setProperty("hive.metastore.init.hooks", "");
-    System.setProperty("hive.metastore.event.listeners", ErrorEventListener.class.getName());
-    int port = MetaStoreTestUtils.findFreePort();
-    try {
-      HiveMetaStore.startMetaStore(port, HadoopThriftAuthBridge.getBridge());
-    } catch (Throwable throwable) {
-      Assert.assertEquals(MetaException.class, throwable.getClass());
-      Assert.assertEquals(
-          "Failed to instantiate listener named: " +
-              "org.apache.hadoop.hive.metastore.TestMetaStoreListenersError$ErrorEventListener, " +
-              "reason: java.lang.IllegalArgumentException: exception on constructor",
-          throwable.getMessage());
-    }
-  }
-
-  public static class ErrorInitListener extends MetaStoreInitListener {
-
-    public ErrorInitListener(Configuration config) {
-      super(config);
-      throw new IllegalArgumentException("exception on constructor");
-    }
-
-    public void onInit(MetaStoreInitContext context) throws MetaException {
-    }
-  }
-
-  public static class ErrorEventListener extends MetaStoreEventListener {
-
-    public ErrorEventListener(Configuration config) {
-      super(config);
-      throw new IllegalArgumentException("exception on constructor");
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/d9801d9c/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStoreInitRetry.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStoreInitRetry.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStoreInitRetry.java
deleted file mode 100644
index 1695bfd..0000000
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStoreInitRetry.java
+++ /dev/null
@@ -1,127 +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 static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-
-import java.sql.DriverManager;
-import java.sql.SQLException;
-
-import javax.jdo.JDOCanRetryException;
-
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-
-public class TestObjectStoreInitRetry {
-
-  private static boolean noisy = true; // switch to true to see line number debug traces for FakeDerby calls
-
-  private static int injectConnectFailure = 0;
-
-  public static void setInjectConnectFailure(int x){
-    injectConnectFailure = x;
-  }
-
-  public static int getInjectConnectFailure(){
-    return injectConnectFailure;
-  }
-
-  public static void decrementInjectConnectFailure(){
-    injectConnectFailure--;
-  }
-
-  @BeforeClass
-  public static void oneTimeSetup() throws SQLException {
-    // dummy instantiation to make sure any static/ctor code blocks of that
-    // driver are loaded and ready to go.
-    DriverManager.registerDriver(new FakeDerby());
-  }
-
-  @AfterClass
-  public static void oneTimeTearDown() throws SQLException {
-    DriverManager.deregisterDriver(new FakeDerby());
-  }
-
-  public static void misbehave() throws RuntimeException{
-    TestObjectStoreInitRetry.debugTrace();
-    if (TestObjectStoreInitRetry.getInjectConnectFailure() > 0){
-      TestObjectStoreInitRetry.decrementInjectConnectFailure();
-      RuntimeException re = new JDOCanRetryException();
-      if (noisy){
-        System.err.println("MISBEHAVE:" + TestObjectStoreInitRetry.getInjectConnectFailure());
-        re.printStackTrace(System.err);
-      }
-      throw re;
-    }
-  }
-
-  // debug instrumenter - useful in finding which fns get called, and how often
-  public static void debugTrace() {
-    if (noisy){
-      Exception e = new Exception();
-      System.err.println("." + e.getStackTrace()[1].getLineNumber() + ":" + TestObjectStoreInitRetry.getInjectConnectFailure());
-    }
-  }
-
-  protected static HiveConf hiveConf;
-
-  @Test
-  public void testObjStoreRetry() throws Exception {
-    hiveConf = new HiveConf(this.getClass());
-
-    hiveConf.setIntVar(ConfVars.HMSHANDLERATTEMPTS, 4);
-    hiveConf.setVar(ConfVars.HMSHANDLERINTERVAL, "1s");
-    hiveConf.setVar(ConfVars.METASTORE_CONNECTION_DRIVER,FakeDerby.class.getName());
-    hiveConf.setBoolVar(ConfVars.METASTORE_TRY_DIRECT_SQL,true);
-    String jdbcUrl = hiveConf.get(ConfVars.METASTORECONNECTURLKEY.varname);
-    jdbcUrl = jdbcUrl.replace("derby","fderby");
-    hiveConf.setVar(ConfVars.METASTORECONNECTURLKEY,jdbcUrl);
-
-    ObjectStore objStore = new ObjectStore();
-
-    Exception savE = null;
-    try {
-      setInjectConnectFailure(5);
-      objStore.setConf(hiveConf);
-    } catch (Exception e) {
-      e.printStackTrace(System.err);
-      savE = e;
-    }
-
-    /**
-     * A note on retries.
-     *
-     * We've configured a total of 4 attempts.
-     * 5 - 4 == 1 connect failure simulation count left after this.
-     */
-
-    assertEquals(1, getInjectConnectFailure());
-    assertNotNull(savE);
-
-    setInjectConnectFailure(0);
-    objStore.setConf(hiveConf);
-    assertEquals(0, getInjectConnectFailure());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/d9801d9c/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestPartitionNameWhitelistValidation.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestPartitionNameWhitelistValidation.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestPartitionNameWhitelistValidation.java
deleted file mode 100644
index e3e175b..0000000
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestPartitionNameWhitelistValidation.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 static org.junit.Assert.*;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.hadoop.hive.cli.CliSessionState;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.ql.session.SessionState;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-// Validate the metastore client call validatePartitionNameCharacters to ensure it throws
-// an exception if partition fields contain Unicode characters or commas
-
-public class TestPartitionNameWhitelistValidation {
-
-  private static final String partitionValidationPattern = "[\\x20-\\x7E&&[^,]]*";
-  private static HiveConf hiveConf;
-  private static HiveMetaStoreClient msc;
-
-  @BeforeClass
-  public static void setupBeforeClass() throws Exception {
-    System.setProperty(HiveConf.ConfVars.METASTORE_PARTITION_NAME_WHITELIST_PATTERN.varname,
-        partitionValidationPattern);
-    hiveConf = new HiveConf();
-    SessionState.start(new CliSessionState(hiveConf));
-    msc = new HiveMetaStoreClient(hiveConf);
-  }
-
-  // Runs an instance of DisallowUnicodePreEventListener
-  // Returns whether or not it succeeded
-  private boolean runValidation(List<String> partVals) {
-    try {
-      msc.validatePartitionNameCharacters(partVals);
-    } catch (Exception e) {
-      return false;
-    }
-
-    return true;
- }
-
-  // Sample data
-  private List<String> getPartValsWithUnicode() {
-    List<String> partVals = new ArrayList<String>();
-    partVals.add("klâwen");
-    partVals.add("tägelîch");
-
-    return partVals;
-  }
-
-  private List<String> getPartValsWithCommas() {
-    List<String> partVals = new ArrayList<String>();
-    partVals.add("a,b");
-    partVals.add("c,d,e,f");
-
-    return partVals;
-  }
-
-  private List<String> getPartValsWithValidCharacters() {
-    List<String> partVals = new ArrayList<String>();
-    partVals.add("part1");
-    partVals.add("part2");
-
-    return partVals;
-  }
-
-  @Test
-  public void testAddPartitionWithCommas() {
-    assertFalse("Add a partition with commas in name",
-        runValidation(getPartValsWithCommas()));
-  }
-
-  @Test
-  public void testAddPartitionWithUnicode() {
-    assertFalse("Add a partition with unicode characters in name",
-        runValidation(getPartValsWithUnicode()));
-  }
-
-  @Test
-  public void testAddPartitionWithValidPartVal() {
-    assertTrue("Add a partition with unicode characters in name",
-        runValidation(getPartValsWithValidCharacters()));
-  }
-
-  @Test
-  public void testAppendPartitionWithUnicode() {
-    assertFalse("Append a partition with unicode characters in name",
-        runValidation(getPartValsWithUnicode()));
-  }
-
-  @Test
-  public void testAppendPartitionWithCommas() {
-    assertFalse("Append a partition with unicode characters in name",
-        runValidation(getPartValsWithCommas()));
-  }
-
-  @Test
-  public void testAppendPartitionWithValidCharacters() {
-    assertTrue("Append a partition with no unicode characters in name",
-        runValidation(getPartValsWithValidCharacters()));
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/d9801d9c/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestRemoteHiveMetaStore.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestRemoteHiveMetaStore.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestRemoteHiveMetaStore.java
deleted file mode 100644
index ec84e66..0000000
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestRemoteHiveMetaStore.java
+++ /dev/null
@@ -1,60 +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;
-import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
-import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge;
-import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
-
-
-public class TestRemoteHiveMetaStore extends TestHiveMetaStore {
-  private static boolean isServerStarted = false;
-  protected static int port;
-
-  public TestRemoteHiveMetaStore() {
-    super();
-    isThriftClient = true;
-  }
-
-  @Override
-  protected void setUp() throws Exception {
-    super.setUp();
-
-    if (isServerStarted) {
-      assertNotNull("Unable to connect to the MetaStore server", client);
-      hiveConf.setVar(HiveConf.ConfVars.METASTOREURIS, "thrift://localhost:" + port);
-      return;
-    }
-
-    port = MetaStoreTestUtils.startMetaStoreWithRetry(hiveConf);
-    System.out.println("Starting MetaStore Server on port " + port);
-    isServerStarted = true;
-
-    // This is default case with setugi off for both client and server
-    client = createClient();
-  }
-
-  @Override
-  protected HiveMetaStoreClient createClient() throws Exception {
-    hiveConf.setVar(HiveConf.ConfVars.METASTOREURIS, "thrift://localhost:" + port);
-    hiveConf.setBoolVar(ConfVars.METASTORE_EXECUTE_SET_UGI, false);
-    return new HiveMetaStoreClient(hiveConf);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/d9801d9c/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestRemoteHiveMetaStoreIpAddress.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestRemoteHiveMetaStoreIpAddress.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestRemoteHiveMetaStoreIpAddress.java
deleted file mode 100644
index c7c35f3..0000000
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestRemoteHiveMetaStoreIpAddress.java
+++ /dev/null
@@ -1,80 +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 junit.framework.TestCase;
-
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
-import org.apache.hadoop.hive.metastore.api.Database;
-import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge;
-import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
-import org.apache.hadoop.util.StringUtils;
-
-/**
- *
- * TestRemoteHiveMetaStoreIpAddress.
- *
- * Test which checks that the remote Hive metastore stores the proper IP address using
- * IpAddressListener
- */
-public class TestRemoteHiveMetaStoreIpAddress extends TestCase {
-  private static boolean isServerStarted = false;
-  private static HiveConf hiveConf;
-  private static HiveMetaStoreClient msc;
-
-  @Override
-  protected void setUp() throws Exception {
-    super.setUp();
-    hiveConf = new HiveConf(this.getClass());
-
-    if (isServerStarted) {
-      assertNotNull("Unable to connect to the MetaStore server", msc);
-      return;
-    }
-
-    System.setProperty(ConfVars.METASTORE_EVENT_LISTENERS.varname,
-        IpAddressListener.class.getName());
-    int port = MetaStoreTestUtils.startMetaStoreWithRetry();
-    System.out.println("Started MetaStore Server on port " + port);
-    isServerStarted = true;
-
-    // This is default case with setugi off for both client and server
-    createClient(port);
-  }
-
-  public void testIpAddress() throws Exception {
-    try {
-
-      Database db = new Database();
-      db.setName("testIpAddressIp");
-      msc.createDatabase(db);
-      msc.dropDatabase(db.getName());
-    } catch (Exception e) {
-      System.err.println(StringUtils.stringifyException(e));
-      System.err.println("testIpAddress() failed.");
-      throw e;
-    }
-  }
-
-  protected void createClient(int port) throws Exception {
-    hiveConf.setVar(HiveConf.ConfVars.METASTOREURIS, "thrift://localhost:" + port);
-    msc = new HiveMetaStoreClient(hiveConf);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/d9801d9c/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestRemoteUGIHiveMetaStoreIpAddress.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestRemoteUGIHiveMetaStoreIpAddress.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestRemoteUGIHiveMetaStoreIpAddress.java
deleted file mode 100644
index 8658262..0000000
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestRemoteUGIHiveMetaStoreIpAddress.java
+++ /dev/null
@@ -1,28 +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 TestRemoteUGIHiveMetaStoreIpAddress extends TestRemoteHiveMetaStoreIpAddress {
-  public TestRemoteUGIHiveMetaStoreIpAddress() {
-    super();
-    System.setProperty(ConfVars.METASTORE_EXECUTE_SET_UGI.varname, "true");
-  }
-
-}


[4/7] hive git commit: HIVE-17982 Move metastore specific itests. This closes #279. (Alan Gates, reviewed by Peter Vary)

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/d9801d9c/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/d9801d9c/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/d9801d9c/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/d9801d9c/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/d9801d9c/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/d9801d9c/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/d9801d9c/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/d9801d9c/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/d9801d9c/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 00f3c8d..3c8d005 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
@@ -485,7 +485,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/d9801d9c/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/d9801d9c/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..d4820b3 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;
@@ -51,6 +52,7 @@ public class MetaStoreTestUtils {
     return startMetaStore(HadoopThriftAuthBridge.getBridge(), conf);
   }
 
+
   public static void startMetaStore(final int port, final HadoopThriftAuthBridge bridge) throws Exception {
     MetaStoreTestUtils.startMetaStore(port, bridge, null);
   }
@@ -91,19 +93,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;
+  }
 
+  private 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 +203,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/d9801d9c/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/d9801d9c/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/d9801d9c/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..2104636
--- /dev/null
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestFilterHooks.java
@@ -0,0 +1,302 @@
+/*
+ * 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 org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Lists;
+
+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.startMetaStoreWithRetry(HadoopThriftAuthBridge.getBridge(), conf);
+    MetastoreConf.setVar(conf, ConfVars.THRIFT_URIS, "thrift://localhost:" + port);
+
+    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
+    }
+  }
+
+}