You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by jd...@apache.org on 2014/08/09 03:28:20 UTC

svn commit: r1616911 - in /hive/trunk: beeline/ beeline/src/java/org/apache/hive/beeline/ itests/hive-unit-hadoop2/src/test/java/org/apache/hadoop/hive/ql/security/ itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/ metastore/src/java/o...

Author: jdere
Date: Sat Aug  9 01:28:19 2014
New Revision: 1616911

URL: http://svn.apache.org/r1616911
Log:
HIVE-7634: Use Configuration.getPassword() if available to eliminate passwords from hive-site.xml (Jason Dere, reviewed by Thejas Nair)

Added:
    hive/trunk/itests/hive-unit-hadoop2/src/test/java/org/apache/hadoop/hive/ql/security/TestPasswordWithCredentialProvider.java
    hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestPasswordWithConfig.java
Modified:
    hive/trunk/beeline/pom.xml
    hive/trunk/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java
    hive/trunk/shims/0.20/src/main/java/org/apache/hadoop/hive/shims/Hadoop20Shims.java
    hive/trunk/shims/0.20S/src/main/java/org/apache/hadoop/hive/shims/Hadoop20SShims.java
    hive/trunk/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java
    hive/trunk/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShims.java

Modified: hive/trunk/beeline/pom.xml
URL: http://svn.apache.org/viewvc/hive/trunk/beeline/pom.xml?rev=1616911&r1=1616910&r2=1616911&view=diff
==============================================================================
--- hive/trunk/beeline/pom.xml (original)
+++ hive/trunk/beeline/pom.xml Sat Aug  9 01:28:19 2014
@@ -48,7 +48,6 @@
       <groupId>org.apache.hive</groupId>
       <artifactId>hive-shims</artifactId>
       <version>${project.version}</version>
-      <scope>runtime</scope>
     </dependency>
     <!-- inter-project -->
     <dependency>

