Accumulo 2.x Documentation >> Security >> Kerberos

Kerberos

Edit this page

Overview

Kerberos is a network authentication protocol that provides a secure way for peers to prove their identity over an unsecure network in a client-server model. A centralized key-distribution center (KDC) is the service that coordinates authentication between a client and a server. Clients and servers use “tickets”, obtained from the KDC via a password or a special file called a “keytab”, to communicate with the KDC and prove their identity. A KDC administrator must create the principal (name for the client/server identity) and the password or keytab, securely passing the necessary information to the actual user/service. Properly securing the KDC and generated ticket material is central to the security model and is mentioned only as a warning to administrators running their own KDC.

To interact with Kerberos programmatically, GSSAPI and SASL are two standards which allow cross-language integration with Kerberos for authentication. GSSAPI, the generic security service application program interface, is a standard which Kerberos implements. In the Java programming language, the language itself also implements GSSAPI which is leveraged by other applications, like Apache Hadoop and Apache Thrift. SASL, simple authentication and security layer, is a framework for authentication and security over the network. SASL provides a number of mechanisms for authentication, one of which is GSSAPI. Thus, SASL provides the transport which authenticates using GSSAPI that Kerberos implements.

Kerberos is a very complicated software application and is deserving of much more description than can be provided here. An explain like I’m 5 blog post is very good at distilling the basics, while MIT Kerberos’s project page contains lots of documentation for users or administrators. Various Hadoop “vendors” also provide free documentation that includes step-by-step instructions for configuring Hadoop and ZooKeeper (which will be henceforth considered as prerequisites).

Within Hadoop

Out of the box, HDFS and YARN have no ability to enforce that a user is who they claim they are. Thus, any basic Hadoop installation should be treated as unsecure: any user with access to the cluster has the ability to access any data. Using Kerberos to provide authentication, users can be strongly identified, delegating to Kerberos to determine who a user is and enforce that a user is who they claim to be. As such, Kerberos is widely used across the entire Hadoop ecosystem for strong authentication. Since server processes accessing HDFS or YARN are required to use Kerberos to authenticate with HDFS, it makes sense that they also require Kerberos authentication from their clients, in addition to other features provided by SASL.

A typical deployment involves the creation of Kerberos principals for all server processes (Hadoop datanodes and namenode(s), ZooKeepers), the creation of a keytab file for each principal and then proper configuration for the Hadoop site xml files. Users also need Kerberos principals created for them; however, a user typically uses a password to identify themselves instead of a keytab. Users can obtain a ticket granting ticket (TGT) from the KDC using their password which allows them to authenticate for the lifetime of the TGT (typically one day by default) and alleviates the need for further password authentication.

For client server applications, like web servers, a keytab can be created which allow for fully-automated Kerberos identification removing the need to enter any password, at the cost of needing to protect the keytab file. These principals will apply directly to authentication for clients accessing Accumulo and the Accumulo processes accessing HDFS.

Delegation Tokens

MapReduce, a common way that clients interact with Accumulo, does not map well to the client-server model that Kerberos was originally designed to support. Specifically, the parallelization of tasks across many nodes introduces the problem of securely sharing the user credentials across these tasks in as safe a manner as possible. To address this problem, Hadoop introduced the notion of a delegation token to be used in distributed execution settings.

A delegation token is nothing more than a short-term, on-the-fly password generated after authenticating with the user’s credentials. In Hadoop itself, the Namenode and ResourceManager, for HDFS and YARN respectively, act as the gateway for delegation tokens requests. For example, before a YARN job is submitted, the implementation will request delegation tokens from the NameNode and ResourceManager so the YARN tasks can communicate with HDFS and YARN. In the same manner, support has been added in the Accumulo Manager to generate delegation tokens to enable interaction with Accumulo via MapReduce when Kerberos authentication is enabled in a manner similar to HDFS and YARN.

