You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ai...@apache.org on 2018/02/13 21:13:12 UTC

[1/2] hive git commit: HIVE-17735: ObjectStore.addNotificationEvent is leaking queries (Aihua Xu, reviewed by Yongzhi Chen)

Repository: hive
Updated Branches:
  refs/heads/master 8cf36e733 -> 35605732b


HIVE-17735: ObjectStore.addNotificationEvent is leaking queries (Aihua Xu, reviewed by Yongzhi Chen)


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

Branch: refs/heads/master
Commit: ec7ccc3a452fa125719ca820b5f751ddd00686ec
Parents: 8cf36e7
Author: Aihua Xu <ai...@apache.org>
Authored: Mon Feb 5 15:35:30 2018 -0800
Committer: Aihua Xu <ai...@apache.org>
Committed: Tue Feb 13 13:04:15 2018 -0800

----------------------------------------------------------------------
 .../hadoop/hive/metastore/ObjectStore.java      | 42 +++++++-------------
 .../hadoop/hive/metastore/TestObjectStore.java  |  2 +-
 2 files changed, 15 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/ec7ccc3a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
index d58ed67..edabaa1 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
@@ -3941,13 +3941,13 @@ public class ObjectStore implements RawStore, Configurable {
     }
 
     boolean success = false;
-    QueryWrapper queryWrapper = new QueryWrapper();
+    Query query = null;
 
     try {
       openTransaction();
       LOG.debug("execute removeUnusedColumnDescriptor");
 
-      Query query = pm.newQuery("select count(1) from " +
+      query = pm.newQuery("select count(1) from " +
         "org.apache.hadoop.hive.metastore.model.MStorageDescriptor where (this.cd == inCD)");
       query.declareParameters("MColumnDescriptor inCD");
       long count = ((Long)query.execute(oldCD)).longValue();
@@ -3960,7 +3960,7 @@ public class ObjectStore implements RawStore, Configurable {
       success = commitTransaction();
       LOG.debug("successfully deleted a CD in removeUnusedColumnDescriptor");
     } finally {
-      rollbackAndCleanup(success, queryWrapper);
+      rollbackAndCleanup(success, query);
     }
   }
 
@@ -8819,14 +8819,13 @@ public class ObjectStore implements RawStore, Configurable {
   public Function getFunction(String dbName, String funcName) throws MetaException {
     boolean commited = false;
     Function func = null;
+    Query query = null;
     try {
       openTransaction();
       func = convertToFunction(getMFunction(dbName, funcName));
       commited = commitTransaction();
     } finally {
-      if (!commited) {
-        rollbackTransaction();
-      }
+      rollbackAndCleanup(commited, query);
     }
     return func;
   }
@@ -8834,17 +8833,16 @@ public class ObjectStore implements RawStore, Configurable {
   @Override
   public List<Function> getAllFunctions() throws MetaException {
     boolean commited = false;
+    Query query = null;
     try {
       openTransaction();
-      Query query = pm.newQuery(MFunction.class);
+      query = pm.newQuery(MFunction.class);
       List<MFunction> allFunctions = (List<MFunction>) query.execute();
       pm.retrieveAll(allFunctions);
       commited = commitTransaction();
       return convertToFunctions(allFunctions);
     } finally {
-      if (!commited) {
-        rollbackTransaction();
-      }
+      rollbackAndCleanup(commited, query);
     }
   }
 
@@ -8905,10 +8903,7 @@ public class ObjectStore implements RawStore, Configurable {
       }
       return result;
     } finally {
-      if (!commited) {
-        rollbackAndCleanup(commited, query);
-        return null;
-      }
+      rollbackAndCleanup(commited, query);
     }
   }
 
@@ -8938,6 +8933,7 @@ public class ObjectStore implements RawStore, Configurable {
       query.setUnique(true);
       // only need to execute it to get db Lock
       query.execute();
+      query.closeAll();
     }).run();
   }
 
