aboutsummaryrefslogtreecommitdiff
path: root/launcher/src/test/java/org/apache/spark/launcher/LauncherServerSuite.java
blob: 13f72b757f30a074a208e73a61d23dad7d3f7233 (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
/*
 * 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.launcher;

import java.io.Closeable;
import java.io.IOException;
import java.net.InetAddress;
import java.net.Socket;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.Semaphore;
import java.util.concurrent.TimeUnit;

import org.junit.Test;
import static org.junit.Assert.*;

import static org.apache.spark.launcher.LauncherProtocol.*;

public class LauncherServerSuite extends BaseSuite {

  @Test
  public void testLauncherServerReuse() throws Exception {
    ChildProcAppHandle handle1 = null;
    ChildProcAppHandle handle2 = null;
    ChildProcAppHandle handle3 = null;

    try {
      handle1 = LauncherServer.newAppHandle();
      handle2 = LauncherServer.newAppHandle();
      LauncherServer server1 = handle1.getServer();
      assertSame(server1, handle2.getServer());

      handle1.kill();
      handle2.kill();

      handle3 = LauncherServer.newAppHandle();
      assertNotSame(server1, handle3.getServer());

      handle3.kill();

      assertNull(LauncherServer.getServerInstance());
    } finally {
      kill(handle1);
      kill(handle2);
      kill(handle3);
    }
  }

  @Test
  public void testCommunication() throws Exception {
    ChildProcAppHandle handle = LauncherServer.newAppHandle();
    TestClient client = null;
    try {
      Socket s = new Socket(InetAddress.getLoopbackAddress(),
        LauncherServer.getServerInstance().getPort());

      final Semaphore semaphore = new Semaphore(0);
      handle.addListener(new SparkAppHandle.Listener() {
        @Override
        public void stateChanged(SparkAppHandle handle) {
          semaphore.release();
        }
        @Override
        public void infoChanged(SparkAppHandle handle) {
          semaphore.release();
        }
      });

      client = new TestClient(s);
      client.send(new Hello(handle.getSecret(), "1.4.0"));
      semaphore.tryAcquire(10, TimeUnit.MILLISECONDS);

      // Make sure the server matched the client to the handle.
      assertNotNull(handle.getConnection());

      client.send(new SetAppId("app-id"));
      semaphore.tryAcquire(10, TimeUnit.MILLISECONDS);
      assertEquals("app-id", handle.getAppId());

      client.send(new SetState(SparkAppHandle.State.RUNNING));
      semaphore.tryAcquire(10, TimeUnit.MILLISECONDS);
      assertEquals(SparkAppHandle.State.RUNNING, handle.getState());

      handle.stop();
      Message stopMsg = client.inbound.poll(10, TimeUnit.SECONDS);
      assertTrue(stopMsg instanceof Stop);
    } finally {
      kill(handle);
      close(client);
      client.clientThread.join();
    }
  }

  @Test
  public void testTimeout() throws Exception {
    ChildProcAppHandle handle = null;
    TestClient client = null;
    try {
      // LauncherServer will immediately close the server-side socket when the timeout is set
      // to 0.
      SparkLauncher.setConfig(SparkLauncher.CHILD_CONNECTION_TIMEOUT, "0");

      handle = LauncherServer.newAppHandle();

      Socket s = new Socket(InetAddress.getLoopbackAddress(),
        LauncherServer.getServerInstance().getPort());
      client = new TestClient(s);

      // Try a few times since the client-side socket may not reflect the server-side close
      // immediately.
      boolean helloSent = false;
      int maxTries = 10;
      for (int i = 0; i < maxTries; i++) {
        try {
          if (!helloSent) {
            client.send(new Hello(handle.getSecret(), "1.4.0"));
            helloSent = true;
          } else {
            client.send(new SetAppId("appId"));
          }
          fail("Expected exception caused by connection timeout.");
        } catch (IllegalStateException | IOException e) {
          // Expected.
          break;
        } catch (AssertionError e) {
          if (i < maxTries - 1) {
            Thread.sleep(100);
          } else {
            throw new AssertionError("Test failed after " + maxTries + " attempts.", e);
          }
        }
      }
    } finally {
      SparkLauncher.launcherConfig.remove(SparkLauncher.CHILD_CONNECTION_TIMEOUT);
      kill(handle);
      close(client);
    }
  }

  private void kill(SparkAppHandle handle) {
    if (handle != null) {
      handle.kill();
    }
  }

  private void close(Closeable c) {
    if (c != null) {
      try {
        c.close();
      } catch (Exception e) {
        // no-op.
      }
    }
  }

  private static class TestClient extends LauncherConnection {

    final BlockingQueue<Message> inbound;
    final Thread clientThread;

    TestClient(Socket s) throws IOException {
      super(s);
      this.inbound = new LinkedBlockingQueue<Message>();
      this.clientThread = new Thread(this);
      clientThread.setName("TestClient");
      clientThread.setDaemon(true);
      clientThread.start();
    }

    @Override
    protected void handle(Message msg) throws IOException {
      inbound.offer(msg);
    }

  }

}