Generating an expiring password is, arguably, more secure than distributing the user’s credentials across the cluster as only access to HDFS, YARN and Accumulo would be compromised in the case of the token being compromised as opposed to the entire Kerberos credential. Additional details for clients and servers will be covered in subsequent sections.

Configuring Accumulo

To configure Accumulo for use with Kerberos, both client-facing and server-facing changes must be made for a functional system on secured Hadoop. As previously mentioned, numerous guidelines already exist on the subject of configuring Hadoop and ZooKeeper for use with Kerberos and won’t be covered here. It is assumed that you have functional Hadoop and ZooKeeper already installed.

Note that on an existing cluster the server side changes will require a full cluster shutdown and restart. You should wait to restart the TraceServers until after you’ve completed the rest of the cluster set up and provisioned a trace user with appropriate permissions.

Servers

The first step is to obtain a Kerberos identity for the Accumulo server processes. When running Accumulo with Kerberos enabled, a valid Kerberos identity will be required to initiate any RPC between Accumulo processes (e.g. Manager and TabletServer) in addition to any HDFS action (e.g. client to HDFS or TabletServer to HDFS).

Generate Principal and Keytab

In the kadmin.local shell or using the -q option on kadmin.local, create a principal for Accumulo for all hosts that are running Accumulo processes. A Kerberos principal is of the form “primary/instance@REALM”. “accumulo” is commonly the “primary” (although not required) and the “instance” is the fully-qualified domain name for the host that will be running the Accumulo process – this is required.

kadmin.local -q "addprinc -randkey accumulo/host.domain.com"

Perform the above for each node running Accumulo processes in the instance, modifying “host.domain.com” for your network. The randkey option generates a random password because we will use a keytab for authentication, not a password, since the Accumulo server processes don’t have an interactive console to enter a password into.

kadmin.local -q "xst -k accumulo.hostname.keytab accumulo/host.domain.com"

To simplify deployments, at the cost of security, all Accumulo principals could be globbed into a single keytab

kadmin.local -q "xst -k accumulo.service.keytab -glob accumulo*"

To ensure that the SASL handshake can occur from clients to servers and servers to servers, all Accumulo servers must share the same instance and realm principal components as the “client” needs to know these to set up the connection with the “server”.

Server Configuration

A number of properties need to be changed to account to properly configure servers in accumulo.properties.

Key Suggested Value Description
general.kerberos.keytab /etc/security/keytabs/accumulo.service.keytab The path to the keytab for Accumulo on local filesystem. Change the value to the actual path on your system.
general.kerberos.principal accumulo/_HOST@REALM The Kerberos principal for Accumulo, needs to match the keytab. “_HOST” can be used instead of the actual hostname in the principal and will be automatically expanded to the current FQDN which reduces the configuration file burden.
instance.rpc.sasl.enabled true Enables SASL for the Thrift Servers (supports GSSAPI)
rpc.sasl.qop auth One of “auth”, “auth-int”, or “auth-conf”. These map to the SASL defined properties for quality of protection. “auth” is authentication only. “auth-int” is authentication and data integrity. “auth-conf” is authentication, data integrity and confidentiality.
instance.security.authenticator org.apache.accumulo.server.security.handler.KerberosAuthenticator Configures Accumulo to use the Kerberos principal as the Accumulo username/principal
instance.security.authorizor org.apache.accumulo.server.security.handler.KerberosAuthorizor Configures Accumulo to use the Kerberos principal for authorization purposes
instance.security.permissionHandler org.apache.accumulo.server.security.handler.KerberosPermissionHandler Configures Accumulo to use the Kerberos principal for permission purposes
trace.token.type org.apache.accumulo.core.client.security.tokens.KerberosToken Configures the Accumulo Tracer to use the KerberosToken for authentication when serializing traces to the trace table.
trace.user accumulo/_HOST@REALM The tracer process needs valid credentials to serialize traces to Accumulo. While the other server processes are creating a SystemToken from the provided keytab and principal, we can still use a normal KerberosToken and the same keytab/principal to serialize traces. Like non-Kerberized instances, the table must be created and permissions granted to the trace.user. The same _HOST replacement is performed on this value, substituted the FQDN for _HOST.
trace.token.property.keytab   You can optionally specify the path to a keytab file for the principal given in the trace.user property. If you don’t set this path, it will default to the value given in general.kerberos.principal.
general.delegation.token.lifetime 7d The length of time that the server-side secret used to create delegation tokens is valid. After a server-side secret expires, a delegation token created with that secret is no longer valid.
general.delegation.token.update.interval 1d The frequency in which new server-side secrets should be generated to create delegation tokens for clients. Generating new secrets reduces the likelihood of cryptographic attacks.

