You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by do...@apache.org on 2024/01/29 06:38:43 UTC

(spark) branch master updated: [SPARK-46899][CORE] Remove `POST` APIs from `MasterWebUI` when `spark.ui.killEnabled` is `false`

This is an automated email from the ASF dual-hosted git repository.

dongjoon pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
     new 95a4abd5b5bc [SPARK-46899][CORE] Remove `POST` APIs from `MasterWebUI` when `spark.ui.killEnabled` is `false`
95a4abd5b5bc is described below

commit 95a4abd5b5bcc36335be9af84b7bbddd7d0034ba
Author: Dongjoon Hyun <dh...@apple.com>
AuthorDate: Sun Jan 28 22:38:32 2024 -0800

    [SPARK-46899][CORE] Remove `POST` APIs from `MasterWebUI` when `spark.ui.killEnabled` is `false`
    
    ### What changes were proposed in this pull request?
    
    This PR aims to remove `POST` APIs from `MasterWebUI` when `spark.ui.killEnabled` is false.
    
    ### Why are the changes needed?
    
    If `spark.ui.killEnabled` is false, we don't need to attach `POST`-related redirect or servlet handlers from the beginning because it will be ignored in `MasterPage`.
    
    https://github.com/apache/spark/blob/8cd0d1854da04334aff3188e4eca08a48f734579/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala#L64-L65
    
    ### Does this PR introduce _any_ user-facing change?
    
    Previously, the user request is ignored silently after redirecting. Now, it will response with a correct HTTP error code, 405 `Method Not Allowed`.
    
    ### How was this patch tested?
    
    Pass the CIs with newly added test suite, `ReadOnlyMasterWebUISuite`.
    
    ### Was this patch authored or co-authored using generative AI tooling?
    
    No.
    
    Closes #44926 from dongjoon-hyun/SPARK-46899.
    
    Authored-by: Dongjoon Hyun <dh...@apple.com>
    Signed-off-by: Dongjoon Hyun <dh...@apple.com>
---
 .../spark/deploy/master/ui/MasterWebUI.scala       | 46 ++++++-------
 .../spark/deploy/master/ui/MasterWebUISuite.scala  |  9 ++-
 .../master/ui/ReadOnlyMasterWebUISuite.scala       | 75 ++++++++++++++++++++++
 3 files changed, 105 insertions(+), 25 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala
index 3025c0bf468b..14ea6dbb3d20 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala
@@ -54,31 +54,33 @@ class MasterWebUI(
     attachPage(new LogPage(this))
     attachPage(masterPage)
     addStaticHandler(MasterWebUI.STATIC_RESOURCE_DIR)
-    attachHandler(createRedirectHandler(
-      "/app/kill", "/", masterPage.handleAppKillRequest, httpMethods = Set("POST")))
-    attachHandler(createRedirectHandler(
-      "/driver/kill", "/", masterPage.handleDriverKillRequest, httpMethods = Set("POST")))
-    attachHandler(createServletHandler("/workers/kill", new HttpServlet {
-      override def doPost(req: HttpServletRequest, resp: HttpServletResponse): Unit = {
-        val hostnames: Seq[String] = Option(req.getParameterValues("host"))
-          .getOrElse(Array[String]()).toImmutableArraySeq
-        if (decommissionDisabled || !isDecommissioningRequestAllowed(req)) {
-          resp.sendError(HttpServletResponse.SC_METHOD_NOT_ALLOWED)
-        } else {
-          val removedWorkers = masterEndpointRef.askSync[Integer](
-            DecommissionWorkersOnHosts(hostnames))
-          logInfo(s"Decommissioning of hosts $hostnames decommissioned $removedWorkers workers")
-          if (removedWorkers > 0) {
-            resp.setStatus(HttpServletResponse.SC_OK)
-          } else if (removedWorkers == 0) {
-            resp.sendError(HttpServletResponse.SC_NOT_FOUND)
+    if (killEnabled) {
+      attachHandler(createRedirectHandler(
+        "/app/kill", "/", masterPage.handleAppKillRequest, httpMethods = Set("POST")))
+      attachHandler(createRedirectHandler(
+        "/driver/kill", "/", masterPage.handleDriverKillRequest, httpMethods = Set("POST")))
+      attachHandler(createServletHandler("/workers/kill", new HttpServlet {
+        override def doPost(req: HttpServletRequest, resp: HttpServletResponse): Unit = {
+          val hostnames: Seq[String] = Option(req.getParameterValues("host"))
+            .getOrElse(Array[String]()).toImmutableArraySeq
+          if (decommissionDisabled || !isDecommissioningRequestAllowed(req)) {
+            resp.sendError(HttpServletResponse.SC_METHOD_NOT_ALLOWED)
           } else {
-            // We shouldn't even see this case.
-            resp.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR)
+            val removedWorkers = masterEndpointRef.askSync[Integer](
+              DecommissionWorkersOnHosts(hostnames))
+            logInfo(s"Decommissioning of hosts $hostnames decommissioned $removedWorkers workers")
+            if (removedWorkers > 0) {
+              resp.setStatus(HttpServletResponse.SC_OK)
+            } else if (removedWorkers == 0) {
+              resp.sendError(HttpServletResponse.SC_NOT_FOUND)
+            } else {
+              // We shouldn't even see this case.
+              resp.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR)
+            }
           }
         }
-      }
-    }, ""))
+      }, ""))
+    }
   }
 
   def addProxy(): Unit = {
diff --git a/core/src/test/scala/org/apache/spark/deploy/master/ui/MasterWebUISuite.scala b/core/src/test/scala/org/apache/spark/deploy/master/ui/MasterWebUISuite.scala
index 40265a12af93..caefab9e0746 100644
--- a/core/src/test/scala/org/apache/spark/deploy/master/ui/MasterWebUISuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/master/ui/MasterWebUISuite.scala
@@ -35,6 +35,7 @@ import org.apache.spark.rpc.{RpcEndpointRef, RpcEnv}
 import org.apache.spark.util.Utils
 
 class MasterWebUISuite extends SparkFunSuite {
+  import MasterWebUISuite._
 
   val conf = new SparkConf().set(DECOMMISSION_ENABLED, true)
   val securityMgr = new SecurityManager(conf)
@@ -105,12 +106,14 @@ class MasterWebUISuite extends SparkFunSuite {
   test("Kill multiple hosts") {
     testKillWorkers(Seq("noSuchHost", "LocalHost"))
   }
+}
 
-  private def convPostDataToString(data: Seq[(String, String)]): String = {
+object MasterWebUISuite {
+  private[ui] def convPostDataToString(data: Seq[(String, String)]): String = {
     (for ((name, value) <- data) yield s"$name=$value").mkString("&")
   }
 
-  private def convPostDataToString(data: Map[String, String]): String = {
+  private[ui] def convPostDataToString(data: Map[String, String]): String = {
     convPostDataToString(data.toSeq)
   }
 
@@ -118,7 +121,7 @@ class MasterWebUISuite extends SparkFunSuite {
    * Send an HTTP request to the given URL using the method and the body specified.
    * Return the connection object.
    */
-  private def sendHttpRequest(
+  private[ui] def sendHttpRequest(
       url: String,
       method: String,
       body: String = ""): HttpURLConnection = {
diff --git a/core/src/test/scala/org/apache/spark/deploy/master/ui/ReadOnlyMasterWebUISuite.scala b/core/src/test/scala/org/apache/spark/deploy/master/ui/ReadOnlyMasterWebUISuite.scala
new file mode 100644
index 000000000000..c52ce91fda8b
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/deploy/master/ui/ReadOnlyMasterWebUISuite.scala
@@ -0,0 +1,75 @@
+/*
+ * 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.master.ui
+
+import javax.servlet.http.HttpServletResponse.SC_METHOD_NOT_ALLOWED
+
+import org.mockito.Mockito.{mock, when}
+
+import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite}
+import org.apache.spark.deploy.master._
+import org.apache.spark.deploy.master.ui.MasterWebUISuite._
+import org.apache.spark.internal.config.UI.UI_KILL_ENABLED
+import org.apache.spark.rpc.{RpcEndpointRef, RpcEnv}
+import org.apache.spark.util.Utils
+
+class ReadOnlyMasterWebUISuite extends SparkFunSuite {
+
+  val conf = new SparkConf().set(UI_KILL_ENABLED, false)
+  val securityMgr = new SecurityManager(conf)
+  val rpcEnv = mock(classOf[RpcEnv])
+  val master = mock(classOf[Master])
+  val masterEndpointRef = mock(classOf[RpcEndpointRef])
+  when(master.securityMgr).thenReturn(securityMgr)
+  when(master.conf).thenReturn(conf)
+  when(master.rpcEnv).thenReturn(rpcEnv)
+  when(master.self).thenReturn(masterEndpointRef)
+  val masterWebUI = new MasterWebUI(master, 0)
+
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+    masterWebUI.bind()
+  }
+
+  override def afterAll(): Unit = {
+    try {
+      masterWebUI.stop()
+    } finally {
+      super.afterAll()
+    }
+  }
+
+  test("/app/kill POST method is not allowed") {
+    val url = s"http://${Utils.localHostNameForURI()}:${masterWebUI.boundPort}/app/kill/"
+    val body = convPostDataToString(Map(("id", "1"), ("terminate", "true")))
+    assert(sendHttpRequest(url, "POST", body).getResponseCode === SC_METHOD_NOT_ALLOWED)
+  }
+
+  test("/driver/kill POST method is not allowed") {
+    val url = s"http://${Utils.localHostNameForURI()}:${masterWebUI.boundPort}/driver/kill/"
+    val body = convPostDataToString(Map(("id", "driver-0"), ("terminate", "true")))
+    assert(sendHttpRequest(url, "POST", body).getResponseCode === SC_METHOD_NOT_ALLOWED)
+  }
+
+  test("/workers/kill POST method is not allowed") {
+    val hostnames = Seq(s"${Utils.localHostNameForURI()}")
+    val url = s"http://${Utils.localHostNameForURI()}:${masterWebUI.boundPort}/workers/kill/"
+    val body = convPostDataToString(hostnames.map(("host", _)))
+    assert(sendHttpRequest(url, "POST", body).getResponseCode === SC_METHOD_NOT_ALLOWED)
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org