aboutsummaryrefslogtreecommitdiff
path: root/common/network-shuffle/src/test/java/org/apache/spark/network/sasl/SaslIntegrationSuite.java
blob: 298a487ebb7522d40e20f40ac6bb57b94120c276 (plain) (blame)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
/*
 * Licensed to the Apache Software Foundation (ASF) under one or more
 * contributor license agreements.  See the NOTICE file distributed with
 * this work for additional information regarding copyright ownership.
 * The ASF licenses this file to You under the Apache License, Version 2.0
 * (the "License"); you may not use this file except in compliance with
 * the License.  You may obtain a copy of the License at
 *
 *    http://www.apache.org/licenses/LICENSE-2.0
 *
 * Unless required by applicable law or agreed to in writing, software
 * distributed under the License is distributed on an "AS IS" BASIS,
 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 * See the License for the specific language governing permissions and
 * limitations under the License.
 */

package org.apache.spark.network.sasl;

import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.Arrays;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicReference;

import com.google.common.collect.Lists;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;

import static org.junit.Assert.*;
import static org.mockito.Mockito.*;

import org.apache.spark.network.TestUtils;
import org.apache.spark.network.TransportContext;
import org.apache.spark.network.buffer.ManagedBuffer;
import org.apache.spark.network.client.ChunkReceivedCallback;
import org.apache.spark.network.client.RpcResponseCallback;
import org.apache.spark.network.client.TransportClient;
import org.apache.spark.network.client.TransportClientBootstrap;
import org.apache.spark.network.client.TransportClientFactory;
import org.apache.spark.network.server.OneForOneStreamManager;
import org.apache.spark.network.server.RpcHandler;
import org.apache.spark.network.server.StreamManager;
import org.apache.spark.network.server.TransportServer;
import org.apache.spark.network.server.TransportServerBootstrap;
import org.apache.spark.network.shuffle.BlockFetchingListener;
import org.apache.spark.network.shuffle.ExternalShuffleBlockHandler;
import org.apache.spark.network.shuffle.ExternalShuffleBlockResolver;
import org.apache.spark.network.shuffle.OneForOneBlockFetcher;
import org.apache.spark.network.shuffle.protocol.BlockTransferMessage;
import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo;
import org.apache.spark.network.shuffle.protocol.OpenBlocks;
import org.apache.spark.network.shuffle.protocol.RegisterExecutor;
import org.apache.spark.network.shuffle.protocol.StreamHandle;
import org.apache.spark.network.util.JavaUtils;
import org.apache.spark.network.util.MapConfigProvider;
import org.apache.spark.network.util.TransportConf;

public class SaslIntegrationSuite {

  // Use a long timeout to account for slow / overloaded build machines. In the normal case,
  // tests should finish way before the timeout expires.
  private static final long TIMEOUT_MS = 10_000;

  static TransportServer server;
  static TransportConf conf;
  static TransportContext context;
  static SecretKeyHolder secretKeyHolder;

  TransportClientFactory clientFactory;

  @BeforeClass
  public static void beforeAll() throws IOException {
    conf = new TransportConf("shuffle", MapConfigProvider.EMPTY);
    context = new TransportContext(conf, new TestRpcHandler());

    secretKeyHolder = mock(SecretKeyHolder.class);
    when(secretKeyHolder.getSaslUser(eq("app-1"))).thenReturn("app-1");
    when(secretKeyHolder.getSecretKey(eq("app-1"))).thenReturn("app-1");
    when(secretKeyHolder.getSaslUser(eq("app-2"))).thenReturn("app-2");
    when(secretKeyHolder.getSecretKey(eq("app-2"))).thenReturn("app-2");
    when(secretKeyHolder.getSaslUser(anyString())).thenReturn("other-app");
    when(secretKeyHolder.getSecretKey(anyString())).thenReturn("correct-password");

    TransportServerBootstrap bootstrap = new SaslServerBootstrap(conf, secretKeyHolder);
    server = context.createServer(Arrays.asList(bootstrap));
  }


