Skip to content
This repository was archived by the owner on Jan 9, 2020. It is now read-only.
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
64 commits
Select commit Hold shift + click to select a range
71b825a
Staging server for receiving application dependencies.
mccheah Apr 3, 2017
e005213
Merge remote-tracking branch 'apache-spark-on-k8s/branch-2.1-kubernet…
mccheah Apr 3, 2017
929d880
Move packages around to split between v1 work and v2 work
mccheah Apr 4, 2017
6e40c4c
Add unit test for file writing
mccheah Apr 4, 2017
e4daf56
Merge branch 'submission-v2-file-server' into branch-packages
mccheah Apr 4, 2017
89c295e
Remove unnecessary main
mccheah Apr 5, 2017
4b96668
Allow the file staging server to be secured with TLS.
mccheah Apr 6, 2017
876bb4a
Add back license header
mccheah Apr 6, 2017
7d00f07
Minor fixes
mccheah Apr 6, 2017
c4f2e64
Merge branch 'submission-v2-file-server' into branch-packages
mccheah Apr 6, 2017
1d6f3f6
Merge branch 'branch-packages' into ssl-submission-v2-file-server
mccheah Apr 6, 2017
0597b4c
Merge branch 'submission-v2-file-server' into split-v1-and-v2-submiss…
mccheah Apr 6, 2017
ea76823
Merge branch 'branch-2.1-kubernetes' into submission-v2-file-server
mccheah Apr 6, 2017
ac0b841
Merge branch 'submission-v2-file-server' into split-v1-and-v2-submiss…
mccheah Apr 6, 2017
b7c1ea5
Merge branch 'split-v1-and-v2-submission-packages' into ssl-submissio…
mccheah Apr 6, 2017
baf10aa
Fix integration test with renamed package for client. Fix scalastyle.
mccheah Apr 6, 2017
757b3dc
Merge branch 'split-v1-and-v2-submission-packages' into ssl-submissio…
mccheah Apr 6, 2017
d041648
Remove unused import
mccheah Apr 6, 2017
d20537d
Force json serialization to consider the different package.
mccheah Apr 10, 2017
2ca73a3
Revert extraneous log
mccheah Apr 10, 2017
ffe8c6c
Fix scalastyle
mccheah Apr 10, 2017
6e5e3a8
Merge branch 'split-v1-and-v2-submission-packages' into ssl-submissio…
mccheah Apr 10, 2017
3dd3504
Remove getting credentials from the API
mccheah Apr 11, 2017
3cbcadf
Merge branch 'submission-v2-file-server' into split-v1-and-v2-submiss…
mccheah Apr 11, 2017
c62910d
Merge branch 'split-v1-and-v2-submission-packages' into ssl-submissio…
mccheah Apr 11, 2017
c1a359a
Fix build
mccheah Apr 11, 2017
fbbad73
Randomize name and namespace in test to prevent collisions
mccheah Apr 12, 2017
9bfb085
Merge branch 'branch-2.1-kubernetes' into submission-v2-file-server
ash211 Apr 12, 2017
df8e0c8
Generalize to resource staging server outside of Spark
mccheah Apr 12, 2017
24452ec
Update code documentation
mccheah Apr 12, 2017
f597171
Val instead of var
mccheah Apr 12, 2017
c20e461
Merge remote-tracking branch 'apache-spark-on-k8s/submission-v2-file-…
mccheah Apr 12, 2017
47704f1
Merge remote-tracking branch 'apache-spark-on-k8s/submission-v2-file-…
mccheah Apr 12, 2017
a6e0856
Merge branch 'split-v1-and-v2-submission-packages' into ssl-submissio…
mccheah Apr 12, 2017
ff0b7cf
Fix unit tests.
mccheah Apr 12, 2017
2e6fe07
Fix build
mccheah Apr 12, 2017
2c1b068
Merge branch 'split-v1-and-v2-submission-packages' into ssl-submissio…
mccheah Apr 12, 2017
e5f26aa
Fix naming, remove unused import
mccheah Apr 13, 2017
a09d935
Merge branch 'submission-v2-file-server' into split-v1-and-v2-submiss…
mccheah Apr 13, 2017
ddd51cc
Merge remote-tracking branch 'apache-spark-on-k8s/split-v1-and-v2-sub…
mccheah Apr 13, 2017
c408ff9
Move suites from integration test package to core
mccheah Apr 13, 2017
928f3a5
Merge branch 'submission-v2-file-server' into split-v1-and-v2-submiss…
mccheah Apr 13, 2017
1f23745
Merge branch 'split-v1-and-v2-submission-packages' into ssl-submissio…
mccheah Apr 13, 2017
539abf5
Fix unit test
mccheah Apr 13, 2017
a527847
Merge branch 'branch-2.1-kubernetes' into submission-v2-file-server
mccheah Apr 18, 2017
0f2f847
Merge branch 'submission-v2-file-server' into split-v1-and-v2-submiss…
mccheah Apr 18, 2017
d969d0b
Merge branch 'split-v1-and-v2-submission-packages' into ssl-submissio…
mccheah Apr 18, 2017
64eddc1
Use TrieMap instead of locks
mccheah Apr 19, 2017
8f79802
Address comments
mccheah Apr 19, 2017
31fd3b6
Merge branch 'submission-v2-file-server' into split-v1-and-v2-submiss…
mccheah Apr 19, 2017
35baecb
Merge branch 'split-v1-and-v2-submission-packages' into ssl-submissio…
mccheah Apr 19, 2017
04099d6
Fix imports
mccheah Apr 19, 2017
cac485c
Merge branch 'submission-v2-file-server' into split-v1-and-v2-submiss…
mccheah Apr 19, 2017
3c331f2
Merge branch 'split-v1-and-v2-submission-packages' into ssl-submissio…
mccheah Apr 19, 2017
b16eafb
Address comments
mccheah Apr 19, 2017
7fb0584
Change main object name
mccheah Apr 19, 2017
f0036f2
Change config variable names
mccheah Apr 20, 2017
d713c27
Change paths, use POST instead of PUT
mccheah Apr 20, 2017
91df628
Merge branch 'submission-v2-file-server' into split-v1-and-v2-submiss…
mccheah Apr 20, 2017
60c1422
Merge branch 'split-v1-and-v2-submission-packages' into ssl-submissio…
mccheah Apr 20, 2017
720c38d
Use a resource identifier as well as a resource secret
mccheah Apr 20, 2017
89b032e
Merge branch 'submission-v2-file-server' into split-v1-and-v2-submiss…
mccheah Apr 20, 2017
86f809f
Merge branch 'split-v1-and-v2-submission-packages' into ssl-submissio…
mccheah Apr 20, 2017
c0711ff
Merge branch 'branch-2.1-kubernetes' into ssl-submission-v2-file-server
ash211 Apr 21, 2017
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -288,4 +288,37 @@ package object config {
.doc("Interval between reports of the current app status in cluster mode.")
.timeConf(TimeUnit.MILLISECONDS)
.createWithDefaultString("1s")

// Spark dependency server for submission v2

private[spark] val RESOURCE_STAGING_SERVER_PORT =
ConfigBuilder("spark.kubernetes.resourceStagingServer.port")
.doc("Port for the Kubernetes resource staging server to listen on.")
.intConf
.createWithDefault(10000)

private[spark] val RESOURCE_STAGING_SERVER_KEY_PEM =
ConfigBuilder("spark.ssl.kubernetes.resourceStagingServer.keyPem")
.doc("Key PEM file to use when having the Kubernetes dependency server listen on TLS.")
.stringConf
.createOptional

private[spark] val RESOURCE_STAGING_SERVER_CERT_PEM =
ConfigBuilder("spark.ssl.kubernetes.resourceStagingServer.serverCertPem")
.doc("Certificate PEM file to use when having the Kubernetes dependency server" +
" listen on TLS.")
.stringConf
.createOptional

private[spark] val RESOURCE_STAGING_SERVER_KEYSTORE_PASSWORD_FILE =
ConfigBuilder("spark.ssl.kubernetes.resourceStagingServer.keyStorePasswordFile")
.doc("File containing the keystore password for the Kubernetes dependency server.")
.stringConf
.createOptional

private[spark] val RESOURCE_STAGING_SERVER_KEYSTORE_KEY_PASSWORD_FILE =
ConfigBuilder("spark.ssl.kubernetes.resourceStagingServer.keyPasswordFile")
.doc("File containing the key password for the Kubernetes dependency server.")
.stringConf
.createOptional
}
Original file line number Diff line number Diff line change
Expand Up @@ -60,11 +60,12 @@ private[spark] object PemsToKeyStoreConverter {
privateKey,
keyPassword.map(_.toCharArray).orNull,
certificates)
val keyStoreOutputPath = Paths.get(s"keystore-${UUID.randomUUID()}.$resolvedKeyStoreType")
Utils.tryWithResource(new FileOutputStream(keyStoreOutputPath.toFile)) { storeStream =>
val keyStoreDir = Utils.createTempDir("temp-keystores")
val keyStoreFile = new File(keyStoreDir, s"keystore-${UUID.randomUUID()}.$resolvedKeyStoreType")
Utils.tryWithResource(new FileOutputStream(keyStoreFile)) { storeStream =>
keyStore.store(storeStream, keyStorePassword.map(_.toCharArray).orNull)
}
keyStoreOutputPath.toFile
keyStoreFile
}

