It is known that Hadoop's implementation of Kerberos authentication tokens is plagued by lack of thread safety (see https://issues.apache.org/jira/browse/HADOOP-13066 for some discussion) and UserGroupInformation is notoriously difficult to reason about.
Accumulo does not currently support the kind of multi-threaded behavior you're using, but with some work, we probably could. Have you any insight into what kinds of code changes would be required to properly support this multi-threaded case with separate Kerberos users in Accumulo? On Tue, Jul 3, 2018 at 7:33 PM mhd wrk <[email protected]> wrote: > Here's the test case conditions: > > -Kerberized cluster > -Thread one authenticates as user1 (using keytab) and start performing a > long running task on a specific table. > -Thread two simply authenticates as user2 (using username and password ). > > My observation is that as soon as thread two logins, thread one runs into > the exception below. > > java.lang.RuntimeException: > org.apache.accumulo.core.client.AccumuloSecurityException: Error > BAD_CREDENTIALS for user Principal in credentials object should match > kerberos principal. Expected 'user2@example' but was 'user1@example' on > table user1.test_table(ID:3) - Username or Password is Invalid > at > org.apache.accumulo.core.client.impl.ScannerIterator.hasNext(ScannerIterator.java:161) > at java.lang.Iterable.forEach(Iterable.java:74) > at com.example.test.TestBug$1.run(TestBug.java:53) > at java.lang.Thread.run(Thread.java:748) > Caused by: org.apache.accumulo.core.client.AccumuloSecurityException: > Error BAD_CREDENTIALS for user Principal in credentials object should match > kerberos principal. Expected 'user2@example' but was 'user1@example' on > table user1.test_table(ID:3) - Username or Password is Invalid > at > org.apache.accumulo.core.client.impl.ThriftScanner.scan(ThriftScanner.java:465) > at > org.apache.accumulo.core.client.impl.ThriftScanner.scan(ThriftScanner.java:285) > at > org.apache.accumulo.core.client.impl.ScannerIterator$Reader.run(ScannerIterator.java:80) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) > at > org.apache.accumulo.fate.util.LoggingRunnable.run(LoggingRunnable.java:35) > ... 1 more > Caused by: ThriftSecurityException(user:Principal in credentials object > should match kerberos principal. Expected 'user2@example' but was > 'user1@example', code:BAD_CREDENTIALS) > at > org.apache.accumulo.core.tabletserver.thrift.TabletClientService$startScan_result$startScan_resultStandardScheme.read(TabletClientService.java:6696) > at > org.apache.accumulo.core.tabletserver.thrift.TabletClientService$startScan_result$startScan_resultStandardScheme.read(TabletClientService.java:6673) > at > org.apache.accumulo.core.tabletserver.thrift.TabletClientService$startScan_result.read(TabletClientService.java:6596) > at org.apache.thrift.TServiceClient.receiveBase(TServiceClient.java:78) > at > org.apache.accumulo.core.tabletserver.thrift.TabletClientService$Client.recv_startScan(TabletClientService.java:232) > at > org.apache.accumulo.core.tabletserver.thrift.TabletClientService$Client.startScan(TabletClientService.java:208) > at > org.apache.accumulo.core.client.impl.ThriftScanner.scan(ThriftScanner.java:410) > ... 6 more > > > > ======================== > Java class to reproduce the issue > ======================== > > package com.example.test; > > import org.apache.accumulo.core.client.ClientConfiguration; > import org.apache.accumulo.core.client.Connector; > import org.apache.accumulo.core.client.ZooKeeperInstance; > import org.apache.accumulo.core.client.security.tokens.KerberosToken; > import org.apache.accumulo.core.security.Authorizations; > import org.apache.hadoop.conf.Configuration; > import org.apache.hadoop.fs.Path; > import org.apache.hadoop.security.UserGroupInformation; > > import javax.security.auth.callback.Callback; > import javax.security.auth.callback.CallbackHandler; > import javax.security.auth.callback.NameCallback; > import javax.security.auth.callback.PasswordCallback; > import javax.security.auth.callback.UnsupportedCallbackException; > import javax.security.auth.login.LoginContext; > import java.io.File; > import java.io.IOException; > import java.security.PrivilegedExceptionAction; > > public class TestBug { > > public static void main(String[] args) throws Exception { > > final String hadoopHome = "/path/to/hadoophome"; > final String hadoopConfigFile = "/path/to/my-site.xml"; > > final String accumuloTableName = "test_table"; > > final String user1Name = "user1@example"; > final String user1Keytab = "/etc/security/keytabs/user1.keytab"; > > final String user2Name = "user2@example"; > final String user2Password = "user2password"; > > System.out.println("===================== Initializing Hadoop"); > System.setProperty("hadoop.home.dir", hadoopHome); > Configuration hadoopConf = new Configuration(); > hadoopConf.addResource(new Path(hadoopConfigFile)); > UserGroupInformation.setConfiguration(hadoopConf); > > Thread scanner = new Thread(new Runnable() { > @Override > public void run() { > try { > System.out.println("===================== Authenticate as > user1 using keytab"); > Connector connector = new > ZooKeeperInstance(ClientConfiguration.loadDefault()) > .getConnector(user1Name, new > KerberosToken(user1Name, new File(user1Keytab), true)); > > for ( int i = 0; true; ++i) { > System.out.println("===================== scan table > - " + i); > connector.createScanner(accumuloTableName, new > Authorizations()).forEach(e -> System.out.println(e)); > Thread.sleep(1000); > } > }catch(Exception x) { > x.printStackTrace(); > } > > } > }); > > Thread authenticator = new Thread(new Runnable() { > @Override > public void run() { > try { > System.out.println("===================== authenticate as > user2 using password"); > > LoginContext loginCtx = new LoginContext("MyClientJaas", > new CallbackHandler() { > public void handle(Callback[] callbacks) throws > IOException, UnsupportedCallbackException { > for (Callback c : callbacks) { > if (c instanceof NameCallback) > ((NameCallback) c).setName(user2Name); > if (c instanceof PasswordCallback) > ((PasswordCallback) > c).setPassword(user2Password.toCharArray()); > } > } > }); > loginCtx.login(); > > > UserGroupInformation.loginUserFromSubject(loginCtx.getSubject()); > UserGroupInformation ugi = > UserGroupInformation.getUGIFromSubject(loginCtx.getSubject()); > Connector newConnector = > ugi.doAs((PrivilegedExceptionAction<Connector>) () -> { > KerberosToken token = new KerberosToken(); > return new > ZooKeeperInstance(ClientConfiguration.loadDefault()).getConnector(token.getPrincipal(), > token); > }); > > } catch (Exception x) { > x.printStackTrace(); > } > } > }); > > scanner.start(); > scanner.join(1000); > > authenticator.start(); > > scanner.join(); > authenticator.join(); > } > } > > > ======================== > my jaas config file content: > ======================== > > MyClientJaas { > com.sun.security.auth.module.Krb5LoginModule required client=TRUE; > }; > > Client { > com.sun.security.auth.module.Krb5LoginModule required > useKeyTab=true > keyTab="/etc/security/keytabs/user1.keytab" > storeKey=true > useTicketCache=false > doNotPrompt=true > debug=true > principal="user1@example"; > }; > >
