You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by na...@apache.org on 2012/03/05 08:16:13 UTC

svn commit: r1296946 - in /hive/trunk/metastore/src: java/org/apache/hadoop/hive/metastore/ test/org/apache/hadoop/hive/metastore/

Author: namit
Date: Mon Mar  5 07:16:13 2012
New Revision: 1296946

URL: http://svn.apache.org/viewvc?rev=1296946&view=rev
Log:
HIVE-2833 Fix test failures caused by HIVE-2716
(Kevin Wilfong via namit)


Added:
    hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/DummyJdoConnectionUrlHook.java
    hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
    hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/TestMetaStoreConnectionUrlHook.java
Modified:
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/RetryingRawStore.java

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java?rev=1296946&r1=1296945&r2=1296946&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java Mon Mar  5 07:16:13 2012
@@ -241,8 +241,8 @@ public class HiveMetaStore extends Thrif
           "hive.metastore.checkForDefaultDb", true);
       String alterHandlerName = hiveConf.get("hive.metastore.alter.impl",
           HiveAlterHandler.class.getName());
-      alterHandler = (AlterHandler) ReflectionUtils.newInstance(getClass(
-          alterHandlerName, AlterHandler.class), hiveConf);
+      alterHandler = (AlterHandler) ReflectionUtils.newInstance(MetaStoreUtils.getClass(
+          alterHandlerName), hiveConf);
       wh = new Warehouse(hiveConf);
 
       createDefaultDB();
@@ -310,10 +310,8 @@ public class HiveMetaStore extends Thrif
       LOG.info(addPrefix("Opening raw store with implemenation class:"
           + rawStoreClassName));
       Configuration conf = getConf();
-      RawStore ms = (RawStore) ReflectionUtils.newInstance(getClass(rawStoreClassName,
-          RawStore.class), conf);
 
-      return RetryingRawStore.getProxy(hiveConf, conf, ms, threadLocalId.get());
+      return RetryingRawStore.getProxy(hiveConf, conf, rawStoreClassName, threadLocalId.get());
     }
 
     private void createDefaultDB_core(RawStore ms) throws MetaException, InvalidObjectException {
@@ -350,15 +348,6 @@ public class HiveMetaStore extends Thrif
 
     }
 
-    private Class<?> getClass(String rawStoreClassName, Class<?> class1)
-        throws MetaException {
-      try {
-        return Class.forName(rawStoreClassName, true, classLoader);
-      } catch (ClassNotFoundException e) {
-        throw new MetaException(rawStoreClassName + " class not found");
-      }
-    }
-
     private void logInfo(String m) {
       LOG.info(threadLocalId.get().toString() + ": " + m);
       logAuditEvent(m);

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java?rev=1296946&r1=1296945&r2=1296946&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java Mon Mar  5 07:16:13 2012
@@ -1015,4 +1015,13 @@ public class MetaStoreUtils {
       sd.setPrimaryRegionName(defaultRegionName);
     }
   }
+
+  public static Class<?> getClass(String rawStoreClassName)
+      throws MetaException {
+    try {
+      return Class.forName(rawStoreClassName, true, JavaUtils.getClassLoader());
+    } catch (ClassNotFoundException e) {
+      throw new MetaException(rawStoreClassName + " class not found");
+    }
+  }
 }

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/RetryingRawStore.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/RetryingRawStore.java?rev=1296946&r1=1296945&r2=1296946&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/RetryingRawStore.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/RetryingRawStore.java Mon Mar  5 07:16:13 2012
@@ -50,22 +50,28 @@ public class RetryingRawStore implements
   private final HiveConf hiveConf;
   private final Configuration conf; // thread local conf from HMS
 