def convertCertPemToTrustStore(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,19 +16,32 @@
*/
package org.apache.spark.deploy.rest.kubernetes.v2

import java.io.{File, FileInputStream}
import java.util.Properties

import com.fasterxml.jackson.databind.ObjectMapper
import com.fasterxml.jackson.jaxrs.json.JacksonJaxbJsonProvider
import com.fasterxml.jackson.module.scala.DefaultScalaModule
import org.eclipse.jetty.server.{Server, ServerConnector}
import com.google.common.collect.Maps
import org.eclipse.jetty.http.HttpVersion
import org.eclipse.jetty.server.{HttpConfiguration, HttpConnectionFactory, Server, ServerConnector, SslConnectionFactory}
import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder}
import org.eclipse.jetty.util.thread.QueuedThreadPool
import org.eclipse.jetty.util.thread.{QueuedThreadPool, ScheduledExecutorScheduler}
import org.glassfish.jersey.media.multipart.MultiPartFeature
import org.glassfish.jersey.server.ResourceConfig
import org.glassfish.jersey.servlet.ServletContainer
import scala.collection.JavaConverters._

import org.apache.spark.SparkConf
import org.apache.spark.deploy.kubernetes.config._
import org.apache.spark.internal.Logging
import org.apache.spark.internal.config.{ConfigReader, SparkConfigProvider}
import org.apache.spark.util.Utils