Although it should be a prerequisite, it is ever important that you have DNS properly configured for your nodes and that Accumulo is configured to use the FQDN. It is extremely important to use the FQDN in each of the “hosts” files for each Accumulo process: managers, monitors, tservers, tracers, and gc.

Normally, no changes are needed in accumulo-env.sh to enable Kerberos. Typically, the krb5.conf is installed on the local machine in /etc/, and the Java library implementations will look here to find the necessary configuration to communicate with the KDC. Some installations may require a different krb5.conf to be used for Accumulo which can be accomplished by adding the JVM system property -Djava.security.krb5.conf=/path/to/other/krb5.conf to JAVA_OPTS in accumulo-env.sh.

KerberosAuthenticator

The KerberosAuthenticator is an implementation of the pluggable security interfaces that Accumulo provides. It builds on top of what the default ZooKeeper-based implementation, but removes the need to create user accounts with passwords in Accumulo for clients. As long as a client has a valid Kerberos identity, they can connect to and interact with Accumulo, but without any permissions (e.g. cannot create tables or write data). Leveraging ZooKeeper removes the need to change the permission handler and authorizor, so other Accumulo functions regarding permissions and cell-level authorizations do not change.

It is extremely important to note that, while user operations like SecurityOperations.listLocalUsers(), SecurityOperations.dropLocalUser(), and SecurityOperations.createLocalUser() will not return errors, these methods are not equivalent to normal installations, as they will only operate on users which have, at one point in time, authenticated with Accumulo using their Kerberos identity. The KDC is still the authoritative entity for user management. The previously mentioned methods are provided as they simplify management of users within Accumulo, especially with respect to granting Authorizations and Permissions to new users.

Administrative User

Out of the box (without Kerberos enabled), Accumulo has a single user with administrative permissions “root”. This user is used to “bootstrap” other users, creating less-privileged users for applications using the system. In Kerberos, to authenticate with the system, it’s required that the client presents Kerberos credentials for the principal (user) the client is trying to authenticate as.

Because of this, an administrative user named “root” would be useless in an instance using Kerberos, because it is very unlikely to have Kerberos credentials for a principal named root. When Kerberos is enabled, Accumulo will prompt for the name of a user to grant the same permissions as what the root user would normally have. The name of the Accumulo user to grant administrative permissions to can also be given by the -u or --user options.

If you are enabling Kerberos on an existing cluster, you will need to reinitialize the security system in order to replace the existing “root” user with one that can be used with Kerberos. These steps should be completed after you have done the previously described configuration changes and will require access to a complete accumulo.properties, including the instance secret. Note that this process will delete all existing users in the system; you will need to reassign user permissions based on Kerberos principals.

  1. Ensure Accumulo is not running.
  2. Given the path to a accumulo.properties with the instance secret, run the security reset tool. If you are prompted for a password you can just hit return, since it won’t be used.
  3. Start the Accumulo cluster