@@ -9003,8 +8999,8 @@ public class ObjectStore implements RawStore, Configurable {
     try {
       openTransaction();
       lockForUpdate();
-      Query objectQuery = pm.newQuery(MNotificationNextId.class);
-      Collection<MNotificationNextId> ids = (Collection) objectQuery.execute();
+      query = pm.newQuery(MNotificationNextId.class);
+      Collection<MNotificationNextId> ids = (Collection) query.execute();
       MNotificationNextId mNotificationNextId = null;
       boolean needToPersistId;
       if (CollectionUtils.isEmpty(ids)) {
@@ -9533,12 +9529,7 @@ public class ObjectStore implements RawStore, Configurable {
       }
       commited = commitTransaction();
     } finally {
-      if (!commited) {
-        rollbackTransaction();
-      }
-      if (query != null) {
-        query.closeAll();
-      }
+      rollbackAndCleanup(commited, query);
     }
     return uniqueConstraints;
   }
@@ -9603,12 +9594,7 @@ public class ObjectStore implements RawStore, Configurable {
       }
       commited = commitTransaction();
     } finally {
-      if (!commited) {
-        rollbackTransaction();
-      }
-      if (query != null) {
-        query.closeAll();
-      }
+      rollbackAndCleanup(commited, query);
     }
     return notNullConstraints;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/ec7ccc3a/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 372dee6..ee5b3e0 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
@@ -442,7 +442,7 @@ public class TestObjectStore {
     spy.getAllFunctions();
     spy.getAllTables(DB1);
     spy.getPartitionCount();
-    Mockito.verify(spy, Mockito.times(2))
+    Mockito.verify(spy, Mockito.times(3))
         .rollbackAndCleanup(Mockito.anyBoolean(), Mockito.<Query>anyObject());
   }
 


[2/2] hive git commit: HIVE-18586: Upgrade Derby to 10.14.1.0 (Janaki Lahorani, reviewed by Aihua Xu)

Posted by ai...@apache.org.
HIVE-18586: Upgrade Derby to 10.14.1.0 (Janaki Lahorani, reviewed by Aihua Xu)


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

Branch: refs/heads/master
Commit: 35605732b2041eee809485718bfd951cdfae0980
Parents: ec7ccc3
Author: Aihua Xu <ai...@apache.org>
Authored: Tue Feb 13 13:06:31 2018 -0800
Committer: Aihua Xu <ai...@apache.org>
Committed: Tue Feb 13 13:06:31 2018 -0800

----------------------------------------------------------------------
 .../org/apache/hive/hcatalog/DerbyPolicy.java   | 90 ++++++++++++++++++++
 .../org/apache/hive/hcatalog/DerbyPolicy.java   | 90 ++++++++++++++++++++
 .../apache/hive/hcatalog/cli/TestPermsGrp.java  |  3 +
 .../mapreduce/TestHCatPartitionPublish.java     |  3 +
 .../org/apache/hive/hcatalog/package-info.java  | 22 +++++
 .../hive/hcatalog/api/TestHCatClient.java       |  4 +
 pom.xml                                         |  2 +-
 .../metastore/TestHiveMetaStoreGetMetaConf.java | 25 ------
 .../TestHiveMetaStorePartitionSpecs.java        | 26 ------
 9 files changed, 213 insertions(+), 52 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/35605732/core/src/test/java/org/apache/hive/hcatalog/DerbyPolicy.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/hive/hcatalog/DerbyPolicy.java b/core/src/test/java/org/apache/hive/hcatalog/DerbyPolicy.java