Modified: hive/trunk/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java
URL: http://svn.apache.org/viewvc/hive/trunk/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java?rev=1616911&r1=1616910&r2=1616911&view=diff
==============================================================================
--- hive/trunk/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java (original)
+++ hive/trunk/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java Sat Aug  9 01:28:19 2014
@@ -48,6 +48,7 @@ import org.apache.hadoop.hive.conf.HiveC
 import org.apache.hadoop.hive.metastore.HiveMetaException;
 import org.apache.hadoop.hive.metastore.MetaStoreSchemaInfo;
 import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.shims.ShimLoader;
 import org.apache.hive.beeline.HiveSchemaHelper.NestedScriptParser;
 
 public class HiveSchemaTool {
@@ -72,7 +73,12 @@ public class HiveSchemaTool {
     this.dbType = dbType;
     this.metaStoreSchemaInfo = new MetaStoreSchemaInfo(hiveHome, hiveConf, dbType);
     userName = hiveConf.get(ConfVars.METASTORE_CONNECTION_USER_NAME.varname);
-    passWord = hiveConf.get(HiveConf.ConfVars.METASTOREPWD.varname);
+    try {
+      passWord = ShimLoader.getHadoopShims().getPassword(hiveConf,
+          HiveConf.ConfVars.METASTOREPWD.varname);
+    } catch (IOException err) {
+      throw new HiveMetaException("Error getting metastore password", err);
+    }
   }
 
   public HiveConf getHiveConf() {

Added: hive/trunk/itests/hive-unit-hadoop2/src/test/java/org/apache/hadoop/hive/ql/security/TestPasswordWithCredentialProvider.java
URL: http://svn.apache.org/viewvc/hive/trunk/itests/hive-unit-hadoop2/src/test/java/org/apache/hadoop/hive/ql/security/TestPasswordWithCredentialProvider.java?rev=1616911&view=auto
==============================================================================
--- hive/trunk/itests/hive-unit-hadoop2/src/test/java/org/apache/hadoop/hive/ql/security/TestPasswordWithCredentialProvider.java (added)
+++ hive/trunk/itests/hive-unit-hadoop2/src/test/java/org/apache/hadoop/hive/ql/security/TestPasswordWithCredentialProvider.java Sat Aug  9 01:28:19 2014
@@ -0,0 +1,120 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.security;
+
+import java.io.File;
+import java.lang.reflect.Method;
+import java.util.List;
+
+import org.junit.*;
+import static org.junit.Assert.*;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.shims.ShimLoader;
+
+public class TestPasswordWithCredentialProvider {
+
+  public static boolean doesHadoopPasswordAPIExist() {
+    boolean foundMethod = false;
+    try {
+      Method getPasswordMethod = Configuration.class.getMethod("getPassword", String.class);
+      foundMethod = true;
+    } catch (NoSuchMethodException err) {
+    }
+    return foundMethod;
+  }
+
+  private static final File tmpDir = 
+      new File(System.getProperty("test.tmp.dir"), "creds");
+
+  private static Object invoke(Class objClass, Object obj, String methodName, Object ... args)
+      throws Exception {
+    Class[] argTypes = new Class[args.length];
+    for (int idx = 0; idx < args.length; ++idx) {
+      argTypes[idx] = args[idx].getClass();
+    }
+    Method method = objClass.getMethod(methodName, argTypes);
+    return method.invoke(obj, args);
+  }
+
+  @Test
+  public void testPassword() throws Exception {
+    if (!doesHadoopPasswordAPIExist()) {
+      System.out.println("Skipping Password API test"
+          + " because this version of hadoop-2 does not support the password API.");
+      return;
+    }
+
+    String credName = "my.password";
+    String credName2 = "my.password2";
+    String credName3 = "my.password3";
+    String hiveConfPassword = "conf value";
+    String credPassword = "cred value";
+    String confOnlyPassword = "abcdefg";
+    String credOnlyPassword = "12345";
+
+    // Set up conf
+    Configuration conf = new Configuration();
+    conf.set(credName, hiveConfPassword);  // Will be superceded by credential provider
+    conf.set(credName2, confOnlyPassword);  // Will not be superceded
+    assertEquals(hiveConfPassword, conf.get(credName));
+    assertEquals(confOnlyPassword, conf.get(credName2));
+    assertNull("credName3 should not exist in HiveConf", conf.get(credName3));
+
+    // Configure getPassword() to fall back to conf if credential doesn't have entry 
+    conf.set("hadoop.security.credential.clear-text-fallback", "true");
+
+    // Set up CredentialProvider
+    conf.set("hadoop.security.credential.provider.path", "jceks://file/" + tmpDir + "/test.jks");
+
+    // CredentialProvider/CredentialProviderFactory may not exist, depending on the version of
+    // hadoop-2 being used to build Hive. Use reflection to do the following lines
+    // to allow the test to compile regardless of what version of hadoop-2.
+    // Update credName entry in the credential provider.
+    //CredentialProvider provider = CredentialProviderFactory.getProviders(conf).get(0);
+    //provider.createCredentialEntry(credName, credPassword.toCharArray());
+    //provider.createCredentialEntry(credName3, credOnlyPassword.toCharArray());
+    //provider.flush();
+
+    Class credentialProviderClass =
+        Class.forName("org.apache.hadoop.security.alias.CredentialProvider");
+    Class credentialProviderFactoryClass =
+        Class.forName("org.apache.hadoop.security.alias.CredentialProviderFactory");
+    Object provider = 
+        ((List) invoke(credentialProviderFactoryClass, null, "getProviders", conf))
+        .get(0);
+    invoke(credentialProviderClass, provider, "createCredentialEntry", credName, credPassword.toCharArray());
+    invoke(credentialProviderClass, provider, "createCredentialEntry", credName3, credOnlyPassword.toCharArray());
+    invoke(credentialProviderClass,  provider, "flush");
+
+    // If credential provider has entry for our credential, then it should be used
+    assertEquals("getPassword() should use match value in credential provider",
+        credPassword, ShimLoader.getHadoopShims().getPassword(conf, credName));
+    // If cred provider doesn't have entry, fall back to conf
+    assertEquals("getPassword() should match value from conf",
+        confOnlyPassword, ShimLoader.getHadoopShims().getPassword(conf, credName2));
+    // If cred provider has entry and conf does not, cred provider is used.
+    // This is our use case of not having passwords stored in in the clear in hive conf files.
+    assertEquals("getPassword() should use credential provider if conf has no value",
+        credOnlyPassword, ShimLoader.getHadoopShims().getPassword(conf, credName3));
+    // If neither cred provider or conf have entry, return null;
+    assertNull("null if neither cred provider or conf have entry",
+        ShimLoader.getHadoopShims().getPassword(conf, "nonexistentkey"));
+  }
+}

Added: hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestPasswordWithConfig.java
URL: http://svn.apache.org/viewvc/hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestPasswordWithConfig.java?rev=1616911&view=auto
==============================================================================
--- hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestPasswordWithConfig.java (added)
+++ hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestPasswordWithConfig.java Sat Aug  9 01:28:19 2014
@@ -0,0 +1,39 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.security;
+
+import org.junit.*;
+import static org.junit.Assert.*;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.shims.ShimLoader;
+
+public class TestPasswordWithConfig {
+  @Test
+  public void testPassword() throws Exception {
+    String key1 = "key1";
+    String key2 = "key2";
+    String val1 = "value1";
+    Configuration conf = new Configuration();
+    conf.set(key1, val1);
+
+    assertEquals("key1 should exist in config", val1, ShimLoader.getHadoopShims().getPassword(conf, key1));
+    assertNull("key2 should not exist in config", ShimLoader.getHadoopShims().getPassword(conf, key2));
+  }
+}

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java?rev=1616911&r1=1616910&r2=1616911&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java Sat Aug  9 01:28:19 2014
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hive.metastore.txn;
 
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.shims.ShimLoader;
 
 import java.sql.Connection;
 import java.sql.Driver;
@@ -201,7 +202,8 @@ public class TxnDbUtil {
     Properties prop = new Properties();
     String driverUrl = HiveConf.getVar(conf, HiveConf.ConfVars.METASTORECONNECTURLKEY);
     String user = HiveConf.getVar(conf, HiveConf.ConfVars.METASTORE_CONNECTION_USER_NAME);
-    String passwd = HiveConf.getVar(conf, HiveConf.ConfVars.METASTOREPWD);
+    String passwd = ShimLoader.getHadoopShims().getPassword(conf,
+        HiveConf.ConfVars.METASTOREPWD.varname);
     prop.put("user", user);
     prop.put("password", passwd);
     return driver.connect(driverUrl, prop);

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java?rev=1616911&r1=1616910&r2=1616911&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java Sat Aug  9 01:28:19 2014
@@ -32,9 +32,12 @@ import org.apache.hadoop.hive.common.Val
 import org.apache.hadoop.hive.common.ValidTxnListImpl;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.*;
+import org.apache.hadoop.hive.shims.ShimLoader;
 import org.apache.hadoop.util.StringUtils;
 
 import javax.sql.DataSource;
+
+import java.io.IOException;
 import java.sql.*;
 import java.util.*;
 
@@ -1602,7 +1605,13 @@ public class TxnHandler {
 
     String driverUrl = HiveConf.getVar(conf, HiveConf.ConfVars.METASTORECONNECTURLKEY);
     String user = HiveConf.getVar(conf, HiveConf.ConfVars.METASTORE_CONNECTION_USER_NAME);
-    String passwd = HiveConf.getVar(conf, HiveConf.ConfVars.METASTOREPWD);
+    String passwd;
+    try {
+      passwd = ShimLoader.getHadoopShims().getPassword(conf,
+          HiveConf.ConfVars.METASTOREPWD.varname);
+    } catch (IOException err) {
+      throw new SQLException("Error getting metastore password", err);
+    }
     String connectionPooler = HiveConf.getVar(conf,
         HiveConf.ConfVars.METASTORE_CONNECTION_POOLING_TYPE).toLowerCase();
 

Modified: hive/trunk/service/src/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java?rev=1616911&r1=1616910&r2=1616911&view=diff
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java (original)
+++ hive/trunk/service/src/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java Sat Aug  9 01:28:19 2014
@@ -20,7 +20,9 @@ package org.apache.hive.service.cli.thri
 
 import java.net.InetSocketAddress;
 
+import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.shims.ShimLoader;
 import org.apache.hive.service.auth.HiveAuthFactory;
 import org.apache.hive.service.cli.CLIService;
 import org.apache.thrift.TProcessorFactory;
@@ -73,8 +75,10 @@ public class ThriftBinaryCLIService exte
           throw new IllegalArgumentException(ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PATH.varname +
               " Not configured for SSL connection");
         }
+        String keyStorePassword = ShimLoader.getHadoopShims().getPassword(hiveConf,
+            HiveConf.ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PASSWORD.varname);
         serverSocket = HiveAuthFactory.getServerSSLSocket(hiveHost, portNum,
-            keyStorePath, hiveConf.getVar(ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PASSWORD));
+            keyStorePath, keyStorePassword);
       }
       TThreadPoolServer.Args sargs = new TThreadPoolServer.Args(serverSocket)
       .processorFactory(processorFactory)

Modified: hive/trunk/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java?rev=1616911&r1=1616910&r2=1616911&view=diff
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java (original)
+++ hive/trunk/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java Sat Aug  9 01:28:19 2014
@@ -20,6 +20,7 @@ package org.apache.hive.service.cli.thri
 
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.shims.ShimLoader;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.Shell;
 import org.apache.hive.service.auth.HiveAuthFactory;
@@ -83,7 +84,8 @@ public class ThriftHttpCLIService extend
 
       if (useSsl) {
         String keyStorePath = hiveConf.getVar(ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PATH).trim();
-        String keyStorePassword = hiveConf.getVar(ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PASSWORD);
+        String keyStorePassword = ShimLoader.getHadoopShims().getPassword(hiveConf,
+            HiveConf.ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PASSWORD.varname);
         if (keyStorePath.isEmpty()) {
           throw new IllegalArgumentException(ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PATH.varname +
               " Not configured for SSL connection");

Modified: hive/trunk/shims/0.20/src/main/java/org/apache/hadoop/hive/shims/Hadoop20Shims.java
URL: http://svn.apache.org/viewvc/hive/trunk/shims/0.20/src/main/java/org/apache/hadoop/hive/shims/Hadoop20Shims.java?rev=1616911&r1=1616910&r2=1616911&view=diff
==============================================================================
--- hive/trunk/shims/0.20/src/main/java/org/apache/hadoop/hive/shims/Hadoop20Shims.java (original)
+++ hive/trunk/shims/0.20/src/main/java/org/apache/hadoop/hive/shims/Hadoop20Shims.java Sat Aug  9 01:28:19 2014
@@ -889,4 +889,10 @@ public class Hadoop20Shims implements Ha
       throws IOException, AccessControlException, Exception {
     DefaultFileAccess.checkFileAccess(fs, stat, action);
   }
+
+  @Override
+  public String getPassword(Configuration conf, String name) {
+    // No password API, just retrieve value from conf
+    return conf.get(name);
+  }
 }

Modified: hive/trunk/shims/0.20S/src/main/java/org/apache/hadoop/hive/shims/Hadoop20SShims.java
URL: http://svn.apache.org/viewvc/hive/trunk/shims/0.20S/src/main/java/org/apache/hadoop/hive/shims/Hadoop20SShims.java?rev=1616911&r1=1616910&r2=1616911&view=diff
==============================================================================
--- hive/trunk/shims/0.20S/src/main/java/org/apache/hadoop/hive/shims/Hadoop20SShims.java (original)
+++ hive/trunk/shims/0.20S/src/main/java/org/apache/hadoop/hive/shims/Hadoop20SShims.java Sat Aug  9 01:28:19 2014
@@ -507,4 +507,10 @@ public class Hadoop20SShims extends Hado
   public void getMergedCredentials(JobConf jobConf) throws IOException {
     throw new IOException("Merging of credentials not supported in this version of hadoop");
   }
+
+  @Override
+  public String getPassword(Configuration conf, String name) {
+    // No password API, just retrieve value from conf
+    return conf.get(name);
+  }
 }

Modified: hive/trunk/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java
URL: http://svn.apache.org/viewvc/hive/trunk/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java?rev=1616911&r1=1616910&r2=1616911&view=diff
==============================================================================
--- hive/trunk/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java (original)
+++ hive/trunk/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java Sat Aug  9 01:28:19 2014
@@ -745,6 +745,7 @@ public class Hadoop23Shims extends Hadoo
   }
 
   protected static final Method accessMethod;
