aboutsummaryrefslogtreecommitdiff
path: root/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala
blob: cebac2097f38071f19c84832adad368594ea9362 (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
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
/*
 * 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.rpc

import java.io.{File, NotSerializableException}
import java.nio.charset.StandardCharsets.UTF_8
import java.util.UUID
import java.util.concurrent.{ConcurrentLinkedQueue, CountDownLatch, TimeUnit}

import scala.collection.mutable
import scala.collection.JavaConverters._
import scala.concurrent.Await
import scala.concurrent.duration._
import scala.language.postfixOps

import com.google.common.io.Files
import org.mockito.Mockito.{mock, when}
import org.scalatest.BeforeAndAfterAll
import org.scalatest.concurrent.Eventually._

import org.apache.spark.{SecurityManager, SparkConf, SparkEnv, SparkException, SparkFunSuite}
import org.apache.spark.deploy.SparkHadoopUtil
import org.apache.spark.util.Utils

/**
 * Common tests for an RpcEnv implementation.
 */
abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll {

  var env: RpcEnv = _

  override def beforeAll(): Unit = {
    super.beforeAll()
    val conf = new SparkConf()
    env = createRpcEnv(conf, "local", 0)

    val sparkEnv = mock(classOf[SparkEnv])
    when(sparkEnv.rpcEnv).thenReturn(env)
    SparkEnv.set(sparkEnv)
  }

  override def afterAll(): Unit = {
    try {
      if (env != null) {
        env.shutdown()
      }
      SparkEnv.set(null)
    } finally {
      super.afterAll()
    }
  }

  def createRpcEnv(conf: SparkConf, name: String, port: Int, clientMode: Boolean = false): RpcEnv

  test("send a message locally") {
    @volatile var message: String = null
    val rpcEndpointRef = env.setupEndpoint("send-locally", new RpcEndpoint {
      override val rpcEnv = env

      override def receive = {
        case msg: String => message = msg
      }
    })
    rpcEndpointRef.send("hello")
    eventually(timeout(5 seconds), interval(10 millis)) {
      assert("hello" === message)
    }
  }

  test("send a message remotely") {
    @volatile var message: String = null
    // Set up a RpcEndpoint using env
    env.setupEndpoint("send-remotely", new RpcEndpoint {
      override val rpcEnv = env

      override def receive: PartialFunction[Any, Unit] = {
        case msg: String => message = msg
      }
    })

    val anotherEnv = createRpcEnv(new SparkConf(), "remote", 0, clientMode = true)
    // Use anotherEnv to find out the RpcEndpointRef
    val rpcEndpointRef = anotherEnv.setupEndpointRef(env.address, "send-remotely")
    try {
      rpcEndpointRef.send("hello")
      eventually(timeout(5 seconds), interval(10 millis)) {
        assert("hello" === message)
      }
    } finally {
      anotherEnv.shutdown()
      anotherEnv.awaitTermination()
    }
  }

  test("send a RpcEndpointRef") {
    val endpoint = new RpcEndpoint {
      override val rpcEnv = env

      override def receiveAndReply(context: RpcCallContext) = {
        case "Hello" => context.reply(self)
        case "Echo" => context.reply("Echo")
      }
    }
    val rpcEndpointRef = env.setupEndpoint("send-ref", endpoint)
    val newRpcEndpointRef = rpcEndpointRef.askWithRetry[RpcEndpointRef]("Hello")
    val reply = newRpcEndpointRef.askWithRetry[String]("Echo")
    assert("Echo" === reply)
  }

  test("ask a message locally") {
    val rpcEndpointRef = env.setupEndpoint("ask-locally", new RpcEndpoint {
      override val rpcEnv = env

      override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = {
        case msg: String =>
          context.reply(msg)
      }
    })
    val reply = rpcEndpointRef.askWithRetry[String]("hello")
    assert("hello" === reply)
  }

  test("ask a message remotely") {
    env.setupEndpoint("ask-remotely", new RpcEndpoint {
      override val rpcEnv = env

      override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = {
        case msg: String =>
          context.reply(msg)
      }
    })

    val anotherEnv = createRpcEnv(new SparkConf(), "remote", 0, clientMode = true)
    // Use anotherEnv to find out the RpcEndpointRef
    val rpcEndpointRef = anotherEnv.setupEndpointRef(env.address, "ask-remotely")
    try {
      val reply = rpcEndpointRef.askWithRetry[String]("hello")
      assert("hello" === reply)
    } finally {
      anotherEnv.shutdown()
      anotherEnv.awaitTermination()
    }
  }

  test("ask a message timeout") {
    env.setupEndpoint("ask-timeout", new RpcEndpoint {
      override val rpcEnv = env

      override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = {
        case msg: String =>
          Thread.sleep(100)
          context.reply(msg)
      }
    })

    val conf = new SparkConf()
    val shortProp = "spark.rpc.short.timeout"
    conf.set("spark.rpc.retry.wait", "0")
    conf.set("spark.rpc.numRetries", "1")
    val anotherEnv = createRpcEnv(conf, "remote", 0, clientMode = true)
    // Use anotherEnv to find out the RpcEndpointRef
    val rpcEndpointRef = anotherEnv.setupEndpointRef(env.address, "ask-timeout")
    try {
      // Any exception thrown in askWithRetry is wrapped with a SparkException and set as the cause
      val e = intercept[SparkException] {
        rpcEndpointRef.askWithRetry[String]("hello", new RpcTimeout(1 millis, shortProp))
      }
      // The SparkException cause should be a RpcTimeoutException with message indicating the
      // controlling timeout property
      assert(e.getCause.isInstanceOf[RpcTimeoutException])
      assert(e.getCause.getMessage.contains(shortProp))
    } finally {
      anotherEnv.shutdown()
      anotherEnv.awaitTermination()
    }
  }

  test("onStart and onStop") {
    val stopLatch = new CountDownLatch(1)
    val calledMethods = mutable.ArrayBuffer[String]()

    val endpoint = new RpcEndpoint {
      override val rpcEnv = env

      override def onStart(): Unit = {
        calledMethods += "start"
      }

      override def receive: PartialFunction[Any, Unit] = {
        case msg: String =>
      }

      override def onStop(): Unit = {
        calledMethods += "stop"
        stopLatch.countDown()
      }
    }
    val rpcEndpointRef = env.setupEndpoint("start-stop-test", endpoint)
    env.stop(rpcEndpointRef)
    stopLatch.await(10, TimeUnit.SECONDS)
    assert(List("start", "stop") === calledMethods)
  }

  test("onError: error in onStart") {
    @volatile var e: Throwable = null
    env.setupEndpoint("onError-onStart", new RpcEndpoint {
      override val rpcEnv = env

      override def onStart(): Unit = {
        throw new RuntimeException("Oops!")
      }

      override def receive: PartialFunction[Any, Unit] = {
        case m =>
      }

      override def onError(cause: Throwable): Unit = {
        e = cause
      }
    })

    eventually(timeout(5 seconds), interval(10 millis)) {
      assert(e.getMessage === "Oops!")
    }
  }

  test("onError: error in onStop") {
    @volatile var e: Throwable = null
    val endpointRef = env.setupEndpoint("onError-onStop", new RpcEndpoint {
      override val rpcEnv = env

      override def receive: PartialFunction[Any, Unit] = {
        case m =>
      }

      override def onError(cause: Throwable): Unit = {
        e = cause
      }

      override def onStop(): Unit = {
        throw new RuntimeException("Oops!")
      }
    })

    env.stop(endpointRef)

    eventually(timeout(5 seconds), interval(10 millis)) {
      assert(e.getMessage === "Oops!")
    }
  }

  test("onError: error in receive") {
    @volatile var e: Throwable = null
    val endpointRef = env.setupEndpoint("onError-receive", new RpcEndpoint {
      override val rpcEnv = env

      override def receive: PartialFunction[Any, Unit] = {
        case m => throw new RuntimeException("Oops!")
      }

      override def onError(cause: Throwable): Unit = {
        e = cause
      }
    })

    endpointRef.send("Foo")

    eventually(timeout(5 seconds), interval(10 millis)) {
      assert(e.getMessage === "Oops!")
    }
  }

  test("self: call in onStart") {
    @volatile var callSelfSuccessfully = false

    env.setupEndpoint("self-onStart", new RpcEndpoint {
      override val rpcEnv = env

      override def onStart(): Unit = {
        self
        callSelfSuccessfully = true
      }

      override def receive: PartialFunction[Any, Unit] = {
        case m =>
      }
    })

    eventually(timeout(5 seconds), interval(10 millis)) {
      // Calling `self` in `onStart` is fine
      assert(callSelfSuccessfully === true)
    }
  }

  test("self: call in receive") {
    @volatile var callSelfSuccessfully = false

    val endpointRef = env.setupEndpoint("self-receive", new RpcEndpoint {
      override val rpcEnv = env

      override def receive: PartialFunction[Any, Unit] = {
        case m =>
          self
          callSelfSuccessfully = true
      }
    })

    endpointRef.send("Foo")

    eventually(timeout(5 seconds), interval(10 millis)) {
      // Calling `self` in `receive` is fine
      assert(callSelfSuccessfully === true)
    }
  }

  test("self: call in onStop") {
    @volatile var selfOption: Option[RpcEndpointRef] = null

    val endpointRef = env.setupEndpoint("self-onStop", new RpcEndpoint {
      override val rpcEnv = env

      override def receive: PartialFunction[Any, Unit] = {
        case m =>
      }

      override def onStop(): Unit = {
        selfOption = Option(self)
      }
    })

    env.stop(endpointRef)

    eventually(timeout(5 seconds), interval(10 millis)) {
      // Calling `self` in `onStop` will return null, so selfOption will be None
      assert(selfOption == None)
    }
  }

  test("call receive in sequence") {
    // If a RpcEnv implementation breaks the `receive` contract, hope this test can expose it
    for (i <- 0 until 100) {
      @volatile var result = 0
      val endpointRef = env.setupEndpoint(s"receive-in-sequence-$i", new ThreadSafeRpcEndpoint {
        override val rpcEnv = env

        override def receive: PartialFunction[Any, Unit] = {
          case m => result += 1
        }

      })

      (0 until 10) foreach { _ =>
        new Thread {
          override def run() {
            (0 until 100) foreach { _ =>
              endpointRef.send("Hello")
            }
          }
        }.start()
      }

      eventually(timeout(5 seconds), interval(5 millis)) {
        assert(result == 1000)
      }

      env.stop(endpointRef)
    }
  }

  test("stop(RpcEndpointRef) reentrant") {
    @volatile var onStopCount = 0
    val endpointRef = env.setupEndpoint("stop-reentrant", new RpcEndpoint {
      override val rpcEnv = env

      override def receive: PartialFunction[Any, Unit] = {
        case m =>
      }

      override def onStop(): Unit = {
        onStopCount += 1
      }
    })

    env.stop(endpointRef)
    env.stop(endpointRef)

    eventually(timeout(5 seconds), interval(5 millis)) {
      // Calling stop twice should only trigger onStop once.
      assert(onStopCount == 1)
    }
  }

  test("sendWithReply") {
    val endpointRef = env.setupEndpoint("sendWithReply", new RpcEndpoint {
      override val rpcEnv = env

      override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = {
        case m => context.reply("ack")
      }
    })

    val f = endpointRef.ask[String]("Hi")
    val ack = Await.result(f, 5 seconds)
    assert("ack" === ack)

    env.stop(endpointRef)
  }

  test("sendWithReply: remotely") {
    env.setupEndpoint("sendWithReply-remotely", new RpcEndpoint {
      override val rpcEnv = env

      override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = {
        case m => context.reply("ack")
      }
    })

    val anotherEnv = createRpcEnv(new SparkConf(), "remote", 0, clientMode = true)
    // Use anotherEnv to find out the RpcEndpointRef
    val rpcEndpointRef = anotherEnv.setupEndpointRef(env.address, "sendWithReply-remotely")
    try {
      val f = rpcEndpointRef.ask[String]("hello")
      val ack = Await.result(f, 5 seconds)
      assert("ack" === ack)
    } finally {
      anotherEnv.shutdown()
      anotherEnv.awaitTermination()
    }
  }

  test("sendWithReply: error") {
    val endpointRef = env.setupEndpoint("sendWithReply-error", new RpcEndpoint {
      override val rpcEnv = env

      override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = {
        case m => context.sendFailure(new SparkException("Oops"))
      }
    })

    val f = endpointRef.ask[String]("Hi")
    val e = intercept[SparkException] {
      Await.result(f, 5 seconds)
    }
    assert("Oops" === e.getMessage)

    env.stop(endpointRef)
  }

  test("sendWithReply: remotely error") {
    env.setupEndpoint("sendWithReply-remotely-error", new RpcEndpoint {
      override val rpcEnv = env

      override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = {
        case msg: String => context.sendFailure(new SparkException("Oops"))
      }
    })

    val anotherEnv = createRpcEnv(new SparkConf(), "remote", 0, clientMode = true)
    // Use anotherEnv to find out the RpcEndpointRef
    val rpcEndpointRef = anotherEnv.setupEndpointRef(env.address, "sendWithReply-remotely-error")
    try {
      val f = rpcEndpointRef.ask[String]("hello")
      val e = intercept[SparkException] {
        Await.result(f, 5 seconds)
      }
      assert("Oops" === e.getMessage)
    } finally {
      anotherEnv.shutdown()
      anotherEnv.awaitTermination()
    }
  }

  /**
   * Setup an [[RpcEndpoint]] to collect all network events.
   * @return the [[RpcEndpointRef]] and an `ConcurrentLinkedQueue` that contains network events.
   */
  private def setupNetworkEndpoint(
      _env: RpcEnv,
      name: String): (RpcEndpointRef, ConcurrentLinkedQueue[(Any, Any)]) = {
    val events = new ConcurrentLinkedQueue[(Any, Any)]
    val ref = _env.setupEndpoint("network-events-non-client", new ThreadSafeRpcEndpoint {
      override val rpcEnv = _env

      override def receive: PartialFunction[Any, Unit] = {
        case "hello" =>
        case m => events.add("receive" -> m)
      }

      override def onConnected(remoteAddress: RpcAddress): Unit = {
        events.add("onConnected" -> remoteAddress)
      }

      override def onDisconnected(remoteAddress: RpcAddress): Unit = {
        events.add("onDisconnected" -> remoteAddress)
      }

      override def onNetworkError(cause: Throwable, remoteAddress: RpcAddress): Unit = {
        events.add("onNetworkError" -> remoteAddress)
      }

    })
    (ref, events)
  }

  test("network events in sever RpcEnv when another RpcEnv is in server mode") {
    val serverEnv1 = createRpcEnv(new SparkConf(), "server1", 0, clientMode = false)
    val serverEnv2 = createRpcEnv(new SparkConf(), "server2", 0, clientMode = false)
    val (_, events) = setupNetworkEndpoint(serverEnv1, "network-events")
    val (serverRef2, _) = setupNetworkEndpoint(serverEnv2, "network-events")
    try {
      val serverRefInServer2 = serverEnv1.setupEndpointRef(serverRef2.address, serverRef2.name)
      // Send a message to set up the connection
      serverRefInServer2.send("hello")

      eventually(timeout(5 seconds), interval(5 millis)) {
        assert(events.contains(("onConnected", serverEnv2.address)))
      }

      serverEnv2.shutdown()
      serverEnv2.awaitTermination()

      eventually(timeout(5 seconds), interval(5 millis)) {
        assert(events.contains(("onConnected", serverEnv2.address)))
        assert(events.contains(("onDisconnected", serverEnv2.address)))
      }
    } finally {
      serverEnv1.shutdown()
      serverEnv2.shutdown()
      serverEnv1.awaitTermination()
      serverEnv2.awaitTermination()
    }
  }

  test("network events in sever RpcEnv when another RpcEnv is in client mode") {
    val serverEnv = createRpcEnv(new SparkConf(), "server", 0, clientMode = false)
    val (serverRef, events) = setupNetworkEndpoint(serverEnv, "network-events")
    val clientEnv = createRpcEnv(new SparkConf(), "client", 0, clientMode = true)
    try {
      val serverRefInClient = clientEnv.setupEndpointRef(serverRef.address, serverRef.name)
      // Send a message to set up the connection
      serverRefInClient.send("hello")

      eventually(timeout(5 seconds), interval(5 millis)) {
        // We don't know the exact client address but at least we can verify the message type
        assert(events.asScala.map(_._1).exists(_ == "onConnected"))
      }

      clientEnv.shutdown()
      clientEnv.awaitTermination()

      eventually(timeout(5 seconds), interval(5 millis)) {
        // We don't know the exact client address but at least we can verify the message type
        assert(events.asScala.map(_._1).exists(_ == "onConnected"))
        assert(events.asScala.map(_._1).exists(_ == "onDisconnected"))
      }
    } finally {
      clientEnv.shutdown()
      serverEnv.shutdown()
      clientEnv.awaitTermination()
      serverEnv.awaitTermination()
    }
  }

  test("network events in client RpcEnv when another RpcEnv is in server mode") {
    val clientEnv = createRpcEnv(new SparkConf(), "client", 0, clientMode = true)
    val serverEnv = createRpcEnv(new SparkConf(), "server", 0, clientMode = false)
    val (_, events) = setupNetworkEndpoint(clientEnv, "network-events")
    val (serverRef, _) = setupNetworkEndpoint(serverEnv, "network-events")
    try {
      val serverRefInClient = clientEnv.setupEndpointRef(serverRef.address, serverRef.name)
      // Send a message to set up the connection
      serverRefInClient.send("hello")

      eventually(timeout(5 seconds), interval(5 millis)) {
        assert(events.contains(("onConnected", serverEnv.address)))
      }

      serverEnv.shutdown()
      serverEnv.awaitTermination()

      eventually(timeout(5 seconds), interval(5 millis)) {
        assert(events.contains(("onConnected", serverEnv.address)))
        assert(events.contains(("onDisconnected", serverEnv.address)))
      }
    } finally {
      clientEnv.shutdown()
      serverEnv.shutdown()
      clientEnv.awaitTermination()
      serverEnv.awaitTermination()
    }
  }

  test("sendWithReply: unserializable error") {
    env.setupEndpoint("sendWithReply-unserializable-error", new RpcEndpoint {
      override val rpcEnv = env

      override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = {
        case msg: String => context.sendFailure(new UnserializableException)
      }
    })

    val anotherEnv = createRpcEnv(new SparkConf(), "remote", 0, clientMode = true)
    // Use anotherEnv to find out the RpcEndpointRef
    val rpcEndpointRef =
      anotherEnv.setupEndpointRef(env.address, "sendWithReply-unserializable-error")
    try {
      val f = rpcEndpointRef.ask[String]("hello")
      val e = intercept[Exception] {
        Await.result(f, 1 seconds)
      }
      assert(e.isInstanceOf[NotSerializableException])
    } finally {
      anotherEnv.shutdown()
      anotherEnv.awaitTermination()
    }
  }

  test("port conflict") {
    val anotherEnv = createRpcEnv(new SparkConf(), "remote", env.address.port)
    assert(anotherEnv.address.port != env.address.port)
  }

  test("send with authentication") {
    val conf = new SparkConf
    conf.set("spark.authenticate", "true")
    conf.set("spark.authenticate.secret", "good")

    val localEnv = createRpcEnv(conf, "authentication-local", 0)
    val remoteEnv = createRpcEnv(conf, "authentication-remote", 0, clientMode = true)

    try {
      @volatile var message: String = null
      localEnv.setupEndpoint("send-authentication", new RpcEndpoint {
        override val rpcEnv = localEnv

        override def receive: PartialFunction[Any, Unit] = {
          case msg: String => message = msg
        }
      })
      val rpcEndpointRef = remoteEnv.setupEndpointRef(localEnv.address, "send-authentication")
      rpcEndpointRef.send("hello")
      eventually(timeout(5 seconds), interval(10 millis)) {
        assert("hello" === message)
      }
    } finally {
      localEnv.shutdown()
      localEnv.awaitTermination()
      remoteEnv.shutdown()
      remoteEnv.awaitTermination()
    }
  }

  test("ask with authentication") {
    val conf = new SparkConf
    conf.set("spark.authenticate", "true")
    conf.set("spark.authenticate.secret", "good")

    val localEnv = createRpcEnv(conf, "authentication-local", 0)
    val remoteEnv = createRpcEnv(conf, "authentication-remote", 0, clientMode = true)

    try {
      localEnv.setupEndpoint("ask-authentication", new RpcEndpoint {
        override val rpcEnv = localEnv

        override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = {
          case msg: String =>
            context.reply(msg)
        }
      })
      val rpcEndpointRef = remoteEnv.setupEndpointRef(localEnv.address, "ask-authentication")
      val reply = rpcEndpointRef.askWithRetry[String]("hello")
      assert("hello" === reply)
    } finally {
      localEnv.shutdown()
      localEnv.awaitTermination()
      remoteEnv.shutdown()
      remoteEnv.awaitTermination()
    }
  }

  test("construct RpcTimeout with conf property") {
    val conf = new SparkConf

    val testProp = "spark.ask.test.timeout"
    val testDurationSeconds = 30
    val secondaryProp = "spark.ask.secondary.timeout"

    conf.set(testProp, s"${testDurationSeconds}s")
    conf.set(secondaryProp, "100s")

    // Construct RpcTimeout with a single property
    val rt1 = RpcTimeout(conf, testProp)
    assert( testDurationSeconds === rt1.duration.toSeconds )

    // Construct RpcTimeout with prioritized list of properties
    val rt2 = RpcTimeout(conf, Seq("spark.ask.invalid.timeout", testProp, secondaryProp), "1s")
    assert( testDurationSeconds === rt2.duration.toSeconds )

    // Construct RpcTimeout with default value,
    val defaultProp = "spark.ask.default.timeout"
    val defaultDurationSeconds = 1
    val rt3 = RpcTimeout(conf, Seq(defaultProp), defaultDurationSeconds.toString + "s")
    assert( defaultDurationSeconds === rt3.duration.toSeconds )
    assert( rt3.timeoutProp.contains(defaultProp) )

    // Try to construct RpcTimeout with an unconfigured property
    intercept[NoSuchElementException] {
      RpcTimeout(conf, "spark.ask.invalid.timeout")
    }
  }

  test("ask a message timeout on Future using RpcTimeout") {
    case class NeverReply(msg: String)

    val rpcEndpointRef = env.setupEndpoint("ask-future", new RpcEndpoint {
      override val rpcEnv = env

      override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = {
        case msg: String => context.reply(msg)
        case _: NeverReply =>
      }
    })

    val longTimeout = new RpcTimeout(1 second, "spark.rpc.long.timeout")
    val shortTimeout = new RpcTimeout(10 millis, "spark.rpc.short.timeout")

    // Ask with immediate response, should complete successfully
    val fut1 = rpcEndpointRef.ask[String]("hello", longTimeout)
    val reply1 = longTimeout.awaitResult(fut1)
    assert("hello" === reply1)

    // Ask with a delayed response and wait for response immediately that should timeout
    val fut2 = rpcEndpointRef.ask[String](NeverReply("doh"), shortTimeout)
    val reply2 =
      intercept[RpcTimeoutException] {
        shortTimeout.awaitResult(fut2)
      }.getMessage

    // RpcTimeout.awaitResult should have added the property to the TimeoutException message
    assert(reply2.contains(shortTimeout.timeoutProp))

    // Ask with delayed response and allow the Future to timeout before Await.result
    val fut3 = rpcEndpointRef.ask[String](NeverReply("goodbye"), shortTimeout)

    // Allow future to complete with failure using plain Await.result, this will return
    // once the future is complete to verify addMessageIfTimeout was invoked
    val reply3 =
      intercept[RpcTimeoutException] {
        Await.result(fut3, 2000 millis)
      }.getMessage

    // When the future timed out, the recover callback should have used
    // RpcTimeout.addMessageIfTimeout to add the property to the TimeoutException message
    assert(reply3.contains(shortTimeout.timeoutProp))

    // Use RpcTimeout.awaitResult to process Future, since it has already failed with
    // RpcTimeoutException, the same RpcTimeoutException should be thrown
    val reply4 =
      intercept[RpcTimeoutException] {
        shortTimeout.awaitResult(fut3)
      }.getMessage

    // Ensure description is not in message twice after addMessageIfTimeout and awaitResult
    assert(shortTimeout.timeoutProp.r.findAllIn(reply4).length === 1)
  }

  test("file server") {
    val conf = new SparkConf()
    val tempDir = Utils.createTempDir()
    val file = new File(tempDir, "file")
    Files.write(UUID.randomUUID().toString(), file, UTF_8)
    val fileWithSpecialChars = new File(tempDir, "file name")
    Files.write(UUID.randomUUID().toString(), fileWithSpecialChars, UTF_8)
    val empty = new File(tempDir, "empty")
    Files.write("", empty, UTF_8);
    val jar = new File(tempDir, "jar")
    Files.write(UUID.randomUUID().toString(), jar, UTF_8)

    val dir1 = new File(tempDir, "dir1")
    assert(dir1.mkdir())
    val subFile1 = new File(dir1, "file1")
    Files.write(UUID.randomUUID().toString(), subFile1, UTF_8)

    val dir2 = new File(tempDir, "dir2")
    assert(dir2.mkdir())
    val subFile2 = new File(dir2, "file2")
    Files.write(UUID.randomUUID().toString(), subFile2, UTF_8)

    val fileUri = env.fileServer.addFile(file)
    val fileWithSpecialCharsUri = env.fileServer.addFile(fileWithSpecialChars)
    val emptyUri = env.fileServer.addFile(empty)
    val jarUri = env.fileServer.addJar(jar)
    val dir1Uri = env.fileServer.addDirectory("/dir1", dir1)
    val dir2Uri = env.fileServer.addDirectory("/dir2", dir2)

    // Try registering directories with invalid names.
    Seq("/files", "/jars").foreach { uri =>
      intercept[IllegalArgumentException] {
        env.fileServer.addDirectory(uri, dir1)
      }
    }

    val destDir = Utils.createTempDir()
    val sm = new SecurityManager(conf)
    val hc = SparkHadoopUtil.get.conf

    val files = Seq(
      (file, fileUri),
      (fileWithSpecialChars, fileWithSpecialCharsUri),
      (empty, emptyUri),
      (jar, jarUri),
      (subFile1, dir1Uri + "/file1"),
      (subFile2, dir2Uri + "/file2"))
    files.foreach { case (f, uri) =>
      val destFile = new File(destDir, f.getName())
      Utils.fetchFile(uri, destDir, conf, sm, hc, 0L, false)
      assert(Files.equal(f, destFile))
    }

    // Try to download files that do not exist.
    Seq("files", "jars", "dir1").foreach { root =>
      intercept[Exception] {
        val uri = env.address.toSparkURL + s"/$root/doesNotExist"
        Utils.fetchFile(uri, destDir, conf, sm, hc, 0L, false)
      }
    }
  }

}

class UnserializableClass

class UnserializableException extends Exception {
  private val unserializableField = new UnserializableClass
}