|
@@ -40,9 +40,21 @@ import java.security.PrivilegedExceptionAction;
|
|
|
import java.security.Security;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.Collection;
|
|
|
+import java.util.Collections;
|
|
|
import java.util.HashMap;
|
|
|
+import java.util.Iterator;
|
|
|
+import java.util.List;
|
|
|
import java.util.Map;
|
|
|
import java.util.Set;
|
|
|
+import java.util.concurrent.Callable;
|
|
|
+import java.util.concurrent.ExecutionException;
|
|
|
+import java.util.concurrent.ExecutorService;
|
|
|
+import java.util.concurrent.Executors;
|
|
|
+import java.util.concurrent.Future;
|
|
|
+import java.util.concurrent.ThreadLocalRandom;
|
|
|
+import java.util.concurrent.TimeUnit;
|
|
|
+import java.util.concurrent.TimeoutException;
|
|
|
+import java.util.concurrent.atomic.AtomicInteger;
|
|
|
import java.util.regex.Pattern;
|
|
|
|
|
|
import javax.security.auth.callback.Callback;
|
|
@@ -65,6 +77,7 @@ import org.apache.hadoop.fs.CommonConfigurationKeys;
|
|
|
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
|
|
|
import org.apache.hadoop.io.Text;
|
|
|
import org.apache.hadoop.ipc.Client.ConnectionId;
|
|
|
+import org.apache.hadoop.ipc.Server.Call;
|
|
|
import org.apache.hadoop.net.NetUtils;
|
|
|
import org.apache.hadoop.security.KerberosInfo;
|
|
|
import org.apache.hadoop.security.SaslInputStream;
|
|
@@ -78,6 +91,7 @@ import org.apache.hadoop.security.SecurityInfo;
|
|
|
import org.apache.hadoop.security.SecurityUtil;
|
|
|
import org.apache.hadoop.security.TestUserGroupInformation;
|
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
|
+import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
|
|
|
import org.apache.hadoop.security.token.SecretManager;
|
|
|
import org.apache.hadoop.security.token.SecretManager.InvalidToken;
|
|
|
import org.apache.hadoop.security.token.Token;
|
|
@@ -85,6 +99,7 @@ import org.apache.hadoop.security.token.TokenIdentifier;
|
|
|
import org.apache.hadoop.security.token.TokenInfo;
|
|
|
import org.apache.hadoop.security.token.TokenSelector;
|
|
|
import org.apache.log4j.Level;
|
|
|
+import org.junit.Assert;
|
|
|
import org.junit.Before;
|
|
|
import org.junit.BeforeClass;
|
|
|
import org.junit.Test;
|
|
@@ -294,10 +309,13 @@ public class TestSaslRPC {
|
|
|
public interface TestSaslProtocol extends TestRPC.TestProtocol {
|
|
|
public AuthMethod getAuthMethod() throws IOException;
|
|
|
public String getAuthUser() throws IOException;
|
|
|
+ public String echoPostponed(String value) throws IOException;
|
|
|
+ public void sendPostponed() throws IOException;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
public static class TestSaslImpl extends TestRPC.TestImpl implements
|
|
|
TestSaslProtocol {
|
|
|
+ private List<Call> postponedCalls = new ArrayList<Call>();
|
|
|
@Override
|
|
|
public AuthMethod getAuthMethod() throws IOException {
|
|
|
return UserGroupInformation.getCurrentUser()
|
|
@@ -307,6 +325,21 @@ public class TestSaslRPC {
|
|
|
public String getAuthUser() throws IOException {
|
|
|
return UserGroupInformation.getCurrentUser().getUserName();
|
|
|
}
|
|
|
+ @Override
|
|
|
+ public String echoPostponed(String value) {
|
|
|
+ Call call = Server.getCurCall().get();
|
|
|
+ call.postponeResponse();
|
|
|
+ postponedCalls.add(call);
|
|
|
+ return value;
|
|
|
+ }
|
|
|
+ @Override
|
|
|
+ public void sendPostponed() throws IOException {
|
|
|
+ Collections.shuffle(postponedCalls);
|
|
|
+ for (Call call : postponedCalls) {
|
|
|
+ call.sendResponse();
|
|
|
+ }
|
|
|
+ postponedCalls.clear();
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
public static class CustomSecurityInfo extends SecurityInfo {
|
|
@@ -844,6 +877,85 @@ public class TestSaslRPC {
|
|
|
assertAuthEquals(KrbFailed, getAuthMethod(KERBEROS, KERBEROS, UseToken.INVALID));
|
|
|
}
|
|
|
|
|
|
+ // ensure that for all qop settings, client can handle postponed rpc
|
|
|
+ // responses. basically ensures that the rpc server isn't encrypting
|
|
|
+ // and queueing the responses out of order.
|
|
|
+ @Test(timeout=10000)
|
|
|
+ public void testSaslResponseOrdering() throws Exception {
|
|
|
+ SecurityUtil.setAuthenticationMethod(
|
|
|
+ AuthenticationMethod.TOKEN, conf);
|
|
|
+ UserGroupInformation.setConfiguration(conf);
|
|
|
+
|
|
|
+ TestTokenSecretManager sm = new TestTokenSecretManager();
|
|
|
+ Server server = new RPC.Builder(conf)
|
|
|
+ .setProtocol(TestSaslProtocol.class)
|
|
|
+ .setInstance(new TestSaslImpl()).setBindAddress(ADDRESS).setPort(0)
|
|
|
+ .setNumHandlers(1) // prevents ordering issues when unblocking calls.
|
|
|
+ .setVerbose(true)
|
|
|
+ .setSecretManager(sm)
|
|
|
+ .build();
|
|
|
+ server.start();
|
|
|
+ try {
|
|
|
+ final InetSocketAddress addr = NetUtils.getConnectAddress(server);
|
|
|
+ final UserGroupInformation clientUgi =
|
|
|
+ UserGroupInformation.createRemoteUser("client");
|
|
|
+ clientUgi.setAuthenticationMethod(AuthenticationMethod.TOKEN);
|
|
|
+
|
|
|
+ TestTokenIdentifier tokenId = new TestTokenIdentifier(
|
|
|
+ new Text(clientUgi.getUserName()));
|
|
|
+ Token<?> token = new Token<TestTokenIdentifier>(tokenId, sm);
|
|
|
+ SecurityUtil.setTokenService(token, addr);
|
|
|
+ clientUgi.addToken(token);
|
|
|
+ clientUgi.doAs(new PrivilegedExceptionAction<Void>() {
|
|
|
+ @Override
|
|
|
+ public Void run() throws Exception {
|
|
|
+ final TestSaslProtocol proxy = RPC.getProxy(TestSaslProtocol.class,
|
|
|
+ TestSaslProtocol.versionID, addr, conf);
|
|
|
+ final ExecutorService executor = Executors.newCachedThreadPool();
|
|
|
+ final AtomicInteger count = new AtomicInteger();
|
|
|
+ try {
|
|
|
+ // queue up a bunch of futures for postponed calls serviced
|
|
|
+ // in a random order.
|
|
|
+ Future<?>[] futures = new Future<?>[10];
|
|
|
+ for (int i=0; i < futures.length; i++) {
|
|
|
+ futures[i] = executor.submit(new Callable<Void>(){
|
|
|
+ @Override
|
|
|
+ public Void call() throws Exception {
|
|
|
+ String expect = "future"+count.getAndIncrement();
|
|
|
+ String answer = proxy.echoPostponed(expect);
|
|
|
+ assertEquals(expect, answer);
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+ });
|
|
|
+ try {
|
|
|
+ // ensures the call is initiated and the response is blocked.
|
|
|
+ futures[i].get(100, TimeUnit.MILLISECONDS);
|
|
|
+ } catch (TimeoutException te) {
|
|
|
+ continue; // expected.
|
|
|
+ }
|
|
|
+ Assert.fail("future"+i+" did not block");
|
|
|
+ }
|
|
|
+ // triggers responses to be unblocked in a random order. having
|
|
|
+ // only 1 handler ensures that the prior calls are already
|
|
|
+ // postponed. 1 handler also ensures that this call will
|
|
|
+ // timeout if the postponing doesn't work (ie. free up handler)
|
|
|
+ proxy.sendPostponed();
|
|
|
+ for (int i=0; i < futures.length; i++) {
|
|
|
+ LOG.info("waiting for future"+i);
|
|
|
+ futures[i].get();
|
|
|
+ }
|
|
|
+ } finally {
|
|
|
+ RPC.stopProxy(proxy);
|
|
|
+ executor.shutdownNow();
|
|
|
+ }
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+ });
|
|
|
+ } finally {
|
|
|
+ server.stop();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
|
|
|
// test helpers
|
|
|
|