+  protected static final Method getPasswordMethod;
 
   static {
     Method m = null;
@@ -754,6 +755,14 @@ public class Hadoop23Shims extends Hadoo
       // This version of Hadoop does not support FileSystem.access().
     }
     accessMethod = m;
+
+    try {
+      m = Configuration.class.getMethod("getPassword", String.class);
+    } catch (NoSuchMethodException err) {
+      // This version of Hadoop does not support getPassword(), just retrieve password from conf.
+      m = null;
+    }
+    getPasswordMethod = m;
   }
 
   @Override
@@ -789,4 +798,22 @@ public class Hadoop23Shims extends Hadoo
     }
     return err;
   }
+
+  @Override
+  public String getPassword(Configuration conf, String name) throws IOException {
+    if (getPasswordMethod == null) {
+      // Just retrieve value from conf
+      return conf.get(name);
+    } else {
+      try {
+        char[] pw = (char[]) getPasswordMethod.invoke(conf, name);
+        if (pw == null) {
+          return null;
+        }
+        return new String(pw);
+      } catch (Exception err) {
+        throw new IOException(err.getMessage(), err);
+      }
+    }
+  }
 }

Modified: hive/trunk/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShims.java
URL: http://svn.apache.org/viewvc/hive/trunk/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShims.java?rev=1616911&r1=1616910&r2=1616911&view=diff
==============================================================================
--- hive/trunk/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShims.java (original)
+++ hive/trunk/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShims.java Sat Aug  9 01:28:19 2014
@@ -685,4 +685,13 @@ public interface HadoopShims {
    */
   public void checkFileAccess(FileSystem fs, FileStatus status, FsAction action)
       throws IOException, AccessControlException, Exception;
+
+  /**
+   * Use password API (if available) to fetch credentials/password
+   * @param conf
+   * @param name
+   * @return
+   */
+  public String getPassword(Configuration conf, String name) throws IOException;
+
 }