  @AfterClass
  public static void afterAll() {
    server.close();
  }

  @After
  public void afterEach() {
    if (clientFactory != null) {
      clientFactory.close();
      clientFactory = null;
    }
  }

  @Test
  public void testGoodClient() throws IOException {
    clientFactory = context.createClientFactory(
      Lists.<TransportClientBootstrap>newArrayList(
        new SaslClientBootstrap(conf, "app-1", secretKeyHolder)));

    TransportClient client = clientFactory.createClient(TestUtils.getLocalHost(), server.getPort());
    String msg = "Hello, World!";
    ByteBuffer resp = client.sendRpcSync(JavaUtils.stringToBytes(msg), TIMEOUT_MS);
    assertEquals(msg, JavaUtils.bytesToString(resp));
  }

  @Test
  public void testBadClient() {
    SecretKeyHolder badKeyHolder = mock(SecretKeyHolder.class);
    when(badKeyHolder.getSaslUser(anyString())).thenReturn("other-app");
    when(badKeyHolder.getSecretKey(anyString())).thenReturn("wrong-password");
    clientFactory = context.createClientFactory(
      Lists.<TransportClientBootstrap>newArrayList(
        new SaslClientBootstrap(conf, "unknown-app", badKeyHolder)));

    try {
      // Bootstrap should fail on startup.
      clientFactory.createClient(TestUtils.getLocalHost(), server.getPort());
      fail("Connection should have failed.");
    } catch (Exception e) {
      assertTrue(e.getMessage(), e.getMessage().contains("Mismatched response"));
    }
  }

  @Test
  public void testNoSaslClient() throws IOException {
    clientFactory = context.createClientFactory(
      Lists.<TransportClientBootstrap>newArrayList());

    TransportClient client = clientFactory.createClient(TestUtils.getLocalHost(), server.getPort());
    try {
      client.sendRpcSync(ByteBuffer.allocate(13), TIMEOUT_MS);
      fail("Should have failed");
    } catch (Exception e) {
      assertTrue(e.getMessage(), e.getMessage().contains("Expected SaslMessage"));
    }

    try {
      // Guessing the right tag byte doesn't magically get you in...
      client.sendRpcSync(ByteBuffer.wrap(new byte[] { (byte) 0xEA }), TIMEOUT_MS);
      fail("Should have failed");
    } catch (Exception e) {
      assertTrue(e.getMessage(), e.getMessage().contains("java.lang.IndexOutOfBoundsException"));
    }
  }

  @Test
  public void testNoSaslServer() {
    RpcHandler handler = new TestRpcHandler();
    TransportContext context = new TransportContext(conf, handler);
    clientFactory = context.createClientFactory(
      Lists.<TransportClientBootstrap>newArrayList(
        new SaslClientBootstrap(conf, "app-1", secretKeyHolder)));
    TransportServer server = context.createServer();
    try {
      clientFactory.createClient(TestUtils.getLocalHost(), server.getPort());
    } catch (Exception e) {
      assertTrue(e.getMessage(), e.getMessage().contains("Digest-challenge format violation"));
    } finally {
      server.close();
    }
  }