$ accumulo-cluster stop
...
$ accumulo init --reset-security
Running against secured HDFS
Principal (user) to grant administrative privileges to : acculumo_admin@EXAMPLE.COM
Enter initial password for accumulo_admin@EXAMPLE.COM (this may not be applicable for your security setup):
Confirm initial password for accumulo_admin@EXAMPLE.COM:
$ accumulo-cluster start
...

Verifying secure access

To verify that servers have correctly started with Kerberos enabled, ensure that the processes are actually running (they should exit immediately if login fails) and verify that you see something similar to the following in the application log.

2015-01-07 11:57:56,826 [security.SecurityUtil] INFO : Attempting to login with keytab as accumulo/hostname@EXAMPLE.COM
2015-01-07 11:57:56,830 [security.UserGroupInformation] INFO : Login successful for user accumulo/hostname@EXAMPLE.COM using keytab file /etc/security/keytabs/accumulo.service.keytab

Impersonation

Impersonation is functionality which allows a certain user to act as another. One direct application of this concept within Accumulo is the Thrift proxy. The Thrift proxy is configured to accept user requests and pass them onto Accumulo, enabling client access to Accumulo via any thrift-compatible language. When the proxy is running with SASL transports, this enforces that clients present a valid Kerberos identity to make a connection. In this situation, the Thrift proxy server does not have access to the secret key material in order to make a secure connection to Accumulo as the client, it can only connect to Accumulo as itself. Impersonation, in this context, refers to the ability of the proxy to authenticate to Accumulo as itself, but act on behalf of an Accumulo user.

Accumulo supports basic impersonation of end-users by a third party via static rules in accumulo.properties. These two properties are semicolon separated properties which are aligned by index. This first element in the user impersonation property value matches the first element in the host impersonation property value, etc.

instance.rpc.sasl.allowed.user.impersonation=$PROXY_USER:*
instance.rpc.sasl.allowed.host.impersonation=*

Here, $PROXY_USER can impersonate any user from any host.

The following is an example of specifying a subset of users $PROXY_USER can impersonate and also limiting the hosts from which $PROXY_USER can initiate requests from.

instance.rpc.sasl.allowed.user.impersonation=$PROXY_USER:user1,user2;$PROXY_USER2:user2,user4
instance.rpc.sasl.allowed.host.impersonation=host1.domain.com,host2.domain.com;*

Here, $PROXY_USER can impersonate user1 and user2 only from host1.domain.com or host2.domain.com. $PROXY_USER2 can impersonate user2 and user4 from any host.

In these examples, the value $PROXY_USER is the Kerberos principal of the server which is acting on behalf of a user. Impersonation is enforced by the Kerberos principal and the host from which the RPC originated (from the perspective of the Accumulo TabletServers/Managers). An asterisk (*) can be used to specify all users or all hosts (depending on the context).

Delegation Tokens

Within Accumulo services, the primary task to implement delegation tokens is the generation and distribution of a shared secret among all Accumulo tabletservers and the manager. The secret key allows for generation of delegation tokens for users and verification of delegation tokens presented by clients. If a server process is unaware of the secret key used to create a delegation token, the client cannot be authenticated. As ZooKeeper distribution is an asynchronous operation (typically on the order of seconds), the value for general.delegation.token.update.interval should be on the order of hours to days to reduce the likelihood of servers rejecting valid clients because the server did not yet see a new secret key.

Supporting authentication with both Kerberos credentials and delegation tokens, the SASL thrift server accepts connections with either GSSAPI and DIGEST-MD5 mechanisms set. The DIGEST-MD5 mechanism enables authentication as a normal username and password exchange which DelegationTokens leverages.

Since delegation tokens are a weaker form of authentication than Kerberos credentials, user access to obtain delegation tokens from Accumulo is protected with the DELEGATION_TOKEN system permission. Only users with the system permission are allowed to obtain delegation tokens. It is also recommended to configure confidentiality with SASL, using the rpc.sasl.qop=auth-conf configuration property, to ensure that prying eyes cannot view the DelegationToken as it passes over the network.

