Skip to content

Commit

Permalink
add security (maybe?) for metrics json
Browse files Browse the repository at this point in the history
  • Loading branch information
squito committed Feb 12, 2015
1 parent f0264a7 commit cba1ef6
Show file tree
Hide file tree
Showing 5 changed files with 74 additions and 9 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -20,12 +20,16 @@ import javax.servlet.ServletContext
import javax.ws.rs._
import javax.ws.rs.core.{Response, Context}

import com.sun.jersey.api.core.ResourceConfig
import com.sun.jersey.spi.container.servlet.ServletContainer
import org.apache.spark.ui.SparkUI

import org.eclipse.jetty.server.handler.ContextHandler
import org.eclipse.jetty.servlet.{ServletHolder, ServletContextHandler}
import org.glassfish.jersey.jackson._

import org.apache.spark.SecurityManager
import org.apache.spark.ui.SparkUI


@Path("/v1")
class JsonRootResource extends UIRootFromServletContext {
Expand Down Expand Up @@ -81,6 +85,8 @@ object JsonRootResource {
"com.sun.jersey.api.core.PackagesResourceConfig")
holder.setInitParameter("com.sun.jersey.config.property.packages",
"org.apache.spark.status.api.v1")
holder.setInitParameter(ResourceConfig.PROPERTY_CONTAINER_REQUEST_FILTERS,
classOf[SecurityFilter].getCanonicalName)
UIRootFromServletContext.setUiRoot(jerseyContext, uiRoot)
jerseyContext.addServlet(holder, "/*")
jerseyContext
Expand All @@ -97,6 +103,7 @@ private[spark] trait UIRoot {
case None => throw new NotFoundException("no such app: " + appId)
}
}
def securityManager: SecurityManager
}

object UIRootFromServletContext {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,38 @@
/*
* 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.status.api.v1

import javax.ws.rs.WebApplicationException
import javax.ws.rs.core.Response

import com.sun.jersey.spi.container.{ContainerRequest,ContainerRequestFilter}

class SecurityFilter extends ContainerRequestFilter with UIRootFromServletContext {
def filter(req: ContainerRequest): ContainerRequest = {
val user = Option(req.getUserPrincipal).map{_.getName}.orNull
if (uiRoot.securityManager.checkUIViewPermissions(user)) {
req
} else {
throw new WebApplicationException(
Response
.status(Response.Status.UNAUTHORIZED)
.entity("user \"" + user + "\"is not authorized")
.build()
)
}
}
}
2 changes: 1 addition & 1 deletion core/src/main/scala/org/apache/spark/ui/SparkUI.scala
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@ import org.apache.spark.ui.storage.{StorageListener, StorageTab}
private[spark] class SparkUI private (
val sc: Option[SparkContext],
val conf: SparkConf,
val securityManager: SecurityManager,
securityManager: SecurityManager,
val environmentListener: EnvironmentListener,
val storageStatusListener: StorageStatusListener,
val executorsListener: ExecutorsListener,
Expand Down
2 changes: 1 addition & 1 deletion core/src/main/scala/org/apache/spark/ui/WebUI.scala
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@ import org.apache.spark.util.Utils
* pages. The use of tabs is optional, however; a WebUI may choose to include pages directly.
*/
private[spark] abstract class WebUI(
securityManager: SecurityManager,
val securityManager: SecurityManager,
port: Int,
conf: SparkConf,
basePath: String = "",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -91,12 +91,32 @@ class HistoryServerSuite extends FunSuite with BeforeAndAfter with Matchers {
}
}

test("fields w/ None are skipped (not written as null)") {
pending
}

test("security") {
pending
val conf = new SparkConf()
.set("spark.history.fs.logDirectory", logDir.getAbsolutePath)
.set("spark.history.fs.updateInterval", "0")
.set("spark.acls.enable", "true")
.set("spark.ui.view.acls", "user1")
val securityManager = new SecurityManager(conf)

val securePort = port + 1
val secureServer = new HistoryServer(conf, provider, securityManager, securePort)
secureServer.initialize()
secureServer.bind()

securityManager.checkUIViewPermissions("user1") should be (true)
securityManager.checkUIViewPermissions("user2") should be (false)

try {

//TODO figure out a way to authenticate as the users in the requests
// getContentAndCode("applications", securePort)._1 should be (200)
pending

} finally {
secureServer.stop()
}

}

test("response codes on bad paths") {
Expand All @@ -117,7 +137,7 @@ class HistoryServerSuite extends FunSuite with BeforeAndAfter with Matchers {

}

def getContentAndCode(path: String): (Int, Option[String], Option[String]) = {
def getContentAndCode(path: String, port: Int = port): (Int, Option[String], Option[String]) = {
val url = new URL(s"http://localhost:$port/json/v1/$path")
val connection = url.openConnection().asInstanceOf[HttpURLConnection]
connection.setRequestMethod("GET")
Expand Down

0 comments on commit cba1ef6

Please sign in to comment.