|
@@ -2448,84 +2448,66 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
|
|
|
waitForLoadingFSImage();
|
|
|
|
|
|
- /*
|
|
|
- * We want to avoid holding any locks while doing KeyProvider operations,
|
|
|
- * since they can be very slow. Since the path can
|
|
|
- * flip flop between being in an encryption zone and not in the meantime,
|
|
|
- * we need to recheck the preconditions and redo KeyProvider operations
|
|
|
- * in some situations.
|
|
|
- *
|
|
|
- * A special RetryStartFileException is used to indicate that we should
|
|
|
- * retry creation of a FileEncryptionInfo.
|
|
|
+ /**
|
|
|
+ * If the file is in an encryption zone, we optimistically create an
|
|
|
+ * EDEK for the file by calling out to the configured KeyProvider.
|
|
|
+ * Since this typically involves doing an RPC, we take the readLock
|
|
|
+ * initially, then drop it to do the RPC.
|
|
|
+ *
|
|
|
+ * Since the path can flip-flop between being in an encryption zone and not
|
|
|
+ * in the meantime, we need to recheck the preconditions when we retake the
|
|
|
+ * lock to do the create. If the preconditions are not met, we throw a
|
|
|
+ * special RetryStartFileException to ask the DFSClient to try the create
|
|
|
+ * again later.
|
|
|
*/
|
|
|
- BlocksMapUpdateInfo toRemoveBlocks = null;
|
|
|
+ CipherSuite suite = null;
|
|
|
+ String ezKeyName = null;
|
|
|
+ readLock();
|
|
|
try {
|
|
|
- boolean shouldContinue = true;
|
|
|
- int iters = 0;
|
|
|
- while (shouldContinue) {
|
|
|
- skipSync = false;
|
|
|
- if (iters >= 10) {
|
|
|
- throw new IOException("Too many retries because of encryption zone " +
|
|
|
- "operations, something might be broken!");
|
|
|
- }
|
|
|
- shouldContinue = false;
|
|
|
- iters++;
|
|
|
-
|
|
|
- // Optimistically determine CipherSuite and ezKeyName if the path is
|
|
|
- // currently within an encryption zone
|
|
|
- CipherSuite suite = null;
|
|
|
- String ezKeyName = null;
|
|
|
- readLock();
|
|
|
- try {
|
|
|
- src = resolvePath(src, pathComponents);
|
|
|
- INodesInPath iip = dir.getINodesInPath4Write(src);
|
|
|
- // Nothing to do if the path is not within an EZ
|
|
|
- if (dir.isInAnEZ(iip)) {
|
|
|
- suite = chooseCipherSuite(iip, cipherSuites);
|
|
|
- if (suite != null) {
|
|
|
- Preconditions.checkArgument(!suite.equals(CipherSuite.UNKNOWN),
|
|
|
- "Chose an UNKNOWN CipherSuite!");
|
|
|
- }
|
|
|
- ezKeyName = dir.getKeyName(iip);
|
|
|
- Preconditions.checkState(ezKeyName != null);
|
|
|
- }
|
|
|
- } finally {
|
|
|
- readUnlock();
|
|
|
+ src = resolvePath(src, pathComponents);
|
|
|
+ INodesInPath iip = dir.getINodesInPath4Write(src);
|
|
|
+ // Nothing to do if the path is not within an EZ
|
|
|
+ if (dir.isInAnEZ(iip)) {
|
|
|
+ suite = chooseCipherSuite(iip, cipherSuites);
|
|
|
+ if (suite != null) {
|
|
|
+ Preconditions.checkArgument(!suite.equals(CipherSuite.UNKNOWN),
|
|
|
+ "Chose an UNKNOWN CipherSuite!");
|
|
|
}
|
|
|
+ ezKeyName = dir.getKeyName(iip);
|
|
|
+ Preconditions.checkState(ezKeyName != null);
|
|
|
+ }
|
|
|
+ } finally {
|
|
|
+ readUnlock();
|
|
|
+ }
|
|
|
|
|
|
- Preconditions.checkState(
|
|
|
- (suite == null && ezKeyName == null) ||
|
|
|
+ Preconditions.checkState(
|
|
|
+ (suite == null && ezKeyName == null) ||
|
|
|
(suite != null && ezKeyName != null),
|
|
|
- "Both suite and ezKeyName should both be null or not null");
|
|
|
- // Generate EDEK if necessary while not holding the lock
|
|
|
- EncryptedKeyVersion edek =
|
|
|
- generateEncryptedDataEncryptionKey(ezKeyName);
|
|
|
- EncryptionFaultInjector.getInstance().startFileAfterGenerateKey();
|
|
|
- // Try to create the file with the computed cipher suite and EDEK
|
|
|
- writeLock();
|
|
|
- try {
|
|
|
- checkOperation(OperationCategory.WRITE);
|
|
|
- checkNameNodeSafeMode("Cannot create file" + src);
|
|
|
- src = resolvePath(src, pathComponents);
|
|
|
- toRemoveBlocks = startFileInternal(pc, src, permissions, holder,
|
|
|
- clientMachine, create, overwrite, createParent, replication,
|
|
|
- blockSize, suite, edek, logRetryCache);
|
|
|
- stat = dir.getFileInfo(src, false,
|
|
|
- FSDirectory.isReservedRawName(srcArg));
|
|
|
- } catch (StandbyException se) {
|
|
|
- skipSync = true;
|
|
|
- throw se;
|
|
|
- } catch (RetryStartFileException e) {
|
|
|
- shouldContinue = true;
|
|
|
- if (LOG.isTraceEnabled()) {
|
|
|
- LOG.trace("Preconditions failed, retrying creation of " +
|
|
|
- "FileEncryptionInfo", e);
|
|
|
- }
|
|
|
- } finally {
|
|
|
- writeUnlock();
|
|
|
- }
|
|
|
- }
|
|
|
+ "Both suite and ezKeyName should both be null or not null");
|
|
|
+
|
|
|
+ // Generate EDEK if necessary while not holding the lock
|
|
|
+ EncryptedKeyVersion edek =
|
|
|
+ generateEncryptedDataEncryptionKey(ezKeyName);
|
|
|
+ EncryptionFaultInjector.getInstance().startFileAfterGenerateKey();
|
|
|
+
|
|
|
+ // Proceed with the create, using the computed cipher suite and
|
|
|
+ // generated EDEK
|
|
|
+ BlocksMapUpdateInfo toRemoveBlocks = null;
|
|
|
+ writeLock();
|
|
|
+ try {
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
+ checkNameNodeSafeMode("Cannot create file" + src);
|
|
|
+ src = resolvePath(src, pathComponents);
|
|
|
+ toRemoveBlocks = startFileInternal(pc, src, permissions, holder,
|
|
|
+ clientMachine, create, overwrite, createParent, replication,
|
|
|
+ blockSize, suite, edek, logRetryCache);
|
|
|
+ stat = dir.getFileInfo(src, false,
|
|
|
+ FSDirectory.isReservedRawName(srcArg));
|
|
|
+ } catch (StandbyException se) {
|
|
|
+ skipSync = true;
|
|
|
+ throw se;
|
|
|
} finally {
|
|
|
+ writeUnlock();
|
|
|
// There might be transactions logged while trying to recover the lease.
|
|
|
// They need to be sync'ed even when an exception was thrown.
|
|
|
if (!skipSync) {
|