  /**
   * This test is not actually testing SASL behavior, but testing that the shuffle service
   * performs correct authorization checks based on the SASL authentication data.
   */
  @Test
  public void testAppIsolation() throws Exception {
    // Start a new server with the correct RPC handler to serve block data.
    ExternalShuffleBlockResolver blockResolver = mock(ExternalShuffleBlockResolver.class);
    ExternalShuffleBlockHandler blockHandler = new ExternalShuffleBlockHandler(
      new OneForOneStreamManager(), blockResolver);
    TransportServerBootstrap bootstrap = new SaslServerBootstrap(conf, secretKeyHolder);
    TransportContext blockServerContext = new TransportContext(conf, blockHandler);
    TransportServer blockServer = blockServerContext.createServer(Arrays.asList(bootstrap));

    TransportClient client1 = null;
    TransportClient client2 = null;
    TransportClientFactory clientFactory2 = null;
    try {
      // Create a client, and make a request to fetch blocks from a different app.
      clientFactory = blockServerContext.createClientFactory(
        Lists.<TransportClientBootstrap>newArrayList(
          new SaslClientBootstrap(conf, "app-1", secretKeyHolder)));
      client1 = clientFactory.createClient(TestUtils.getLocalHost(),
        blockServer.getPort());

      final AtomicReference<Throwable> exception = new AtomicReference<>();

      final CountDownLatch blockFetchLatch = new CountDownLatch(1);
      BlockFetchingListener listener = new BlockFetchingListener() {
        @Override
        public void onBlockFetchSuccess(String blockId, ManagedBuffer data) {
          blockFetchLatch.countDown();
        }
        @Override
        public void onBlockFetchFailure(String blockId, Throwable t) {
          exception.set(t);
          blockFetchLatch.countDown();
        }
      };

      String[] blockIds = { "shuffle_2_3_4", "shuffle_6_7_8" };
      OneForOneBlockFetcher fetcher =
          new OneForOneBlockFetcher(client1, "app-2", "0", blockIds, listener);
      fetcher.start();
      blockFetchLatch.await();
      checkSecurityException(exception.get());

      // Register an executor so that the next steps work.
      ExecutorShuffleInfo executorInfo = new ExecutorShuffleInfo(
        new String[] { System.getProperty("java.io.tmpdir") }, 1,
          "org.apache.spark.shuffle.sort.SortShuffleManager");
      RegisterExecutor regmsg = new RegisterExecutor("app-1", "0", executorInfo);
      client1.sendRpcSync(regmsg.toByteBuffer(), TIMEOUT_MS);

      // Make a successful request to fetch blocks, which creates a new stream. But do not actually
      // fetch any blocks, to keep the stream open.
      OpenBlocks openMessage = new OpenBlocks("app-1", "0", blockIds);
      ByteBuffer response = client1.sendRpcSync(openMessage.toByteBuffer(), TIMEOUT_MS);
      StreamHandle stream = (StreamHandle) BlockTransferMessage.Decoder.fromByteBuffer(response);
      long streamId = stream.streamId;

      // Create a second client, authenticated with a different app ID, and try to read from
      // the stream created for the previous app.
      clientFactory2 = blockServerContext.createClientFactory(
        Lists.<TransportClientBootstrap>newArrayList(
          new SaslClientBootstrap(conf, "app-2", secretKeyHolder)));
      client2 = clientFactory2.createClient(TestUtils.getLocalHost(),
        blockServer.getPort());

      final CountDownLatch chunkReceivedLatch = new CountDownLatch(1);
      ChunkReceivedCallback callback = new ChunkReceivedCallback() {
        @Override
        public void onSuccess(int chunkIndex, ManagedBuffer buffer) {
          chunkReceivedLatch.countDown();
        }
        @Override
        public void onFailure(int chunkIndex, Throwable t) {
          exception.set(t);
          chunkReceivedLatch.countDown();
        }
      };

      exception.set(null);
      client2.fetchChunk(streamId, 0, callback);
      chunkReceivedLatch.await();
      checkSecurityException(exception.get());
    } finally {
      if (client1 != null) {
        client1.close();
      }
      if (client2 != null) {
        client2.close();
      }
      if (clientFactory2 != null) {
        clientFactory2.close();
      }
      blockServer.close();
    }
  }

  /** RPC handler which simply responds with the message it received. */
  public static class TestRpcHandler extends RpcHandler {
    @Override
    public void receive(TransportClient client, ByteBuffer message, RpcResponseCallback callback) {
      callback.onSuccess(message);
    }

    @Override
    public StreamManager getStreamManager() {
      return new OneForOneStreamManager();
    }
  }

  private void checkSecurityException(Throwable t) {
    assertNotNull("No exception was caught.", t);
    assertTrue("Expected SecurityException.",
      t.getMessage().contains(SecurityException.class.getName()));
  }
}