# Check a user's permissions
admin@REALM@accumulo> userpermissions -u user@REALM

# Grant the DELEGATION_TOKEN system permission to a user
admin@REALM@accumulo> grant System.DELEGATION_TOKEN -s -u user@REALM

Clients

Create client principal

Like the Accumulo servers, clients must also have a Kerberos principal created for them. The primary difference between a server principal is that principals for users are created with a password and also not qualified to a specific instance (host).

kadmin.local -q "addprinc $user"

The above will prompt for a password for that user which will be used to identify that $user. The user can verify that they can authenticate with the KDC using the command kinit $user. Upon entering the correct password, a local credentials cache will be made which can be used to authenticate with Accumulo, access HDFS, etc.

The user can verify the state of their local credentials cache by using the command klist.

$ klist
Ticket cache: FILE:/tmp/krb5cc_123
Default principal: user@EXAMPLE.COM

Valid starting       Expires              Service principal
01/07/2015 11:56:35  01/08/2015 11:56:35  krbtgt/EXAMPLE.COM@EXAMPLE.COM
    renew until 01/14/2015 11:56:35

Configuration

The second thing clients need to do is to configure kerberos when an Accumulo client is created. This can be done using client builder methods or by setting the properties below in accumulo-client.properties which can be provided to Accumulo utilities using the --config-file command line option.

Each of these properties must match the configuration of the accumulo servers; this is required to set up the SASL transport.

Verifying Administrative Access

At this point you should have enough configured on the server and client side to interact with the system. You should verify that the administrative user you chose earlier can successfully interact with the system.

While this example logs in via kinit with a password, any login method that caches Kerberos tickets should work.

$ kinit accumulo_admin@EXAMPLE.COM
Password for accumulo_admin@EXAMPLE.COM: ******************************
$ accumulo shell

Shell - Apache Accumulo Interactive Shell
-
- version: 1.7.2
- instance name: MYACCUMULO
- instance id: 483b9038-889f-4b2d-b72b-dfa2bb5dbd07
-
- type 'help' for a list of available commands
-
accumulo_admin@EXAMPLE.COM@MYACCUMULO> userpermissions
System permissions: System.GRANT, System.CREATE_TABLE, System.DROP_TABLE, System.ALTER_TABLE, System.CREATE_USER, System.DROP_USER, System.ALTER_USER, System.SYSTEM, System.CREATE_NAMESPACE, System.DROP_NAMESPACE, System.ALTER_NAMESPACE, System.OBTAIN_DELEGATION_TOKEN

Namespace permissions (accumulo): Namespace.READ, Namespace.ALTER_TABLE

Table permissions (accumulo.metadata): Table.READ, Table.ALTER_TABLE
Table permissions (accumulo.replication): Table.READ
Table permissions (accumulo.root): Table.READ, Table.ALTER_TABLE

accumulo_admin@EXAMPLE.COM@MYACCUMULO> quit
$ kdestroy
$

DelegationTokens with MapReduce

To use DelegationTokens in a custom MapReduce job, the user should create an AccumuloClient using a KerberosToken and use it to call SecurityOperations.getDelegationToken. The DelegationToken that is created can then be used to create a new client using this delegation token. The ClientInfo object from this client can be passed into the MapReduce job. It is expected that the user launching the MapReduce job is already logged in via Kerberos via a keytab or via a locally-cached Kerberos ticket-granting-ticket (TGT).

KerberosToken kt = new KerberosToken();
AccumuloClient client = Accumulo.newClient().to("myinstance", "zoo1,zoo2")
                          .as(principal, kt).build();
DelegationToken dt = client.securityOperations().getDelegationToken();
Properties props = Accumulo.newClientProperties().from(client.properties())
                          .as(principal, dt).build();

// Reading from Accumulo
AccumuloInputFormat.configure().clientProperties(props).store(job);