-  protected RetryingRawStore(HiveConf hiveConf, Configuration conf, RawStore base, int id)
-      throws MetaException {
-    this.base = base;
+  protected RetryingRawStore(HiveConf hiveConf, Configuration conf,
+      Class<? extends RawStore> rawStoreClass, int id) throws MetaException {
     this.conf = conf;
     this.hiveConf = hiveConf;
     this.id = id;
+
+    // This has to be called before initializing the instance of RawStore
     init();
+
+    this.base = (RawStore) ReflectionUtils.newInstance(rawStoreClass, conf);
   }
 
-  public static RawStore getProxy(HiveConf hiveConf, Configuration conf, RawStore base, int id)
-      throws MetaException {
+  public static RawStore getProxy(HiveConf hiveConf, Configuration conf, String rawStoreClassName,
+      int id) throws MetaException {
+
+    Class<? extends RawStore> baseClass = (Class<? extends RawStore>) MetaStoreUtils.getClass(
+        rawStoreClassName);
 
-    RetryingRawStore handler = new RetryingRawStore(hiveConf, conf, base, id);
+    RetryingRawStore handler = new RetryingRawStore(hiveConf, conf, baseClass, id);
 
     return (RawStore) Proxy.newProxyInstance(RetryingRawStore.class.getClassLoader()
-        , base.getClass().getInterfaces(), handler);
+        , baseClass.getInterfaces(), handler);
   }
 
   private void init() throws MetaException {
@@ -74,9 +80,9 @@ public class RetryingRawStore implements
     retryLimit = HiveConf.getIntVar(hiveConf,
         HiveConf.ConfVars.METASTOREATTEMPTS);
     // Using the hook on startup ensures that the hook always has priority
-    // over settings in *.xml. We can use hiveConf as only a single thread
-    // will be calling the constructor.
-    updateConnectionURL(hiveConf, null);
+    // over settings in *.xml.  The thread local conf needs to be used because at this point
+    // it has already been initialized using hiveConf.
+    updateConnectionURL(getConf(), null);
   }
 
   private void initMS() {

Added: hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/DummyJdoConnectionUrlHook.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/DummyJdoConnectionUrlHook.java?rev=1296946&view=auto
==============================================================================
--- hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/DummyJdoConnectionUrlHook.java (added)
+++ hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/DummyJdoConnectionUrlHook.java Mon Mar  5 07:16:13 2012
@@ -0,0 +1,45 @@
+/**
+ * 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.hooks.JDOConnectionURLHook;
+
+/**
+ *
+ * DummyJdoConnectionUrlHook.
+ *
+ * An implementation of JDOConnectionURLHook which simply returns CORRECT_URL when
+ * getJdoConnectionUrl is called.
+ */
+public class DummyJdoConnectionUrlHook implements JDOConnectionURLHook {
+
+  public static final String initialUrl = "BAD_URL";
+  public static final String newUrl = "CORRECT_URL";
+
+  @Override
+  public String getJdoConnectionUrl(Configuration conf) throws Exception {
+    return newUrl;
+  }
+
+  @Override
+  public void notifyBadConnectionUrl(String url) {
+  }
+
+}

Added: hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java?rev=1296946&view=auto
==============================================================================
--- hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java (added)
+++ hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java Mon Mar  5 07:16:13 2012
@@ -0,0 +1,507 @@
+/**
+ * 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 java.util.Map;
+
+import junit.framework.Assert;
+
+import org.apache.hadoop.conf.Configuration;
+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.InvalidObjectException;
+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.Partition;
+import org.apache.hadoop.hive.metastore.api.PartitionEventType;
+import org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet;
+import org.apache.hadoop.hive.metastore.api.PrincipalType;
+import org.apache.hadoop.hive.metastore.api.PrivilegeBag;
+import org.apache.hadoop.hive.metastore.api.Role;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.Type;
+import org.apache.hadoop.hive.metastore.api.UnknownDBException;
+import org.apache.hadoop.hive.metastore.api.UnknownPartitionException;
+import org.apache.hadoop.hive.metastore.api.UnknownTableException;
+import org.apache.hadoop.hive.metastore.model.MDBPrivilege;
+import org.apache.hadoop.hive.metastore.model.MGlobalPrivilege;
+import org.apache.hadoop.hive.metastore.model.MPartitionColumnPrivilege;
+import org.apache.hadoop.hive.metastore.model.MPartitionPrivilege;
+import org.apache.hadoop.hive.metastore.model.MRoleMap;
+import org.apache.hadoop.hive.metastore.model.MTableColumnPrivilege;
+import org.apache.hadoop.hive.metastore.model.MTablePrivilege;
+
+/**
+ *
+ * DummyRawStoreForJdoConnection.
+ *
+ * An implementation of RawStore that verifies the DummyJdoConnectionUrlHook has already been
+ * applied when this class's setConf method is called, by checking that the value of the
+ * METASTORECONNECTURLKEY ConfVar has been updated.
+ *
+ * All non-void methods return default values.
+ */
+public class DummyRawStoreForJdoConnection implements RawStore {
+
+  @Override
+  public Configuration getConf() {
+
+    return null;
+  }
+
+  @Override
+  public void setConf(Configuration arg0) {
+    String expected = DummyJdoConnectionUrlHook.newUrl;
+    String actual = arg0.get(HiveConf.ConfVars.METASTORECONNECTURLKEY.varname);
+
+    Assert.assertEquals("The expected URL used by JDO to connect to the metastore: " + expected +
+        " did not match the actual value when the Raw Store was initialized: " + actual,
+        expected, actual);
+  }
+
+  @Override
+  public void shutdown() {
+
+
+  }
+
+  @Override
+  public boolean openTransaction() {
+
+    return false;
+  }
+
+  @Override
+  public boolean commitTransaction() {
+
+    return false;
+  }
+
+  @Override
+  public void rollbackTransaction() {
+
+
+  }
+
+  @Override
+  public void createDatabase(Database db) throws InvalidObjectException, MetaException {
+
+
+  }
+
+  @Override
+  public Database getDatabase(String name) throws NoSuchObjectException {
+
+    return null;
+  }
+
+  @Override
+  public boolean dropDatabase(String dbname) throws NoSuchObjectException, MetaException {
+
+    return false;
+  }
+
+  @Override
+  public boolean alterDatabase(String dbname, Database db) throws NoSuchObjectException,
+      MetaException {
+
+    return false;
+  }
+
+  @Override
+  public List<String> getDatabases(String pattern) throws MetaException {
+
+    return null;
+  }
+
+  @Override
+  public List<String> getAllDatabases() throws MetaException {
+
+    return null;
+  }
+
+  @Override
+  public boolean createType(Type type) {
+
+    return false;
+  }
+
+  @Override
+  public Type getType(String typeName) {
+
+    return null;
+  }
+
+  @Override
+  public boolean dropType(String typeName) {
+
+    return false;
+  }
+
+  @Override
+  public void createTable(Table tbl) throws InvalidObjectException, MetaException {
+
+
+  }
+
+  @Override
+  public boolean dropTable(String dbName, String tableName) throws MetaException {
+
+    return false;
+  }
+
+  @Override
+  public Table getTable(String dbName, String tableName) throws MetaException {
+
+    return null;
+  }
+
+  @Override
+  public boolean addPartition(Partition part) throws InvalidObjectException, MetaException {
+
+    return false;
+  }
+
+  @Override
+  public Partition getPartition(String dbName, String tableName, List<String> part_vals)
+      throws MetaException, NoSuchObjectException {
+
+    return null;
+  }
+
+  @Override
+  public boolean dropPartition(String dbName, String tableName, List<String> part_vals)
+      throws MetaException {
+
+    return false;
+  }
+
+  @Override
+  public List<Partition> getPartitions(String dbName, String tableName, int max)
+      throws MetaException {
+
+    return null;
+  }
+
+  @Override
+  public void alterTable(String dbname, String name, Table newTable) throws InvalidObjectException,
+      MetaException {
+
+
+  }
+
+  @Override
+  public List<String> getTables(String dbName, String pattern) throws MetaException {
+
+    return null;
+  }
+
+  @Override
+  public List<Table> getTableObjectsByName(String dbname, List<String> tableNames)
+      throws MetaException, UnknownDBException {
+
+    return null;
+  }
+
+  @Override
+  public List<String> getAllTables(String dbName) throws MetaException {
+
+    return null;
+  }
+
+  @Override
+  public List<String> listTableNamesByFilter(String dbName, String filter, short max_tables)
+      throws MetaException, UnknownDBException {
+
+    return null;
+  }
+
+  @Override
+  public List<String> listPartitionNames(String db_name, String tbl_name, short max_parts)
+      throws MetaException {
+
+    return null;
+  }
+
+  @Override
+  public List<String> listPartitionNamesByFilter(String db_name, String tbl_name, String filter,
+      short max_parts) throws MetaException {
+
+    return null;
+  }
+
+  @Override
+  public void alterPartition(String db_name, String tbl_name, List<String> part_vals,
+      Partition new_part) throws InvalidObjectException, MetaException {
+
+
+  }
+
+  @Override
+  public boolean addIndex(Index index) throws InvalidObjectException, MetaException {
+
+    return false;
+  }
+
+  @Override
+  public Index getIndex(String dbName, String origTableName, String indexName)
+      throws MetaException {
+
+    return null;
+  }
+
+  @Override
+  public boolean dropIndex(String dbName, String origTableName, String indexName)
+      throws MetaException {
+
+    return false;
+  }
+
+  @Override
+  public List<Index> getIndexes(String dbName, String origTableName, int max)
+      throws MetaException {
+
+    return null;
+  }
+
+  @Override
+  public List<String> listIndexNames(String dbName, String origTableName, short max)
+      throws MetaException {
+
+    return null;
+  }
+
+  @Override
+  public void alterIndex(String dbname, String baseTblName, String name, Index newIndex)
+      throws InvalidObjectException, MetaException {
+
+
+  }
+
+  @Override
+  public List<Partition> getPartitionsByFilter(String dbName, String tblName, String filter,
+      short maxParts) throws MetaException, NoSuchObjectException {
+
+    return null;
+  }
+
+  @Override
+  public List<Partition> getPartitionsByNames(String dbName, String tblName,
+      List<String> partNames) throws MetaException, NoSuchObjectException {
+
+    return null;
+  }
+
+  @Override
+  public Table markPartitionForEvent(String dbName, String tblName, Map<String, String> partVals,
+      PartitionEventType evtType) throws MetaException, UnknownTableException,
+      InvalidPartitionException, UnknownPartitionException {
+
+    return null;
+  }
+
+  @Override
+  public boolean isPartitionMarkedForEvent(String dbName, String tblName,
+      Map<String, String> partName, PartitionEventType evtType) throws MetaException,
+      UnknownTableException, InvalidPartitionException, UnknownPartitionException {
+
+    return false;
+  }
+
+  @Override
+  public boolean addRole(String rowName, String ownerName) throws InvalidObjectException,
+      MetaException, NoSuchObjectException {
+
+    return false;
+  }
+
+  @Override
+  public boolean removeRole(String roleName) throws MetaException, NoSuchObjectException {
+
+    return false;
+  }
+
+  @Override
+  public boolean grantRole(Role role, String userName, PrincipalType principalType, String grantor,
+      PrincipalType grantorType, boolean grantOption) throws MetaException, NoSuchObjectException,
+      InvalidObjectException {
+
+    return false;
+  }
+
+  @Override
+  public boolean revokeRole(Role role, String userName, PrincipalType principalType)
+      throws MetaException, NoSuchObjectException {
+
+    return false;
+  }
+
+  @Override
+  public PrincipalPrivilegeSet getUserPrivilegeSet(String userName, List<String> groupNames)
+      throws InvalidObjectException, MetaException {
+
+    return null;
+  }
+
+  @Override
+  public PrincipalPrivilegeSet getDBPrivilegeSet(String dbName, String userName,
+      List<String> groupNames) throws InvalidObjectException, MetaException {
+
+    return null;
+  }
+
+  @Override
+  public PrincipalPrivilegeSet getTablePrivilegeSet(String dbName, String tableName,
+      String userName, List<String> groupNames) throws InvalidObjectException, MetaException {
+
+    return null;
+  }
+
+  @Override
+  public PrincipalPrivilegeSet getPartitionPrivilegeSet(String dbName, String tableName,
+      String partition, String userName, List<String> groupNames) throws InvalidObjectException,
+      MetaException {
+
+    return null;
+  }
+
+  @Override
+  public PrincipalPrivilegeSet getColumnPrivilegeSet(String dbName, String tableName,
+      String partitionName, String columnName, String userName, List<String> groupNames)
+      throws InvalidObjectException, MetaException {
+
+    return null;
+  }
+
+  @Override
+  public List<MGlobalPrivilege> listPrincipalGlobalGrants(String principalName,
+      PrincipalType principalType) {
+
+    return null;
+  }
+
+  @Override
+  public List<MDBPrivilege> listPrincipalDBGrants(String principalName,
+      PrincipalType principalType, String dbName) {
+
+    return null;
+  }
+
+  @Override
+  public List<MTablePrivilege> listAllTableGrants(String principalName,
+      PrincipalType principalType, String dbName, String tableName) {
+
+    return null;
+  }
+
+  @Override
+  public List<MPartitionPrivilege> listPrincipalPartitionGrants(String principalName,
+      PrincipalType principalType, String dbName, String tableName, String partName) {
+
+    return null;
+  }
+
+  @Override
+  public List<MTableColumnPrivilege> listPrincipalTableColumnGrants(String principalName,
+      PrincipalType principalType, String dbName, String tableName, String columnName) {
+
+    return null;
+  }
+
+  @Override
+  public List<MPartitionColumnPrivilege> listPrincipalPartitionColumnGrants(String principalName,
+      PrincipalType principalType, String dbName, String tableName, String partName,
+      String columnName) {
+
+    return null;
+  }
+
+  @Override
+  public boolean grantPrivileges(PrivilegeBag privileges) throws InvalidObjectException,
+      MetaException, NoSuchObjectException {
+
+    return false;
+  }
+
+  @Override
+  public boolean revokePrivileges(PrivilegeBag privileges) throws InvalidObjectException,
+      MetaException, NoSuchObjectException {
+
+    return false;
+  }
+
+  @Override
+  public Role getRole(String roleName) throws NoSuchObjectException {
+
+    return null;
+  }
+
+  @Override
+  public List<String> listRoleNames() {
+
+    return null;
+  }
+
+  @Override
+  public List<MRoleMap> listRoles(String principalName, PrincipalType principalType) {
+
+    return null;
+  }
+
+  @Override
+  public Partition getPartitionWithAuth(String dbName, String tblName, List<String> partVals,
+      String user_name, List<String> group_names) throws MetaException, NoSuchObjectException,
+      InvalidObjectException {
+
+    return null;
+  }
+
+  @Override
+  public List<Partition> getPartitionsWithAuth(String dbName, String tblName, short maxParts,
+      String userName, List<String> groupNames) throws MetaException, NoSuchObjectException,
+      InvalidObjectException {
+
+    return null;
+  }
+
+  @Override
+  public List<String> listPartitionNamesPs(String db_name, String tbl_name, List<String> part_vals,
+      short max_parts) throws MetaException, NoSuchObjectException {
+
+    return null;
+  }
+
+  @Override
+  public List<Partition> listPartitionsPsWithAuth(String db_name, String tbl_name,
+      List<String> part_vals, short max_parts, String userName, List<String> groupNames)
+      throws MetaException, InvalidObjectException, NoSuchObjectException {
+
+    return null;
+  }
+
+  @Override
+  public long cleanupEvents() {
+
+    return 0;
+  }
+
+}

Added: hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/TestMetaStoreConnectionUrlHook.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/TestMetaStoreConnectionUrlHook.java?rev=1296946&view=auto
==============================================================================
--- hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/TestMetaStoreConnectionUrlHook.java (added)
+++ hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/TestMetaStoreConnectionUrlHook.java Mon Mar  5 07:16:13 2012
@@ -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 junit.framework.TestCase;
+
+import org.apache.hadoop.hive.cli.CliSessionState;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.session.SessionState;
+
+/**
+ * TestMetaStoreConnectionUrlHook
+ * Verifies that when an instance of an implementation of RawStore is initialized, the connection
+ * URL has already been updated by any metastore connect URL hooks.
+ */
+public class TestMetaStoreConnectionUrlHook extends TestCase {
+  private HiveConf hiveConf;
+
+  @Override
+  protected void setUp() throws Exception {
+
+    super.setUp();
+  }
+
+  @Override
+  protected void tearDown() throws Exception {
+    super.tearDown();
+  }
+
+  public void testUrlHook() throws Exception {
+    hiveConf = new HiveConf(this.getClass());
+    hiveConf.setVar(HiveConf.ConfVars.METASTORECONNECTURLHOOK,
+        DummyJdoConnectionUrlHook.class.getName());
+    hiveConf.setVar(HiveConf.ConfVars.METASTORECONNECTURLKEY,
+        DummyJdoConnectionUrlHook.initialUrl);
+    hiveConf.setVar(HiveConf.ConfVars.METASTORE_RAW_STORE_IMPL,
+        DummyRawStoreForJdoConnection.class.getName());
+    hiveConf.setBoolean("hive.metastore.checkForDefaultDb", true);
+    SessionState.start(new CliSessionState(hiveConf));
+
+    // Instantiating the HMSHandler with hive.metastore.checkForDefaultDb will cause it to
+    // initialize an instance of the DummyRawStoreForJdoConnection
+    HiveMetaStore.HMSHandler hms = new HiveMetaStore.HMSHandler(
+        "test_metastore_connection_url_hook_hms_handler", hiveConf);
+  }
+}