aboutsummaryrefslogtreecommitdiff
path: root/core/src/main/scala/spark/broadcast/ChainedBroadcast.scala
blob: 3afe923bae338da7ea12a501952efc2d2163175e (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
package spark.broadcast

import java.io._
import java.net._
import java.util.{Comparator, PriorityQueue, Random, UUID}

import scala.collection.mutable.{Map, Set}
import scala.math

import spark._

@serializable
class ChainedBroadcast[T](@transient var value_ : T, isLocal: Boolean)
extends Broadcast[T] with Logging {

  def value = value_

  ChainedBroadcast.synchronized {
    ChainedBroadcast.values.put(uuid, value_)
  }

  @transient var arrayOfBlocks: Array[BroadcastBlock] = null
  @transient var totalBytes = -1
  @transient var totalBlocks = -1
  @transient var hasBlocks = 0
  // CHANGED: BlockSize in the Broadcast object is expected to change over time
  @transient var blockSize = Broadcast.BlockSize

  @transient var listenPortLock = new Object
  @transient var guidePortLock = new Object
  @transient var totalBlocksLock = new Object
  @transient var hasBlocksLock = new Object

  @transient var pqOfSources = new PriorityQueue[SourceInfo]

  @transient var serveMR: ServeMultipleRequests = null
  @transient var guideMR: GuideMultipleRequests = null

  @transient var hostAddress = Utils.localIpAddress
  @transient var listenPort = -1
  @transient var guidePort = -1

  @transient var hasCopyInHDFS = false
  @transient var stopBroadcast = false

  // Must call this after all the variables have been created/initialized
  if (!isLocal) {
    sendBroadcast
  }

  def sendBroadcast(): Unit = {
    logInfo("Local host address: " + hostAddress)

    // Store a persistent copy in HDFS
    // TODO: Turned OFF for now
    // val out = new ObjectOutputStream(DfsBroadcast.openFileForWriting(uuid))
    // out.writeObject(value_)
    // out.close()
    // TODO: Fix this at some point
    hasCopyInHDFS = true

    // Create a variableInfo object and store it in valueInfos
    var variableInfo = Broadcast.blockifyObject(value_)

    guideMR = new GuideMultipleRequests
    guideMR.setDaemon(true)
    guideMR.start()
    logInfo("GuideMultipleRequests started...")

    serveMR = new ServeMultipleRequests
    serveMR.setDaemon(true)
    serveMR.start()
    logInfo("ServeMultipleRequests started...")

    // Prepare the value being broadcasted
    // TODO: Refactoring and clean-up required here
    arrayOfBlocks = variableInfo.arrayOfBlocks
    totalBytes = variableInfo.totalBytes
    totalBlocks = variableInfo.totalBlocks
    hasBlocks = variableInfo.totalBlocks

    while (listenPort == -1) {
      listenPortLock.synchronized {
        listenPortLock.wait
      }
    }

    pqOfSources = new PriorityQueue[SourceInfo]
    val masterSource =
      SourceInfo(hostAddress, listenPort, totalBlocks, totalBytes, blockSize)
    pqOfSources.add(masterSource)

    // Register with the Tracker
    while (guidePort == -1) {
      guidePortLock.synchronized {
        guidePortLock.wait
      }
    }
    ChainedBroadcast.registerValue(uuid, guidePort)
  }

  private def readObject(in: ObjectInputStream): Unit = {
    in.defaultReadObject
    ChainedBroadcast.synchronized {
      val cachedVal = ChainedBroadcast.values.get(uuid)
      if (cachedVal != null) {
        value_ = cachedVal.asInstanceOf[T]
      } else {
        // Initializing everything because Master will only send null/0 values
        initializeSlaveVariables

        logInfo("Local host address: " + hostAddress)

        serveMR = new ServeMultipleRequests
        serveMR.setDaemon(true)
        serveMR.start()
        logInfo("ServeMultipleRequests started...")

        val start = System.nanoTime

        val receptionSucceeded = receiveBroadcast(uuid)
        // If does not succeed, then get from HDFS copy
        if (receptionSucceeded) {
          value_ = Broadcast.unBlockifyObject[T](arrayOfBlocks, totalBytes, totalBlocks)
          ChainedBroadcast.values.put(uuid, value_)
        }  else {
          val fileIn = new ObjectInputStream(DfsBroadcast.openFileForReading(uuid))
          value_ = fileIn.readObject.asInstanceOf[T]
          ChainedBroadcast.values.put(uuid, value_)
          fileIn.close()
        }

        val time =(System.nanoTime - start) / 1e9
        logInfo("Reading Broadcasted variable " + uuid + " took " + time + " s")
      }
    }
  }

  private def initializeSlaveVariables: Unit = {
    arrayOfBlocks = null
    totalBytes = -1
    totalBlocks = -1
    hasBlocks = 0
    blockSize = -1

    listenPortLock = new Object
    totalBlocksLock = new Object
    hasBlocksLock = new Object

    serveMR =  null

    hostAddress = Utils.localIpAddress
    listenPort = -1

    stopBroadcast = false
  }

  def getMasterListenPort(variableUUID: UUID): Int = {
    var clientSocketToTracker: Socket = null
    var oosTracker: ObjectOutputStream = null
    var oisTracker: ObjectInputStream = null

    var masterListenPort: Int = SourceInfo.TxOverGoToHDFS

    var retriesLeft = Broadcast.MaxRetryCount
    do {
      try {
        // Connect to the tracker to find out the guide
        clientSocketToTracker =
          new Socket(Broadcast.MasterHostAddress, Broadcast.MasterTrackerPort)
        oosTracker =
          new ObjectOutputStream(clientSocketToTracker.getOutputStream)
        oosTracker.flush()
        oisTracker =
          new ObjectInputStream(clientSocketToTracker.getInputStream)

        // Send UUID and receive masterListenPort
        oosTracker.writeObject(uuid)
        oosTracker.flush()
        masterListenPort = oisTracker.readObject.asInstanceOf[Int]
      } catch {
        case e: Exception => {
          logInfo("getMasterListenPort had a " + e)
        }
      } finally {
        if (oisTracker != null) {
          oisTracker.close()
        }
        if (oosTracker != null) {
          oosTracker.close()
        }
        if (clientSocketToTracker != null) {
          clientSocketToTracker.close()
        }
      }
      retriesLeft -= 1

      Thread.sleep(ChainedBroadcast.ranGen.nextInt(
        Broadcast.MaxKnockInterval - Broadcast.MinKnockInterval) +
        Broadcast.MinKnockInterval)

    } while (retriesLeft > 0 && masterListenPort == SourceInfo.TxNotStartedRetry)

    logInfo("Got this guidePort from Tracker: " + masterListenPort)
    return masterListenPort
  }

  def receiveBroadcast(variableUUID: UUID): Boolean = {
    val masterListenPort = getMasterListenPort(variableUUID)

    if (masterListenPort == SourceInfo.TxOverGoToHDFS ||
        masterListenPort == SourceInfo.TxNotStartedRetry) {
      // TODO: SourceInfo.TxNotStartedRetry is not really in use because we go
      // to HDFS anyway when receiveBroadcast returns false
      return false
    }

    // Wait until hostAddress and listenPort are created by the
    // ServeMultipleRequests thread
    while (listenPort == -1) {
      listenPortLock.synchronized {
        listenPortLock.wait
      }
    }

    var clientSocketToMaster: Socket = null
    var oosMaster: ObjectOutputStream = null
    var oisMaster: ObjectInputStream = null

    // Connect and receive broadcast from the specified source, retrying the
    // specified number of times in case of failures
    var retriesLeft = Broadcast.MaxRetryCount
    do {
      // Connect to Master and send this worker's Information
      clientSocketToMaster =
        new Socket(Broadcast.MasterHostAddress, masterListenPort)
      // TODO: Guiding object connection is reusable
      oosMaster =
        new ObjectOutputStream(clientSocketToMaster.getOutputStream)
      oosMaster.flush()
      oisMaster =
        new ObjectInputStream(clientSocketToMaster.getInputStream)

      logInfo("Connected to Master's guiding object")

      // Send local source information
      oosMaster.writeObject(SourceInfo(hostAddress, listenPort))
      oosMaster.flush()

      // Receive source information from Master
      var sourceInfo = oisMaster.readObject.asInstanceOf[SourceInfo]
      totalBlocks = sourceInfo.totalBlocks
      arrayOfBlocks = new Array[BroadcastBlock](totalBlocks)
      totalBlocksLock.synchronized {
        totalBlocksLock.notifyAll
      }
      totalBytes = sourceInfo.totalBytes

      logInfo("Received SourceInfo from Master:" + sourceInfo + " My Port: " + listenPort)

      val start = System.nanoTime
      val receptionSucceeded = receiveSingleTransmission(sourceInfo)
      val time =(System.nanoTime - start) / 1e9

      // Updating some statistics in sourceInfo. Master will be using them later
      if (!receptionSucceeded) {
        sourceInfo.receptionFailed = true
      }

      // Send back statistics to the Master
      oosMaster.writeObject(sourceInfo)

      if (oisMaster != null) {
        oisMaster.close()
      }
      if (oosMaster != null) {
        oosMaster.close()
      }
      if (clientSocketToMaster != null) {
        clientSocketToMaster.close()
      }

      retriesLeft -= 1
    } while (retriesLeft > 0 && hasBlocks < totalBlocks)

    return(hasBlocks == totalBlocks)
  }

  // Tries to receive broadcast from the source and returns Boolean status.
  // This might be called multiple times to retry a defined number of times.
  private def receiveSingleTransmission(sourceInfo: SourceInfo): Boolean = {
    var clientSocketToSource: Socket = null
    var oosSource: ObjectOutputStream = null
    var oisSource: ObjectInputStream = null

    var receptionSucceeded = false
    try {
      // Connect to the source to get the object itself
      clientSocketToSource =
        new Socket(sourceInfo.hostAddress, sourceInfo.listenPort)
      oosSource =
        new ObjectOutputStream(clientSocketToSource.getOutputStream)
      oosSource.flush()
      oisSource =
        new ObjectInputStream(clientSocketToSource.getInputStream)

      logInfo("Inside receiveSingleTransmission")
      logInfo("totalBlocks: "+ totalBlocks + " " + "hasBlocks: " + hasBlocks)

      // Send the range
      oosSource.writeObject((hasBlocks, totalBlocks))
      oosSource.flush()

      for (i <- hasBlocks until totalBlocks) {
        val recvStartTime = System.currentTimeMillis
        val bcBlock = oisSource.readObject.asInstanceOf[BroadcastBlock]
        val receptionTime =(System.currentTimeMillis - recvStartTime)

        logInfo("Received block: " + bcBlock.blockID + " from " + sourceInfo + " in " + receptionTime + " millis.")

        arrayOfBlocks(hasBlocks) = bcBlock
        hasBlocks += 1
        // Set to true if at least one block is received
        receptionSucceeded = true
        hasBlocksLock.synchronized {
          hasBlocksLock.notifyAll
        }
      }
    } catch {
      case e: Exception => {
        logInfo("receiveSingleTransmission had a " + e)
      }
    } finally {
      if (oisSource != null) {
        oisSource.close()
      }
      if (oosSource != null) {
        oosSource.close()
      }
      if (clientSocketToSource != null) {
        clientSocketToSource.close()
      }
    }

    return receptionSucceeded
  }

  class GuideMultipleRequests
  extends Thread with Logging {
    // Keep track of sources that have completed reception
    private var setOfCompletedSources = Set[SourceInfo]()

    override def run: Unit = {
      var threadPool = Utils.newDaemonCachedThreadPool()
      var serverSocket: ServerSocket = null

      serverSocket = new ServerSocket(0)
      guidePort = serverSocket.getLocalPort
      logInfo("GuideMultipleRequests => " + serverSocket + " " + guidePort)

      guidePortLock.synchronized {
        guidePortLock.notifyAll
      }

      try {
        // Don't stop until there is a copy in HDFS
        while (!stopBroadcast || !hasCopyInHDFS) {
          var clientSocket: Socket = null
          try {
            serverSocket.setSoTimeout(Broadcast.ServerSocketTimeout)
            clientSocket = serverSocket.accept
          } catch {
            case e: Exception => {
              logInfo("GuideMultipleRequests Timeout.")

              // Stop broadcast if at least one worker has connected and
              // everyone connected so far are done. Comparing with
              // pqOfSources.size - 1, because it includes the Guide itself
              if (pqOfSources.size > 1 &&
                setOfCompletedSources.size == pqOfSources.size - 1) {
                stopBroadcast = true
              }
            }
          }
          if (clientSocket != null) {
            logInfo("Guide: Accepted new client connection: " + clientSocket)
            try {
              threadPool.execute(new GuideSingleRequest(clientSocket))
            } catch {
              // In failure, close the socket here; else, the thread will close it
              case ioe: IOException => clientSocket.close()
            }
          }
        }

        logInfo("Sending stopBroadcast notifications...")
        sendStopBroadcastNotifications

        ChainedBroadcast.unregisterValue(uuid)
      } finally {
        if (serverSocket != null) {
          logInfo("GuideMultipleRequests now stopping...")
          serverSocket.close()
        }
      }

      // Shutdown the thread pool
      threadPool.shutdown()
    }

    private def sendStopBroadcastNotifications: Unit = {
      pqOfSources.synchronized {
        var pqIter = pqOfSources.iterator
        while (pqIter.hasNext) {
          var sourceInfo = pqIter.next

          var guideSocketToSource: Socket = null
          var gosSource: ObjectOutputStream = null
          var gisSource: ObjectInputStream = null

          try {
            // Connect to the source
            guideSocketToSource =
              new Socket(sourceInfo.hostAddress, sourceInfo.listenPort)
            gosSource =
              new ObjectOutputStream(guideSocketToSource.getOutputStream)
            gosSource.flush()
            gisSource =
              new ObjectInputStream(guideSocketToSource.getInputStream)

            // Send stopBroadcast signal. Range = SourceInfo.StopBroadcast*2
            gosSource.writeObject((SourceInfo.StopBroadcast,
              SourceInfo.StopBroadcast))
            gosSource.flush()
          } catch {
            case e: Exception => {
              logInfo("sendStopBroadcastNotifications had a " + e)
            }
          } finally {
            if (gisSource != null) {
              gisSource.close()
            }
            if (gosSource != null) {
              gosSource.close()
            }
            if (guideSocketToSource != null) {
              guideSocketToSource.close()
            }
          }
        }
      }
    }

    class GuideSingleRequest(val clientSocket: Socket)
    extends Thread with Logging {
      private val oos = new ObjectOutputStream(clientSocket.getOutputStream)
      oos.flush()
      private val ois = new ObjectInputStream(clientSocket.getInputStream)

      private var selectedSourceInfo: SourceInfo = null
      private var thisWorkerInfo:SourceInfo = null

      override def run: Unit = {
        try {
          logInfo("new GuideSingleRequest is running")
          // Connecting worker is sending in its hostAddress and listenPort it will
          // be listening to. Other fields are invalid(SourceInfo.UnusedParam)
          var sourceInfo = ois.readObject.asInstanceOf[SourceInfo]

          pqOfSources.synchronized {
            // Select a suitable source and send it back to the worker
            selectedSourceInfo = selectSuitableSource(sourceInfo)
            logInfo("Sending selectedSourceInfo: " + selectedSourceInfo)
            oos.writeObject(selectedSourceInfo)
            oos.flush()

            // Add this new(if it can finish) source to the PQ of sources
            thisWorkerInfo = SourceInfo(sourceInfo.hostAddress,
              sourceInfo.listenPort, totalBlocks, totalBytes, blockSize)
            logInfo("Adding possible new source to pqOfSources: " + thisWorkerInfo)
            pqOfSources.add(thisWorkerInfo)
          }

          // Wait till the whole transfer is done. Then receive and update source
          // statistics in pqOfSources
          sourceInfo = ois.readObject.asInstanceOf[SourceInfo]

          pqOfSources.synchronized {
            // This should work since SourceInfo is a case class
            assert(pqOfSources.contains(selectedSourceInfo))

            // Remove first
            pqOfSources.remove(selectedSourceInfo)
            // TODO: Removing a source based on just one failure notification!

            // Update sourceInfo and put it back in, IF reception succeeded
            if (!sourceInfo.receptionFailed) {
              // Add thisWorkerInfo to sources that have completed reception
              setOfCompletedSources.synchronized {
                setOfCompletedSources += thisWorkerInfo
              }

              selectedSourceInfo.currentLeechers -= 1

              // Put it back
              pqOfSources.add(selectedSourceInfo)
            }
          }
        } catch {
          // If something went wrong, e.g., the worker at the other end died etc.
          // then close everything up
          case e: Exception => {
            // Assuming that exception caused due to receiver worker failure.
            // Remove failed worker from pqOfSources and update leecherCount of
            // corresponding source worker
            pqOfSources.synchronized {
              if (selectedSourceInfo != null) {
                // Remove first
                pqOfSources.remove(selectedSourceInfo)
                // Update leecher count and put it back in
                selectedSourceInfo.currentLeechers -= 1
                pqOfSources.add(selectedSourceInfo)
              }

              // Remove thisWorkerInfo
              if (pqOfSources != null) {
                pqOfSources.remove(thisWorkerInfo)
              }
            }
          }
        } finally {
          ois.close()
          oos.close()
          clientSocket.close()
        }
      }

      // FIXME: Caller must have a synchronized block on pqOfSources
      // FIXME: If a worker fails to get the broadcasted variable from a source and
      // comes back to Master, this function might choose the worker itself as a
      // source tp create a dependency cycle(this worker was put into pqOfSources
      // as a streming source when it first arrived). The length of this cycle can
      // be arbitrarily long.
      private def selectSuitableSource(skipSourceInfo: SourceInfo): SourceInfo = {
        // Select one based on the ordering strategy(e.g., least leechers etc.)
        // take is a blocking call removing the element from PQ
        var selectedSource = pqOfSources.poll
        assert(selectedSource != null)
        // Update leecher count
        selectedSource.currentLeechers += 1
        // Add it back and then return
        pqOfSources.add(selectedSource)
        return selectedSource
      }
    }
  }

  class ServeMultipleRequests
  extends Thread with Logging {
    override def run: Unit = {
      var threadPool = Utils.newDaemonCachedThreadPool()
      var serverSocket: ServerSocket = null

      serverSocket = new ServerSocket(0)
      listenPort = serverSocket.getLocalPort
      logInfo("ServeMultipleRequests started with " + serverSocket)

      listenPortLock.synchronized {
        listenPortLock.notifyAll
      }

      try {
        while (!stopBroadcast) {
          var clientSocket: Socket = null
          try {
            serverSocket.setSoTimeout(Broadcast.ServerSocketTimeout)
            clientSocket = serverSocket.accept
          } catch {
            case e: Exception => {
              logInfo("ServeMultipleRequests Timeout.")
            }
          }
          if (clientSocket != null) {
            logInfo("Serve: Accepted new client connection: " + clientSocket)
            try {
              threadPool.execute(new ServeSingleRequest(clientSocket))
            } catch {
              // In failure, close socket here; else, the thread will close it
              case ioe: IOException => clientSocket.close()
            }
          }
        }
      } finally {
        if (serverSocket != null) {
          logInfo("ServeMultipleRequests now stopping...")
          serverSocket.close()
        }
      }

      // Shutdown the thread pool
      threadPool.shutdown()
    }

    class ServeSingleRequest(val clientSocket: Socket)
    extends Thread with Logging {
      private val oos = new ObjectOutputStream(clientSocket.getOutputStream)
      oos.flush()
      private val ois = new ObjectInputStream(clientSocket.getInputStream)

      private var sendFrom = 0
      private var sendUntil = totalBlocks

      override def run: Unit = {
        try {
          logInfo("new ServeSingleRequest is running")

          // Receive range to send
          var rangeToSend = ois.readObject.asInstanceOf[(Int, Int)]
          sendFrom = rangeToSend._1
          sendUntil = rangeToSend._2

          if (sendFrom == SourceInfo.StopBroadcast &&
            sendUntil == SourceInfo.StopBroadcast) {
            stopBroadcast = true
          } else {
            // Carry on
            sendObject
          }
        } catch {
          // If something went wrong, e.g., the worker at the other end died etc.
          // then close everything up
          case e: Exception => {
            logInfo("ServeSingleRequest had a " + e)
          }
        } finally {
          logInfo("ServeSingleRequest is closing streams and sockets")
          ois.close()
          oos.close()
          clientSocket.close()
        }
      }

      private def sendObject: Unit = {
        // Wait till receiving the SourceInfo from Master
        while (totalBlocks == -1) {
          totalBlocksLock.synchronized {
            totalBlocksLock.wait
          }
        }

        for (i <- sendFrom until sendUntil) {
          while (i == hasBlocks) {
            hasBlocksLock.synchronized {
              hasBlocksLock.wait
            }
          }
          try {
            oos.writeObject(arrayOfBlocks(i))
            oos.flush()
          } catch {
            case e: Exception => {
              logInfo("sendObject had a " + e)
            }
          }
          logInfo("Sent block: " + i + " to " + clientSocket)
        }
      }
    }
  }
}

class ChainedBroadcastFactory
extends BroadcastFactory {
  def initialize(isMaster: Boolean) = ChainedBroadcast.initialize(isMaster)
  def newBroadcast[T](value_ : T, isLocal: Boolean) =
    new ChainedBroadcast[T](value_, isLocal)
}

private object ChainedBroadcast
extends Logging {
  val values = SparkEnv.get.cache.newKeySpace()

  var valueToGuidePortMap = Map[UUID, Int]()

  // Random number generator
  var ranGen = new Random

  private var initialized = false
  private var isMaster_ = false

  private var trackMV: TrackMultipleValues = null

  def initialize(isMaster__ : Boolean): Unit = {
    synchronized {
      if (!initialized) {
        isMaster_ = isMaster__

        if (isMaster) {
          trackMV = new TrackMultipleValues
          trackMV.setDaemon(true)
          trackMV.start()
          // TODO: Logging the following line makes the Spark framework ID not
          // getting logged, cause it calls logInfo before log4j is initialized
          logInfo("TrackMultipleValues started...")
        }

        // Initialize DfsBroadcast to be used for broadcast variable persistence
        DfsBroadcast.initialize

        initialized = true
      }
    }
  }

  def isMaster = isMaster_

  def registerValue(uuid: UUID, guidePort: Int): Unit = {
    valueToGuidePortMap.synchronized {
      valueToGuidePortMap +=(uuid -> guidePort)
      logInfo("New value registered with the Tracker " + valueToGuidePortMap)
    }
  }

  def unregisterValue(uuid: UUID): Unit = {
    valueToGuidePortMap.synchronized {
      valueToGuidePortMap(uuid) = SourceInfo.TxOverGoToHDFS
      logInfo("Value unregistered from the Tracker " + valueToGuidePortMap)
    }
  }

  class TrackMultipleValues
  extends Thread with Logging {
    override def run: Unit = {
      var threadPool = Utils.newDaemonCachedThreadPool()
      var serverSocket: ServerSocket = null

      serverSocket = new ServerSocket(Broadcast.MasterTrackerPort)
      logInfo("TrackMultipleValues" + serverSocket)

      try {
        while (true) {
          var clientSocket: Socket = null
          try {
            serverSocket.setSoTimeout(Broadcast.TrackerSocketTimeout)
            clientSocket = serverSocket.accept
          } catch {
            case e: Exception => {
              logInfo("TrackMultipleValues Timeout. Stopping listening...")
            }
          }

          if (clientSocket != null) {
            try {
              threadPool.execute(new Thread {
                override def run: Unit = {
                  val oos = new ObjectOutputStream(clientSocket.getOutputStream)
                  oos.flush()
                  val ois = new ObjectInputStream(clientSocket.getInputStream)
                  try {
                    val uuid = ois.readObject.asInstanceOf[UUID]
                    var guidePort =
                      if (valueToGuidePortMap.contains(uuid)) {
                        valueToGuidePortMap(uuid)
                      } else SourceInfo.TxNotStartedRetry
                    logInfo("TrackMultipleValues: Got new request: " + clientSocket + " for " + uuid + " : " + guidePort)
                    oos.writeObject(guidePort)
                  } catch {
                    case e: Exception => {
                      logInfo("TrackMultipleValues had a " + e)
                    }
                  } finally {
                    ois.close()
                    oos.close()
                    clientSocket.close()
                  }
                }
              })
            } catch {
              // In failure, close socket here; else, client thread will close
              case ioe: IOException => clientSocket.close()
            }
          }
        }
      } finally {
        serverSocket.close()
      }

      // Shutdown the thread pool
      threadPool.shutdown()
    }
  }
}