// Writing to Accumulo
AccumuloOutputFormat.configure().clientProperties(props).store(job);

Users must have the DELEGATION_TOKEN system permission to call the getDelegationToken method. The obtained delegation token is only valid for the requesting user for a period of time dependent on Accumulo’s configuration (general.delegation.token.lifetime).

For the duration of validity of the DelegationToken, the user must take the necessary precautions to protect the DelegationToken from prying eyes as it can be used by any user on any host to impersonate the user who requested the DelegationToken. YARN ensures that passing the delegation token from the client JVM to each YARN task is secure, even in multi-tenant instances.

Debugging

Q: I have valid Kerberos credentials and a correct client configuration file, but I still get errors like:

java.io.IOException: Failed on local exception: java.io.IOException: javax.security.sasl.SaslException: GSS initiate failed [Caused by GSSException: No valid credentials provided (Mechanism level: Failed to find any Kerberos tgt)]

A: When you have a valid client configuration and Kerberos TGT, it is possible that the search path for your local credentials cache is incorrect. Check the value of the KRB5CCNAME environment value, and ensure it matches the value reported by klist.

$ echo $KRB5CCNAME

$ klist
Ticket cache: FILE:/tmp/krb5cc_123
Default principal: user@EXAMPLE.COM

Valid starting       Expires              Service principal
01/07/2015 11:56:35  01/08/2015 11:56:35  krbtgt/EXAMPLE.COM@EXAMPLE.COM
    renew until 01/14/2015 11:56:35
$ export KRB5CCNAME=/tmp/krb5cc_123
$ echo $KRB5CCNAME
/tmp/krb5cc_123

Q: I thought I had everything configured correctly, but my client/server still fails to log in. I don’t know what is actually failing.

A: Add the following system property to the JVM invocation:

-Dsun.security.krb5.debug=true

This will enable lots of extra debugging at the JVM level which is often sufficient to diagnose some high-level configuration problem. Client applications can add this system property by hand to the command line and Accumulo server processes or applications started using the accumulo script by adding the property to JAVA_OPTS in accumulo-env.sh.

Additionally, you can increase the log4j levels on org.apache.hadoop.security, which includes the Hadoop UserGroupInformation class, which will include some high-level debug statements. This can be controlled in your client application, or using log4j-service.properties

Q: All of my Accumulo processes successfully start and log in with their keytab, but they are unable to communicate with each other, showing the following errors:

2015-01-12 14:47:27,055 [transport.TSaslTransport] ERROR: SASL negotiation failure
javax.security.sasl.SaslException: GSS initiate failed [Caused by GSSException: No valid credentials provided (Mechanism level: Server not found in Kerberos database (7) - LOOKING_UP_SERVER)]
        at com.sun.security.sasl.gsskerb.GssKrb5Client.evaluateChallenge(GssKrb5Client.java:212)
        at org.apache.thrift.transport.TSaslClientTransport.handleSaslStartMessage(TSaslClientTransport.java:94)
        at org.apache.thrift.transport.TSaslTransport.open(TSaslTransport.java:253)
        at org.apache.thrift.transport.TSaslClientTransport.open(TSaslClientTransport.java:37)
        at org.apache.accumulo.core.rpc.UGIAssumingTransport$1.run(UGIAssumingTransport.java:53)
        at org.apache.accumulo.core.rpc.UGIAssumingTransport$1.run(UGIAssumingTransport.java:49)
        at java.security.AccessController.doPrivileged(Native Method)
        at javax.security.auth.Subject.doAs(Subject.java:415)
        at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1628)
        at org.apache.accumulo.core.rpc.UGIAssumingTransport.open(UGIAssumingTransport.java:49)
        at org.apache.accumulo.core.rpc.ThriftUtil.createClientTransport(ThriftUtil.java:357)
        at org.apache.accumulo.core.rpc.ThriftUtil.createTransport(ThriftUtil.java:255)
        at org.apache.accumulo.server.manager.LiveTServerSet$TServerConnection.getTableMap(LiveTServerSet.java:106)
        at org.apache.accumulo.manager.Manager.gatherTableInformation(Manager.java:996)
        at org.apache.accumulo.manager.Manager.access$600(Manager.java:160)
        at org.apache.accumulo.manager.Manager$StatusThread.updateStatus(Manager.java:911)
        at org.apache.accumulo.manager.Manager$StatusThread.run(Manager.java:901)
