|
@@ -25,6 +25,7 @@ import java.io.DataOutputStream;
|
|
|
import java.io.FileNotFoundException;
|
|
|
import java.io.IOException;
|
|
|
import java.util.ArrayList;
|
|
|
+import java.util.EnumSet;
|
|
|
import java.util.List;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
@@ -38,6 +39,7 @@ import org.apache.hadoop.fs.FileStatus;
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.fs.PathFilter;
|
|
|
+import org.apache.hadoop.fs.XAttrSetFlag;
|
|
|
import org.apache.hadoop.io.IOUtils;
|
|
|
import org.apache.hadoop.security.token.delegation.DelegationKey;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
|
@@ -84,7 +86,8 @@ public class FileSystemRMStateStore extends RMStateStore {
|
|
|
protected static final String AMRMTOKEN_SECRET_MANAGER_NODE =
|
|
|
"AMRMTokenSecretManagerNode";
|
|
|
|
|
|
- @VisibleForTesting
|
|
|
+ private static final String UNREADABLE_BY_SUPERUSER_XATTRIB =
|
|
|
+ "security.hdfs.unreadable.by.superuser";
|
|
|
protected FileSystem fs;
|
|
|
@VisibleForTesting
|
|
|
protected Configuration fsConf;
|
|
@@ -97,6 +100,7 @@ public class FileSystemRMStateStore extends RMStateStore {
|
|
|
private Path dtSequenceNumberPath = null;
|
|
|
private int fsNumRetries;
|
|
|
private long fsRetryInterval;
|
|
|
+ private boolean isHDFS;
|
|
|
|
|
|
@VisibleForTesting
|
|
|
Path fsWorkingPath;
|
|
@@ -141,11 +145,17 @@ public class FileSystemRMStateStore extends RMStateStore {
|
|
|
}
|
|
|
|
|
|
fs = fsWorkingPath.getFileSystem(fsConf);
|
|
|
+ isHDFS = fs.getScheme().toLowerCase().contains("hdfs");
|
|
|
mkdirsWithRetries(rmDTSecretManagerRoot);
|
|
|
mkdirsWithRetries(rmAppRoot);
|
|
|
mkdirsWithRetries(amrmTokenSecretManagerRoot);
|
|
|
}
|
|
|
|
|
|
+ @VisibleForTesting
|
|
|
+ void setIsHDFS(boolean isHDFS) {
|
|
|
+ this.isHDFS = isHDFS;
|
|
|
+ }
|
|
|
+
|
|
|
@Override
|
|
|
protected synchronized void closeInternal() throws Exception {
|
|
|
closeWithRetries();
|
|
@@ -175,9 +185,9 @@ public class FileSystemRMStateStore extends RMStateStore {
|
|
|
byte[] data =
|
|
|
((VersionPBImpl) CURRENT_VERSION_INFO).getProto().toByteArray();
|
|
|
if (existsWithRetries(versionNodePath)) {
|
|
|
- updateFile(versionNodePath, data);
|
|
|
+ updateFile(versionNodePath, data, false);
|
|
|
} else {
|
|
|
- writeFileWithRetries(versionNodePath, data);
|
|
|
+ writeFileWithRetries(versionNodePath, data, false);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -194,12 +204,12 @@ public class FileSystemRMStateStore extends RMStateStore {
|
|
|
// increment epoch and store it
|
|
|
byte[] storeData = Epoch.newInstance(currentEpoch + 1).getProto()
|
|
|
.toByteArray();
|
|
|
- updateFile(epochNodePath, storeData);
|
|
|
+ updateFile(epochNodePath, storeData, false);
|
|
|
} else {
|
|
|
// initialize epoch file with 1 for the next time.
|
|
|
byte[] storeData = Epoch.newInstance(currentEpoch + 1).getProto()
|
|
|
.toByteArray();
|
|
|
- writeFileWithRetries(epochNodePath, storeData);
|
|
|
+ writeFileWithRetries(epochNodePath, storeData, false);
|
|
|
}
|
|
|
return currentEpoch;
|
|
|
}
|
|
@@ -253,7 +263,9 @@ public class FileSystemRMStateStore extends RMStateStore {
|
|
|
continue;
|
|
|
}
|
|
|
byte[] childData = readFileWithRetries(childNodeStatus.getPath(),
|
|
|
- childNodeStatus.getLen());
|
|
|
+ childNodeStatus.getLen());
|
|
|
+ // Set attribute if not already set
|
|
|
+ setUnreadableBySuperuserXattrib(childNodeStatus.getPath());
|
|
|
if (childNodeName.startsWith(ApplicationId.appIdStrPrefix)) {
|
|
|
// application
|
|
|
if (LOG.isDebugEnabled()) {
|
|
@@ -326,7 +338,7 @@ public class FileSystemRMStateStore extends RMStateStore {
|
|
|
assert newChildNodeStatus.isFile();
|
|
|
String newChildNodeName = newChildNodeStatus.getPath().getName();
|
|
|
String childNodeName = newChildNodeName.substring(
|
|
|
- 0, newChildNodeName.length() - ".new".length());
|
|
|
+ 0, newChildNodeName.length() - ".new".length());
|
|
|
Path childNodePath =
|
|
|
new Path(newChildNodeStatus.getPath().getParent(), childNodeName);
|
|
|
replaceFile(newChildNodeStatus.getPath(), childNodePath);
|
|
@@ -394,7 +406,7 @@ public class FileSystemRMStateStore extends RMStateStore {
|
|
|
try {
|
|
|
// currently throw all exceptions. May need to respond differently for HA
|
|
|
// based on whether we have lost the right to write to FS
|
|
|
- writeFileWithRetries(nodeCreatePath, appStateData);
|
|
|
+ writeFileWithRetries(nodeCreatePath, appStateData, true);
|
|
|
} catch (Exception e) {
|
|
|
LOG.info("Error storing info for app: " + appId, e);
|
|
|
throw e;
|
|
@@ -412,7 +424,7 @@ public class FileSystemRMStateStore extends RMStateStore {
|
|
|
try {
|
|
|
// currently throw all exceptions. May need to respond differently for HA
|
|
|
// based on whether we have lost the right to write to FS
|
|
|
- updateFile(nodeCreatePath, appStateData);
|
|
|
+ updateFile(nodeCreatePath, appStateData, true);
|
|
|
} catch (Exception e) {
|
|
|
LOG.info("Error updating info for app: " + appId, e);
|
|
|
throw e;
|
|
@@ -433,7 +445,7 @@ public class FileSystemRMStateStore extends RMStateStore {
|
|
|
try {
|
|
|
// currently throw all exceptions. May need to respond differently for HA
|
|
|
// based on whether we have lost the right to write to FS
|
|
|
- writeFileWithRetries(nodeCreatePath, attemptStateData);
|
|
|
+ writeFileWithRetries(nodeCreatePath, attemptStateData, true);
|
|
|
} catch (Exception e) {
|
|
|
LOG.info("Error storing info for attempt: " + appAttemptId, e);
|
|
|
throw e;
|
|
@@ -454,7 +466,7 @@ public class FileSystemRMStateStore extends RMStateStore {
|
|
|
try {
|
|
|
// currently throw all exceptions. May need to respond differently for HA
|
|
|
// based on whether we have lost the right to write to FS
|
|
|
- updateFile(nodeCreatePath, attemptStateData);
|
|
|
+ updateFile(nodeCreatePath, attemptStateData, true);
|
|
|
} catch (Exception e) {
|
|
|
LOG.info("Error updating info for attempt: " + appAttemptId, e);
|
|
|
throw e;
|
|
@@ -483,7 +495,7 @@ public class FileSystemRMStateStore extends RMStateStore {
|
|
|
public synchronized void removeRMDelegationTokenState(
|
|
|
RMDelegationTokenIdentifier identifier) throws Exception {
|
|
|
Path nodeCreatePath = getNodePath(rmDTSecretManagerRoot,
|
|
|
- DELEGATION_TOKEN_PREFIX + identifier.getSequenceNumber());
|
|
|
+ DELEGATION_TOKEN_PREFIX + identifier.getSequenceNumber());
|
|
|
LOG.info("Removing RMDelegationToken_" + identifier.getSequenceNumber());
|
|
|
deleteFileWithRetries(nodeCreatePath);
|
|
|
}
|
|
@@ -505,10 +517,10 @@ public class FileSystemRMStateStore extends RMStateStore {
|
|
|
new RMDelegationTokenIdentifierData(identifier, renewDate);
|
|
|
if (isUpdate) {
|
|
|
LOG.info("Updating RMDelegationToken_" + identifier.getSequenceNumber());
|
|
|
- updateFile(nodeCreatePath, identifierData.toByteArray());
|
|
|
+ updateFile(nodeCreatePath, identifierData.toByteArray(), true);
|
|
|
} else {
|
|
|
LOG.info("Storing RMDelegationToken_" + identifier.getSequenceNumber());
|
|
|
- writeFileWithRetries(nodeCreatePath, identifierData.toByteArray());
|
|
|
+ writeFileWithRetries(nodeCreatePath, identifierData.toByteArray(), true);
|
|
|
|
|
|
// store sequence number
|
|
|
Path latestSequenceNumberPath = getNodePath(rmDTSecretManagerRoot,
|
|
@@ -539,7 +551,7 @@ public class FileSystemRMStateStore extends RMStateStore {
|
|
|
try (DataOutputStream fsOut = new DataOutputStream(os)) {
|
|
|
LOG.info("Storing RMDelegationKey_" + masterKey.getKeyId());
|
|
|
masterKey.write(fsOut);
|
|
|
- writeFileWithRetries(nodeCreatePath, os.toByteArray());
|
|
|
+ writeFileWithRetries(nodeCreatePath, os.toByteArray(), true);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -572,6 +584,16 @@ public class FileSystemRMStateStore extends RMStateStore {
|
|
|
return getNodePath(root, appId.toString());
|
|
|
}
|
|
|
|
|
|
+ @VisibleForTesting
|
|
|
+ protected Path getAppDir(ApplicationId appId) {
|
|
|
+ return getAppDir(rmAppRoot, appId);
|
|
|
+ }
|
|
|
+
|
|
|
+ @VisibleForTesting
|
|
|
+ protected Path getAppAttemptDir(ApplicationAttemptId appAttId) {
|
|
|
+ return getNodePath(getAppDir(appAttId.getApplicationId()), appAttId
|
|
|
+ .toString());
|
|
|
+ }
|
|
|
// FileSystem related code
|
|
|
|
|
|
private boolean checkAndRemovePartialRecordWithRetries(final Path record)
|
|
@@ -594,12 +616,13 @@ public class FileSystemRMStateStore extends RMStateStore {
|
|
|
}.runWithRetries();
|
|
|
}
|
|
|
|
|
|
- private void writeFileWithRetries(final Path outputPath,final byte[] data)
|
|
|
- throws Exception {
|
|
|
+ private void writeFileWithRetries(final Path outputPath, final byte[] data,
|
|
|
+ final boolean makeUnreadableByAdmin)
|
|
|
+ throws Exception {
|
|
|
new FSAction<Void>() {
|
|
|
@Override
|
|
|
public Void run() throws Exception {
|
|
|
- writeFile(outputPath, data);
|
|
|
+ writeFile(outputPath, data, makeUnreadableByAdmin);
|
|
|
return null;
|
|
|
}
|
|
|
}.runWithRetries();
|
|
@@ -746,7 +769,8 @@ public class FileSystemRMStateStore extends RMStateStore {
|
|
|
* data to .tmp file and then rename it. Here we are assuming that rename is
|
|
|
* atomic for underlying file system.
|
|
|
*/
|
|
|
- private void writeFile(Path outputPath, byte[] data) throws Exception {
|
|
|
+ protected void writeFile(Path outputPath, byte[] data, boolean
|
|
|
+ makeUnradableByAdmin) throws Exception {
|
|
|
Path tempPath =
|
|
|
new Path(outputPath.getParent(), outputPath.getName() + ".tmp");
|
|
|
FSDataOutputStream fsOut = null;
|
|
@@ -754,6 +778,9 @@ public class FileSystemRMStateStore extends RMStateStore {
|
|
|
// final status.
|
|
|
try {
|
|
|
fsOut = fs.create(tempPath, true);
|
|
|
+ if (makeUnradableByAdmin) {
|
|
|
+ setUnreadableBySuperuserXattrib(tempPath);
|
|
|
+ }
|
|
|
fsOut.write(data);
|
|
|
fsOut.close();
|
|
|
fsOut = null;
|
|
@@ -768,10 +795,11 @@ public class FileSystemRMStateStore extends RMStateStore {
|
|
|
* data to .new file and then rename it. Here we are assuming that rename is
|
|
|
* atomic for underlying file system.
|
|
|
*/
|
|
|
- protected void updateFile(Path outputPath, byte[] data) throws Exception {
|
|
|
+ protected void updateFile(Path outputPath, byte[] data, boolean
|
|
|
+ makeUnradableByAdmin) throws Exception {
|
|
|
Path newPath = new Path(outputPath.getParent(), outputPath.getName() + ".new");
|
|
|
// use writeFileWithRetries to make sure .new file is created atomically
|
|
|
- writeFileWithRetries(newPath, data);
|
|
|
+ writeFileWithRetries(newPath, data, makeUnradableByAdmin);
|
|
|
replaceFile(newPath, outputPath);
|
|
|
}
|
|
|
|
|
@@ -810,9 +838,9 @@ public class FileSystemRMStateStore extends RMStateStore {
|
|
|
AMRMTokenSecretManagerState.newInstance(amrmTokenSecretManagerState);
|
|
|
byte[] stateData = data.getProto().toByteArray();
|
|
|
if (isUpdate) {
|
|
|
- updateFile(nodeCreatePath, stateData);
|
|
|
+ updateFile(nodeCreatePath, stateData, true);
|
|
|
} else {
|
|
|
- writeFileWithRetries(nodeCreatePath, stateData);
|
|
|
+ writeFileWithRetries(nodeCreatePath, stateData, true);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -825,4 +853,13 @@ public class FileSystemRMStateStore extends RMStateStore {
|
|
|
public long getRetryInterval() {
|
|
|
return fsRetryInterval;
|
|
|
}
|
|
|
+
|
|
|
+ private void setUnreadableBySuperuserXattrib(Path p)
|
|
|
+ throws IOException {
|
|
|
+ if (isHDFS &&
|
|
|
+ !fs.getXAttrs(p).containsKey(UNREADABLE_BY_SUPERUSER_XATTRIB)) {
|
|
|
+ fs.setXAttr(p, UNREADABLE_BY_SUPERUSER_XATTRIB, null,
|
|
|
+ EnumSet.of(XAttrSetFlag.CREATE));
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|