Github user andrewor14 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/7820#discussion_r36049340
  
    --- Diff: 
core/src/main/scala/org/apache/spark/deploy/mesos/MesosExternalShuffleService.scala
 ---
    @@ -0,0 +1,142 @@
    +/*
    + * 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.deploy.mesos
    +
    +import java.net.SocketAddress
    +import java.util.concurrent.CountDownLatch
    +
    +import org.apache.spark.deploy.ExternalShuffleService
    +import org.apache.spark.network.client.{RpcResponseCallback, 
TransportClient}
    +import org.apache.spark.network.netty.SparkTransportConf
    +import org.apache.spark.network.shuffle.ExternalShuffleBlockHandler
    +import org.apache.spark.network.shuffle.protocol.BlockTransferMessage
    +import org.apache.spark.network.shuffle.protocol.mesos.RegisterDriver
    +import org.apache.spark.network.util.TransportConf
    +import org.apache.spark.util.Utils
    +import org.apache.spark.{Logging, SecurityManager, SparkConf}
    +
    +import scala.collection.mutable
    +
    +
    +/**
    + * MesosExternalShuffleServiceEndpoint is a RPC endpoint that receives
    + * registration requests from Spark drivers launched with Mesos.
    + * It detects driver termination and calls the cleanup callback to 
[[ExternalShuffleService]]
    + */
    +private[mesos] class MesosExternalShuffleBlockHandler(transportConf: 
TransportConf)
    +  extends ExternalShuffleBlockHandler(transportConf) with Logging {
    +
    +  // Stores a map of driver socket addresses to app ids
    +  private val connectedApps = new mutable.HashMap[SocketAddress, String]
    +
    +  protected override def handleMessage(
    +      message: BlockTransferMessage,
    +      client: TransportClient,
    +      callback: RpcResponseCallback): Unit = {
    +    message match {
    +      case RegisterDriverParam(appId) =>
    +        val address = client.getSocketAddress()
    +        logDebug(s"Received registration request from app $appId, address 
$address")
    +        if (connectedApps.contains(address)) {
    +          val existingAppId: String = connectedApps(address)
    +          if (!existingAppId.equals(appId)) {
    +            logError(s"A new app id $appId has connected to existing 
address $address" +
    +              s", removing registered app $existingAppId")
    +            applicationRemoved(existingAppId, true)
    +          }
    +        }
    +        connectedApps(address) = appId
    +        callback.onSuccess(new Array[Byte](0))
    +      case _ => super.handleMessage(message, client, callback)
    +    }
    +  }
    +
    +  override def connectionTerminated(client: TransportClient): Unit = {
    +    val address = client.getSocketAddress()
    +    if (connectedApps.contains(address)) {
    +      val appId = connectedApps(address)
    +      logInfo(s"Application $appId disconnected (address was $address)")
    +      applicationRemoved(appId, true)
    +      connectedApps.remove(address)
    +    } else {
    +      logWarning(s"Address $address not found in mesos shuffle service")
    +    }
    +  }
    +}
    +
    +/**
    + * An extractor object for matching RegisterDriver message.
    + */
    +private[mesos] object RegisterDriverParam {
    +  def unapply(r: RegisterDriver): Option[String] = Some(r.getAppId())
    +}
    +
    +/**
    + * MesosExternalShuffleService wraps [[ExternalShuffleService]] which 
provides an additional
    + * endpoint for drivers to associate with. This allows the shuffle service 
to detect when
    + * a driver is terminated and can further clean up the cached shuffle data.
    + */
    +private[mesos] class MesosExternalShuffleService(
    +    conf: SparkConf,
    +    securityManager: SecurityManager,
    +    transportConf: TransportConf)
    +  extends ExternalShuffleService(
    +    conf, securityManager, transportConf, new 
MesosExternalShuffleBlockHandler(transportConf)) {
    +}
    +
    +private[spark] object MesosExternalShuffleService extends Logging {
    --- End diff --
    
    this whole thing is duplicated from `ExternalShuffleService`. We should at 
least attempt to reuse some of the code... I think if you make this object 
inherit the `ExternalShuffleService` object we can get a lot done.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to