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

Reviewed-by: Steve Loughran <stevel@cloudera.com>
Reviewed-by: Wei-Chiu Chuang <weichiu@apache.org>
This commit is contained in:
Akira Ajisaka 2020-11-04 04:20:23 +09:00 committed by GitHub
parent e580280a8b
commit 7f5caca04c
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
1 changed files with 4 additions and 4 deletions

View File

@ -23,6 +23,7 @@ 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,6 +105,7 @@ 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;
@ -131,6 +133,7 @@ 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();
@ -639,10 +642,7 @@ public class JavaKeyStoreProvider extends KeyProvider {
private void renameOrFail(Path src, Path dest)
throws IOException {
if (!fs.rename(src, dest)) {
throw new IOException("Rename unsuccessful : "
+ String.format("'%s' to '%s'", src, dest));
}
context.rename(src, dest, org.apache.hadoop.fs.Options.Rename.NONE);
}
@Override