Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Add permission checks before reading from HDFS stream #26716

Merged
merged 3 commits into from
Sep 21, 2017
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Options.CreateOpts;
import org.apache.hadoop.fs.Path;
import org.elasticsearch.SpecialPermission;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.blobstore.BlobMetaData;
import org.elasticsearch.common.blobstore.BlobPath;
Expand All @@ -45,12 +46,14 @@

final class HdfsBlobContainer extends AbstractBlobContainer {
private final HdfsBlobStore store;
private final HdfsSecurityContext securityContext;
private final Path path;
private final int bufferSize;

HdfsBlobContainer(BlobPath blobPath, HdfsBlobStore store, Path path, int bufferSize) {
HdfsBlobContainer(BlobPath blobPath, HdfsBlobStore store, Path path, int bufferSize, HdfsSecurityContext hdfsSecurityContext) {
super(blobPath);
this.store = store;
this.securityContext = hdfsSecurityContext;
this.path = path;
this.bufferSize = bufferSize;
}
Expand Down Expand Up @@ -90,7 +93,9 @@ public InputStream readBlob(String blobName) throws IOException {
// FSDataInputStream can open connections on read() or skip() so we wrap in
// HDFSPrivilegedInputSteam which will ensure that underlying methods will
// be called with the proper privileges.
return store.execute(fileContext -> new HDFSPrivilegedInputSteam(fileContext.open(new Path(path, blobName), bufferSize)));
return store.execute(fileContext ->
new HDFSPrivilegedInputSteam(fileContext.open(new Path(path, blobName), bufferSize), securityContext)
);
}

@Override
Expand Down Expand Up @@ -144,8 +149,11 @@ public Map<String, BlobMetaData> listBlobs() throws IOException {
*/
private static class HDFSPrivilegedInputSteam extends FilterInputStream {

HDFSPrivilegedInputSteam(InputStream in) {
private final HdfsSecurityContext securityContext;

HDFSPrivilegedInputSteam(InputStream in, HdfsSecurityContext hdfsSecurityContext) {
super(in);
this.securityContext = hdfsSecurityContext;
}

public int read() throws IOException {
Expand Down Expand Up @@ -175,9 +183,10 @@ public synchronized void reset() throws IOException {
});
}

private static <T> T doPrivilegedOrThrow(PrivilegedExceptionAction<T> action) throws IOException {
private <T> T doPrivilegedOrThrow(PrivilegedExceptionAction<T> action) throws IOException {
SpecialPermission.check();
try {
return AccessController.doPrivileged(action);
return AccessController.doPrivileged(action, null, securityContext.getRestrictedExecutionPermissions());
} catch (PrivilegedActionException e) {
throw (IOException) e.getCause();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -75,7 +75,7 @@ public String toString() {

@Override
public BlobContainer blobContainer(BlobPath path) {
return new HdfsBlobContainer(path, this, buildHdfsPath(path), bufferSize);
return new HdfsBlobContainer(path, this, buildHdfsPath(path), bufferSize, securityContext);
}

private Path buildHdfsPath(BlobPath blobPath) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -56,7 +56,9 @@ class HdfsSecurityContext {
// 1) hadoop dynamic proxy is messy with access rules
new ReflectPermission("suppressAccessChecks"),
// 2) allow hadoop to add credentials to our Subject
new AuthPermission("modifyPrivateCredentials")
new AuthPermission("modifyPrivateCredentials"),
// 3) RPC Engine requires this for re-establishing pooled connections over the lifetime of the client
new PrivateCredentialPermission("org.apache.hadoop.security.Credentials * \"*\"", "read")
};

// If Security is enabled, we need all the following elevated permissions:
Expand Down