aboutsummaryrefslogtreecommitdiff
path: root/network/shuffle/src/test/java/org/apache/spark/network/sasl/SaslIntegrationSuite.java
blob: 84781207861ed6fe57cedb92a96f7edde0fc3db0 (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
/*
 * 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 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 org.apache.spark.network.TestUtils;
import org.apache.spark.network.TransportContext;
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.shuffle.ExternalShuffleBlockHandler;
import org.apache.spark.network.util.SystemPropertyConfigProvider;
import org.apache.spark.network.util.TransportConf;

public class SaslIntegrationSuite {
  static ExternalShuffleBlockHandler handler;
  static TransportServer server;
  static TransportConf conf;
  static TransportContext context;

  TransportClientFactory clientFactory;

  /** Provides a secret key holder which always returns the given secret key. */
  static class TestSecretKeyHolder implements SecretKeyHolder {

    private final String secretKey;

    TestSecretKeyHolder(String secretKey) {
      this.secretKey = secretKey;
    }

    @Override
    public String getSaslUser(String appId) {
      return "user";
    }
    @Override
    public String getSecretKey(String appId) {
      return secretKey;
    }
  }


  @BeforeClass
  public static void beforeAll() throws IOException {
    SecretKeyHolder secretKeyHolder = new TestSecretKeyHolder("good-key");
    SaslRpcHandler handler = new SaslRpcHandler(new TestRpcHandler(), secretKeyHolder);
    conf = new TransportConf(new SystemPropertyConfigProvider());
    context = new TransportContext(conf, handler);
    server = context.createServer();
  }


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

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

  @Test
  public void testGoodClient() {
    clientFactory = context.createClientFactory(
      Lists.<TransportClientBootstrap>newArrayList(
        new SaslClientBootstrap(conf, "app-id", new TestSecretKeyHolder("good-key"))));

    TransportClient client = clientFactory.createClient(TestUtils.getLocalHost(), server.getPort());
    String msg = "Hello, World!";
    byte[] resp = client.sendRpcSync(msg.getBytes(), 1000);
    assertEquals(msg, new String(resp)); // our rpc handler should just return the given msg
  }

  @Test
  public void testBadClient() {
    clientFactory = context.createClientFactory(
      Lists.<TransportClientBootstrap>newArrayList(
        new SaslClientBootstrap(conf, "app-id", new TestSecretKeyHolder("bad-key"))));

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

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

    TransportClient client = clientFactory.createClient(TestUtils.getLocalHost(), server.getPort());
    try {
      client.sendRpcSync(new byte[13], 1000);
      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(new byte[] { (byte) 0xEA }, 1000);
      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-id", new TestSecretKeyHolder("key"))));
    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();
    }
  }

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

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