瀏覽代碼

Revert "HADOOP-17255. JavaKeyStoreProvider fails to create a new key if the keystore is HDFS. (#2291)"

This reverts commit dd1634ec3b676d6c3da18af7da46734584ddd9f6.
Wei-Chiu Chuang 4 年之前
父節點
當前提交
cfa0986d00

+ 4 - 4
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/JavaKeyStoreProvider.java

@@ -23,7 +23,6 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -105,7 +104,6 @@ public class JavaKeyStoreProvider extends KeyProvider {
   private final Path path;
   private final FileSystem fs;
   private FsPermission permissions;
-  private FileContext context;
   private KeyStore keyStore;
   private char[] password;
   private boolean changed = false;
@@ -133,7 +131,6 @@ public class JavaKeyStoreProvider extends KeyProvider {
     this.uri = uri;
     path = ProviderUtils.unnestUri(uri);
     fs = path.getFileSystem(conf);
-    context = FileContext.getFileContext(conf);
     locateKeystore();
     ReadWriteLock lock = new ReentrantReadWriteLock(true);
     readLock = lock.readLock();
@@ -642,7 +639,10 @@ public class JavaKeyStoreProvider extends KeyProvider {
 
   private void renameOrFail(Path src, Path dest)
       throws IOException {
-    context.rename(src, dest, org.apache.hadoop.fs.Options.Rename.NONE);
+    if (!fs.rename(src, dest)) {
+      throw new IOException("Rename unsuccessful : "
+          + String.format("'%s' to '%s'", src, dest));
+    }
   }
 
   @Override