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

Fixing permission errors for `KERBEROS` security mode for HDFS Repository #23439

Merged
merged 36 commits into from May 4, 2017

Conversation

@jbaiera
Copy link
Contributor

commented Mar 1, 2017

This PR is meant to address the permission errors that are encountered in the HDFS Repository Plugin as described in #22156.

When Hadoop security is enabled, the HDFS client requests the current logged in Subject for a hadoop based Credentials object, which trips a missing permission in the plugin's policy file. This is not caught during testing since we neither use the actual HDFS client code nor do we execute with Kerberos security enabled.

I'm working on testing this on a local environment at the moment since it requires a secured HDFS service to activate the code path. My main concern is that there may be other permissions that have not yet had the chance to trip up the plugin because they have not yet been reached in the code.

Closes #22156

@rjernst

This comment has been minimized.

Copy link
Member

commented Mar 1, 2017

I do not think we should add a permission that we cannot test (all the time, not manually). Can you improve the hdfs fixture so that we can use credentials with it to test this actually works?

@jbaiera

This comment has been minimized.

Copy link
Contributor Author

commented Mar 2, 2017

@rjernst I completely agree with you on that. I'm still trying to dig through all the permission violations at the moment against a local kerberized cluster, then will circle back to assess the effort for the test fixtures.

@surekhabalaji

This comment has been minimized.

Copy link

commented Mar 14, 2017

So glad to see this issue being worked on. W are waiting on this fix for the hdfs plugin issue with Elastic Search 5.2 in order to take snapshots on hdfs repository. Any ETA on when this will get released?

@jbaiera

This comment has been minimized.

Copy link
Contributor Author

commented Mar 15, 2017

@surekhabalaji It's hard to say at this stage. It's been a long process of testing, seeing new permissions that are missing, retesting, repeat, and the iteration cycle for testing is fairly large due to the overhead of requiring a secure HDFS environment. Rest assured that it is being actively worked on.

@jbaiera jbaiera changed the title Adding missing private credential permission for HDFS Repository Fixing permission errors for `KERBEROS` security mode for HDFS Repository Mar 20, 2017
@jbaiera jbaiera requested review from rjernst and jasontedor Mar 22, 2017
@jbaiera

This comment has been minimized.

Copy link
Contributor Author

commented Mar 22, 2017

At this point the code should be complete. I'm able to manually test this in a sandbox environment with a Kerberos-enabled HDFS cluster. Repository creation, snapshots and restores are working in that environment. @rjernst and @jasontedor, does it make sense to get this change in then immediately circle back around for the testing fixtures after?

@jasontedor

This comment has been minimized.

Copy link
Member

commented Mar 22, 2017

@jbaiera This looks like an elaborate and difficult change to have worked through, and I appreciate the effort and the documentation within the change. However, building off what @rjernst said, I think that we need a testing plan before this can be integrated (n.b.: plan). As we discussed via another channel, if the fixture approach is too much effort I'm fine with testing this through a VM.

Copy link
Member

left a comment

Thanks @jbaiera. I left some first-round review comments.