private[spark] class ResourceStagingServer(
port: Int,
serviceInstance: ResourceStagingService) {
serviceInstance: ResourceStagingService,
sslOptionsProvider: ResourceStagingServerSslOptionsProvider) extends Logging {

private var jettyServer: Option[Server] = None

Expand All @@ -45,17 +58,72 @@ private[spark] class ResourceStagingServer(
contextHandler.setContextPath("/api/")
contextHandler.addServlet(servletHolder, "/*")
threadPool.setDaemon(true)
val resolvedConnectionFactories = sslOptionsProvider.getSslOptions
.createJettySslContextFactory()
.map(sslFactory => {
val sslConnectionFactory = new SslConnectionFactory(
sslFactory, HttpVersion.HTTP_1_1.asString())
val rawHttpConfiguration = new HttpConfiguration()
rawHttpConfiguration.setSecureScheme("https")
rawHttpConfiguration.setSecurePort(port)
val rawHttpConnectionFactory = new HttpConnectionFactory(rawHttpConfiguration)
Array(sslConnectionFactory, rawHttpConnectionFactory)
}).getOrElse(Array(new HttpConnectionFactory()))
val server = new Server(threadPool)
val connector = new ServerConnector(server)
val connector = new ServerConnector(
server,
null,
// Call this full constructor to set this, which forces daemon threads:
new ScheduledExecutorScheduler("DependencyServer-Executor", true),
null,
-1,
-1,
resolvedConnectionFactories: _*)
connector.setPort(port)
server.addConnector(connector)
server.setHandler(contextHandler)
server.start()
Copy link

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

does this log some kind of "server started on hostname:port" message? Want to have a clear "I'm ready" message to look for in server logs

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't think we have the hostname here but Spark tends to log "Started server listening on port X", and we should do something similar here.

jettyServer = Some(server)
logInfo(s"Resource staging server started on port $port.")
}

def join(): Unit = jettyServer.foreach(_.join())

def stop(): Unit = synchronized {
jettyServer.foreach(_.stop())
jettyServer = None
}
}