new file mode 100644
index 0000000..cecf6dc
--- /dev/null
+++ b/core/src/test/java/org/apache/hive/hcatalog/DerbyPolicy.java
@@ -0,0 +1,90 @@
+/*
+ * 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.hive.hcatalog;
+
+import org.apache.derby.security.SystemPermission;
+
+import java.security.CodeSource;
+import java.security.Permission;
+import java.security.PermissionCollection;
+import java.security.Policy;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Enumeration;
+import java.util.Iterator;
+
+/**
+ * A security policy that grants usederbyinternals
+ *
+ * <p>
+ *   HCatalog tests use Security Manager to handle exits.  With Derby version 10.14.1, if a
+ *   security manager is configured, embedded Derby requires usederbyinternals permission, and
+ *   that is checked directly using AccessController.checkPermission.  This class will be used to
+ *   setup a security policy to grant usederbyinternals, in tests that use NoExitSecurityManager.
+ * </p>
+ */
+public class DerbyPolicy extends Policy {
+
+  private static PermissionCollection perms;
+
+  public DerbyPolicy() {
+    super();
+    if (perms == null) {
+      perms = new DerbyPermissionCollection();
+      addPermissions();
+    }
+  }
+
+  @Override
+  public PermissionCollection getPermissions(CodeSource codesource) {
+    return perms;
+  }
+
+  private void addPermissions() {
+    SystemPermission systemPermission = new SystemPermission("engine", "usederbyinternals");
+    perms.add(systemPermission);
+  }
+
+  class DerbyPermissionCollection extends PermissionCollection {
+
+    ArrayList<Permission> perms = new ArrayList<Permission>();
+
+    public void add(Permission p) {
+      perms.add(p);
+    }
+
+    public boolean implies(Permission p) {
+      for (Iterator<Permission> i = perms.iterator(); i.hasNext();) {
+        if (((Permission) i.next()).implies(p)) {
+          return true;
+        }
+      }
+      return false;
+    }
+
+    public Enumeration<Permission> elements() {
+      return Collections.enumeration(perms);
+    }
+
+    public boolean isReadOnly() {
+      return false;
+    }
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/35605732/hcatalog/core/src/test/java/org/apache/hive/hcatalog/DerbyPolicy.java
----------------------------------------------------------------------
diff --git a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/DerbyPolicy.java b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/DerbyPolicy.java
new file mode 100644
index 0000000..cecf6dc
--- /dev/null
+++ b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/DerbyPolicy.java
@@ -0,0 +1,90 @@
+/*
+ * 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.hive.hcatalog;
+
+import org.apache.derby.security.SystemPermission;
+
+import java.security.CodeSource;
+import java.security.Permission;
+import java.security.PermissionCollection;
+import java.security.Policy;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Enumeration;
+import java.util.Iterator;
+
+/**
+ * A security policy that grants usederbyinternals
+ *
+ * <p>
+ *   HCatalog tests use Security Manager to handle exits.  With Derby version 10.14.1, if a
+ *   security manager is configured, embedded Derby requires usederbyinternals permission, and
+ *   that is checked directly using AccessController.checkPermission.  This class will be used to
+ *   setup a security policy to grant usederbyinternals, in tests that use NoExitSecurityManager.
+ * </p>
+ */
+public class DerbyPolicy extends Policy {
+
+  private static PermissionCollection perms;
+
+  public DerbyPolicy() {
+    super();
+    if (perms == null) {
+      perms = new DerbyPermissionCollection();
+      addPermissions();
+    }
+  }
+
+  @Override
+  public PermissionCollection getPermissions(CodeSource codesource) {
+    return perms;
+  }
+
+  private void addPermissions() {
+    SystemPermission systemPermission = new SystemPermission("engine", "usederbyinternals");
+    perms.add(systemPermission);
+  }
+
+  class DerbyPermissionCollection extends PermissionCollection {
+
+    ArrayList<Permission> perms = new ArrayList<Permission>();
+
+    public void add(Permission p) {
+      perms.add(p);
+    }
+
+    public boolean implies(Permission p) {
+      for (Iterator<Permission> i = perms.iterator(); i.hasNext();) {
+        if (((Permission) i.next()).implies(p)) {
+          return true;
+        }
+      }
+      return false;
+    }
+
+    public Enumeration<Permission> elements() {
+      return Collections.enumeration(perms);
+    }
+
+    public boolean isReadOnly() {
+      return false;
+    }
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/35605732/hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/TestPermsGrp.java
----------------------------------------------------------------------
diff --git a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/TestPermsGrp.java b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/TestPermsGrp.java
index d78ab78..a686579 100644
--- a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/TestPermsGrp.java
+++ b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/TestPermsGrp.java
@@ -19,6 +19,7 @@
 package org.apache.hive.hcatalog.cli;
 
 import java.io.FileNotFoundException;
+import java.security.Policy;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
@@ -47,6 +48,7 @@ import org.apache.hadoop.hive.ql.io.HiveOutputFormat;
 import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe;
+import org.apache.hive.hcatalog.DerbyPolicy;
 import org.apache.hive.hcatalog.ExitException;
 import org.apache.hive.hcatalog.NoExitSecurityManager;
 import org.apache.hive.hcatalog.cli.SemanticAnalysis.HCatSemanticAnalyzer;
@@ -84,6 +86,7 @@ public class TestPermsGrp extends TestCase {
 
     securityManager = System.getSecurityManager();
     System.setSecurityManager(new NoExitSecurityManager());
+    Policy.setPolicy(new DerbyPolicy());
 
     hcatConf = new HiveConf(this.getClass());
     hcatConf.setVar(HiveConf.ConfVars.METASTOREURIS, "thrift://127.0.0.1:" + msPort);

http://git-wip-us.apache.org/repos/asf/hive/blob/35605732/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatPartitionPublish.java
----------------------------------------------------------------------
diff --git a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatPartitionPublish.java b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatPartitionPublish.java
index 7205a79..fb6a7f4 100644
--- a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatPartitionPublish.java
+++ b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatPartitionPublish.java
@@ -20,6 +20,7 @@ package org.apache.hive.hcatalog.mapreduce;
 
 import java.io.File;
 import java.io.IOException;
+import java.security.Policy;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
@@ -55,6 +56,7 @@ import org.apache.hadoop.mapred.MiniMRCluster;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
+import org.apache.hive.hcatalog.DerbyPolicy;
 import org.apache.hive.hcatalog.NoExitSecurityManager;
 import org.apache.hive.hcatalog.cli.SemanticAnalysis.HCatSemanticAnalyzer;
 import org.apache.hive.hcatalog.data.DefaultHCatRecord;
@@ -110,6 +112,7 @@ public class TestHCatPartitionPublish {
     isServerRunning = true;
     securityManager = System.getSecurityManager();
     System.setSecurityManager(new NoExitSecurityManager());
+    Policy.setPolicy(new DerbyPolicy());
 
     hcatConf = new HiveConf(TestHCatPartitionPublish.class);
     hcatConf.setVar(HiveConf.ConfVars.METASTOREURIS, "thrift://localhost:"

http://git-wip-us.apache.org/repos/asf/hive/blob/35605732/hcatalog/core/src/test/java/org/apache/hive/hcatalog/package-info.java
----------------------------------------------------------------------
diff --git a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/package-info.java b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/package-info.java
new file mode 100644
index 0000000..a1c3043
--- /dev/null
+++ b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * A package info file for hcatalog tests.
+ */
+package org.apache.hive.hcatalog;

http://git-wip-us.apache.org/repos/asf/hive/blob/35605732/hcatalog/webhcat/java-client/src/test/java/org/apache/hive/hcatalog/api/TestHCatClient.java
----------------------------------------------------------------------
diff --git a/hcatalog/webhcat/java-client/src/test/java/org/apache/hive/hcatalog/api/TestHCatClient.java b/hcatalog/webhcat/java-client/src/test/java/org/apache/hive/hcatalog/api/TestHCatClient.java
index 2a36c85..515eef2 100644
--- a/hcatalog/webhcat/java-client/src/test/java/org/apache/hive/hcatalog/api/TestHCatClient.java
+++ b/hcatalog/webhcat/java-client/src/test/java/org/apache/hive/hcatalog/api/TestHCatClient.java
@@ -20,6 +20,7 @@ package org.apache.hive.hcatalog.api;
 
 import java.io.IOException;
 import java.math.BigInteger;
+import java.security.Policy;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.EnumSet;
@@ -52,6 +53,7 @@ import org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe;
 import org.apache.hadoop.hive.shims.Utils;
 import org.apache.hadoop.mapred.TextInputFormat;
 import org.apache.hadoop.security.authorize.ProxyUsers;
+import org.apache.hive.hcatalog.DerbyPolicy;
 import org.apache.hive.hcatalog.api.repl.Command;
 import org.apache.hive.hcatalog.api.repl.ReplicationTask;
 import org.apache.hive.hcatalog.api.repl.ReplicationUtils;
@@ -121,6 +123,8 @@ public class TestHCatClient {
     msPort = MetaStoreTestUtils.startMetaStore();
     securityManager = System.getSecurityManager();
     System.setSecurityManager(new NoExitSecurityManager());
+    Policy.setPolicy(new DerbyPolicy());
+
     hcatConf.setVar(HiveConf.ConfVars.METASTOREURIS, "thrift://localhost:"
       + msPort);
     hcatConf.setIntVar(HiveConf.ConfVars.METASTORETHRIFTCONNECTIONRETRIES, 3);

http://git-wip-us.apache.org/repos/asf/hive/blob/35605732/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 5ae63da..e220891 100644
--- a/pom.xml
+++ b/pom.xml
@@ -137,7 +137,7 @@
     <commons-lang3.version>3.2</commons-lang3.version>
     <commons-pool.version>1.5.4</commons-pool.version>
     <commons-dbcp.version>1.4</commons-dbcp.version>
-    <derby.version>10.11.1.1</derby.version>
+    <derby.version>10.14.1.0</derby.version>
     <dropwizard.version>3.1.0</dropwizard.version>
     <dropwizard-metrics-hadoop-metrics2-reporter.version>0.1.2</dropwizard-metrics-hadoop-metrics2-reporter.version>
     <druid.version>0.11.0</druid.version>

http://git-wip-us.apache.org/repos/asf/hive/blob/35605732/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreGetMetaConf.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreGetMetaConf.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreGetMetaConf.java
index ba86e05..14e5d75 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreGetMetaConf.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreGetMetaConf.java
@@ -18,8 +18,6 @@
 
 package org.apache.hadoop.hive.metastore;
 
-import java.security.Permission;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
@@ -45,40 +43,17 @@ public class TestHiveMetaStoreGetMetaConf {
 
   private static final Logger LOG = LoggerFactory.getLogger(TestHiveMetaStoreGetMetaConf.class);
   private static Configuration conf;
-  private static SecurityManager securityManager;
 
   private HiveMetaStoreClient hmsc;
 
-  public static class NoExitSecurityManager extends SecurityManager {
-
-    @Override
-    public void checkPermission(Permission perm) {
-      // allow anything.
-    }
-
-    @Override
-    public void checkPermission(Permission perm, Object context) {
-      // allow anything.
-    }
-
-    @Override
-    public void checkExit(int status) {
-      super.checkExit(status);
-      throw new RuntimeException("System.exit() was called. Raising exception.");
-    }
-  }
-
   @AfterClass
   public static void tearDown() throws Exception {
     LOG.info("Shutting down metastore.");
-    System.setSecurityManager(securityManager);
   }
 
   @BeforeClass
   public static void startMetaStoreServer() throws Exception {
 
-    securityManager = System.getSecurityManager();
-    System.setSecurityManager(new NoExitSecurityManager());
     Configuration metastoreConf = MetastoreConf.newMetastoreConf();
     MetastoreConf.setClass(metastoreConf, ConfVars.EXPRESSION_PROXY_CLASS,
       MockPartitionExpressionForMetastore.class, PartitionExpressionProxy.class);

http://git-wip-us.apache.org/repos/asf/hive/blob/35605732/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStorePartitionSpecs.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStorePartitionSpecs.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStorePartitionSpecs.java
index 57e5a41..843cf13 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStorePartitionSpecs.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStorePartitionSpecs.java
@@ -37,7 +37,6 @@ import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.security.Permission;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
@@ -52,33 +51,10 @@ public class TestHiveMetaStorePartitionSpecs {
   private static final Logger LOG = LoggerFactory.getLogger(TestHiveMetaStorePartitionSpecs.class);
   private static int msPort;
   private static Configuration conf;
-  private static SecurityManager securityManager;
-
-  public static class NoExitSecurityManager extends SecurityManager {
-
-    @Override
-    public void checkPermission(Permission perm) {
-      // allow anything.
-    }
-
-    @Override
-    public void checkPermission(Permission perm, Object context) {
-      // allow anything.
-    }
-
-    @Override
-    public void checkExit(int status) {
-
-      super.checkExit(status);
-      throw new RuntimeException("System.exit() was called. Raising exception. ");
-    }
-  }
-
 
   @AfterClass
   public static void tearDown() throws Exception {
     LOG.info("Shutting down metastore.");
-    System.setSecurityManager(securityManager);
 
     HiveMetaStoreClient hmsc = new HiveMetaStoreClient(conf);
     hmsc.dropDatabase(dbName, true, true, true);
@@ -92,8 +68,6 @@ public class TestHiveMetaStorePartitionSpecs {
       MockPartitionExpressionForMetastore.class, PartitionExpressionProxy.class);
     MetaStoreTestUtils.setConfForStandloneMode(metastoreConf);
     msPort = MetaStoreTestUtils.startMetaStore(metastoreConf);
-    securityManager = System.getSecurityManager();
-    System.setSecurityManager(new NoExitSecurityManager());
     conf = MetastoreConf.newMetastoreConf();
     MetastoreConf.setVar(conf, ConfVars.THRIFT_URIS, "thrift://localhost:" + msPort);
     MetastoreConf.setLongVar(conf, ConfVars.THRIFT_CONNECTION_RETRIES, 3);