Caused by: GSSException: No valid credentials provided (Mechanism level: Server not found in Kerberos database (7) - LOOKING_UP_SERVER)
        at sun.security.jgss.krb5.Krb5Context.initSecContext(Krb5Context.java:710)
        at sun.security.jgss.GSSContextImpl.initSecContext(GSSContextImpl.java:248)
        at sun.security.jgss.GSSContextImpl.initSecContext(GSSContextImpl.java:179)
        at com.sun.security.sasl.gsskerb.GssKrb5Client.evaluateChallenge(GssKrb5Client.java:193)
        ... 16 more
Caused by: KrbException: Server not found in Kerberos database (7) - LOOKING_UP_SERVER
        at sun.security.krb5.KrbTgsRep.<init>(KrbTgsRep.java:73)
        at sun.security.krb5.KrbTgsReq.getReply(KrbTgsReq.java:192)
        at sun.security.krb5.KrbTgsReq.sendAndGetCreds(KrbTgsReq.java:203)
        at sun.security.krb5.internal.CredentialsUtil.serviceCreds(CredentialsUtil.java:309)
        at sun.security.krb5.internal.CredentialsUtil.acquireServiceCreds(CredentialsUtil.java:115)
        at sun.security.krb5.Credentials.acquireServiceCreds(Credentials.java:454)
        at sun.security.jgss.krb5.Krb5Context.initSecContext(Krb5Context.java:641)
        ... 19 more
Caused by: KrbException: Identifier doesn't match expected value (906)
        at sun.security.krb5.internal.KDCRep.init(KDCRep.java:143)
        at sun.security.krb5.internal.TGSRep.init(TGSRep.java:66)
        at sun.security.krb5.internal.TGSRep.<init>(TGSRep.java:61)
        at sun.security.krb5.KrbTgsRep.<init>(KrbTgsRep.java:55)
        ... 25 more

or

2015-01-12 14:47:29,440 [server.TThreadPoolServer] ERROR: Error occurred during processing of message.
java.lang.RuntimeException: org.apache.thrift.transport.TTransportException: Peer indicated failure: GSS initiate failed
        at org.apache.thrift.transport.TSaslServerTransport$Factory.getTransport(TSaslServerTransport.java:219)
        at org.apache.accumulo.core.rpc.UGIAssumingTransportFactory$1.run(UGIAssumingTransportFactory.java:51)
        at org.apache.accumulo.core.rpc.UGIAssumingTransportFactory$1.run(UGIAssumingTransportFactory.java:48)
        at java.security.AccessController.doPrivileged(Native Method)
        at javax.security.auth.Subject.doAs(Subject.java:356)
        at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1608)
        at org.apache.accumulo.core.rpc.UGIAssumingTransportFactory.getTransport(UGIAssumingTransportFactory.java:48)
        at org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:208)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
        at java.lang.Thread.run(Thread.java:745)
Caused by: org.apache.thrift.transport.TTransportException: Peer indicated failure: GSS initiate failed
        at org.apache.thrift.transport.TSaslTransport.receiveSaslMessage(TSaslTransport.java:190)
        at org.apache.thrift.transport.TSaslServerTransport.handleSaslStartMessage(TSaslServerTransport.java:125)
        at org.apache.thrift.transport.TSaslTransport.open(TSaslTransport.java:253)
        at org.apache.thrift.transport.TSaslServerTransport.open(TSaslServerTransport.java:41)
        at org.apache.thrift.transport.TSaslServerTransport$Factory.getTransport(TSaslServerTransport.java:216)
        ... 10 more