object ResourceStagingServer {
def main(args: Array[String]): Unit = {
val sparkConf = new SparkConf(true)
if (args.nonEmpty) {
val propertiesFile = new File(args(0))
if (!propertiesFile.isFile) {
throw new IllegalArgumentException(s"Server properties file given at" +
s" ${propertiesFile.getAbsoluteFile} does not exist or is not a file.")
}
val properties = new Properties
Utils.tryWithResource(new FileInputStream(propertiesFile))(properties.load)
val propertiesMap = Maps.fromProperties(properties)
val configReader = new ConfigReader(new SparkConfigProvider(propertiesMap))
propertiesMap.asScala.keys.foreach { key =>
configReader.get(key).foreach(sparkConf.set(key, _))
}
}
val dependenciesRootDir = Utils.createTempDir(namePrefix = "local-application-dependencies")
val serviceInstance = new ResourceStagingServiceImpl(dependenciesRootDir)
val sslOptionsProvider = new ResourceStagingServerSslOptionsProviderImpl(sparkConf)
val server = new ResourceStagingServer(
port = sparkConf.get(RESOURCE_STAGING_SERVER_PORT),
serviceInstance = serviceInstance,
sslOptionsProvider = sslOptionsProvider)
server.start()
try {
server.join()
} finally {
server.stop()
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,133 @@
/*
* 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.rest.kubernetes.v2

import java.io.File

import com.google.common.base.Charsets
import com.google.common.io.Files

import org.apache.spark.{SecurityManager => SparkSecurityManager, SparkConf, SparkException, SSLOptions}
import org.apache.spark.deploy.kubernetes.config._
import org.apache.spark.deploy.rest.kubernetes.v1.PemsToKeyStoreConverter
import org.apache.spark.internal.Logging

private[spark] trait ResourceStagingServerSslOptionsProvider {
def getSslOptions: SSLOptions
}

private[spark] class ResourceStagingServerSslOptionsProviderImpl(sparkConf: SparkConf)
extends ResourceStagingServerSslOptionsProvider with Logging {
def getSslOptions: SSLOptions = {
val baseSslOptions = new SparkSecurityManager(sparkConf)
.getSSLOptions("kubernetes.resourceStagingServer")
val maybeKeyPem = sparkConf.get(RESOURCE_STAGING_SERVER_KEY_PEM)
val maybeCertPem = sparkConf.get(RESOURCE_STAGING_SERVER_CERT_PEM)
val maybeKeyStorePasswordFile = sparkConf.get(RESOURCE_STAGING_SERVER_KEYSTORE_PASSWORD_FILE)
val maybeKeyPasswordFile = sparkConf.get(RESOURCE_STAGING_SERVER_KEYSTORE_KEY_PASSWORD_FILE)

logSslConfigurations(
baseSslOptions, maybeKeyPem, maybeCertPem, maybeKeyStorePasswordFile, maybeKeyPasswordFile)

requireNandDefined(baseSslOptions.keyStore, maybeKeyPem,
"Shouldn't provide both key PEM and keyStore files for TLS.")
requireNandDefined(baseSslOptions.keyStore, maybeCertPem,
"Shouldn't provide both certificate PEM and keyStore files for TLS.")
requireNandDefined(baseSslOptions.keyStorePassword, maybeKeyStorePasswordFile,
"Shouldn't provide both the keyStore password value and the keyStore password file.")
requireNandDefined(baseSslOptions.keyPassword, maybeKeyPasswordFile,
"Shouldn't provide both the keyStore key password value and the keyStore key password file.")
requireBothOrNeitherDefined(
maybeKeyPem,
maybeCertPem,
"When providing a certificate PEM file, the key PEM file must also be provided.",
"When providing a key PEM file, the certificate PEM file must also be provided.")

val resolvedKeyStorePassword = baseSslOptions.keyStorePassword
.orElse(maybeKeyStorePasswordFile.map { keyStorePasswordFile =>
safeFileToString(keyStorePasswordFile, "KeyStore password file")
})
val resolvedKeyStoreKeyPassword = baseSslOptions.keyPassword
.orElse(maybeKeyPasswordFile.map { keyPasswordFile =>
safeFileToString(keyPasswordFile, "KeyStore key password file")
})
val resolvedKeyStore = baseSslOptions.keyStore
.orElse(maybeKeyPem.map { keyPem =>
val keyPemFile = new File(keyPem)
val certPemFile = new File(maybeCertPem.get)
PemsToKeyStoreConverter.convertPemsToTempKeyStoreFile(
keyPemFile,
certPemFile,
"key",
resolvedKeyStorePassword,
resolvedKeyStoreKeyPassword,
baseSslOptions.keyStoreType)
})
baseSslOptions.copy(
keyStore = resolvedKeyStore,
keyStorePassword = resolvedKeyStorePassword,
keyPassword = resolvedKeyStoreKeyPassword)
}

private def logSslConfigurations(
baseSslOptions: SSLOptions,
maybeKeyPem: Option[String],
maybeCertPem: Option[String],
maybeKeyStorePasswordFile: Option[String],
maybeKeyPasswordFile: Option[String]) = {
logDebug("The following SSL configurations were provided for the resource staging server:")
logDebug(s"KeyStore File: ${baseSslOptions.keyStore.map(_.getAbsolutePath).getOrElse("N/A")}")
logDebug("KeyStore Password: " +
baseSslOptions.keyStorePassword.map(_ => "<present_but_redacted>").getOrElse("N/A"))
logDebug(s"KeyStore Password File: ${maybeKeyStorePasswordFile.getOrElse("N/A")}")
logDebug("Key Password: " +
baseSslOptions.keyPassword.map(_ => "<present_but_redacted>").getOrElse("N/A"))
logDebug(s"Key Password File: ${maybeKeyPasswordFile.getOrElse("N/A")}")
logDebug(s"KeyStore Type: ${baseSslOptions.keyStoreType.getOrElse("N/A")}")
logDebug(s"Key PEM: ${maybeKeyPem.getOrElse("N/A")}")
logDebug(s"Certificate PEM: ${maybeCertPem.getOrElse("N/A")}")
}

private def requireBothOrNeitherDefined(
opt1: Option[_],
opt2: Option[_],
errMessageWhenFirstIsMissing: String,
errMessageWhenSecondIsMissing: String): Unit = {
requireSecondIfFirstIsDefined(opt1, opt2, errMessageWhenSecondIsMissing)
requireSecondIfFirstIsDefined(opt2, opt1, errMessageWhenFirstIsMissing)
}

private def requireSecondIfFirstIsDefined(
opt1: Option[_], opt2: Option[_], errMessageWhenSecondIsMissing: String): Unit = {
opt1.foreach { _ =>
require(opt2.isDefined, errMessageWhenSecondIsMissing)
}
}

private def requireNandDefined(opt1: Option[_], opt2: Option[_], errMessage: String): Unit = {
opt1.foreach { _ => require(opt2.isEmpty, errMessage) }
}

private def safeFileToString(filePath: String, fileType: String): String = {
val file = new File(filePath)
if (!file.isFile) {
throw new SparkException(s"$fileType provided at ${file.getAbsolutePath} does not exist or"
+ s" is not a file.")
}
Files.toString(file, Charsets.UTF_8)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -16,21 +16,50 @@
*/
package org.apache.spark.deploy.rest.kubernetes.v2

import java.io.FileInputStream
import java.security.{KeyStore, SecureRandom}
import javax.net.ssl.{SSLContext, TrustManagerFactory, X509TrustManager}

import com.fasterxml.jackson.databind.ObjectMapper
import com.fasterxml.jackson.module.scala.DefaultScalaModule
import okhttp3.OkHttpClient
import retrofit2.Retrofit
import retrofit2.converter.jackson.JacksonConverterFactory
import retrofit2.converter.scalars.ScalarsConverterFactory

import org.apache.spark.SSLOptions
import org.apache.spark.util.Utils

private[spark] object RetrofitUtils {

private val OBJECT_MAPPER = new ObjectMapper().registerModule(new DefaultScalaModule)
private val SECURE_RANDOM = new SecureRandom()

def createRetrofitClient[T](baseUrl: String, serviceType: Class[T]): T = {
def createRetrofitClient[T](baseUrl: String, serviceType: Class[T], sslOptions: SSLOptions): T = {
val okHttpClientBuilder = new OkHttpClient.Builder()
sslOptions.trustStore.foreach { trustStoreFile =>
require(trustStoreFile.isFile, s"TrustStore provided at ${trustStoreFile.getAbsolutePath}"
+ " does not exist, or is not a file.")
val trustStoreType = sslOptions.trustStoreType.getOrElse(KeyStore.getDefaultType)
val trustStore = KeyStore.getInstance(trustStoreType)
val trustStorePassword = sslOptions.trustStorePassword.map(_.toCharArray).orNull
Utils.tryWithResource(new FileInputStream(trustStoreFile)) {
trustStore.load(_, trustStorePassword)
}
val trustManagerFactory = TrustManagerFactory.getInstance(
TrustManagerFactory.getDefaultAlgorithm)
trustManagerFactory.init(trustStore)
val trustManagers = trustManagerFactory.getTrustManagers
val sslContext = SSLContext.getInstance("TLSv1.2")
sslContext.init(null, trustManagers, SECURE_RANDOM)
okHttpClientBuilder.sslSocketFactory(sslContext.getSocketFactory,
trustManagers(0).asInstanceOf[X509TrustManager])
}
new Retrofit.Builder()
.baseUrl(baseUrl)
.addConverterFactory(ScalarsConverterFactory.create())
.addConverterFactory(JacksonConverterFactory.create(OBJECT_MAPPER))
.client(okHttpClientBuilder.build())
.build()
.create(serviceType)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.spark.deploy.kubernetes.integrationtest.sslutil
package org.apache.spark.deploy.kubernetes

import java.io.{File, FileOutputStream, OutputStreamWriter}
import java.math.BigInteger
Expand Down
Loading