plugins/repository-hdfs/src/main/java/org/elasticsearch/repositories/hdfs/HdfsBlobStore.java Outdated
@@ -118,7 +119,8 @@ private Path translateToHdfsPath(BlobPath blobPath) {
try {
return AccessController.doPrivileged((PrivilegedExceptionAction<V>)
() -> operation.run(fileContext), null, new ReflectPermission("suppressAccessChecks"),
new AuthPermission("modifyPrivateCredentials"), new SocketPermission("*", "connect"));
new AuthPermission("modifyPrivateCredentials"), new SocketPermission("*", "connect"),
new AuthPermission("doAs"), new ServicePermission("*", "initiate,accept"));

This comment has been minimized.

Copy link
@jasontedor

jasontedor Mar 22, 2017

Member

I think that the comment on this method is no longer accurate. Can you adjust it?

This comment has been minimized.

Copy link
@jbaiera

jbaiera Mar 22, 2017

Author Contributor

I pushed d973296

plugins/repository-hdfs/src/main/plugin-metadata/plugin-security.policy Outdated
@@ -25,17 +25,44 @@ grant {
permission java.lang.RuntimePermission "accessDeclaredMembers";
permission java.lang.reflect.ReflectPermission "suppressAccessChecks";

// Needed so that Hadoop can load the correct classes for SPI and JAAS
// org.apache.hadoop.security.SecurityUtil clinit
// org.apache.hadoop.security.UserGroupInformation.newLoginContext()

This comment has been minimized.

Copy link
@jasontedor

jasontedor Mar 22, 2017

Member

Nit: please fix the indentation here

* jars. Since plugins don't execute with their class loaders installed as the thread's
* context class loader, we need to install the loader briefly, allow the util to be
* initialized, then restore the old loader since we don't actually own this thread.
*/

This comment has been minimized.

Copy link
@jasontedor

jasontedor Mar 22, 2017

Member

I appreciate the thoroughness of this comment.

plugins/repository-hdfs/src/main/java/org/elasticsearch/repositories/hdfs/HdfsRepository.java Outdated
@@ -51,6 +49,8 @@

public final class HdfsRepository extends BlobStoreRepository {

private static final Logger LOGGER = Logger.getLogger(HdfsRepository.class);

This comment has been minimized.

Copy link
@jasontedor

jasontedor Mar 22, 2017

Member

Why are we using the old Log4j API here? I understand that Hadoop needs it, but we can use the new Log4j API here, and our Log4j API to acquire a logger, no?

plugins/repository-hdfs/src/main/java/org/elasticsearch/repositories/hdfs/HdfsRepository.java Outdated
LOGGER.info("Hadoop Security Is [ENABLED]");
} else {
LOGGER.info("Hadoop Security is [DISABLED]");
}

This comment has been minimized.

Copy link
@jasontedor

jasontedor Mar 22, 2017

Member

Can this whole block be replaced by:

diff --git a/plugins/repository-hdfs/src/main/java/org/elasticsearch/repositories/hdfs/HdfsRepository.java b/plugins/repository-hdfs/src/main/java/org/elasticsearch/repositories/hdfs/HdfsRepository.java
index 95619b1d40..d65686e356 100644
--- a/plugins/repository-hdfs/src/main/java/org/elasticsearch/repositories/hdfs/HdfsRepository.java
+++ b/plugins/repository-hdfs/src/main/java/org/elasticsearch/repositories/hdfs/HdfsRepository.java
@@ -122,11 +122,7 @@ public final class HdfsRepository extends BlobStoreRepository {
         UserGroupInformation.setConfiguration(cfg);
 
         // Debugging
-        if (UserGroupInformation.isSecurityEnabled()) {
-            LOGGER.info("Hadoop Security Is [ENABLED]");
-        } else {
-            LOGGER.info("Hadoop Security is [DISABLED]");
-        }
+        LOGGER.info("Hadoop security enabled: [{}]", UserGroupInformation.isSecurityEnabled());
         UserGroupInformation.AuthenticationMethod method = SecurityUtil.getAuthenticationMethod(cfg);
         LOGGER.info("Using Hadoop authentication method : [" + method + "]");
plugins/repository-hdfs/src/main/java/org/elasticsearch/repositories/hdfs/HdfsRepository.java Outdated
// Initialize the UGI with the configuration.
UserGroupInformation.setConfiguration(cfg);

// Debugging

This comment has been minimized.

Copy link
@jasontedor

jasontedor Mar 22, 2017

Member

If this is debugging, why we are logging at the info level?

This comment has been minimized.

Copy link
@jbaiera

jbaiera Mar 22, 2017

Author Contributor

Good catch. Should be fixed in 8845f2d.

plugins/repository-hdfs/src/main/java/org/elasticsearch/repositories/hdfs/HdfsRepository.java Outdated
LOGGER.info("Hadoop Security is [DISABLED]");
}
UserGroupInformation.AuthenticationMethod method = SecurityUtil.getAuthenticationMethod(cfg);
LOGGER.info("Using Hadoop authentication method : [" + method + "]");

This comment has been minimized.

Copy link
@jasontedor

jasontedor Mar 22, 2017

Member

Is this better on the debug level?

Copy link
Member

left a comment

I left one more comment that I forgot leave on the first review, sorry.

plugins/repository-hdfs/src/main/java/org/elasticsearch/repositories/hdfs/HdfsRepository.java Outdated
LOGGER.info("Hadoop Security is [DISABLED]");
}
UserGroupInformation.AuthenticationMethod method = SecurityUtil.getAuthenticationMethod(cfg);
LOGGER.info("Using Hadoop authentication method : [" + method + "]");

This comment has been minimized.

Copy link
@jasontedor

jasontedor Mar 22, 2017

Member

This will make the logger usage checker unhappy, can you rewrite this to be parameterized?

@rjernst

This comment has been minimized.

Copy link
Member

commented Mar 22, 2017

The changes look alright, but as I said before, I really don't trust any of it without tests. This may fix the issue for one particular configuration (which you tested manually), but we don't know that it works for other configurations. In addition to a plan for how to test this, I think we need an explicitly documented set of authentication types that we claim to support with hadoop. Then what needs to be tested should come naturally from that.

@jasontedor jasontedor removed the blocker label Mar 23, 2017
@jbaiera

This comment has been minimized.

Copy link
Contributor Author

commented Mar 23, 2017

After further scoping out testing requirements for this, I've come across some potentially nasty problems with how Hadoop tries to refresh Kerberos tickets in the CCACHE.

  1. Hadoop spawns a thread that refreshes the ticket in the background
  2. Hadoop needs the kerberos client packages installed on the local OS because...
  3. Hadoop spawns a child process (default is kinit -R) to do the ticket refresh

I haven't personally run into this issue yet, but that's because I haven't let the repository soak long enough for the testing ticket I've been using to need refreshing. Hadoop is basically sitting on its hands waiting to execute code that is questionable for our security model. I'm going to work on a clean reproduction of this to confirm, but the uncertainty around it makes me uncomfortable about making it a blocker for 5.3.0.

@jbaiera jbaiera removed the v5.3.0 label Mar 23, 2017
@jbaiera

This comment has been minimized.

Copy link
Contributor Author

commented Mar 23, 2017

I've made some local changes to see if I could force the command execution to trip the security checks and, to my dismay, succeeded. Alas, it seems that using a Kerberos TGT from the CCACHE in Elasticsearch is not going to be supportable unless Hadoop finds a better way to refresh Kerberos Tickets instead of forking a process to do so.

I have seen that logging in with a keytab does not launch this background refresh process in the Hadoop code. I'll dig into how the Kerberos auth module works to see if there's some potential to still allow authentication.

Copy link
Member

left a comment

I left one comment, it does not constitute a full review but only something that I noticed when I took a quick peek. I'll do a full review tomorrow.

...s/repository-hdfs/src/main/java/org/elasticsearch/repositories/hdfs/HdfsSecurityContext.java Outdated
* Expects keytab file to exist at {@code $CONFIG_DIR$/kerberos/krb5.keytab}
*/
public static Path locateKeytabFile(Environment environment) {
Path keytabPath = environment.configFile().resolve("kerberos").resolve("krb5.keytab");

This comment has been minimized.

Copy link
@jasontedor

jasontedor Apr 6, 2017

Member

I think that this should be in config/repository-hdfs/krb5.keytab rather than in a separate directory. This is the convention for plugin-specific configuration files.

@jasontedor

This comment has been minimized.

Copy link
Member

commented Apr 7, 2017

test this please

Copy link
Member

left a comment

Thanks @jbaiera, I left some comments.

docs/plugins/repository-hdfs.asciidoc Outdated

`kerberos`::

Authenticates to Hadoop through the usage of a Kerberos principal and ticket. The Kerberos ticket must be available to the user running the Elasticsearch process by means of the credential cache (ccache).

This comment has been minimized.

Copy link
@jasontedor

jasontedor Apr 7, 2017

Member

Is this documentation inaccurate now? I think that it should mention the keytab?

This comment has been minimized.

Copy link
@jbaiera

jbaiera Apr 7, 2017

Author Contributor

The docs still need some work at this point. None of this is accurate any longer. I'll address this soon.

plugins/repository-hdfs/src/main/java/org/elasticsearch/repositories/hdfs/HdfsRepository.java Outdated
@SuppressForbidden(reason = "InetAddress.getLocalHost()")
private static String findHostName() {
try {
return InetAddress.getLocalHost().getCanonicalHostName();

This comment has been minimized.

Copy link
@jasontedor

jasontedor Apr 7, 2017

Member

Can we leave a comment here explaining why this should not block as it's a bit indirect (Log4j does a lookup, it's cached by the JVM, default TTL with a security manager is infinite, etc.)?

plugins/repository-hdfs/src/main/java/org/elasticsearch/repositories/hdfs/HdfsRepository.java Outdated
}

@SuppressForbidden(reason = "InetAddress.getLocalHost()")
private static String findHostName() {

This comment has been minimized.

Copy link
@jasontedor

jasontedor Apr 7, 2017

Member

Nit: findHostName -> getHostName

...s/repository-hdfs/src/main/java/org/elasticsearch/repositories/hdfs/HdfsSecurityContext.java Outdated
// 4) Listen and resolve permissions for kerberos server principals
new SocketPermission("localhost:0", "listen,resolve"),
// We add the following since hadoop requires the client to re-login when the kerberos ticket expires:
// 5) All the permissions needed for UGI to do it's weird JAAS hack

This comment has been minimized.

Copy link
@jasontedor

jasontedor Apr 7, 2017

Member

Nit: it's -> its

...s/repository-hdfs/src/main/java/org/elasticsearch/repositories/hdfs/HdfsSecurityContext.java Outdated
* Locates the keytab file in the environment and verifies that it exists.
* Expects keytab file to exist at {@code $CONFIG_DIR$/kerberos/krb5.keytab}
*/
public static Path locateKeytabFile(Environment environment) {

This comment has been minimized.

Copy link
@jasontedor

jasontedor Apr 7, 2017

Member

I think that this can be package-private.

plugins/repository-hdfs/src/main/java/org/elasticsearch/repositories/hdfs/HdfsRepository.java Outdated
}
return UserGroupInformation.getCurrentUser();
} catch (IOException e) {
throw new RuntimeException("Could not retrieve the current user information", e);

This comment has been minimized.

Copy link
@jasontedor

jasontedor Apr 7, 2017

Member

UncheckedIOException?

...s/repository-hdfs/src/main/java/org/elasticsearch/repositories/hdfs/HdfsSecurityContext.java Outdated
try {
ugi.checkTGTAndReloginFromKeytab();
} catch (IOException ioe) {
throw new RuntimeException("Could not re-authenticate", ioe);

This comment has been minimized.

Copy link
@jasontedor

jasontedor Apr 7, 2017

Member

UncheckedIOException?

...s/repository-hdfs/src/main/java/org/elasticsearch/repositories/hdfs/HdfsSecurityContext.java Outdated
try {
if (Files.exists(keytabPath) == false) {
LOGGER.error("Could not locate keytab file at [{}].", keytabPath);
throw new RuntimeException("Could not locate keytab");

This comment has been minimized.

Copy link
@jasontedor

jasontedor Apr 7, 2017

Member

Include the path in the exception message? I question whether the logging statement is needed given that the exception message will effectively say the same thing? Can you throw this as a FileNotFoundException from here and let it bubble all the way up to a higher layer for wrapping?

This comment has been minimized.

Copy link
@jbaiera

jbaiera Apr 7, 2017

Author Contributor

Good point. Originally I was logging the path but since it was going to be user configurable at the start I figured it would be wise to not broadcast a potentially sensitive file location to any requestor who asks. Since we're standardizing on one canonical place for the file now I don't see why we can't add it back in.

@@ -107,43 +115,105 @@ protected void doStart() {
}

This comment has been minimized.

Copy link
@jasontedor

jasontedor Apr 7, 2017

Member

Commenting here since the line that I want to comment on is not changed here. I see that doStart catches an IOException and wraps it in an ElasticsearchGenerationException. That seems odd to me, I think this should be either UncheckedIOException or RuntimeException.

This comment has been minimized.

Copy link
@jbaiera

jbaiera Apr 7, 2017

Author Contributor

I pushed 6b325a1

...s/repository-hdfs/src/main/java/org/elasticsearch/repositories/hdfs/HdfsSecurityContext.java Outdated

// Append a kerberos.ServicePermission to only allow initiating kerberos connections
// as the logged in user.
permissions[permissions.length-1] = new ServicePermission(ugi.getUserName(), "initiate");

This comment has been minimized.

Copy link
@jasontedor

jasontedor Apr 7, 2017

Member

Nit: permissions.length-1 -> permissions.length - 1

@jbaiera jbaiera force-pushed the jbaiera:jbaiera-hdfs-repo-permissions branch Apr 20, 2017
jbaiera added 14 commits Mar 30, 2017
Revert back to property for keytab location.
…lob store execute code section and moved it to its own class.

Restricting kerberos initiation logic in the execute method to use the user specified principal only.
Removed the TicketEnforcer as it is no longer needed.
…ho might come along.
Removed the references to supporting TGT ticket cache login.
Added support documentation for keytabs.
Added guiding principles on ... principal ... names....
Added section on Authorization.
Added security.principal setting to the docs.
@clintongormley clintongormley added v6.0.0 and removed v6.0.0-alpha1 labels May 3, 2017
@jbaiera

This comment has been minimized.

Copy link
Contributor Author

commented May 3, 2017

@rjernst @jasontedor Any thing else needed for this at this time? I'd like to see if I can get this merged in.

Copy link
Member

left a comment

I left one more comment, it does not require another round from me but please make the change.

plugins/repository-hdfs/src/main/java/org/elasticsearch/repositories/hdfs/HdfsRepository.java Outdated
AbstractFileSystem fs = AbstractFileSystem.get(uri, cfg);
return FileContext.getFileContext(fs, cfg);
AbstractFileSystem fs = AbstractFileSystem.get(uri, hadoopConfiguration);
return FileContext.getFileContext(fs, hadoopConfiguration);
} catch (UnsupportedFileSystemException e) {
throw new RuntimeException(e);

This comment has been minimized.

Copy link
@jasontedor

jasontedor May 3, 2017

Member

Let's make this an UncheckedIOException too.

@Divyakoganti

This comment has been minimized.

Copy link

commented May 3, 2017

Hi,
Thanks for the information.
I need some information on configure hdfs repository with kerebos which i am not able to do.
i read from the posts that its still in progress as :
Kerberos authentication for the HDFS Repository Snapshot plugin does not function correctly in 5.x at the moment. There is currently an effort to re-add this functionality so that it may function well with the internal security manager.
Can you please help me when can i get this functionality added or any other method for using this plugin.

Divya

@jbaiera jbaiera merged commit f5edd50 into elastic:master May 4, 2017
2 checks passed
2 checks passed
CLA Commit author is a member of Elasticsearch
Details
elasticsearch-ci Build finished.
Details
@jbaiera jbaiera deleted the jbaiera:jbaiera-hdfs-repo-permissions branch May 4, 2017
@clintongormley clintongormley added v6.0.0-alpha1 and removed v6.0.0 labels May 9, 2017
jbaiera added a commit that referenced this pull request May 10, 2017
…tory (#23439)

Added missing permissions required for authenticating with Kerberos to HDFS. Also implemented
code to support authentication in the form of using a Kerberos keytab file. In order to support
HDFS authentication, users must install a Kerberos keytab file on each node and transfer it to the
configuration directory. When a user specifies a Kerberos principal in the repository settings the
plugin automatically enables security for Hadoop and begins the login process. There will be a
separate PR and commit for the testing infrastructure to support these changes.
jbaiera added a commit that referenced this pull request May 10, 2017
…tory (#23439)

Added missing permissions required for authenticating with Kerberos to HDFS. Also implemented
code to support authentication in the form of using a Kerberos keytab file. In order to support
HDFS authentication, users must install a Kerberos keytab file on each node and transfer it to the
configuration directory. When a user specifies a Kerberos principal in the repository settings the
plugin automatically enables security for Hadoop and begins the login process. There will be a
separate PR and commit for the testing infrastructure to support these changes.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
6 participants
You can’t perform that action at this time.