A: As previously mentioned, the hostname, and subsequently the address each Accumulo process is bound/listening on, is extremely important when negotiating an SASL connection. This problem commonly arises when the Accumulo servers are not configured to listen on the address denoted by their FQDN.

The values in the Accumulo “hosts” files (In accumulo/conf: managers, monitors, tservers, tracers, and gc) should match the instance component of the Kerberos server principal (e.g. host in accumulo/host@EXAMPLE.COM).

Q: After configuring my system for Kerberos, server processes come up normally and I can interact with the system. However, when I attempt to use the “Recent Traces” page on the Monitor UI I get a stacktrace similar to:

java.lang.AssertionError: AuthenticationToken should not be null
    at org.apache.accumulo.monitor.servlets.trace.Basic.getScanner(Basic.java:139)
    at org.apache.accumulo.monitor.servlets.trace.Summary.pageBody(Summary.java:164)
    at org.apache.accumulo.monitor.servlets.BasicServlet.doGet(BasicServlet.java:63)
    at javax.servlet.http.HttpServlet.service(HttpServlet.java:687)
    at javax.servlet.http.HttpServlet.service(HttpServlet.java:790)
    at org.eclipse.jetty.servlet.ServletHolder.handle(ServletHolder.java:738)
    at org.eclipse.jetty.servlet.ServletHandler.doHandle(ServletHandler.java:551)
    at org.eclipse.jetty.server.handler.ScopedHandler.handle(ScopedHandler.java:143)
    at org.eclipse.jetty.security.SecurityHandler.handle(SecurityHandler.java:568)
    at org.eclipse.jetty.server.session.SessionHandler.doHandle(SessionHandler.java:221)
    at org.eclipse.jetty.server.handler.ContextHandler.doHandle(ContextHandler.java:1111)
    at org.eclipse.jetty.servlet.ServletHandler.doScope(ServletHandler.java:478)
    at org.eclipse.jetty.server.session.SessionHandler.doScope(SessionHandler.java:183)
    at org.eclipse.jetty.server.handler.ContextHandler.doScope(ContextHandler.java:1045)
    at org.eclipse.jetty.server.handler.ScopedHandler.handle(ScopedHandler.java:141)
    at org.eclipse.jetty.server.handler.HandlerWrapper.handle(HandlerWrapper.java:97)
    at org.eclipse.jetty.server.Server.handle(Server.java:462)
    at org.eclipse.jetty.server.HttpChannel.handle(HttpChannel.java:279)
    at org.eclipse.jetty.server.HttpConnection.onFillable(HttpConnection.java:232)
    at org.eclipse.jetty.io.AbstractConnection$2.run(AbstractConnection.java:534)
    at org.eclipse.jetty.util.thread.QueuedThreadPool.runJob(QueuedThreadPool.java:607)
    at org.eclipse.jetty.util.thread.QueuedThreadPool$3.run(QueuedThreadPool.java:536)
    at java.lang.Thread.run(Thread.java:745)

A: This indicates that the Monitor has not been able to successfully log in a client-side user to read from the trace table. Accumulo allows the TraceServer to rely on the property general.kerberos.keytab as a fallback when logging in the trace user if the trace.token.property.keytab property isn’t defined. Some earlier versions of Accumulo did not do this same fallback for the Monitor’s use of the trace user. The end result is that if you configure general.kerberos.keytab and not trace.token.property.keytab you will end up with a system that properly logs trace information but can’t view it.

Ensure you have set trace.token.property.keytab to point to a keytab for the principal defined in trace.user in the accumulo.properties file for the Monitor, since that should work in all versions of Accumulo.

Find documentation for all releases in the archive Edit this page