|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one or more |
| 3 | + * contributor license agreements. See the NOTICE file distributed with |
| 4 | + * this work for additional information regarding copyright ownership. |
| 5 | + * The ASF licenses this file to You under the Apache License, Version 2.0 |
| 6 | + * (the "License"); you may not use this file except in compliance with |
| 7 | + * the License. You may obtain a copy of the License at |
| 8 | + * |
| 9 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 10 | + * |
| 11 | + * Unless required by applicable law or agreed to in writing, software |
| 12 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 13 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 14 | + * See the License for the specific language governing permissions and |
| 15 | + * limitations under the License. |
| 16 | + */ |
| 17 | + |
| 18 | +package org.apache.spark.network.sasl; |
| 19 | + |
| 20 | +import java.io.IOException; |
| 21 | + |
| 22 | +import com.google.common.collect.Lists; |
| 23 | +import org.junit.After; |
| 24 | +import org.junit.AfterClass; |
| 25 | +import org.junit.BeforeClass; |
| 26 | +import org.junit.Test; |
| 27 | + |
| 28 | +import static org.junit.Assert.*; |
| 29 | + |
| 30 | +import org.apache.spark.network.TestUtils; |
| 31 | +import org.apache.spark.network.TransportContext; |
| 32 | +import org.apache.spark.network.client.RpcResponseCallback; |
| 33 | +import org.apache.spark.network.client.TransportClient; |
| 34 | +import org.apache.spark.network.client.TransportClientBootstrap; |
| 35 | +import org.apache.spark.network.client.TransportClientFactory; |
| 36 | +import org.apache.spark.network.server.OneForOneStreamManager; |
| 37 | +import org.apache.spark.network.server.RpcHandler; |
| 38 | +import org.apache.spark.network.server.StreamManager; |
| 39 | +import org.apache.spark.network.server.TransportServer; |
| 40 | +import org.apache.spark.network.shuffle.ExternalShuffleBlockHandler; |
| 41 | +import org.apache.spark.network.util.SystemPropertyConfigProvider; |
| 42 | +import org.apache.spark.network.util.TransportConf; |
| 43 | + |
| 44 | +public class SaslIntegrationSuite { |
| 45 | + static ExternalShuffleBlockHandler handler; |
| 46 | + static TransportServer server; |
| 47 | + static TransportConf conf; |
| 48 | + static TransportContext context; |
| 49 | + |
| 50 | + TransportClientFactory clientFactory; |
| 51 | + |
| 52 | + /** Provides a secret key holder which always returns the given secret key. */ |
| 53 | + static class TestSecretKeyHolder implements SecretKeyHolder { |
| 54 | + |
| 55 | + private final String secretKey; |
| 56 | + |
| 57 | + TestSecretKeyHolder(String secretKey) { |
| 58 | + this.secretKey = secretKey; |
| 59 | + } |
| 60 | + |
| 61 | + @Override |
| 62 | + public String getSaslUser(String appId) { |
| 63 | + return "user"; |
| 64 | + } |
| 65 | + @Override |
| 66 | + public String getSecretKey(String appId) { |
| 67 | + return secretKey; |
| 68 | + } |
| 69 | + } |
| 70 | + |
| 71 | + |
| 72 | + @BeforeClass |
| 73 | + public static void beforeAll() throws IOException { |
| 74 | + SecretKeyHolder secretKeyHolder = new TestSecretKeyHolder("good-key"); |
| 75 | + SaslRpcHandler handler = new SaslRpcHandler(new TestRpcHandler(), secretKeyHolder); |
| 76 | + conf = new TransportConf(new SystemPropertyConfigProvider()); |
| 77 | + context = new TransportContext(conf, handler); |
| 78 | + server = context.createServer(); |
| 79 | + } |
| 80 | + |
| 81 | + |
| 82 | + @AfterClass |
| 83 | + public static void afterAll() { |
| 84 | + server.close(); |
| 85 | + } |
| 86 | + |
| 87 | + @After |
| 88 | + public void afterEach() { |
| 89 | + if (clientFactory != null) { |
| 90 | + clientFactory.close(); |
| 91 | + clientFactory = null; |
| 92 | + } |
| 93 | + } |
| 94 | + |
| 95 | + @Test |
| 96 | + public void testGoodClient() { |
| 97 | + clientFactory = context.createClientFactory( |
| 98 | + Lists.<TransportClientBootstrap>newArrayList( |
| 99 | + new SaslBootstrap("app-id", new TestSecretKeyHolder("good-key")))); |
| 100 | + |
| 101 | + TransportClient client = clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); |
| 102 | + String msg = "Hello, World!"; |
| 103 | + byte[] resp = client.sendRpcSync(msg.getBytes(), 1000); |
| 104 | + assertEquals(msg, new String(resp)); // our rpc handler should just return the given msg |
| 105 | + } |
| 106 | + |
| 107 | + @Test |
| 108 | + public void testBadClient() { |
| 109 | + clientFactory = context.createClientFactory( |
| 110 | + Lists.<TransportClientBootstrap>newArrayList( |
| 111 | + new SaslBootstrap("app-id", new TestSecretKeyHolder("bad-key")))); |
| 112 | + |
| 113 | + try { |
| 114 | + // Bootstrap should fail on startup. |
| 115 | + clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); |
| 116 | + } catch (Exception e) { |
| 117 | + assertTrue(e.getMessage(), e.getMessage().contains("Mismatched response")); |
| 118 | + } |
| 119 | + } |
| 120 | + |
| 121 | + @Test |
| 122 | + public void testNoSaslClient() { |
| 123 | + clientFactory = context.createClientFactory( |
| 124 | + Lists.<TransportClientBootstrap>newArrayList()); |
| 125 | + |
| 126 | + TransportClient client = clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); |
| 127 | + try { |
| 128 | + client.sendRpcSync(new byte[13], 1000); |
| 129 | + fail("Should have failed"); |
| 130 | + } catch (Exception e) { |
| 131 | + assertTrue(e.getMessage(), e.getMessage().contains("Expected SaslMessage")); |
| 132 | + } |
| 133 | + |
| 134 | + try { |
| 135 | + // Guessing the right tag byte doesn't magically get you in... |
| 136 | + client.sendRpcSync(new byte[] { (byte) 0xEA }, 1000); |
| 137 | + fail("Should have failed"); |
| 138 | + } catch (Exception e) { |
| 139 | + assertTrue(e.getMessage(), e.getMessage().contains("java.lang.IndexOutOfBoundsException")); |
| 140 | + } |
| 141 | + } |
| 142 | + |
| 143 | + @Test |
| 144 | + public void testNoSaslServer() { |
| 145 | + RpcHandler handler = new TestRpcHandler(); |
| 146 | + TransportContext context = new TransportContext(conf, handler); |
| 147 | + clientFactory = context.createClientFactory( |
| 148 | + Lists.<TransportClientBootstrap>newArrayList( |
| 149 | + new SaslBootstrap("app-id", new TestSecretKeyHolder("key")))); |
| 150 | + TransportServer server = context.createServer(); |
| 151 | + try { |
| 152 | + clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); |
| 153 | + } catch (Exception e) { |
| 154 | + assertTrue(e.getMessage(), e.getMessage().contains("Digest-challenge format violation")); |
| 155 | + } finally { |
| 156 | + server.close(); |
| 157 | + } |
| 158 | + } |
| 159 | + |
| 160 | + /** RPC handler which simply responds with the message it received. */ |
| 161 | + public static class TestRpcHandler implements RpcHandler { |
| 162 | + @Override |
| 163 | + public void receive(TransportClient client, byte[] message, RpcResponseCallback callback) { |
| 164 | + callback.onSuccess(message); |
| 165 | + } |
| 166 | + |
| 167 | + @Override |
| 168 | + public StreamManager getStreamManager() { |
| 169 | + return new OneForOneStreamManager(); |
| 170 | + } |
| 171 | + } |
| 172 | +} |
0 commit comments