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

This reverts commit 1c7043b1ff.
This commit is contained in:
Wei-Chiu Chuang 2020-11-04 16:18:41 -08:00
parent ce883fe1bb
commit d6fd82ca56
1 changed files with 4 additions and 4 deletions

View File

@ -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;
@ -104,7 +103,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;
@ -132,7 +130,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();
@ -641,7 +638,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