From 48533ff0fed5b28dab70857de15547c622a5b90e Mon Sep 17 00:00:00 2001 From: Kimoon Kim Date: Tue, 18 Jul 2017 10:24:08 -0700 Subject: [PATCH 1/7] Mount a hadoop secret in the driver pod --- .../spark/deploy/kubernetes/config.scala | 2 + .../spark/deploy/kubernetes/constants.scala | 8 +++ ...DriverConfigurationStepsOrchestrator.scala | 2 + .../submitsteps/DriverHadoopTokensStep.scala | 61 +++++++++++++++++++ 4 files changed, 73 insertions(+) create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverHadoopTokensStep.scala diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/config.scala index c6772c1cb5ae4..15e7ce350bdf7 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/config.scala @@ -69,6 +69,8 @@ package object config extends Logging { private[spark] val CLIENT_CERT_FILE_CONF_SUFFIX = "clientCertFile" private[spark] val CA_CERT_FILE_CONF_SUFFIX = "caCertFile" + private[spark] val MOUNTED_HADOOP_SECRET_CONF = "spark.kubernetes.mounted.hadoopSecret" + private[spark] val RESOURCE_STAGING_SERVER_USE_SERVICE_ACCOUNT_CREDENTIALS = ConfigBuilder( s"$APISERVER_AUTH_RESOURCE_STAGING_SERVER_CONF_PREFIX.useServiceAccountCredentials") diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/constants.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/constants.scala index 92f051b2ac298..f39a3fc2f4bb5 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/constants.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/constants.scala @@ -43,6 +43,13 @@ package object constants { s"$DRIVER_CREDENTIALS_SECRETS_BASE_DIR/$DRIVER_CREDENTIALS_OAUTH_TOKEN_SECRET_NAME" private[spark] val DRIVER_CREDENTIALS_SECRET_VOLUME_NAME = "kubernetes-credentials" + // Hadoop credentials secrets for the Spark app. + private[spark] val SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR = "/mnt/secrets/hadoop-credentials" + private[spark] val SPARK_APP_HADOOP_TOKEN_FILE_SECRET_NAME = "hadoop-token-file" + private[spark] val SPARK_APP_HADOOP_TOKEN_FILE_PATH = + s"$SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR/$SPARK_APP_HADOOP_TOKEN_FILE_SECRET_NAME" + private[spark] val SPARK_APP_HADOOP_SECRET_VOLUME_NAME = "hadoop-secret" + // Default and fixed ports private[spark] val SUBMISSION_SERVER_PORT = 7077 private[spark] val DEFAULT_DRIVER_PORT = 7078 @@ -69,6 +76,7 @@ package object constants { private[spark] val ENV_MOUNTED_FILES_DIR = "SPARK_MOUNTED_FILES_DIR" private[spark] val ENV_PYSPARK_FILES = "PYSPARK_FILES" private[spark] val ENV_PYSPARK_PRIMARY = "PYSPARK_PRIMARY" + private[spark] val ENV_HADOOP_TOKEN_FILE_LOCATION = "HADOOP_TOKEN_FILE_LOCATION" // Bootstrapping dependencies with the init-container private[spark] val INIT_CONTAINER_ANNOTATION = "pod.beta.kubernetes.io/init-containers" diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestrator.scala index 82abe55ac6989..8467cae18a619 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestrator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestrator.scala @@ -94,6 +94,7 @@ private[spark] class DriverConfigurationStepsOrchestrator( submissionSparkConf) val kubernetesCredentialsStep = new DriverKubernetesCredentialsStep( submissionSparkConf, kubernetesResourceNamePrefix) + val hadoopTokensStep = new DriverHadoopTokensStep(submissionSparkConf) val pythonStep = mainAppResource match { case PythonMainAppResource(mainPyResource) => Option(new PythonStep(mainPyResource, additionalPythonFiles, filesDownloadPath)) @@ -131,6 +132,7 @@ private[spark] class DriverConfigurationStepsOrchestrator( Seq( initialSubmissionStep, kubernetesCredentialsStep, + hadoopTokensStep, dependencyResolutionStep) ++ initContainerBootstrapStep.toSeq ++ pythonStep.toSeq diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverHadoopTokensStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverHadoopTokensStep.scala new file mode 100644 index 0000000000000..315921546637d --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverHadoopTokensStep.scala @@ -0,0 +1,61 @@ +/* + * 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.kubernetes.submit.submitsteps + +import io.fabric8.kubernetes.api.model.{ContainerBuilder, PodBuilder} + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.kubernetes.constants._ + + +class DriverHadoopTokensStep(submissionSparkConf: SparkConf) extends DriverConfigurationStep { + + private val maybeMountedHadoopSecret = submissionSparkConf.getOption(MOUNTED_HADOOP_SECRET_CONF) + + override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { + val driverPodWithMountedHadoopTokens = maybeMountedHadoopSecret.map { secret => + new PodBuilder(driverSpec.driverPod) + .editOrNewSpec() + .addNewVolume() + .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) + .withNewSecret() + .withSecretName(secret) + .endSecret() + .endVolume() + .endSpec() + .build() + }.getOrElse(driverSpec.driverPod) + val driverContainerWithMountedSecretVolume = maybeMountedHadoopSecret.map { secret => + new ContainerBuilder(driverSpec.driverContainer) + .addNewVolumeMount() + .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) + .withMountPath(SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR) + .endVolumeMount() + .addNewEnv() + .withName(ENV_HADOOP_TOKEN_FILE_LOCATION) + .withValue(SPARK_APP_HADOOP_TOKEN_FILE_PATH) + .endEnv() + .build() + }.getOrElse(driverSpec.driverContainer) + driverSpec.copy( + driverPod = driverPodWithMountedHadoopTokens, + otherKubernetesResources = driverSpec.otherKubernetesResources, + driverSparkConf = driverSpec.driverSparkConf, + driverContainer = driverContainerWithMountedSecretVolume) + } +} From 3c3331aca5228ed5de2e3e4814bde909bcdca00a Mon Sep 17 00:00:00 2001 From: Kimoon Kim Date: Tue, 18 Jul 2017 11:33:39 -0700 Subject: [PATCH 2/7] Fix compile error --- .../submit/DriverConfigurationStepsOrchestrator.scala | 6 +++--- ...opTokensStep.scala => DriverHadoopCredentialsStep.scala} | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/{DriverHadoopTokensStep.scala => DriverHadoopCredentialsStep.scala} (96%) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestrator.scala index 8467cae18a619..06f1547e6a5b1 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestrator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestrator.scala @@ -20,7 +20,7 @@ import org.apache.spark.SparkConf import org.apache.spark.deploy.kubernetes.ConfigurationUtils import org.apache.spark.deploy.kubernetes.config._ import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.submitsteps.{BaseDriverConfigurationStep, DependencyResolutionStep, DriverConfigurationStep, DriverKubernetesCredentialsStep, InitContainerBootstrapStep, PythonStep} +import org.apache.spark.deploy.kubernetes.submit.submitsteps._ import org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer.InitContainerConfigurationStepsOrchestrator import org.apache.spark.launcher.SparkLauncher import org.apache.spark.util.Utils @@ -94,7 +94,7 @@ private[spark] class DriverConfigurationStepsOrchestrator( submissionSparkConf) val kubernetesCredentialsStep = new DriverKubernetesCredentialsStep( submissionSparkConf, kubernetesResourceNamePrefix) - val hadoopTokensStep = new DriverHadoopTokensStep(submissionSparkConf) + val hadoopCredentialsStep = new DriverHadoopCredentialsStep(submissionSparkConf) val pythonStep = mainAppResource match { case PythonMainAppResource(mainPyResource) => Option(new PythonStep(mainPyResource, additionalPythonFiles, filesDownloadPath)) @@ -132,7 +132,7 @@ private[spark] class DriverConfigurationStepsOrchestrator( Seq( initialSubmissionStep, kubernetesCredentialsStep, - hadoopTokensStep, + hadoopCredentialsStep, dependencyResolutionStep) ++ initContainerBootstrapStep.toSeq ++ pythonStep.toSeq diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverHadoopTokensStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverHadoopCredentialsStep.scala similarity index 96% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverHadoopTokensStep.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverHadoopCredentialsStep.scala index 315921546637d..ac4418724d57c 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverHadoopTokensStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverHadoopCredentialsStep.scala @@ -23,7 +23,7 @@ import org.apache.spark.deploy.kubernetes.config._ import org.apache.spark.deploy.kubernetes.constants._ -class DriverHadoopTokensStep(submissionSparkConf: SparkConf) extends DriverConfigurationStep { +class DriverHadoopCredentialsStep(submissionSparkConf: SparkConf) extends DriverConfigurationStep { private val maybeMountedHadoopSecret = submissionSparkConf.getOption(MOUNTED_HADOOP_SECRET_CONF) From f2a4033c77ab8ff6cc86dc2b339f42538b94bb64 Mon Sep 17 00:00:00 2001 From: Kimoon Kim Date: Tue, 18 Jul 2017 15:21:20 -0700 Subject: [PATCH 3/7] Mount a hadoop secret in the executor pod --- .../kubernetes/submit/HadoopSecretUtil.scala | 55 +++++++++++++++++++ .../DriverHadoopCredentialsStep.scala | 40 +++----------- .../KubernetesClusterSchedulerBackend.scala | 13 ++++- 3 files changed, 74 insertions(+), 34 deletions(-) create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/HadoopSecretUtil.scala diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/HadoopSecretUtil.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/HadoopSecretUtil.scala new file mode 100644 index 0000000000000..b7a9e9e3fd6cd --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/HadoopSecretUtil.scala @@ -0,0 +1,55 @@ +/* + * 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.kubernetes.submit + +import io.fabric8.kubernetes.api.model.{Container, ContainerBuilder, Pod, PodBuilder} + +import org.apache.spark.deploy.kubernetes.constants._ + +object HadoopSecretUtil { + + def configurePod(secretNameOption: Option[String], pod: Pod) : Pod = { + secretNameOption.map { secret => + new PodBuilder(pod) + .editOrNewSpec() + .addNewVolume() + .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) + .withNewSecret() + .withSecretName(secret) + .endSecret() + .endVolume() + .endSpec() + .build() + }.getOrElse(pod) + } + + def configureContainer(secretNameOption: Option[String], + containerSpec: Container) : Container = { + secretNameOption.map { secret => + new ContainerBuilder(containerSpec) + .addNewVolumeMount() + .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) + .withMountPath(SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR) + .endVolumeMount() + .addNewEnv() + .withName(ENV_HADOOP_TOKEN_FILE_LOCATION) + .withValue(SPARK_APP_HADOOP_TOKEN_FILE_PATH) + .endEnv() + .build() + }.getOrElse(containerSpec) + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverHadoopCredentialsStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverHadoopCredentialsStep.scala index ac4418724d57c..fbd89a7280260 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverHadoopCredentialsStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverHadoopCredentialsStep.scala @@ -16,46 +16,24 @@ */ package org.apache.spark.deploy.kubernetes.submit.submitsteps -import io.fabric8.kubernetes.api.model.{ContainerBuilder, PodBuilder} - import org.apache.spark.SparkConf import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ - +import org.apache.spark.deploy.kubernetes.submit.HadoopSecretUtil -class DriverHadoopCredentialsStep(submissionSparkConf: SparkConf) extends DriverConfigurationStep { +private[spark] class DriverHadoopCredentialsStep(submissionSparkConf: SparkConf) + extends DriverConfigurationStep { private val maybeMountedHadoopSecret = submissionSparkConf.getOption(MOUNTED_HADOOP_SECRET_CONF) override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { - val driverPodWithMountedHadoopTokens = maybeMountedHadoopSecret.map { secret => - new PodBuilder(driverSpec.driverPod) - .editOrNewSpec() - .addNewVolume() - .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) - .withNewSecret() - .withSecretName(secret) - .endSecret() - .endVolume() - .endSpec() - .build() - }.getOrElse(driverSpec.driverPod) - val driverContainerWithMountedSecretVolume = maybeMountedHadoopSecret.map { secret => - new ContainerBuilder(driverSpec.driverContainer) - .addNewVolumeMount() - .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) - .withMountPath(SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR) - .endVolumeMount() - .addNewEnv() - .withName(ENV_HADOOP_TOKEN_FILE_LOCATION) - .withValue(SPARK_APP_HADOOP_TOKEN_FILE_PATH) - .endEnv() - .build() - }.getOrElse(driverSpec.driverContainer) + val podWithMountedHadoopToken = HadoopSecretUtil.configurePod(maybeMountedHadoopSecret, + driverSpec.driverPod) + val containerWithMountedHadoopToken = HadoopSecretUtil.configureContainer( + maybeMountedHadoopSecret, driverSpec.driverContainer) driverSpec.copy( - driverPod = driverPodWithMountedHadoopTokens, + driverPod = podWithMountedHadoopToken, otherKubernetesResources = driverSpec.otherKubernetesResources, driverSparkConf = driverSpec.driverSparkConf, - driverContainer = driverContainerWithMountedSecretVolume) + driverContainer = containerWithMountedHadoopToken) } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala index a50a9c8bb9c3b..17eff4ee395f3 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala @@ -37,7 +37,7 @@ import org.apache.spark.{SparkContext, SparkEnv, SparkException} import org.apache.spark.deploy.kubernetes.{ConfigurationUtils, InitContainerResourceStagingServerSecretPlugin, PodWithDetachedInitContainer, SparkPodInitContainerBootstrap} import org.apache.spark.deploy.kubernetes.config._ import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.InitContainerUtil +import org.apache.spark.deploy.kubernetes.submit.{HadoopSecretUtil, InitContainerUtil} import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.network.shuffle.kubernetes.KubernetesExternalShuffleClient import org.apache.spark.rpc.{RpcAddress, RpcCallContext, RpcEndpointAddress, RpcEnv} @@ -130,6 +130,8 @@ private[spark] class KubernetesClusterSchedulerBackend( private implicit val requestExecutorContext = ExecutionContext.fromExecutorService( ThreadUtils.newDaemonCachedThreadPool("kubernetes-executor-requests")) + private val maybeMountedHadoopSecret = conf.getOption(MOUNTED_HADOOP_SECRET_CONF) + private val driverPod = try { kubernetesClient.pods().inNamespace(kubernetesNamespace). withName(kubernetesDriverPodName).get() @@ -582,9 +584,14 @@ private[spark] class KubernetesClusterSchedulerBackend( val executorPodWithNodeAffinity = addNodeAffinityAnnotationIfUseful( executorPodWithInitContainer, nodeToLocalTaskCount) - val resolvedExecutorPod = new PodBuilder(executorPodWithNodeAffinity) + val executorPodWithMountedHadoopToken = HadoopSecretUtil.configurePod(maybeMountedHadoopSecret, + executorPodWithNodeAffinity) + val containerWithMountedHadoopToken = HadoopSecretUtil.configureContainer( + maybeMountedHadoopSecret, initBootstrappedExecutorContainer) + + val resolvedExecutorPod = new PodBuilder(executorPodWithMountedHadoopToken) .editSpec() - .addToContainers(initBootstrappedExecutorContainer) + .addToContainers(containerWithMountedHadoopToken) .endSpec() .build() try { From aa04b04d8057a69cd187acdafdec794540caa371 Mon Sep 17 00:00:00 2001 From: Kimoon Kim Date: Thu, 20 Jul 2017 10:22:35 -0700 Subject: [PATCH 4/7] Document the experimental config option --- .../org/apache/spark/deploy/kubernetes/config.scala | 12 +++++++++++- .../submitsteps/DriverHadoopCredentialsStep.scala | 2 +- .../KubernetesClusterSchedulerBackend.scala | 2 +- 3 files changed, 13 insertions(+), 3 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/config.scala index 15e7ce350bdf7..e746410772f62 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/config.scala @@ -69,7 +69,17 @@ package object config extends Logging { private[spark] val CLIENT_CERT_FILE_CONF_SUFFIX = "clientCertFile" private[spark] val CA_CERT_FILE_CONF_SUFFIX = "caCertFile" - private[spark] val MOUNTED_HADOOP_SECRET_CONF = "spark.kubernetes.mounted.hadoopSecret" + // TODO: This option is intended to be used for internal prototype only until the submission + // client automatically creates the secret file. Remove this option afterward + // unless other use is found. + private[spark] val MOUNTED_HADOOP_SECRET_CONF = + ConfigBuilder("spark.kubernetes.mounted.hadoopSecret") + .doc("Use a Kubernetes secret containing Hadoop tokens such as an HDFS delegation token." + + " The secret should have an entry named 'hadoop-token-file' under the data section," + + " which contains binary dumps of Hadoop tokens.") + .internal() + .stringConf + .createOptional private[spark] val RESOURCE_STAGING_SERVER_USE_SERVICE_ACCOUNT_CREDENTIALS = ConfigBuilder( diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverHadoopCredentialsStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverHadoopCredentialsStep.scala index fbd89a7280260..7685e194c5f84 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverHadoopCredentialsStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverHadoopCredentialsStep.scala @@ -23,7 +23,7 @@ import org.apache.spark.deploy.kubernetes.submit.HadoopSecretUtil private[spark] class DriverHadoopCredentialsStep(submissionSparkConf: SparkConf) extends DriverConfigurationStep { - private val maybeMountedHadoopSecret = submissionSparkConf.getOption(MOUNTED_HADOOP_SECRET_CONF) + private val maybeMountedHadoopSecret = submissionSparkConf.get(MOUNTED_HADOOP_SECRET_CONF) override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { val podWithMountedHadoopToken = HadoopSecretUtil.configurePod(maybeMountedHadoopSecret, diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala index 17eff4ee395f3..f6aecc402c0fe 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala @@ -130,7 +130,7 @@ private[spark] class KubernetesClusterSchedulerBackend( private implicit val requestExecutorContext = ExecutionContext.fromExecutorService( ThreadUtils.newDaemonCachedThreadPool("kubernetes-executor-requests")) - private val maybeMountedHadoopSecret = conf.getOption(MOUNTED_HADOOP_SECRET_CONF) + private val maybeMountedHadoopSecret = conf.get(MOUNTED_HADOOP_SECRET_CONF) private val driverPod = try { kubernetesClient.pods().inNamespace(kubernetesNamespace). From 0141c0a54d523d12587491aae83e641da78cb0b0 Mon Sep 17 00:00:00 2001 From: Kimoon Kim Date: Thu, 20 Jul 2017 12:22:04 -0700 Subject: [PATCH 5/7] Address review comments and fix styles --- .../submit/submitsteps/DriverHadoopCredentialsStep.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverHadoopCredentialsStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverHadoopCredentialsStep.scala index 7685e194c5f84..88f2e7b3836c8 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverHadoopCredentialsStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverHadoopCredentialsStep.scala @@ -29,11 +29,10 @@ private[spark] class DriverHadoopCredentialsStep(submissionSparkConf: SparkConf) val podWithMountedHadoopToken = HadoopSecretUtil.configurePod(maybeMountedHadoopSecret, driverSpec.driverPod) val containerWithMountedHadoopToken = HadoopSecretUtil.configureContainer( - maybeMountedHadoopSecret, driverSpec.driverContainer) + maybeMountedHadoopSecret, + driverSpec.driverContainer) driverSpec.copy( driverPod = podWithMountedHadoopToken, - otherKubernetesResources = driverSpec.otherKubernetesResources, - driverSparkConf = driverSpec.driverSparkConf, driverContainer = containerWithMountedHadoopToken) } } From 569f73c02da393ef6a264400d7dd750a86c52d33 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Wed, 27 Sep 2017 20:39:19 -0400 Subject: [PATCH 6/7] Secure HDFS Support (#414) * Initial architecture design for HDFS support * Minor styling * Added proper logic for mounting ConfigMaps * styling * modified otherKubernetesResource logic * fixed Integration tests and modified HADOOP_CONF_DIR variable to be FILE_DIR for Volume mount * setting HADOOP_CONF_DIR env variables * Included integration tests for Stage 1 * Initial Kerberos support * initial Stage 2 architecture using deprecated 2.1 methods * Added current, BROKEN, integration test environment for review * working hadoop cluster * Using locks and monitors to ensure proper configs for setting up kerberized cluster in integration tests * working Stage 2 * documentation * Integration Stages 1,2 and 3 * further testing work * fixing imports * Stage 3 Integration tests pass * uncommented SparkDockerBuilder * testing fix * handled comments and increased test hardening * Solve failing integration test problem and lower TIMEOUT time * modify security.authoization * Modifying HADOOP_CONF flags * Refactored tests and included modifications to pass all tests regardless of environment * Adding unit test and one more integration test * completed unit tests w/o UGI mocking * cleanup and various small fixes * added back sparkdockerbuilder images * address initial comments and scalastyle issues * addresses comments from PR * mocking hadoopUGI * Fix executor env to include simple authn * Fix a bug in executor env handling * Fix a bug in how the driver sets simple authn * handling Pr comments --- .../org/apache/spark/deploy/SparkSubmit.scala | 2 +- docs/running-on-kubernetes.md | 55 ++++ .../kubernetes/HadoopConfBootstrap.scala | 86 ++++++ .../deploy/kubernetes/HadoopUGIUtil.scala | 78 ++++++ .../KerberosTokenConfBootstrap.scala | 76 ++++++ .../kubernetes/PodWithMainContainer.scala | 28 ++ .../spark/deploy/kubernetes/config.scala | 43 +++ .../spark/deploy/kubernetes/constants.scala | 24 +- .../deploy/kubernetes/submit/Client.scala | 16 +- ...DriverConfigurationStepsOrchestrator.scala | 13 +- .../kubernetes/submit/HadoopSecretUtil.scala | 55 ---- .../DriverHadoopCredentialsStep.scala | 38 --- .../HadoopConfigBootstrapStep.scala | 68 +++++ .../hadoopsteps/HadoopConfMounterStep.scala | 57 ++++ .../hadoopsteps/HadoopConfigSpec.scala | 42 +++ .../hadoopsteps/HadoopConfigurationStep.scala | 25 ++ .../HadoopKerberosKeytabResolverStep.scala | 134 +++++++++ .../HadoopKerberosSecretResolverStep.scala | 56 ++++ .../hadoopsteps/HadoopStepsOrchestrator.scala | 104 +++++++ .../kubernetes/KubernetesClusterManager.scala | 46 +++- .../KubernetesClusterSchedulerBackend.scala | 57 ++-- ...sourceStagingServerSecretPluginSuite.scala | 3 +- .../kubernetes/submit/ClientSuite.scala | 22 +- ...rConfigurationStepsOrchestratorSuite.scala | 35 ++- .../submit/HadoopConfBootstrapSuite.scala | 89 ++++++ .../KerberosTokenConfBootstrapSuite.scala | 60 ++++ .../HadoopConfigBootstrapStepSuite.scala | 90 ++++++ ... => InitContainerBootstrapStepSuite.scala} | 2 +- .../HadoopConfMounterStepSuite.scala | 102 +++++++ ...adoopKerberosKeytabResolverStepSuite.scala | 146 ++++++++++ ...adoopKerberosSecretResolverStepSuite.scala | 63 +++++ .../HadoopStepsOrchestratorSuite.scala | 85 ++++++ ...eInitContainerConfigurationStepSuite.scala | 3 +- ...ittedResourcesInitContainerStepSuite.scala | 3 +- .../src/main/docker/driver-py/Dockerfile | 1 + .../src/main/docker/driver/Dockerfile | 1 + .../src/main/docker/executor-py/Dockerfile | 1 + .../src/main/docker/executor/Dockerfile | 1 + .../src/main/docker/kerberos-test/Dockerfile | 27 ++ .../integrationtest/jobs/HDFSTest.scala | 48 ++++ .../data-populator-deployment.yml | 30 ++ .../kerberos-yml/data-populator-service.yml | 17 ++ .../kerberos-yml/dn1-deployment.yml | 30 ++ .../kerberos-yml/dn1-service.yml | 17 ++ .../kerberos-yml/kerberos-deployment.yml | 30 ++ .../kerberos-yml/kerberos-service.yml | 17 ++ .../kerberos-yml/kerberos-test.yml | 25 ++ .../kerberos-yml/kerberos-test2.yml | 25 ++ .../kerberos-yml/nn-deployment.yml | 35 +++ .../kerberos-yml/nn-hadoop.yml | 12 + .../kerberos-yml/nn-service.yml | 17 ++ .../kerberos-yml/server-keytab.yml | 12 + .../kerberos-yml/test-env.sh | 27 ++ .../kubernetes/integration-tests/pom.xml | 34 +++ .../KerberizedHadoopClusterLauncher.scala | 57 ++++ .../KerberosTestPodLauncher.scala | 121 +++++++++ .../integrationtest/KubernetesSuite.scala | 110 +++++++- .../KubernetesTestComponents.scala | 16 +- .../docker/SparkDockerImageBuilder.scala | 4 +- .../kerberos/KerberosCMWatcherCache.scala | 104 +++++++ .../kerberos/KerberosDeployment.scala | 24 ++ .../kerberos/KerberosDriverWatcherCache.scala | 99 +++++++ .../kerberos/KerberosPVWatcherCache.scala | 184 +++++++++++++ .../kerberos/KerberosPodWatcherCache.scala | 256 ++++++++++++++++++ .../kerberos/KerberosStorage.scala | 23 ++ .../kerberos/KerberosUtils.scala | 154 +++++++++++ .../test-data/hadoop-conf/core-site.xml | 38 +++ .../test-data/hadoop-conf/hdfs-site.xml | 157 +++++++++++ .../test-data/hadoop-conf/krb5.conf | 25 ++ .../test-data/hadoop-conf/yarn-site.xml | 26 ++ .../simple-hadoop-conf/core-site.xml | 24 ++ .../simple-hadoop-conf/hdfs-site.xml | 24 ++ 72 files changed, 3503 insertions(+), 156 deletions(-) create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/HadoopConfBootstrap.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/HadoopUGIUtil.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KerberosTokenConfBootstrap.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/PodWithMainContainer.scala delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/HadoopSecretUtil.scala delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverHadoopCredentialsStep.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/HadoopConfigBootstrapStep.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopConfMounterStep.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopConfigSpec.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopConfigurationStep.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStep.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosSecretResolverStep.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopStepsOrchestrator.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/HadoopConfBootstrapSuite.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/KerberosTokenConfBootstrapSuite.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/HadoopConfigBootstrapStepSuite.scala rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/{initContainerBootstrapStepSuite.scala => InitContainerBootstrapStepSuite.scala} (99%) create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopConfMounterStepSuite.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStepSuite.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosSecretResolverStepSuite.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopStepsOrchestratorSuite.scala create mode 100644 resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/kerberos-test/Dockerfile create mode 100644 resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/HDFSTest.scala create mode 100644 resource-managers/kubernetes/integration-tests/kerberos-yml/data-populator-deployment.yml create mode 100644 resource-managers/kubernetes/integration-tests/kerberos-yml/data-populator-service.yml create mode 100644 resource-managers/kubernetes/integration-tests/kerberos-yml/dn1-deployment.yml create mode 100644 resource-managers/kubernetes/integration-tests/kerberos-yml/dn1-service.yml create mode 100644 resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-deployment.yml create mode 100644 resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-service.yml create mode 100644 resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-test.yml create mode 100644 resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-test2.yml create mode 100644 resource-managers/kubernetes/integration-tests/kerberos-yml/nn-deployment.yml create mode 100644 resource-managers/kubernetes/integration-tests/kerberos-yml/nn-hadoop.yml create mode 100644 resource-managers/kubernetes/integration-tests/kerberos-yml/nn-service.yml create mode 100644 resource-managers/kubernetes/integration-tests/kerberos-yml/server-keytab.yml create mode 100644 resource-managers/kubernetes/integration-tests/kerberos-yml/test-env.sh create mode 100644 resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberizedHadoopClusterLauncher.scala create mode 100644 resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberosTestPodLauncher.scala create mode 100644 resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosCMWatcherCache.scala create mode 100644 resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosDeployment.scala create mode 100644 resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosDriverWatcherCache.scala create mode 100644 resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosPVWatcherCache.scala create mode 100644 resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosPodWatcherCache.scala create mode 100644 resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosStorage.scala create mode 100644 resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosUtils.scala create mode 100644 resource-managers/kubernetes/integration-tests/test-data/hadoop-conf/core-site.xml create mode 100644 resource-managers/kubernetes/integration-tests/test-data/hadoop-conf/hdfs-site.xml create mode 100644 resource-managers/kubernetes/integration-tests/test-data/hadoop-conf/krb5.conf create mode 100644 resource-managers/kubernetes/integration-tests/test-data/hadoop-conf/yarn-site.xml create mode 100644 resource-managers/kubernetes/integration-tests/test-data/simple-hadoop-conf/core-site.xml create mode 100644 resource-managers/kubernetes/integration-tests/test-data/simple-hadoop-conf/hdfs-site.xml diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index ed46adcbe9dfb..c04938fdf26da 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -575,7 +575,7 @@ object SparkSubmit extends CommandLineUtils { } // assure a keytab is available from any place in a JVM - if (clusterManager == YARN || clusterManager == LOCAL) { + if (clusterManager == YARN || clusterManager == KUBERNETES || clusterManager == LOCAL) { if (args.principal != null) { require(args.keytab != null, "Keytab must be specified when principal is specified") if (!new File(args.keytab).exists()) { diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index 5346e54e62aec..f66fa7935b15c 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -783,6 +783,61 @@ from the other deployment modes. See the [configuration page](configuration.html + spark.kubernetes.kerberos.enabled + false + + Specify whether your job requires a Kerberos Authentication to access HDFS. By default, we + will assume that you will not require secure HDFS access. + + + + spark.kubernetes.kerberos.keytab + (none) + + Assuming you have set spark.kubernetes.kerberos.enabled to be true. This will let you specify + the location of your Kerberos keytab to be used in order to access Secure HDFS. This is optional as you + may login by running kinit before running the spark-submit, and the submission client + will look within your local TGT cache to resolve this. + + + + spark.kubernetes.kerberos.principal + (none) + + Assuming you have set spark.kubernetes.kerberos.enabled to be true. This will let you specify + your Kerberos principal that you wish to use to access Secure HDFS. This is optional as you + may login by running kinit before running the spark-submit, and the submission client + will look within your local TGT cache to resolve this. + + + + spark.kubernetes.kerberos.rewewer.principal + (none) + + Assuming you have set spark.kubernetes.kerberos.enabled to be true. This will let you specify + the principal that you wish to use to handle renewing of Delegation Tokens. This is optional as you + we will set the principal to be the job users principal by default. + + + + spark.kubernetes.kerberos.tokensecret.name + (none) + + Assuming you have set spark.kubernetes.kerberos.enabled to be true. This will let you specify + the name of the secret where your existing delegation token data is stored. You must also specify the + item key spark.kubernetes.kerberos.tokensecret.itemkey where your data is stored on the secret. + This is optional in the case that you want to use pre-existing secret, otherwise a new secret will be automatically + created. + + + + spark.kubernetes.kerberos.tokensecret.itemkey + spark.kubernetes.kerberos.dt.label + + Assuming you have set spark.kubernetes.kerberos.enabled to be true. This will let you specify + the data item key name within the pre-specified secret where the data of your existing delegation token data is stored. + We have a default value of spark.kubernetes.kerberos.tokensecret.itemkey should you not include it. But + you should always include this if you are proposing a pre-existing secret contain the delegation token data. spark.executorEnv.[EnvironmentVariableName] (none) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/HadoopConfBootstrap.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/HadoopConfBootstrap.scala new file mode 100644 index 0000000000000..17efe19b8f206 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/HadoopConfBootstrap.scala @@ -0,0 +1,86 @@ +/* + * 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.kubernetes + +import java.io.File + +import scala.collection.JavaConverters._ + +import io.fabric8.kubernetes.api.model.{ContainerBuilder, KeyToPathBuilder, PodBuilder} + +import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.internal.Logging + +/** + * This is separated out from the HadoopConf steps API because this component can be reused to + * set up the Hadoop Configuration for executors as well. + */ +private[spark] trait HadoopConfBootstrap { + /** + * Bootstraps a main container with the ConfigMaps containing Hadoop config files + * mounted as volumes and an ENV variable pointing to the mounted file. + */ + def bootstrapMainContainerAndVolumes( + originalPodWithMainContainer: PodWithMainContainer) + : PodWithMainContainer +} + +private[spark] class HadoopConfBootstrapImpl( + hadoopConfConfigMapName: String, + hadoopConfigFiles: Seq[File], + hadoopUGI: HadoopUGIUtil) extends HadoopConfBootstrap with Logging{ + + override def bootstrapMainContainerAndVolumes( + originalPodWithMainContainer: PodWithMainContainer) + : PodWithMainContainer = { + logInfo("HADOOP_CONF_DIR defined. Mounting HDFS specific .xml files") + val keyPaths = hadoopConfigFiles.map(file => + new KeyToPathBuilder() + .withKey(file.toPath.getFileName.toString) + .withPath(file.toPath.getFileName.toString) + .build()).toList + val hadoopSupportedPod = new PodBuilder(originalPodWithMainContainer.pod) + .editSpec() + .addNewVolume() + .withName(HADOOP_FILE_VOLUME) + .withNewConfigMap() + .withName(hadoopConfConfigMapName) + .withItems(keyPaths.asJava) + .endConfigMap() + .endVolume() + .endSpec() + .build() + val mainContainerWithMountedHadoopConf = new ContainerBuilder( + originalPodWithMainContainer.mainContainer) + .addNewVolumeMount() + .withName(HADOOP_FILE_VOLUME) + .withMountPath(HADOOP_CONF_DIR_PATH) + .endVolumeMount() + .addNewEnv() + .withName(ENV_HADOOP_CONF_DIR) + .withValue(HADOOP_CONF_DIR_PATH) + .endEnv() + .addNewEnv() + .withName(ENV_SPARK_USER) + .withValue(hadoopUGI.getShortName) + .endEnv() + .build() + originalPodWithMainContainer.copy( + pod = hadoopSupportedPod, + mainContainer = mainContainerWithMountedHadoopConf) + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/HadoopUGIUtil.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/HadoopUGIUtil.scala new file mode 100644 index 0000000000000..8a004c0e5414f --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/HadoopUGIUtil.scala @@ -0,0 +1,78 @@ +/* + * 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.kubernetes + +import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataInputStream, DataOutputStream} + +import scala.util.Try + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.FileSystem +import org.apache.hadoop.security.{Credentials, UserGroupInformation} +import org.apache.hadoop.security.token.{Token, TokenIdentifier} +import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier + + +// Function of this class is merely for mocking reasons +private[spark] class HadoopUGIUtil{ + def getCurrentUser: UserGroupInformation = UserGroupInformation.getCurrentUser + + def getShortName: String = getCurrentUser.getShortUserName + + def isSecurityEnabled: Boolean = UserGroupInformation.isSecurityEnabled + + def loginUserFromKeytabAndReturnUGI(principal: String, keytab: String): UserGroupInformation = + UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keytab) + + def dfsAddDelegationToken(hadoopConf: Configuration, renewer: String, creds: Credentials) + : Iterable[Token[_ <: TokenIdentifier]] = + FileSystem.get(hadoopConf).addDelegationTokens(renewer, creds) + + def getCurrentTime: Long = System.currentTimeMillis() + + // Functions that should be in Core with Rebase to 2.3 + @deprecated("Moved to core in 2.2", "2.2") + def getTokenRenewalInterval( + renewedTokens: Iterable[Token[_ <: TokenIdentifier]], + hadoopConf: Configuration): Option[Long] = { + val renewIntervals = renewedTokens.filter { + _.decodeIdentifier().isInstanceOf[AbstractDelegationTokenIdentifier]} + .flatMap { token => + Try { + val newExpiration = token.renew(hadoopConf) + val identifier = token.decodeIdentifier().asInstanceOf[AbstractDelegationTokenIdentifier] + val interval = newExpiration - identifier.getIssueDate + interval + }.toOption} + if (renewIntervals.isEmpty) None else Some(renewIntervals.min) + } + + @deprecated("Moved to core in 2.2", "2.2") + def serialize(creds: Credentials): Array[Byte] = { + val byteStream = new ByteArrayOutputStream + val dataStream = new DataOutputStream(byteStream) + creds.writeTokenStorageToStream(dataStream) + byteStream.toByteArray + } + + @deprecated("Moved to core in 2.2", "2.2") + def deserialize(tokenBytes: Array[Byte]): Credentials = { + val creds = new Credentials() + creds.readTokenStorageStream(new DataInputStream(new ByteArrayInputStream(tokenBytes))) + creds + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KerberosTokenConfBootstrap.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KerberosTokenConfBootstrap.scala new file mode 100644 index 0000000000000..6225142fa4b52 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KerberosTokenConfBootstrap.scala @@ -0,0 +1,76 @@ +/* + * 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.kubernetes + +import io.fabric8.kubernetes.api.model.{ContainerBuilder, PodBuilder} + +import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.internal.Logging + + + /** + * This is separated out from the HadoopConf steps API because this component can be reused to + * mounted the DT secret for executors as well. + */ +private[spark] trait KerberosTokenConfBootstrap { + // Bootstraps a main container with the Secret mounted as volumes and an ENV variable + // pointing to the mounted file containing the DT for Secure HDFS interaction + def bootstrapMainContainerAndVolumes( + originalPodWithMainContainer: PodWithMainContainer) + : PodWithMainContainer +} + +private[spark] class KerberosTokenConfBootstrapImpl( + secretName: String, + secretItemKey: String, + userName: String) extends KerberosTokenConfBootstrap with Logging{ + + + override def bootstrapMainContainerAndVolumes( + originalPodWithMainContainer: PodWithMainContainer) + : PodWithMainContainer = { + logInfo("Mounting HDFS DT from Secret for Secure HDFS") + val dtMountedPod = new PodBuilder(originalPodWithMainContainer.pod) + .editOrNewSpec() + .addNewVolume() + .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) + .withNewSecret() + .withSecretName(secretName) + .endSecret() + .endVolume() + .endSpec() + .build() + val mainContainerWithMountedKerberos = new ContainerBuilder( + originalPodWithMainContainer.mainContainer) + .addNewVolumeMount() + .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) + .withMountPath(SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR) + .endVolumeMount() + .addNewEnv() + .withName(ENV_HADOOP_TOKEN_FILE_LOCATION) + .withValue(s"$SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR/$secretItemKey") + .endEnv() + .addNewEnv() + .withName(ENV_SPARK_USER) + .withValue(userName) + .endEnv() + .build() + originalPodWithMainContainer.copy( + pod = dtMountedPod, + mainContainer = mainContainerWithMountedKerberos) + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/PodWithMainContainer.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/PodWithMainContainer.scala new file mode 100644 index 0000000000000..664eb41bd68d5 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/PodWithMainContainer.scala @@ -0,0 +1,28 @@ +/* + * 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.kubernetes + +import io.fabric8.kubernetes.api.model.{Container, Pod} + + /** + * The purpose of this case class is so that we can package together + * the driver pod with its container so we can bootstrap and modify + * the class instead of each component seperately + */ +private[spark] case class PodWithMainContainer( + pod: Pod, + mainContainer: Container) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/config.scala index 5bd61328f3a0b..442bdb01b979b 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/config.scala @@ -542,6 +542,49 @@ package object config extends Logging { private[spark] val KUBERNETES_NODE_SELECTOR_PREFIX = "spark.kubernetes.node.selector." + private[spark] val KUBERNETES_KERBEROS_SUPPORT = + ConfigBuilder("spark.kubernetes.kerberos.enabled") + .doc("Specify whether your job is a job that will require a Delegation Token to access HDFS") + .booleanConf + .createWithDefault(false) + + private[spark] val KUBERNETES_KERBEROS_KEYTAB = + ConfigBuilder("spark.kubernetes.kerberos.keytab") + .doc("Specify the location of keytab" + + " for Kerberos in order to access Secure HDFS") + .stringConf + .createOptional + + private[spark] val KUBERNETES_KERBEROS_PRINCIPAL = + ConfigBuilder("spark.kubernetes.kerberos.principal") + .doc("Specify the principal" + + " for Kerberos in order to access Secure HDFS") + .stringConf + .createOptional + + private[spark] val KUBERNETES_KERBEROS_RENEWER_PRINCIPAL = + ConfigBuilder("spark.kubernetes.kerberos.rewnewer.principal") + .doc("Specify the principal" + + " you wish to renew and retrieve your Kerberos values with") + .stringConf + .createOptional + + private[spark] val KUBERNETES_KERBEROS_DT_SECRET_NAME = + ConfigBuilder("spark.kubernetes.kerberos.tokensecret.name") + .doc("Specify the name of the secret where " + + " your existing delegation token is stored. This removes the need" + + " for the job user to provide any keytab for launching a job") + .stringConf + .createOptional + + private[spark] val KUBERNETES_KERBEROS_DT_SECRET_ITEM_KEY = + ConfigBuilder("spark.kubernetes.kerberos.tokensecret.itemkey") + .doc("Specify the item key of the data where " + + " your existing delegation token is stored. This removes the need" + + " for the job user to provide any keytab for launching a job") + .stringConf + .createOptional + private[spark] def resolveK8sMaster(rawMasterString: String): String = { if (!rawMasterString.startsWith("k8s://")) { throw new IllegalArgumentException("Master URL should start with k8s:// in Kubernetes mode.") diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/constants.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/constants.scala index 3faf387aadfc0..dfb4e0838113f 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/constants.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/constants.scala @@ -45,9 +45,6 @@ package object constants { // Hadoop credentials secrets for the Spark app. private[spark] val SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR = "/mnt/secrets/hadoop-credentials" - private[spark] val SPARK_APP_HADOOP_TOKEN_FILE_SECRET_NAME = "hadoop-token-file" - private[spark] val SPARK_APP_HADOOP_TOKEN_FILE_PATH = - s"$SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR/$SPARK_APP_HADOOP_TOKEN_FILE_SECRET_NAME" private[spark] val SPARK_APP_HADOOP_SECRET_VOLUME_NAME = "hadoop-secret" // Default and fixed ports @@ -79,6 +76,7 @@ package object constants { private[spark] val ENV_JAVA_OPT_PREFIX = "SPARK_JAVA_OPT_" private[spark] val ENV_MOUNTED_FILES_FROM_SECRET_DIR = "SPARK_MOUNTED_FILES_FROM_SECRET_DIR" private[spark] val ENV_HADOOP_TOKEN_FILE_LOCATION = "HADOOP_TOKEN_FILE_LOCATION" + private[spark] val ENV_SPARK_USER = "SPARK_USER" // Bootstrapping dependencies with the init-container private[spark] val INIT_CONTAINER_ANNOTATION = "pod.beta.kubernetes.io/init-containers" @@ -101,6 +99,26 @@ package object constants { private[spark] val DEFAULT_SHUFFLE_MOUNT_NAME = "shuffle" private[spark] val INIT_CONTAINER_SECRET_VOLUME_NAME = "spark-init-secret" + // Hadoop Configuration + private[spark] val HADOOP_FILE_VOLUME = "hadoop-properties" + private[spark] val HADOOP_CONF_DIR_PATH = "/etc/hadoop/conf" + private[spark] val ENV_HADOOP_CONF_DIR = "HADOOP_CONF_DIR" + private[spark] val HADOOP_CONF_DIR_LOC = "spark.kubernetes.hadoop.conf.dir" + private[spark] val HADOOP_CONFIG_MAP_SPARK_CONF_NAME = + "spark.kubernetes.hadoop.executor.hadoopconfigmapname" + + // Kerberos Configuration + private[spark] val HADOOP_KERBEROS_SECRET_NAME = + "spark.kubernetes.kerberos.dt" + private[spark] val HADOOP_KERBEROS_CONF_SECRET = + "spark.kubernetes.kerberos.secretname" + private[spark] val HADOOP_KERBEROS_CONF_ITEM_KEY = + "spark.kubernetes.kerberos.itemkeyname" + private[spark] val KERBEROS_SECRET_LABEL_PREFIX = + "hadoop-tokens" + private[spark] val SPARK_HADOOP_PREFIX = "spark.hadoop." + private[spark] val HADOOP_SECURITY_AUTHENTICATION = + SPARK_HADOOP_PREFIX + "hadoop.security.authentication" // Bootstrapping dependencies via a secret private[spark] val MOUNTED_SMALL_FILES_SECRET_MOUNT_PATH = "/etc/spark-submitted-files" diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala index 1901f1198a84f..15f6c9d7ce0d8 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala @@ -79,6 +79,9 @@ private[spark] class Client( private val driverJavaOptions = submissionSparkConf.get( org.apache.spark.internal.config.DRIVER_JAVA_OPTIONS) + private val isKerberosEnabled = submissionSparkConf.get(KUBERNETES_KERBEROS_SUPPORT) + private val maybeSimpleAuthentication = + if (isKerberosEnabled) Some(s"-D$HADOOP_SECURITY_AUTHENTICATION=simple") else None /** * Run command that initalizes a DriverSpec that will be updated after each @@ -99,7 +102,8 @@ private[spark] class Client( .getAll .map { case (confKey, confValue) => s"-D$confKey=$confValue" - } ++ driverJavaOptions.map(Utils.splitCommandString).getOrElse(Seq.empty) + } ++ driverJavaOptions.map(Utils.splitCommandString).getOrElse(Seq.empty) ++ + maybeSimpleAuthentication val driverJavaOptsEnvs: Seq[EnvVar] = resolvedDriverJavaOpts.zipWithIndex.map { case (option, index) => new EnvVarBuilder() .withName(s"$ENV_JAVA_OPT_PREFIX$index") @@ -153,7 +157,9 @@ private[spark] class Client( } private[spark] object Client { - def run(sparkConf: SparkConf, clientArguments: ClientArguments): Unit = { + def run(sparkConf: SparkConf, + clientArguments: ClientArguments, + hadoopConfDir: Option[String]): Unit = { val namespace = sparkConf.get(KUBERNETES_NAMESPACE) val kubernetesAppId = s"spark-${UUID.randomUUID().toString.replaceAll("-", "")}" val launchTime = System.currentTimeMillis() @@ -172,6 +178,7 @@ private[spark] object Client { clientArguments.mainClass, clientArguments.driverArgs, clientArguments.otherPyFiles, + hadoopConfDir, sparkConf) Utils.tryWithResource(SparkKubernetesClientFactory.createKubernetesClient( master, @@ -199,6 +206,9 @@ private[spark] object Client { def main(args: Array[String]): Unit = { val parsedArguments = ClientArguments.fromCommandLineArgs(args) val sparkConf = new SparkConf() - run(sparkConf, parsedArguments) + // hadoopConfDir is passed into Client#run() to allow for us to + // test this env variable within the integration test environment + val hadoopConfDir = sys.env.get("HADOOP_CONF_DIR") + run(sparkConf, parsedArguments, hadoopConfDir) } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestrator.scala index 1e9f5863a0d96..bc0e29ec6980d 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestrator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestrator.scala @@ -21,6 +21,7 @@ import org.apache.spark.deploy.kubernetes.ConfigurationUtils import org.apache.spark.deploy.kubernetes.config._ import org.apache.spark.deploy.kubernetes.constants._ import org.apache.spark.deploy.kubernetes.submit.submitsteps._ +import org.apache.spark.deploy.kubernetes.submit.submitsteps.hadoopsteps.HadoopStepsOrchestrator import org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer.InitContainerConfigurationStepsOrchestrator import org.apache.spark.launcher.SparkLauncher import org.apache.spark.util.Utils @@ -37,6 +38,7 @@ private[spark] class DriverConfigurationStepsOrchestrator( mainClass: String, appArgs: Array[String], additionalPythonFiles: Seq[String], + hadoopConfDir: Option[String], submissionSparkConf: SparkConf) { // The resource name prefix is derived from the application name, making it easy to connect the @@ -51,6 +53,7 @@ private[spark] class DriverConfigurationStepsOrchestrator( private val filesDownloadPath = submissionSparkConf.get(INIT_CONTAINER_FILES_DOWNLOAD_LOCATION) private val dockerImagePullPolicy = submissionSparkConf.get(DOCKER_IMAGE_PULL_POLICY) private val initContainerConfigMapName = s"$kubernetesResourceNamePrefix-init-config" + private val hadoopConfigMapName = s"$kubernetesResourceNamePrefix-hadoop-config" def getAllConfigurationSteps(): Seq[DriverConfigurationStep] = { val additionalMainAppJar = mainAppResource match { @@ -94,7 +97,13 @@ private[spark] class DriverConfigurationStepsOrchestrator( submissionSparkConf) val kubernetesCredentialsStep = new DriverKubernetesCredentialsStep( submissionSparkConf, kubernetesResourceNamePrefix) - val hadoopCredentialsStep = new DriverHadoopCredentialsStep(submissionSparkConf) + val hadoopConfigSteps = + hadoopConfDir.map { conf => + val hadoopStepsOrchestrator = + new HadoopStepsOrchestrator(namespace, hadoopConfigMapName, submissionSparkConf, conf) + val hadoopConfSteps = hadoopStepsOrchestrator.getHadoopSteps() + Some(new HadoopConfigBootstrapStep(hadoopConfSteps, hadoopConfigMapName))} + .getOrElse(Option.empty[DriverConfigurationStep]) val pythonStep = mainAppResource match { case PythonMainAppResource(mainPyResource) => Option(new PythonStep(mainPyResource, additionalPythonFiles, filesDownloadPath)) @@ -162,9 +171,9 @@ private[spark] class DriverConfigurationStepsOrchestrator( Seq( initialSubmissionStep, kubernetesCredentialsStep, - hadoopCredentialsStep, dependencyResolutionStep) ++ submittedDependenciesBootstrapSteps ++ + hadoopConfigSteps.toSeq ++ pythonStep.toSeq } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/HadoopSecretUtil.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/HadoopSecretUtil.scala deleted file mode 100644 index b7a9e9e3fd6cd..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/HadoopSecretUtil.scala +++ /dev/null @@ -1,55 +0,0 @@ -/* - * 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.kubernetes.submit - -import io.fabric8.kubernetes.api.model.{Container, ContainerBuilder, Pod, PodBuilder} - -import org.apache.spark.deploy.kubernetes.constants._ - -object HadoopSecretUtil { - - def configurePod(secretNameOption: Option[String], pod: Pod) : Pod = { - secretNameOption.map { secret => - new PodBuilder(pod) - .editOrNewSpec() - .addNewVolume() - .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) - .withNewSecret() - .withSecretName(secret) - .endSecret() - .endVolume() - .endSpec() - .build() - }.getOrElse(pod) - } - - def configureContainer(secretNameOption: Option[String], - containerSpec: Container) : Container = { - secretNameOption.map { secret => - new ContainerBuilder(containerSpec) - .addNewVolumeMount() - .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) - .withMountPath(SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR) - .endVolumeMount() - .addNewEnv() - .withName(ENV_HADOOP_TOKEN_FILE_LOCATION) - .withValue(SPARK_APP_HADOOP_TOKEN_FILE_PATH) - .endEnv() - .build() - }.getOrElse(containerSpec) - } -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverHadoopCredentialsStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverHadoopCredentialsStep.scala deleted file mode 100644 index 88f2e7b3836c8..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverHadoopCredentialsStep.scala +++ /dev/null @@ -1,38 +0,0 @@ -/* - * 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.kubernetes.submit.submitsteps - -import org.apache.spark.SparkConf -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.submit.HadoopSecretUtil - -private[spark] class DriverHadoopCredentialsStep(submissionSparkConf: SparkConf) - extends DriverConfigurationStep { - - private val maybeMountedHadoopSecret = submissionSparkConf.get(MOUNTED_HADOOP_SECRET_CONF) - - override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { - val podWithMountedHadoopToken = HadoopSecretUtil.configurePod(maybeMountedHadoopSecret, - driverSpec.driverPod) - val containerWithMountedHadoopToken = HadoopSecretUtil.configureContainer( - maybeMountedHadoopSecret, - driverSpec.driverContainer) - driverSpec.copy( - driverPod = podWithMountedHadoopToken, - driverContainer = containerWithMountedHadoopToken) - } -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/HadoopConfigBootstrapStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/HadoopConfigBootstrapStep.scala new file mode 100644 index 0000000000000..a9c66aed7075e --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/HadoopConfigBootstrapStep.scala @@ -0,0 +1,68 @@ +/* + * 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.kubernetes.submit.submitsteps + +import scala.collection.JavaConverters._ + +import io.fabric8.kubernetes.api.model.ConfigMapBuilder + +import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.kubernetes.submit.submitsteps.hadoopsteps.{HadoopConfigSpec, HadoopConfigurationStep} + + /** + * This class configures the driverSpec with hadoop configuration logic which includes + * volume mounts, config maps, and environment variable manipulation. The steps are + * resolved with the orchestrator and they are run modifying the HadoopSpec with each + * step. The final HadoopSpec's contents will be appended to the driverSpec. + */ +private[spark] class HadoopConfigBootstrapStep( + hadoopConfigurationSteps: Seq[HadoopConfigurationStep], + hadoopConfigMapName: String ) + extends DriverConfigurationStep { + + override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { + var currentHadoopSpec = HadoopConfigSpec( + driverPod = driverSpec.driverPod, + driverContainer = driverSpec.driverContainer, + configMapProperties = Map.empty[String, String], + additionalDriverSparkConf = Map.empty[String, String], + dtSecret = None, + dtSecretName = HADOOP_KERBEROS_SECRET_NAME, + dtSecretItemKey = "") + for (nextStep <- hadoopConfigurationSteps) { + currentHadoopSpec = nextStep.configureContainers(currentHadoopSpec) + } + val configMap = + new ConfigMapBuilder() + .withNewMetadata() + .withName(hadoopConfigMapName) + .endMetadata() + .addToData(currentHadoopSpec.configMapProperties.asJava) + .build() + val executorSparkConf = driverSpec.driverSparkConf.clone() + .set(HADOOP_CONFIG_MAP_SPARK_CONF_NAME, hadoopConfigMapName) + .setAll(currentHadoopSpec.additionalDriverSparkConf) + driverSpec.copy( + driverPod = currentHadoopSpec.driverPod, + driverContainer = currentHadoopSpec.driverContainer, + driverSparkConf = executorSparkConf, + otherKubernetesResources = + driverSpec.otherKubernetesResources ++ + Seq(configMap) ++ currentHadoopSpec.dtSecret.toSeq + ) + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopConfMounterStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopConfMounterStep.scala new file mode 100644 index 0000000000000..53b979df326c2 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopConfMounterStep.scala @@ -0,0 +1,57 @@ +/* + * 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.kubernetes.submit.submitsteps.hadoopsteps + +import java.io.File + +import org.apache.commons.io.FileUtils.readFileToString + +import org.apache.spark.deploy.kubernetes.{HadoopConfBootstrap, PodWithMainContainer} +import org.apache.spark.deploy.kubernetes.constants._ + + /** + * This step is responsible for taking the contents from each file in + * HADOOP_CONF_DIR, grabbing its contents as a string and storing each of them + * as a key-value pair in a configmap. Each key-value pair will be stored + * as a file, via Volume Mounts, later. The HADOOP_CONF_DIR_LOC is passed into the + * SchedulerBackend via sparkConf. + */ +private[spark] class HadoopConfMounterStep( + hadoopConfigMapName: String, + hadoopConfigurationFiles: Seq[File], + hadoopConfBootstrapConf: HadoopConfBootstrap, + hadoopConfDir: String) + extends HadoopConfigurationStep { + + override def configureContainers(hadoopConfigSpec: HadoopConfigSpec): HadoopConfigSpec = { + val bootstrappedPodAndMainContainer = + hadoopConfBootstrapConf.bootstrapMainContainerAndVolumes( + PodWithMainContainer( + hadoopConfigSpec.driverPod, + hadoopConfigSpec.driverContainer + )) + hadoopConfigSpec.copy( + driverPod = bootstrappedPodAndMainContainer.pod, + driverContainer = bootstrappedPodAndMainContainer.mainContainer, + configMapProperties = + hadoopConfigurationFiles.map(file => + (file.toPath.getFileName.toString, readFileToString(file))).toMap, + additionalDriverSparkConf = hadoopConfigSpec.additionalDriverSparkConf ++ + Map(HADOOP_CONF_DIR_LOC -> hadoopConfDir) + ) + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopConfigSpec.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopConfigSpec.scala new file mode 100644 index 0000000000000..51e9e506bcc22 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopConfigSpec.scala @@ -0,0 +1,42 @@ +/* + * 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.kubernetes.submit.submitsteps.hadoopsteps + +import io.fabric8.kubernetes.api.model.{Container, Pod, Secret} + + /** + * Represents a given configuration of the hadoop configuration logic, informing the + * HadoopConfigBootstrapStep of how the driver should be configured. This includes: + *

+ * - What Spark properties should be set on the driver's SparkConf for the executors + * - The spec of the main container so that it can be modified to share volumes + * - The spec of the driver pod EXCEPT for the addition of the given hadoop configs (e.g. volumes + * the hadoop logic needs) + * - The properties that will be stored into the config map which have (key, value) + * pairs of (path, data) + * - The secret containing a DT, either previously specified or built on the fly + * - The name of the secret where the DT will be stored + * - The data item-key on the secret which correlates with where the current DT data is stored + */ +private[spark] case class HadoopConfigSpec( + additionalDriverSparkConf: Map[String, String], + driverPod: Pod, + driverContainer: Container, + configMapProperties: Map[String, String], + dtSecret: Option[Secret], + dtSecretName: String, + dtSecretItemKey: String) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopConfigurationStep.scala new file mode 100644 index 0000000000000..54f46b9a17fc9 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopConfigurationStep.scala @@ -0,0 +1,25 @@ +/* + * 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.kubernetes.submit.submitsteps.hadoopsteps + + /** + * Represents a step in preparing the driver with Hadoop Configuration logic. + */ +private[spark] trait HadoopConfigurationStep { + + def configureContainers(hadoopConfigSpec: HadoopConfigSpec): HadoopConfigSpec +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStep.scala new file mode 100644 index 0000000000000..4e0a0cfd8fe9b --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStep.scala @@ -0,0 +1,134 @@ +/* + * 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.kubernetes.submit.submitsteps.hadoopsteps + +import java.io._ +import java.security.PrivilegedExceptionAction + +import scala.collection.JavaConverters._ + +import io.fabric8.kubernetes.api.model.SecretBuilder +import org.apache.commons.codec.binary.Base64 +import org.apache.hadoop.fs.FileSystem +import org.apache.hadoop.security.Credentials +import org.apache.hadoop.security.token.{Token, TokenIdentifier} + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.kubernetes.{HadoopUGIUtil, KerberosTokenConfBootstrapImpl, PodWithMainContainer} +import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.internal.Logging + + /** + * This step does all the heavy lifting for Delegation Token logic. This step + * assumes that the job user has either specified a principal and keytab or ran + * $kinit before running spark-submit. With a TGT stored locally, by running + * UGI.getCurrentUser you are able to obtain the current user, alternatively + * you can run UGI.loginUserFromKeytabAndReturnUGI and by running .doAs run + * as the logged into user instead of the current user. With the Job User principal + * you then retrieve the delegation token from the NameNode and store values in + * DelegationToken. Lastly, the class puts the data into a secret. All this is + * appended to the current HadoopSpec which in turn will append to the current + * DriverSpec. + */ +private[spark] class HadoopKerberosKeytabResolverStep( + submissionSparkConf: SparkConf, + maybePrincipal: Option[String], + maybeKeytab: Option[File], + maybeRenewerPrincipal: Option[String], + hadoopUGI: HadoopUGIUtil) extends HadoopConfigurationStep with Logging{ + private var originalCredentials: Credentials = _ + private var dfs : FileSystem = _ + private var renewer: String = _ + private var credentials: Credentials = _ + private var tokens: Iterable[Token[_ <: TokenIdentifier]] = _ + + override def configureContainers(hadoopConfigSpec: HadoopConfigSpec): HadoopConfigSpec = { + val hadoopConf = SparkHadoopUtil.get.newConfiguration(submissionSparkConf) + logDebug(s"Hadoop Configuration: ${hadoopConf.toString}") + if (hadoopUGI.isSecurityEnabled) logDebug("Hadoop not configured with Kerberos") + val maybeJobUserUGI = + for { + principal <- maybePrincipal + keytab <- maybeKeytab + } yield { + // Not necessary with [Spark-16742] + // Reliant on [Spark-20328] for changing to YARN principal + submissionSparkConf.set("spark.yarn.principal", principal) + submissionSparkConf.set("spark.yarn.keytab", keytab.toURI.toString) + logDebug("Logged into KDC with keytab using Job User UGI") + hadoopUGI.loginUserFromKeytabAndReturnUGI( + principal, + keytab.toURI.toString) + } + // In the case that keytab is not specified we will read from Local Ticket Cache + val jobUserUGI = maybeJobUserUGI.getOrElse(hadoopUGI.getCurrentUser) + // It is necessary to run as jobUserUGI because logged in user != Current User + jobUserUGI.doAs(new PrivilegedExceptionAction[Void] { + override def run(): Void = { + logDebug(s"Retrieved Job User UGI: $jobUserUGI") + originalCredentials = jobUserUGI.getCredentials + logDebug(s"Original tokens: ${originalCredentials.toString}") + logDebug(s"All tokens: ${originalCredentials.getAllTokens}") + logDebug(s"All secret keys: ${originalCredentials.getAllSecretKeys}") + // TODO: This is not necessary with [Spark-20328] since we would be using + // Spark core providers to handle delegation token renewal + renewer = maybeRenewerPrincipal.getOrElse(jobUserUGI.getShortUserName) + logDebug(s"Renewer is: $renewer") + credentials = new Credentials(originalCredentials) + hadoopUGI.dfsAddDelegationToken(hadoopConf, renewer, credentials) + tokens = credentials.getAllTokens.asScala + logDebug(s"Tokens: ${credentials.toString}") + logDebug(s"All tokens: ${tokens.mkString(",")}") + logDebug(s"All secret keys: ${credentials.getAllSecretKeys}") + null + }}) + if (tokens.isEmpty) logDebug("Did not obtain any Delegation Tokens") + val data = hadoopUGI.serialize(credentials) + val renewalInterval = + hadoopUGI.getTokenRenewalInterval(tokens, hadoopConf).getOrElse(Long.MaxValue) + val currentTime: Long = hadoopUGI.getCurrentTime + val initialTokenDataKeyName = s"$KERBEROS_SECRET_LABEL_PREFIX-$currentTime-$renewalInterval" + val uniqueSecretName = s"$HADOOP_KERBEROS_SECRET_NAME.$currentTime" + val secretDT = + new SecretBuilder() + .withNewMetadata() + .withName(uniqueSecretName) + .withLabels(Map("refresh-hadoop-tokens" -> "yes").asJava) + .endMetadata() + .addToData(initialTokenDataKeyName, Base64.encodeBase64String(data)) + .build() + val bootstrapKerberos = new KerberosTokenConfBootstrapImpl( + uniqueSecretName, + initialTokenDataKeyName, + jobUserUGI.getShortUserName) + val withKerberosEnvPod = bootstrapKerberos.bootstrapMainContainerAndVolumes( + PodWithMainContainer( + hadoopConfigSpec.driverPod, + hadoopConfigSpec.driverContainer)) + hadoopConfigSpec.copy( + additionalDriverSparkConf = + hadoopConfigSpec.additionalDriverSparkConf ++ Map( + HADOOP_KERBEROS_CONF_ITEM_KEY -> initialTokenDataKeyName, + HADOOP_KERBEROS_CONF_SECRET -> uniqueSecretName), + driverPod = withKerberosEnvPod.pod, + driverContainer = withKerberosEnvPod.mainContainer, + dtSecret = Some(secretDT), + dtSecretName = uniqueSecretName, + dtSecretItemKey = initialTokenDataKeyName) + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosSecretResolverStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosSecretResolverStep.scala new file mode 100644 index 0000000000000..3eeec1374ef96 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosSecretResolverStep.scala @@ -0,0 +1,56 @@ +/* + * 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.kubernetes.submit.submitsteps.hadoopsteps + +import org.apache.hadoop.security.UserGroupInformation + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.kubernetes.{KerberosTokenConfBootstrapImpl, PodWithMainContainer} +import org.apache.spark.deploy.kubernetes.constants._ + + /** + * This step assumes that you have already done all the heavy lifting in retrieving a + * delegation token and storing the following data in a secret before running this job. + * This step requires that you just specify the secret name and data item-key corresponding + * to the data where the delegation token is stored. + */ +private[spark] class HadoopKerberosSecretResolverStep( + submissionSparkConf: SparkConf, + tokenSecretName: String, + tokenItemKeyName: String) extends HadoopConfigurationStep { + + override def configureContainers(hadoopConfigSpec: HadoopConfigSpec): HadoopConfigSpec = { + val bootstrapKerberos = new KerberosTokenConfBootstrapImpl( + tokenSecretName, + tokenItemKeyName, + UserGroupInformation.getCurrentUser.getShortUserName) + val withKerberosEnvPod = bootstrapKerberos.bootstrapMainContainerAndVolumes( + PodWithMainContainer( + hadoopConfigSpec.driverPod, + hadoopConfigSpec.driverContainer)) + hadoopConfigSpec.copy( + driverPod = withKerberosEnvPod.pod, + driverContainer = withKerberosEnvPod.mainContainer, + additionalDriverSparkConf = + hadoopConfigSpec.additionalDriverSparkConf ++ Map( + HADOOP_KERBEROS_CONF_ITEM_KEY -> tokenItemKeyName, + HADOOP_KERBEROS_CONF_SECRET -> tokenSecretName), + dtSecret = None, + dtSecretName = tokenSecretName, + dtSecretItemKey = tokenItemKeyName) + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopStepsOrchestrator.scala new file mode 100644 index 0000000000000..93d184fedbcf1 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopStepsOrchestrator.scala @@ -0,0 +1,104 @@ +/* + * 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.kubernetes.submit.submitsteps.hadoopsteps + +import java.io.File + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.kubernetes.{HadoopConfBootstrapImpl, HadoopUGIUtil, OptionRequirements} +import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.internal.Logging + + /** + * Returns the complete ordered list of steps required to configure the hadoop configurations. + */ +private[spark] class HadoopStepsOrchestrator( + namespace: String, + hadoopConfigMapName: String, + submissionSparkConf: SparkConf, + hadoopConfDir: String) extends Logging{ + private val isKerberosEnabled = submissionSparkConf.get(KUBERNETES_KERBEROS_SUPPORT) + private val maybePrincipal = submissionSparkConf.get(KUBERNETES_KERBEROS_PRINCIPAL) + private val maybeKeytab = submissionSparkConf.get(KUBERNETES_KERBEROS_KEYTAB) + .map(k => new File(k)) + private val maybeExistingSecret = submissionSparkConf.get(KUBERNETES_KERBEROS_DT_SECRET_NAME) + private val maybeExistingSecretItemKey = + submissionSparkConf.get(KUBERNETES_KERBEROS_DT_SECRET_ITEM_KEY) + private val maybeRenewerPrincipal = + submissionSparkConf.get(KUBERNETES_KERBEROS_RENEWER_PRINCIPAL) + private val hadoopConfigurationFiles = getHadoopConfFiles(hadoopConfDir) + private val hadoopUGI = new HadoopUGIUtil + logInfo(s"Hadoop Conf directory: $hadoopConfDir") + + require(maybeKeytab.forall( _ => isKerberosEnabled ), + "You must enable Kerberos support if you are specifying a Kerberos Keytab") + + require(maybeExistingSecret.forall( _ => isKerberosEnabled ), + "You must enable Kerberos support if you are specifying a Kerberos Secret") + + OptionRequirements.requireBothOrNeitherDefined( + maybeKeytab, + maybePrincipal, + "If a Kerberos keytab is specified you must also specify a Kerberos principal", + "If a Kerberos principal is specified you must also specify a Kerberos keytab") + + OptionRequirements.requireBothOrNeitherDefined( + maybeExistingSecret, + maybeExistingSecretItemKey, + "If a secret storing a Kerberos Delegation Token is specified you must also" + + " specify the label where the data is stored", + "If a secret data item-key where the data of the Kerberos Delegation Token is specified" + + " you must also specify the name of the secret") + + def getHadoopSteps(): Seq[HadoopConfigurationStep] = { + val hadoopConfBootstrapImpl = new HadoopConfBootstrapImpl( + hadoopConfigMapName, + hadoopConfigurationFiles, + hadoopUGI) + val hadoopConfMounterStep = new HadoopConfMounterStep( + hadoopConfigMapName, + hadoopConfigurationFiles, + hadoopConfBootstrapImpl, + hadoopConfDir) + val maybeKerberosStep = + if (isKerberosEnabled) { + maybeExistingSecret.map(existingSecretName => Some(new HadoopKerberosSecretResolverStep( + submissionSparkConf, + existingSecretName, + maybeExistingSecretItemKey.get))).getOrElse(Some( + new HadoopKerberosKeytabResolverStep( + submissionSparkConf, + maybePrincipal, + maybeKeytab, + maybeRenewerPrincipal, + hadoopUGI))) + } else { + Option.empty[HadoopConfigurationStep] + } + Seq(hadoopConfMounterStep) ++ maybeKerberosStep.toSeq + } + + private def getHadoopConfFiles(path: String) : Seq[File] = { + def isFile(file: File) = if (file.isFile) Some(file) else None + val dir = new File(path) + if (dir.isDirectory) { + dir.listFiles.flatMap { file => isFile(file) }.toSeq + } else { + Seq.empty[File] + } + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala index b89e81bcb0be9..ff02c08ee2dce 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala @@ -20,16 +20,16 @@ import java.io.File import io.fabric8.kubernetes.client.Config -import org.apache.spark.SparkContext -import org.apache.spark.deploy.kubernetes.{InitContainerResourceStagingServerSecretPluginImpl, SparkKubernetesClientFactory, SparkPodInitContainerBootstrapImpl} +import org.apache.spark.deploy.kubernetes._ import org.apache.spark.deploy.kubernetes.config._ import org.apache.spark.deploy.kubernetes.constants._ import org.apache.spark.deploy.kubernetes.submit.MountSmallFilesBootstrapImpl import org.apache.spark.internal.Logging import org.apache.spark.scheduler.{ExternalClusterManager, SchedulerBackend, TaskScheduler, TaskSchedulerImpl} +import org.apache.spark.util.Utils +import org.apache.spark.SparkContext private[spark] class KubernetesClusterManager extends ExternalClusterManager with Logging { - override def canCreate(masterURL: String): Boolean = masterURL.startsWith("k8s") override def createTaskScheduler(sc: SparkContext, masterURL: String): TaskScheduler = { @@ -41,6 +41,10 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit override def createSchedulerBackend(sc: SparkContext, masterURL: String, scheduler: TaskScheduler) : SchedulerBackend = { val sparkConf = sc.getConf + val maybeHadoopConfigMap = sparkConf.getOption(HADOOP_CONFIG_MAP_SPARK_CONF_NAME) + val maybeHadoopConfDir = sparkConf.getOption(HADOOP_CONF_DIR_LOC) + val maybeDTSecretName = sparkConf.getOption(HADOOP_KERBEROS_CONF_SECRET) + val maybeDTDataItem = sparkConf.getOption(HADOOP_KERBEROS_CONF_ITEM_KEY) val maybeInitContainerConfigMap = sparkConf.get(EXECUTOR_INIT_CONTAINER_CONFIG_MAP) val maybeInitContainerConfigMapKey = sparkConf.get(EXECUTOR_INIT_CONTAINER_CONFIG_MAP_KEY) val maybeSubmittedFilesSecret = sparkConf.get(EXECUTOR_SUBMITTED_SMALL_FILES_SECRET) @@ -75,6 +79,27 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit configMap, configMapKey) } + val hadoopBootStrap = for { + hadoopConfigMap <- maybeHadoopConfigMap + } yield { + val hadoopUtil = new HadoopUGIUtil + val hadoopConfigurations = maybeHadoopConfDir.map( + conf_dir => getHadoopConfFiles(conf_dir)).getOrElse(Array.empty[File]) + new HadoopConfBootstrapImpl( + hadoopConfigMap, + hadoopConfigurations, + hadoopUtil + ) + } + val kerberosBootstrap = for { + secretName <- maybeDTSecretName + secretItemKey <- maybeDTDataItem + } yield { + new KerberosTokenConfBootstrapImpl( + secretName, + secretItemKey, + Utils.getCurrentUserName) + } val mountSmallFilesBootstrap = for { secretName <- maybeSubmittedFilesSecret secretMountPath <- maybeSubmittedFilesSecretMountPath @@ -89,6 +114,10 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit logWarning("The executor's init-container config map key was not specified. Executors will" + " therefore not attempt to fetch remote or submitted dependencies.") } + if (maybeHadoopConfigMap.isEmpty) { + logWarning("The executor's hadoop config map key was not specified. Executors will" + + " therefore not attempt to fetch hadoop configuration files.") + } val kubernetesClient = SparkKubernetesClientFactory.createKubernetesClient( KUBERNETES_MASTER_INTERNAL_URL, Some(sparkConf.get(KUBERNETES_NAMESPACE)), @@ -100,6 +129,8 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit sc.taskScheduler.asInstanceOf[TaskSchedulerImpl], sc, executorInitContainerbootStrap, + hadoopBootStrap, + kerberosBootstrap, executorInitContainerSecretVolumePlugin, mountSmallFilesBootstrap, kubernetesClient) @@ -108,4 +139,13 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit override def initialize(scheduler: TaskScheduler, backend: SchedulerBackend): Unit = { scheduler.asInstanceOf[TaskSchedulerImpl].initialize(backend) } + private def getHadoopConfFiles(path: String) : Array[File] = { + def isFile(file: File) = if (file.isFile) Some(file) else None + val dir = new File(path) + if (dir.isDirectory) { + dir.listFiles.flatMap { file => isFile(file) } + } else { + Array.empty[File] + } + } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala index 415c1bda7b632..3738e1b8d3991 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala @@ -33,10 +33,10 @@ import scala.collection.JavaConverters._ import scala.concurrent.{ExecutionContext, Future} import org.apache.spark.{SparkContext, SparkEnv, SparkException} -import org.apache.spark.deploy.kubernetes.{ConfigurationUtils, InitContainerResourceStagingServerSecretPlugin, PodWithDetachedInitContainer, SparkPodInitContainerBootstrap} +import org.apache.spark.deploy.kubernetes._ import org.apache.spark.deploy.kubernetes.config._ import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.{HadoopSecretUtil, InitContainerUtil, MountSmallFilesBootstrap} +import org.apache.spark.deploy.kubernetes.submit.{InitContainerUtil, MountSmallFilesBootstrap} import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.network.shuffle.kubernetes.KubernetesExternalShuffleClient import org.apache.spark.rpc.{RpcAddress, RpcCallContext, RpcEndpointAddress, RpcEnv} @@ -49,6 +49,8 @@ private[spark] class KubernetesClusterSchedulerBackend( scheduler: TaskSchedulerImpl, val sc: SparkContext, executorInitContainerBootstrap: Option[SparkPodInitContainerBootstrap], + executorHadoopBootStrap: Option[HadoopConfBootstrap], + executorKerberosBootStrap: Option[KerberosTokenConfBootstrap], executorMountInitContainerSecretPlugin: Option[InitContainerResourceStagingServerSecretPlugin], mountSmallFilesBootstrap: Option[MountSmallFilesBootstrap], kubernetesClient: KubernetesClient) @@ -73,7 +75,9 @@ private[spark] class KubernetesClusterSchedulerBackend( private val executorExtraClasspath = conf.get( org.apache.spark.internal.config.EXECUTOR_CLASS_PATH) private val executorJarsDownloadDir = conf.get(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION) - + private val isKerberosEnabled = conf.get(KUBERNETES_KERBEROS_SUPPORT) + private val maybeSimpleAuthentication = + if (isKerberosEnabled) Some(s"-D$HADOOP_SECURITY_AUTHENTICATION=simple") else None private val executorLabels = ConfigurationUtils.combinePrefixedKeyValuePairsWithDeprecatedConf( conf, KUBERNETES_EXECUTOR_LABEL_PREFIX, @@ -130,8 +134,6 @@ private[spark] class KubernetesClusterSchedulerBackend( private implicit val requestExecutorContext = ExecutionContext.fromExecutorService( ThreadUtils.newDaemonCachedThreadPool("kubernetes-executor-requests")) - private val maybeMountedHadoopSecret = conf.get(MOUNTED_HADOOP_SECRET_CONF) - private val driverPod = try { kubernetesClient.pods().inNamespace(kubernetesNamespace). withName(kubernetesDriverPodName).get() @@ -457,15 +459,19 @@ private[spark] class KubernetesClusterSchedulerBackend( .withValue(cp) .build() } - val executorExtraJavaOptionsEnv = conf - .get(org.apache.spark.internal.config.EXECUTOR_JAVA_OPTIONS) - .map { opts => - val delimitedOpts = Utils.splitCommandString(opts) - delimitedOpts.zipWithIndex.map { - case (opt, index) => - new EnvVarBuilder().withName(s"$ENV_JAVA_OPT_PREFIX$index").withValue(opt).build() - } - }.getOrElse(Seq.empty[EnvVar]) + val executorExtraJavaOptions = ( + conf.get(org.apache.spark.internal.config.EXECUTOR_JAVA_OPTIONS) + ++ maybeSimpleAuthentication).mkString(" ") match { + case "" => None + case str => Some(str) + } + val executorExtraJavaOptionsEnv = executorExtraJavaOptions.map { opts => + val delimitedOpts = Utils.splitCommandString(opts) + delimitedOpts.zipWithIndex.map { + case (opt, index) => + new EnvVarBuilder().withName(s"$ENV_JAVA_OPT_PREFIX$index").withValue(opt).build() + } + }.getOrElse(Seq.empty[EnvVar]) val executorEnv = (Seq( (ENV_EXECUTOR_PORT, executorPort.toString), (ENV_DRIVER_URL, driverUrl), @@ -597,14 +603,23 @@ private[spark] class KubernetesClusterSchedulerBackend( val executorPodWithNodeAffinity = addNodeAffinityAnnotationIfUseful( executorPodWithInitContainer, nodeToLocalTaskCount) - val executorPodWithMountedHadoopToken = HadoopSecretUtil.configurePod(maybeMountedHadoopSecret, - executorPodWithNodeAffinity) - val containerWithMountedHadoopToken = HadoopSecretUtil.configureContainer( - maybeMountedHadoopSecret, initBootstrappedExecutorContainer) - - val resolvedExecutorPod = new PodBuilder(executorPodWithMountedHadoopToken) + val (executorHadoopConfPod, executorHadoopConfContainer) = + executorHadoopBootStrap.map { bootstrap => + val podWithMainContainer = bootstrap.bootstrapMainContainerAndVolumes( + PodWithMainContainer(executorPodWithNodeAffinity, initBootstrappedExecutorContainer) + ) + (podWithMainContainer.pod, podWithMainContainer.mainContainer) + }.getOrElse(executorPodWithNodeAffinity, initBootstrappedExecutorContainer) + + val (executorKerberosPod, executorKerberosContainer) = + executorKerberosBootStrap.map { bootstrap => + val podWithMainContainer = bootstrap.bootstrapMainContainerAndVolumes( + PodWithMainContainer(executorHadoopConfPod, executorHadoopConfContainer)) + (podWithMainContainer.pod, podWithMainContainer.mainContainer) + }.getOrElse((executorHadoopConfPod, executorHadoopConfContainer)) + val resolvedExecutorPod = new PodBuilder(executorKerberosPod) .editSpec() - .addToContainers(containerWithMountedHadoopToken) + .addToContainers(executorKerberosContainer) .endSpec() .build() try { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/InitContainerResourceStagingServerSecretPluginSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/InitContainerResourceStagingServerSecretPluginSuite.scala index 597bcdb416fc0..51e695fc08a36 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/InitContainerResourceStagingServerSecretPluginSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/InitContainerResourceStagingServerSecretPluginSuite.scala @@ -23,7 +23,8 @@ import scala.collection.JavaConverters._ import org.apache.spark.deploy.kubernetes.constants._ import org.apache.spark.SparkFunSuite -class InitContainerResourceStagingServerSecretPluginSuite extends SparkFunSuite with BeforeAndAfter{ +class InitContainerResourceStagingServerSecretPluginSuite + extends SparkFunSuite with BeforeAndAfter{ private val INIT_CONTAINER_SECRET_NAME = "init-secret" private val INIT_CONTAINER_SECRET_MOUNT = "/tmp/secret" diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientSuite.scala index 0100dce454a3f..35fd5c1f206fa 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientSuite.scala @@ -16,23 +16,25 @@ */ package org.apache.spark.deploy.kubernetes.submit +import scala.collection.JavaConverters._ + import com.google.common.collect.Iterables -import io.fabric8.kubernetes.api.model.{ContainerBuilder, DoneablePod, HasMetadata, Pod, PodBuilder, PodList, Secret, SecretBuilder} +import io.fabric8.kubernetes.api.model._ import io.fabric8.kubernetes.client.{KubernetesClient, Watch} -import io.fabric8.kubernetes.client.dsl.{MixedOperation, NamespaceListVisitFromServerGetDeleteRecreateWaitApplicable, NamespaceVisitFromServerGetWatchDeleteRecreateWaitApplicable, PodResource, Resource} +import io.fabric8.kubernetes.client.dsl.{MixedOperation, NamespaceListVisitFromServerGetDeleteRecreateWaitApplicable, PodResource} import org.mockito.{ArgumentCaptor, Mock, MockitoAnnotations} -import org.mockito.Mockito.{doReturn, verify, when} import org.mockito.invocation.InvocationOnMock +import org.mockito.Mockito.{doReturn, verify, when} import org.mockito.stubbing.Answer import org.scalatest.BeforeAndAfter import org.scalatest.mock.MockitoSugar._ -import scala.collection.JavaConverters._ import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.kubernetes.config._ import org.apache.spark.deploy.kubernetes.constants._ import org.apache.spark.deploy.kubernetes.submit.submitsteps.{DriverConfigurationStep, KubernetesDriverSpec} -class ClientSuite extends SparkFunSuite with BeforeAndAfter { +private[spark] class ClientSuite extends SparkFunSuite with BeforeAndAfter { private val DRIVER_POD_UID = "pod-id" private val DRIVER_POD_API_VERSION = "v1" @@ -136,6 +138,10 @@ class ClientSuite extends SparkFunSuite with BeforeAndAfter { .set( org.apache.spark.internal.config.DRIVER_JAVA_OPTIONS, "-XX:+HeapDumpOnOutOfMemoryError -XX:+PrintGCDetails") + .set( + KUBERNETES_KERBEROS_SUPPORT, + true + ) val submissionClient = new Client( submissionSteps, sparkConf, @@ -150,14 +156,16 @@ class ClientSuite extends SparkFunSuite with BeforeAndAfter { val driverJvmOptsEnvs = driverContainer.getEnv.asScala.filter { env => env.getName.startsWith(ENV_JAVA_OPT_PREFIX) }.sortBy(_.getName) - assert(driverJvmOptsEnvs.size === 4) + assert(driverJvmOptsEnvs.size === 6) val expectedJvmOptsValues = Seq( + "-Dspark.kubernetes.kerberos.enabled=true", "-Dspark.logConf=true", s"-D${SecondTestConfigurationStep.sparkConfKey}=" + s"${SecondTestConfigurationStep.sparkConfValue}", s"-XX:+HeapDumpOnOutOfMemoryError", - s"-XX:+PrintGCDetails") + s"-XX:+PrintGCDetails", + "-Dspark.hadoop.hadoop.security.authentication=simple") driverJvmOptsEnvs.zip(expectedJvmOptsValues).zipWithIndex.foreach { case ((resolvedEnv, expectedJvmOpt), index) => assert(resolvedEnv.getName === s"$ENV_JAVA_OPT_PREFIX$index") diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestratorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestratorSuite.scala index c168e7b5407ba..b87fa48291a8c 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestratorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestratorSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.deploy.kubernetes.submit import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.submit.submitsteps.{BaseDriverConfigurationStep, DependencyResolutionStep, DriverConfigurationStep, DriverKubernetesCredentialsStep, InitContainerBootstrapStep, MountSmallLocalFilesStep, PythonStep} +import org.apache.spark.deploy.kubernetes.submit.submitsteps.{BaseDriverConfigurationStep, DependencyResolutionStep, DriverConfigurationStep, DriverKubernetesCredentialsStep, HadoopConfigBootstrapStep, InitContainerBootstrapStep, MountSmallLocalFilesStep, PythonStep} private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunSuite { @@ -43,6 +43,7 @@ private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunS MAIN_CLASS, APP_ARGS, ADDITIONAL_PYTHON_FILES, + None, sparkConf) validateStepTypes( orchestrator, @@ -65,6 +66,7 @@ private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunS MAIN_CLASS, APP_ARGS, ADDITIONAL_PYTHON_FILES, + None, sparkConf) validateStepTypes( orchestrator, @@ -86,6 +88,7 @@ private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunS MAIN_CLASS, APP_ARGS, ADDITIONAL_PYTHON_FILES, + None, sparkConf) validateStepTypes( orchestrator, @@ -107,6 +110,7 @@ private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunS MAIN_CLASS, APP_ARGS, ADDITIONAL_PYTHON_FILES, + None, sparkConf) validateStepTypes( orchestrator, @@ -116,11 +120,36 @@ private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunS classOf[MountSmallLocalFilesStep]) } + test("Submission steps with hdfs interaction and HADOOP_CONF_DIR defined") { + val sparkConf = new SparkConf(false) + val mainAppResource = JavaMainAppResource("local:///var/apps/jars/main.jar") + val hadoopConf = Some("/etc/hadoop/conf") + val orchestrator = new DriverConfigurationStepsOrchestrator( + NAMESPACE, + APP_ID, + LAUNCH_TIME, + mainAppResource, + APP_NAME, + MAIN_CLASS, + APP_ARGS, + ADDITIONAL_PYTHON_FILES, + hadoopConf, + sparkConf) + val steps = orchestrator.getAllConfigurationSteps() + validateStepTypes( + orchestrator, + classOf[BaseDriverConfigurationStep], + classOf[DriverKubernetesCredentialsStep], + classOf[DependencyResolutionStep], + classOf[HadoopConfigBootstrapStep]) + } + private def validateStepTypes( - orchestrator: DriverConfigurationStepsOrchestrator, - types: Class[_ <: DriverConfigurationStep]*): Unit = { + orchestrator: DriverConfigurationStepsOrchestrator, + types: Class[_ <: DriverConfigurationStep]*): Unit = { val steps = orchestrator.getAllConfigurationSteps() assert(steps.size === types.size) assert(steps.map(_.getClass) === types) } + } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/HadoopConfBootstrapSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/HadoopConfBootstrapSuite.scala new file mode 100644 index 0000000000000..64426f0deb15e --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/HadoopConfBootstrapSuite.scala @@ -0,0 +1,89 @@ +/* + * 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.kubernetes.submit + +import java.io.File +import java.util.UUID + +import scala.collection.JavaConverters._ + +import com.google.common.io.Files +import io.fabric8.kubernetes.api.model._ +import org.mockito.{Mock, MockitoAnnotations} +import org.mockito.Mockito.when +import org.scalatest.BeforeAndAfter + +import org.apache.spark.SparkFunSuite +import org.apache.spark.deploy.kubernetes.{HadoopConfBootstrapImpl, HadoopUGIUtil, PodWithMainContainer} +import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.util.Utils + +private[spark] class HadoopConfBootstrapSuite extends SparkFunSuite with BeforeAndAfter{ + private val CONFIG_MAP_NAME = "config-map" + private val TEMP_HADOOP_FILE = createTempFile("core-site.xml") + private val HADOOP_FILES = Seq(TEMP_HADOOP_FILE) + private val SPARK_USER_VALUE = "sparkUser" + + @Mock + private var hadoopUtil: HadoopUGIUtil = _ + + before { + MockitoAnnotations.initMocks(this) + when(hadoopUtil.getShortName).thenReturn(SPARK_USER_VALUE) + } + + test("Test of bootstrapping hadoop_conf_dir files") { + val hadoopConfStep = new HadoopConfBootstrapImpl( + CONFIG_MAP_NAME, + HADOOP_FILES, + hadoopUtil) + val expectedKeyPaths = Seq( + new KeyToPathBuilder() + .withKey(TEMP_HADOOP_FILE.toPath.getFileName.toString) + .withPath(TEMP_HADOOP_FILE.toPath.getFileName.toString) + .build()) + val expectedPod = new PodBuilder() + .editOrNewSpec() + .addNewVolume() + .withName(HADOOP_FILE_VOLUME) + .withNewConfigMap() + .withName(CONFIG_MAP_NAME) + .withItems(expectedKeyPaths.asJava) + .endConfigMap() + .endVolume() + .endSpec() + .build() + + val podWithMain = PodWithMainContainer( + new PodBuilder().withNewSpec().endSpec().build(), + new Container()) + val returnedPodContainer = hadoopConfStep.bootstrapMainContainerAndVolumes(podWithMain) + assert(expectedPod === returnedPodContainer.pod) + assert(returnedPodContainer.mainContainer.getVolumeMounts.asScala.map(vm => + (vm.getName, vm.getMountPath)).head === (HADOOP_FILE_VOLUME, HADOOP_CONF_DIR_PATH)) + assert(returnedPodContainer.mainContainer.getEnv.asScala.head === + new EnvVarBuilder().withName(ENV_HADOOP_CONF_DIR).withValue(HADOOP_CONF_DIR_PATH).build()) + assert(returnedPodContainer.mainContainer.getEnv.asScala(1) === + new EnvVarBuilder().withName(ENV_SPARK_USER).withValue(SPARK_USER_VALUE).build()) + } + private def createTempFile(contents: String): File = { + val dir = Utils.createTempDir() + val file = new File(dir, s"${UUID.randomUUID().toString}") + Files.write(contents.getBytes, file) + file + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/KerberosTokenConfBootstrapSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/KerberosTokenConfBootstrapSuite.scala new file mode 100644 index 0000000000000..95d7159736213 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/KerberosTokenConfBootstrapSuite.scala @@ -0,0 +1,60 @@ +/* + * 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.kubernetes.submit + +import scala.collection.JavaConverters._ + +import io.fabric8.kubernetes.api.model._ + +import org.apache.spark.SparkFunSuite +import org.apache.spark.deploy.kubernetes.{KerberosTokenConfBootstrapImpl, PodWithMainContainer} +import org.apache.spark.deploy.kubernetes.constants._ + + +private[spark] class KerberosTokenConfBootstrapSuite extends SparkFunSuite { + private val SECRET_NAME = "dtSecret" + private val SECRET_LABEL = "dtLabel" + private val TEST_SPARK_USER = "hdfs" + + test("Test of bootstrapping kerberos secrets and env") { + val kerberosConfStep = new KerberosTokenConfBootstrapImpl( + SECRET_NAME, + SECRET_LABEL, + TEST_SPARK_USER) + val expectedPod = new PodBuilder() + .editOrNewSpec() + .addNewVolume() + .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) + .withNewSecret() + .withSecretName(SECRET_NAME) + .endSecret() + .endVolume() + .endSpec() + .build() + val podWithMain = PodWithMainContainer( + new PodBuilder().withNewSpec().endSpec().build(), + new Container()) + val returnedPodContainer = kerberosConfStep.bootstrapMainContainerAndVolumes(podWithMain) + assert(expectedPod === returnedPodContainer.pod) + assert(returnedPodContainer.mainContainer.getVolumeMounts.asScala.map(vm => + (vm.getName, vm.getMountPath)).head === + (SPARK_APP_HADOOP_SECRET_VOLUME_NAME, SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR)) + assert(returnedPodContainer.mainContainer.getEnv.asScala.head.getName === + ENV_HADOOP_TOKEN_FILE_LOCATION) + assert(returnedPodContainer.mainContainer.getEnv.asScala(1).getName === ENV_SPARK_USER) + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/HadoopConfigBootstrapStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/HadoopConfigBootstrapStepSuite.scala new file mode 100644 index 0000000000000..074a02c339d95 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/HadoopConfigBootstrapStepSuite.scala @@ -0,0 +1,90 @@ +/* + * 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.kubernetes.submit.submitsteps + +import scala.collection.JavaConverters._ + +import io.fabric8.kubernetes.api.model._ +import org.mockito.{Mock, MockitoAnnotations} +import org.mockito.Matchers.any +import org.mockito.Mockito.when +import org.scalatest.BeforeAndAfter + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.kubernetes.submit.submitsteps.hadoopsteps.{HadoopConfigSpec, HadoopConfigurationStep} + + +private[spark] class HadoopConfigBootstrapStepSuite extends SparkFunSuite with BeforeAndAfter{ + private val CONFIG_MAP_NAME = "config-map" + private val POD_LABEL = Map("bootstrap" -> "true") + private val DRIVER_CONTAINER_NAME = "driver-container" + private val EXPECTED_SECRET = new SecretBuilder() + .withNewMetadata() + .withName(HADOOP_KERBEROS_SECRET_NAME) + .endMetadata() + .addToData("data", "secretata") + .build() + + @Mock + private var hadoopConfigStep : HadoopConfigurationStep = _ + + before { + MockitoAnnotations.initMocks(this) + when(hadoopConfigStep.configureContainers(any[HadoopConfigSpec])).thenReturn( + HadoopConfigSpec( + configMapProperties = Map("data" -> "dataBytesToString"), + driverPod = new PodBuilder() + .withNewMetadata() + .addToLabels("bootstrap", "true") + .endMetadata() + .withNewSpec().endSpec() + .build(), + driverContainer = new ContainerBuilder().withName(DRIVER_CONTAINER_NAME).build(), + additionalDriverSparkConf = Map("sparkConf" -> "confValue"), + dtSecret = + Some(EXPECTED_SECRET), + dtSecretName = HADOOP_KERBEROS_SECRET_NAME, + dtSecretItemKey = "")) + } + + test("Test modification of driverSpec with Hadoop Steps") { + val hadoopConfStep = new HadoopConfigBootstrapStep( + Seq(hadoopConfigStep), + CONFIG_MAP_NAME) + val expectedDriverSparkConf = new SparkConf(true) + .set(HADOOP_CONFIG_MAP_SPARK_CONF_NAME, CONFIG_MAP_NAME) + .set("sparkConf", "confValue") + val expectedConfigMap = new ConfigMapBuilder() + .withNewMetadata() + .withName(CONFIG_MAP_NAME) + .endMetadata() + .addToData(Map("data" -> "dataBytesToString").asJava) + .build() + val expectedResources = Seq(expectedConfigMap, EXPECTED_SECRET) + val driverSpec = KubernetesDriverSpec( + driverPod = new Pod(), + driverContainer = new Container(), + driverSparkConf = new SparkConf(true), + otherKubernetesResources = Seq.empty[HasMetadata]) + val returnContainerSpec = hadoopConfStep.configureDriver(driverSpec) + assert(expectedDriverSparkConf.getAll === returnContainerSpec.driverSparkConf.getAll) + assert(returnContainerSpec.driverContainer.getName == DRIVER_CONTAINER_NAME) + assert(returnContainerSpec.driverPod.getMetadata.getLabels.asScala === POD_LABEL) + assert(returnContainerSpec.otherKubernetesResources === expectedResources) + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initContainerBootstrapStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/InitContainerBootstrapStepSuite.scala similarity index 99% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initContainerBootstrapStepSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/InitContainerBootstrapStepSuite.scala index b11b487111496..1c9a1ac50229e 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initContainerBootstrapStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/InitContainerBootstrapStepSuite.scala @@ -31,7 +31,7 @@ import org.apache.spark.deploy.kubernetes.constants._ import org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer.{InitContainerConfigurationStep, InitContainerSpec} import org.apache.spark.util.Utils -private[spark] class initContainerBootstrapStepSuite extends SparkFunSuite { +class InitContainerBootstrapStepSuite extends SparkFunSuite { private val OBJECT_MAPPER = new ObjectMapper().registerModule(DefaultScalaModule) private val CONFIG_MAP_NAME = "spark-init-config-map" diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopConfMounterStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopConfMounterStepSuite.scala new file mode 100644 index 0000000000000..876675d5a4d96 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopConfMounterStepSuite.scala @@ -0,0 +1,102 @@ +/* + * 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.kubernetes.submit.submitsteps.hadoopsteps + +import java.io.File +import java.util.UUID + +import scala.collection.JavaConverters._ + +import com.google.common.io.Files +import io.fabric8.kubernetes.api.model._ +import org.apache.commons.io.FileUtils.readFileToString +import org.mockito.{Mock, MockitoAnnotations} +import org.mockito.Matchers.any +import org.mockito.Mockito.when +import org.mockito.invocation.InvocationOnMock +import org.mockito.stubbing.Answer +import org.scalatest.BeforeAndAfter + +import org.apache.spark.SparkFunSuite +import org.apache.spark.deploy.kubernetes.{HadoopConfBootstrap, PodWithMainContainer} +import org.apache.spark.deploy.kubernetes.constants.HADOOP_CONF_DIR_LOC +import org.apache.spark.util.Utils + + +private[spark] class HadoopConfMounterStepSuite extends SparkFunSuite with BeforeAndAfter{ + private val CONFIG_MAP_NAME = "config-map" + private val HADOOP_CONF_DIR_VAL = "/etc/hadoop" + private val POD_LABEL = Map("bootstrap" -> "true") + private val DRIVER_CONTAINER_NAME = "driver-container" + private val TEMP_HADOOP_FILE = createTempFile("core-site.xml") + private val HADOOP_FILES = Seq(TEMP_HADOOP_FILE) + + @Mock + private var hadoopConfBootstrap : HadoopConfBootstrap = _ + + before { + MockitoAnnotations.initMocks(this) + when(hadoopConfBootstrap.bootstrapMainContainerAndVolumes( + any[PodWithMainContainer])).thenAnswer(new Answer[PodWithMainContainer] { + override def answer(invocation: InvocationOnMock) : PodWithMainContainer = { + val pod = invocation.getArgumentAt(0, classOf[PodWithMainContainer]) + pod.copy( + pod = + new PodBuilder(pod.pod) + .withNewMetadata() + .addToLabels("bootstrap", "true") + .endMetadata() + .withNewSpec().endSpec() + .build(), + mainContainer = + new ContainerBuilder() + .withName(DRIVER_CONTAINER_NAME).build() + )}}) + } + + test("Test of mounting hadoop_conf_dir files into HadoopConfigSpec") { + val hadoopConfStep = new HadoopConfMounterStep( + CONFIG_MAP_NAME, + HADOOP_FILES, + hadoopConfBootstrap, + HADOOP_CONF_DIR_VAL) + val expectedDriverSparkConf = Map(HADOOP_CONF_DIR_LOC -> HADOOP_CONF_DIR_VAL) + val expectedConfigMap = Map( + TEMP_HADOOP_FILE.toPath.getFileName.toString -> + readFileToString(TEMP_HADOOP_FILE) + ) + val hadoopConfSpec = HadoopConfigSpec( + Map.empty[String, String], + new Pod(), + new Container(), + Map.empty[String, String], + None, + "", + "") + val returnContainerSpec = hadoopConfStep.configureContainers(hadoopConfSpec) + assert(expectedDriverSparkConf === returnContainerSpec.additionalDriverSparkConf) + assert(returnContainerSpec.driverContainer.getName == DRIVER_CONTAINER_NAME) + assert(returnContainerSpec.driverPod.getMetadata.getLabels.asScala === POD_LABEL) + assert(returnContainerSpec.configMapProperties === expectedConfigMap) + } + private def createTempFile(contents: String): File = { + val dir = Utils.createTempDir() + val file = new File(dir, s"${UUID.randomUUID().toString}") + Files.write(contents.getBytes, file) + file + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStepSuite.scala new file mode 100644 index 0000000000000..997556d1508a8 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStepSuite.scala @@ -0,0 +1,146 @@ +/* + * 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.kubernetes.submit.submitsteps.hadoopsteps + +import java.io.File +import java.util.UUID + +import scala.collection.JavaConverters._ + +import com.google.common.io.Files +import io.fabric8.kubernetes.api.model._ +import org.apache.commons.codec.binary.Base64 +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.security.{Credentials, UserGroupInformation} +import org.apache.hadoop.security.token.{Token, TokenIdentifier} +import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier +import org.mockito.{Mock, MockitoAnnotations} +import org.mockito.Matchers.any +import org.mockito.Mockito.when +import org.mockito.invocation.InvocationOnMock +import org.mockito.stubbing.Answer +import org.scalatest.BeforeAndAfter + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.kubernetes.HadoopUGIUtil +import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.util.Utils + + + +private[spark] class HadoopKerberosKeytabResolverStepSuite + extends SparkFunSuite with BeforeAndAfter{ + private val POD_LABEL = Map("bootstrap" -> "true") + private val DRIVER_CONTAINER_NAME = "driver-container" + private val TEMP_KEYTAB_FILE = createTempFile("keytab") + private val KERB_PRINCIPAL = "user@k8s.com" + private val SPARK_USER_VALUE = "sparkUser" + private val TEST_TOKEN_VALUE = "data" + private def getByteArray(input: String) = input.toCharArray.map(_.toByte) + private val TEST_DATA = getByteArray(TEST_TOKEN_VALUE) + private val OUTPUT_TEST_DATA = Base64.encodeBase64String(TEST_DATA) + private val INTERVAL = 500L + private val CURR_TIME = System.currentTimeMillis() + private val DATA_KEY_NAME = + s"$KERBEROS_SECRET_LABEL_PREFIX-$CURR_TIME-$INTERVAL" + private val SECRET_NAME = s"$HADOOP_KERBEROS_SECRET_NAME.$CURR_TIME" + + private val hadoopUGI = new HadoopUGIUtil() + + @Mock + private var hadoopUtil: HadoopUGIUtil = _ + + @Mock + private var ugi: UserGroupInformation = _ + + @Mock + private var creds: Credentials = _ + + @Mock + private var token: Token[AbstractDelegationTokenIdentifier] = _ + + @Mock + private var identifier: AbstractDelegationTokenIdentifier = _ + + before { + MockitoAnnotations.initMocks(this) + when(hadoopUtil.loginUserFromKeytabAndReturnUGI(any[String], any[String])) + .thenAnswer(new Answer[UserGroupInformation] { + override def answer(invocation: InvocationOnMock): UserGroupInformation = { + hadoopUGI.getCurrentUser + } + }) + when(hadoopUtil.getCurrentUser).thenReturn(ugi) + when(hadoopUtil.getShortName).thenReturn(SPARK_USER_VALUE) + when(hadoopUtil.dfsAddDelegationToken(any(), any(), any())).thenReturn(null) + when(ugi.getCredentials).thenReturn(creds) + val tokens = List[Token[_ <: TokenIdentifier]](token).asJavaCollection + when(creds.getAllTokens).thenReturn(tokens) + when(hadoopUtil.serialize(any[Credentials])) + .thenReturn(TEST_DATA) + when(token.decodeIdentifier()).thenReturn(identifier) + when(hadoopUtil.getCurrentTime).thenReturn(CURR_TIME) + when(hadoopUtil.getTokenRenewalInterval(any[Iterable[Token[_ <: TokenIdentifier]]], + any[Configuration])).thenReturn(Some(INTERVAL)) + } + + test("Testing keytab login") { + when(hadoopUtil.isSecurityEnabled).thenReturn(true) + val keytabStep = new HadoopKerberosKeytabResolverStep( + new SparkConf(), + Some(KERB_PRINCIPAL), + Some(TEMP_KEYTAB_FILE), + None, + hadoopUtil) + val hadoopConfSpec = HadoopConfigSpec( + Map.empty[String, String], + new PodBuilder() + .withNewMetadata() + .addToLabels("bootstrap", "true") + .endMetadata() + .withNewSpec().endSpec() + .build(), + new ContainerBuilder().withName(DRIVER_CONTAINER_NAME).build(), + Map.empty[String, String], + None, + "", + "") + val returnContainerSpec = keytabStep.configureContainers(hadoopConfSpec) + assert(returnContainerSpec.additionalDriverSparkConf(HADOOP_KERBEROS_CONF_ITEM_KEY) + .contains(KERBEROS_SECRET_LABEL_PREFIX)) + assert(returnContainerSpec.additionalDriverSparkConf === + Map(HADOOP_KERBEROS_CONF_ITEM_KEY -> DATA_KEY_NAME, + HADOOP_KERBEROS_CONF_SECRET -> SECRET_NAME)) + assert(returnContainerSpec.driverContainer.getName == DRIVER_CONTAINER_NAME) + assert(returnContainerSpec.driverPod.getMetadata.getLabels.asScala === POD_LABEL) + assert(returnContainerSpec.dtSecretItemKey === DATA_KEY_NAME) + assert(returnContainerSpec.dtSecret.get.getData.asScala === Map( + DATA_KEY_NAME -> OUTPUT_TEST_DATA)) + assert(returnContainerSpec.dtSecretName === SECRET_NAME) + assert(returnContainerSpec.dtSecret.get.getMetadata.getLabels.asScala === + Map("refresh-hadoop-tokens" -> "yes")) + assert(returnContainerSpec.dtSecret.nonEmpty) + assert(returnContainerSpec.dtSecret.get.getMetadata.getName === SECRET_NAME) + } + + private def createTempFile(contents: String): File = { + val dir = Utils.createTempDir() + val file = new File(dir, s"${UUID.randomUUID().toString}") + Files.write(contents.getBytes, file) + file + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosSecretResolverStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosSecretResolverStepSuite.scala new file mode 100644 index 0000000000000..bb0fa237a2d5e --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosSecretResolverStepSuite.scala @@ -0,0 +1,63 @@ +/* + * 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.kubernetes.submit.submitsteps.hadoopsteps + +import scala.collection.JavaConverters._ + +import io.fabric8.kubernetes.api.model._ + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.kubernetes.constants._ + +private[spark] class HadoopKerberosSecretResolverStepSuite extends SparkFunSuite { + private val CONFIG_MAP_NAME = "config-map" + private val HADOOP_CONF_DIR_VAL = "/etc/hadoop" + private val POD_LABEL = Map("bootstrap" -> "true") + private val DRIVER_CONTAINER_NAME = "driver-container" + private val TOKEN_SECRET_NAME = "secretName" + private val TOKEN_SECRET_DATA_ITEM_KEY = "secretItemKey" + + test("Testing kerberos with Secret") { + val keytabStep = new HadoopKerberosSecretResolverStep( + new SparkConf(), + TOKEN_SECRET_NAME, + TOKEN_SECRET_DATA_ITEM_KEY) + val expectedDriverSparkConf = Map( + HADOOP_KERBEROS_CONF_ITEM_KEY -> TOKEN_SECRET_DATA_ITEM_KEY, + HADOOP_KERBEROS_CONF_SECRET -> TOKEN_SECRET_NAME) + val hadoopConfSpec = HadoopConfigSpec( + Map.empty[String, String], + new PodBuilder() + .withNewMetadata() + .addToLabels("bootstrap", "true") + .endMetadata() + .withNewSpec().endSpec() + .build(), + new ContainerBuilder().withName(DRIVER_CONTAINER_NAME).build(), + Map.empty[String, String], + None, + "", + "") + val returnContainerSpec = keytabStep.configureContainers(hadoopConfSpec) + assert(expectedDriverSparkConf === returnContainerSpec.additionalDriverSparkConf) + assert(returnContainerSpec.driverContainer.getName == DRIVER_CONTAINER_NAME) + assert(returnContainerSpec.driverPod.getMetadata.getLabels.asScala === POD_LABEL) + assert(returnContainerSpec.dtSecret === None) + assert(returnContainerSpec.dtSecretItemKey === TOKEN_SECRET_DATA_ITEM_KEY) + assert(returnContainerSpec.dtSecretName === TOKEN_SECRET_NAME) + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopStepsOrchestratorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopStepsOrchestratorSuite.scala new file mode 100644 index 0000000000000..1f946666f53ce --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopStepsOrchestratorSuite.scala @@ -0,0 +1,85 @@ +/* + * 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.kubernetes.submit.submitsteps.hadoopsteps + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.kubernetes.config._ + +private[spark] class HadoopStepsOrchestratorSuite extends SparkFunSuite { + private val NAMESPACE = "testNamespace" + private val HADOOP_CONFIG_MAP = "hadoop-config-map" + private val HADOOP_CONF_DIR_VAL = "/etc/hadoop/conf" + + test("Testing without Kerberos") { + val sparkTestConf = new SparkConf(true) + .set(KUBERNETES_KERBEROS_SUPPORT, false) + val hadoopOrchestrator = new HadoopStepsOrchestrator( + NAMESPACE, + HADOOP_CONFIG_MAP, + sparkTestConf, + HADOOP_CONF_DIR_VAL) + val steps = hadoopOrchestrator.getHadoopSteps() + assert(steps.length === 1) + assert(steps.head.isInstanceOf[HadoopConfMounterStep]) + } + + test("Testing with Keytab Kerberos Login") { + val sparkTestConf = new SparkConf(true) + .set(KUBERNETES_KERBEROS_SUPPORT, true) + .set(KUBERNETES_KERBEROS_KEYTAB, "keytab.file") + .set(KUBERNETES_KERBEROS_PRINCIPAL, "user@kerberos") + val hadoopOrchestrator = new HadoopStepsOrchestrator( + NAMESPACE, + HADOOP_CONFIG_MAP, + sparkTestConf, + HADOOP_CONF_DIR_VAL) + val steps = hadoopOrchestrator.getHadoopSteps() + assert(steps.length === 2) + assert(steps.head.isInstanceOf[HadoopConfMounterStep]) + assert(steps(1).isInstanceOf[HadoopKerberosKeytabResolverStep]) + } + + test("Testing with kinit Kerberos Login") { + val sparkTestConf = new SparkConf(true) + .set(KUBERNETES_KERBEROS_SUPPORT, true) + val hadoopOrchestrator = new HadoopStepsOrchestrator( + NAMESPACE, + HADOOP_CONFIG_MAP, + sparkTestConf, + HADOOP_CONF_DIR_VAL) + val steps = hadoopOrchestrator.getHadoopSteps() + assert(steps.length === 2) + assert(steps.head.isInstanceOf[HadoopConfMounterStep]) + assert(steps(1).isInstanceOf[HadoopKerberosKeytabResolverStep]) + } + + test("Testing with Secret stored Kerberos") { + val sparkTestConf = new SparkConf(true) + .set(KUBERNETES_KERBEROS_SUPPORT, true) + .set(KUBERNETES_KERBEROS_DT_SECRET_NAME, "dtSecret") + .set(KUBERNETES_KERBEROS_DT_SECRET_ITEM_KEY, "dtItemKey") + val hadoopOrchestrator = new HadoopStepsOrchestrator( + NAMESPACE, + HADOOP_CONFIG_MAP, + sparkTestConf, + HADOOP_CONF_DIR_VAL) + val steps = hadoopOrchestrator.getHadoopSteps() + assert(steps.length === 2) + assert(steps.head.isInstanceOf[HadoopConfMounterStep]) + assert(steps(1).isInstanceOf[HadoopKerberosSecretResolverStep]) + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStepSuite.scala index fe1af4bc5be2a..79545a55f33b2 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStepSuite.scala @@ -29,7 +29,8 @@ import org.apache.spark.SparkFunSuite import org.apache.spark.deploy.kubernetes.{PodWithDetachedInitContainer, SparkPodInitContainerBootstrap} import org.apache.spark.deploy.kubernetes.config._ -class BaseInitContainerConfigurationStepSuite extends SparkFunSuite with BeforeAndAfter{ +private[spark] class BaseInitContainerConfigurationStepSuite + extends SparkFunSuite with BeforeAndAfter{ private val SPARK_JARS = Seq( "hdfs://localhost:9000/app/jars/jar1.jar", "file:///app/jars/jar2.jar") private val SPARK_FILES = Seq( diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerStepSuite.scala index 2edaba93fe07f..e9c828fdb397c 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerStepSuite.scala @@ -37,7 +37,8 @@ import org.apache.spark.deploy.kubernetes.constants._ import org.apache.spark.deploy.kubernetes.submit.{SubmittedDependencyUploader, SubmittedResourceIdAndSecret} import org.apache.spark.util.Utils -class SubmittedResourcesInitContainerStepSuite extends SparkFunSuite with BeforeAndAfter { +private[spark] class SubmittedResourcesInitContainerStepSuite + extends SparkFunSuite with BeforeAndAfter { private val RESOURCE_SECRET_NAME = "secret" private val JARS_RESOURCE_ID = "jarsID" private val JARS_SECRET = "jarsSecret" diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver-py/Dockerfile b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver-py/Dockerfile index 7b1effa911f19..27339d72cfd38 100644 --- a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver-py/Dockerfile +++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver-py/Dockerfile @@ -44,6 +44,7 @@ CMD SPARK_CLASSPATH="${SPARK_HOME}/jars/*" && \ if ! [ -z ${SPARK_MOUNTED_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_MOUNTED_CLASSPATH:$SPARK_CLASSPATH"; fi && \ if ! [ -z ${SPARK_SUBMIT_EXTRA_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_SUBMIT_EXTRA_CLASSPATH:$SPARK_CLASSPATH"; fi && \ if ! [ -z ${SPARK_EXTRA_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_EXTRA_CLASSPATH:$SPARK_CLASSPATH"; fi && \ + if ! [ -z ${HADOOP_CONF_DIR+x} ]; then SPARK_CLASSPATH="$HADOOP_CONF_DIR:$SPARK_CLASSPATH"; fi && \ if ! [ -z ${SPARK_MOUNTED_FILES_DIR+x} ]; then cp -R "$SPARK_MOUNTED_FILES_DIR/." .; fi && \ if ! [ -z ${SPARK_MOUNTED_FILES_FROM_SECRET_DIR+x} ]; then cp -R "$SPARK_MOUNTED_FILES_FROM_SECRET_DIR/." .; fi && \ ${JAVA_HOME}/bin/java "${SPARK_DRIVER_JAVA_OPTS[@]}" -cp $SPARK_CLASSPATH -Xms$SPARK_DRIVER_MEMORY -Xmx$SPARK_DRIVER_MEMORY $SPARK_DRIVER_CLASS $PYSPARK_PRIMARY $PYSPARK_FILES $SPARK_DRIVER_ARGS diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver/Dockerfile b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver/Dockerfile index 26d1d805fde2b..3e99ef2809dcb 100644 --- a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver/Dockerfile +++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver/Dockerfile @@ -29,6 +29,7 @@ CMD SPARK_CLASSPATH="${SPARK_HOME}/jars/*" && \ if ! [ -z ${SPARK_MOUNTED_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_MOUNTED_CLASSPATH:$SPARK_CLASSPATH"; fi && \ if ! [ -z ${SPARK_SUBMIT_EXTRA_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_SUBMIT_EXTRA_CLASSPATH:$SPARK_CLASSPATH"; fi && \ if ! [ -z ${SPARK_EXTRA_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_EXTRA_CLASSPATH:$SPARK_CLASSPATH"; fi && \ + if ! [ -z ${HADOOP_CONF_DIR+x} ]; then SPARK_CLASSPATH="$HADOOP_CONF_DIR:$SPARK_CLASSPATH"; fi && \ if ! [ -z ${SPARK_MOUNTED_FILES_DIR+x} ]; then cp -R "$SPARK_MOUNTED_FILES_DIR/." .; fi && \ if ! [ -z ${SPARK_MOUNTED_FILES_FROM_SECRET_DIR} ]; then cp -R "$SPARK_MOUNTED_FILES_FROM_SECRET_DIR/." .; fi && \ ${JAVA_HOME}/bin/java "${SPARK_DRIVER_JAVA_OPTS[@]}" -cp $SPARK_CLASSPATH -Xms$SPARK_DRIVER_MEMORY -Xmx$SPARK_DRIVER_MEMORY $SPARK_DRIVER_CLASS $SPARK_DRIVER_ARGS diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/executor-py/Dockerfile b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/executor-py/Dockerfile index a8bb5b362ab52..4e1b2ed91a1b5 100644 --- a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/executor-py/Dockerfile +++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/executor-py/Dockerfile @@ -43,6 +43,7 @@ CMD SPARK_CLASSPATH="${SPARK_HOME}/jars/*" && \ readarray -t SPARK_EXECUTOR_JAVA_OPTS < /tmp/java_opts.txt && \ if ! [ -z ${SPARK_MOUNTED_CLASSPATH}+x} ]; then SPARK_CLASSPATH="$SPARK_MOUNTED_CLASSPATH:$SPARK_CLASSPATH"; fi && \ if ! [ -z ${SPARK_EXECUTOR_EXTRA_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_EXECUTOR_EXTRA_CLASSPATH:$SPARK_CLASSPATH"; fi && \ + if ! [ -z ${HADOOP_CONF_DIR+x} ]; then SPARK_CLASSPATH="$HADOOP_CONF_DIR:$SPARK_CLASSPATH"; fi && \ if ! [ -z ${SPARK_MOUNTED_FILES_DIR+x} ]; then cp -R "$SPARK_MOUNTED_FILES_DIR/." .; fi && \ if ! [ -z ${SPARK_MOUNTED_FILES_FROM_SECRET_DIR+x} ]; then cp -R "$SPARK_MOUNTED_FILES_FROM_SECRET_DIR/." .; fi && \ ${JAVA_HOME}/bin/java "${SPARK_EXECUTOR_JAVA_OPTS[@]}" -Dspark.executor.port=$SPARK_EXECUTOR_PORT -Xms$SPARK_EXECUTOR_MEMORY -Xmx$SPARK_EXECUTOR_MEMORY -cp $SPARK_CLASSPATH org.apache.spark.executor.CoarseGrainedExecutorBackend --driver-url $SPARK_DRIVER_URL --executor-id $SPARK_EXECUTOR_ID --cores $SPARK_EXECUTOR_CORES --app-id $SPARK_APPLICATION_ID --hostname $SPARK_EXECUTOR_POD_IP diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/executor/Dockerfile b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/executor/Dockerfile index ab9f67e95a8e5..28e7b8ec3b44f 100644 --- a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/executor/Dockerfile +++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/executor/Dockerfile @@ -29,6 +29,7 @@ CMD SPARK_CLASSPATH="${SPARK_HOME}/jars/*" && \ if ! [ -z ${SPARK_MOUNTED_CLASSPATH}+x} ]; then SPARK_CLASSPATH="$SPARK_MOUNTED_CLASSPATH:$SPARK_CLASSPATH"; fi && \ if ! [ -z ${SPARK_EXECUTOR_EXTRA_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_EXECUTOR_EXTRA_CLASSPATH:$SPARK_CLASSPATH"; fi && \ if ! [ -z ${SPARK_EXTRA_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_EXTRA_CLASSPATH:$SPARK_CLASSPATH"; fi && \ + if ! [ -z ${HADOOP_CONF_DIR+x} ]; then SPARK_CLASSPATH="$HADOOP_CONF_DIR:$SPARK_CLASSPATH"; fi && \ if ! [ -z ${SPARK_MOUNTED_FILES_DIR+x} ]; then cp -R "$SPARK_MOUNTED_FILES_DIR/." .; fi && \ if ! [ -z ${SPARK_MOUNTED_FILES_FROM_SECRET_DIR+x} ]; then cp -R "$SPARK_MOUNTED_FILES_FROM_SECRET_DIR/." .; fi && \ ${JAVA_HOME}/bin/java "${SPARK_EXECUTOR_JAVA_OPTS[@]}" -Dspark.executor.port=$SPARK_EXECUTOR_PORT -Xms$SPARK_EXECUTOR_MEMORY -Xmx$SPARK_EXECUTOR_MEMORY -cp $SPARK_CLASSPATH org.apache.spark.executor.CoarseGrainedExecutorBackend --driver-url $SPARK_DRIVER_URL --executor-id $SPARK_EXECUTOR_ID --cores $SPARK_EXECUTOR_CORES --app-id $SPARK_APPLICATION_ID --hostname $SPARK_EXECUTOR_POD_IP diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/kerberos-test/Dockerfile b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/kerberos-test/Dockerfile new file mode 100644 index 0000000000000..c4ba43ad511d8 --- /dev/null +++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/kerberos-test/Dockerfile @@ -0,0 +1,27 @@ +# +# 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. +# + +FROM spark-base + +# If this docker file is being used in the context of building your images from a Spark distribution, the docker build +# command should be invoked from the top level directory of the Spark distribution. E.g.: +# docker build -t kerberos-test:latest -f dockerfiles/kerberos-test/Dockerfile . + +RUN apk add --no-cache --update krb5 krb5-libs +COPY examples /opt/spark/examples +COPY test-scripts/test-env.sh /opt/spark/ +COPY hconf /opt/spark/hconf \ No newline at end of file diff --git a/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/HDFSTest.scala b/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/HDFSTest.scala new file mode 100644 index 0000000000000..463dd25d14a1f --- /dev/null +++ b/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/HDFSTest.scala @@ -0,0 +1,48 @@ +/* + * 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.kubernetes.integrationtest.jobs + +import org.apache.spark.sql.SparkSession + +private[spark] object HDFSTest{ + + def main(args: Array[String]): Unit = { + // scalastyle:off println + if (args.length < 1) { + System.err.println("Usage: HdfsTest ") + System.exit(1) + } + // scalastyle:on println + val spark = SparkSession + .builder + .appName("HdfsTest") + .getOrCreate() + val file = spark.read.text(args(0)).rdd + val mapped = file.map(s => s.length).cache() + // scalastyle:off println + for (iter <- 1 to 10) { + val start = System.currentTimeMillis() + for (x <- mapped) { x + 2 } + val end = System.currentTimeMillis() + println("Iteration " + iter + " took " + (end-start) + " ms") + } + println(s"File contents: ${file.map(s => s.toString).collect().mkString(",")}") + println(s"Returned length(s) of: ${file.map(s => s.length).collect().mkString(",")}") + // scalastyle:on println + spark.stop() + } +} diff --git a/resource-managers/kubernetes/integration-tests/kerberos-yml/data-populator-deployment.yml b/resource-managers/kubernetes/integration-tests/kerberos-yml/data-populator-deployment.yml new file mode 100644 index 0000000000000..b87f8ceafe6cf --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/data-populator-deployment.yml @@ -0,0 +1,30 @@ +apiVersion: extensions/v1beta1 +kind: Deployment +metadata: + name: data-populator +spec: + replicas: 1 + template: + metadata: + annotations: + pod.beta.kubernetes.io/hostname: data-populator + labels: + name: hdfs-data-populator + kerberosService: data-populator + job: kerberostest + spec: + containers: + - command: + - /populate-data.sh + name: data-populator + image: ifilonenko/hadoop-base:latest + imagePullPolicy: IfNotPresent + runAsNonRoot: false + volumeMounts: + - mountPath: /var/keytabs + name: data-populator-keytab + restartPolicy: Always + volumes: + - name: data-populator-keytab + persistentVolumeClaim: + claimName: server-keytab \ No newline at end of file diff --git a/resource-managers/kubernetes/integration-tests/kerberos-yml/data-populator-service.yml b/resource-managers/kubernetes/integration-tests/kerberos-yml/data-populator-service.yml new file mode 100644 index 0000000000000..45b4bfb4bfeb5 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/data-populator-service.yml @@ -0,0 +1,17 @@ +apiVersion: v1 +kind: Service +metadata: + annotations: + service.alpha.kubernetes.io/tolerate-unready-endpoints: "true" + labels: + kerberosService: data-populator + job: kerberostest + name: data-populator +spec: + clusterIP: None + ports: + - protocol: TCP + port: 55555 + targetPort: 0 + selector: + kerberosService: data-populator \ No newline at end of file diff --git a/resource-managers/kubernetes/integration-tests/kerberos-yml/dn1-deployment.yml b/resource-managers/kubernetes/integration-tests/kerberos-yml/dn1-deployment.yml new file mode 100644 index 0000000000000..9f028d31e4424 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/dn1-deployment.yml @@ -0,0 +1,30 @@ +apiVersion: extensions/v1beta1 +kind: Deployment +metadata: + name: dn1 +spec: + replicas: 1 + template: + metadata: + annotations: + pod.beta.kubernetes.io/hostname: dn1 + labels: + name: hdfs-dn1 + kerberosService: dn1 + job: kerberostest + spec: + containers: + - command: + - /start-datanode.sh + name: dn1 + image: ifilonenko/hadoop-base:latest + imagePullPolicy: IfNotPresent + runAsNonRoot: false + volumeMounts: + - mountPath: /var/keytabs + name: dn1-keytab + restartPolicy: Always + volumes: + - name: dn1-keytab + persistentVolumeClaim: + claimName: server-keytab \ No newline at end of file diff --git a/resource-managers/kubernetes/integration-tests/kerberos-yml/dn1-service.yml b/resource-managers/kubernetes/integration-tests/kerberos-yml/dn1-service.yml new file mode 100644 index 0000000000000..ee258c87942f8 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/dn1-service.yml @@ -0,0 +1,17 @@ +apiVersion: v1 +kind: Service +metadata: + annotations: + service.alpha.kubernetes.io/tolerate-unready-endpoints: "true" + labels: + kerberosService: dn1 + job: kerberostest + name: dn1 +spec: + clusterIP: None + ports: + - protocol: TCP + port: 55555 + targetPort: 0 + selector: + kerberosService: dn1 \ No newline at end of file diff --git a/resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-deployment.yml b/resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-deployment.yml new file mode 100644 index 0000000000000..6037a3c50b11e --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-deployment.yml @@ -0,0 +1,30 @@ +apiVersion: extensions/v1beta1 +kind: Deployment +metadata: + name: kerberos +spec: + replicas: 1 + template: + metadata: + annotations: + pod.beta.kubernetes.io/hostname: kerberos + labels: + name: hdfs-kerberos + kerberosService: kerberos + job: kerberostest + spec: + containers: + - command: + - /start-kdc.sh + name: kerberos + image: ifilonenko/hadoop-base:latest + imagePullPolicy: IfNotPresent + runAsNonRoot: false + volumeMounts: + - mountPath: /var/keytabs + name: kerb-keytab + restartPolicy: Always + volumes: + - name: kerb-keytab + persistentVolumeClaim: + claimName: server-keytab \ No newline at end of file diff --git a/resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-service.yml b/resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-service.yml new file mode 100644 index 0000000000000..da7b994f6e2af --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-service.yml @@ -0,0 +1,17 @@ +apiVersion: v1 +kind: Service +metadata: + annotations: + service.alpha.kubernetes.io/tolerate-unready-endpoints: "true" + labels: + kerberosService: kerberos + job: kerberostest + name: kerberos +spec: + clusterIP: None + ports: + - protocol: TCP + port: 55555 + targetPort: 0 + selector: + kerberosService: kerberos diff --git a/resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-test.yml b/resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-test.yml new file mode 100644 index 0000000000000..9115b45cc0135 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-test.yml @@ -0,0 +1,25 @@ +apiVersion: extensions/v1beta1 +kind: Deployment +metadata: + name: kerberos-test +spec: + replicas: 1 + template: + metadata: + labels: + name: kerberos-test + spec: + containers: + - command: ["/bin/bash"] + args: ["/opt/spark/test-env.sh"] + name: kerberos-test + image: kerberos-test:latest + imagePullPolicy: IfNotPresent + volumeMounts: + - mountPath: /var/keytabs + name: kerberos-test-keytab + restartPolicy: Always + volumes: + - name: kerberos-test-keytab + persistentVolumeClaim: + claimName: server-keytab \ No newline at end of file diff --git a/resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-test2.yml b/resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-test2.yml new file mode 100644 index 0000000000000..3aaa442bbe3d8 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/kerberos-test2.yml @@ -0,0 +1,25 @@ +apiVersion: extensions/v1beta1 +kind: Deployment +metadata: + name: kerberos-test +spec: + replicas: 1 + template: + metadata: + labels: + name: kerberos-test + spec: + containers: + - command: ["/bin/bash"] + args: ["/opt/spark/test-env2.sh"] + name: kerberos-test + image: kerberos-test:latest + imagePullPolicy: IfNotPresent + volumeMounts: + - mountPath: /var/keytabs + name: kerberos-test-keytab + restartPolicy: Always + volumes: + - name: kerberos-test-keytab + persistentVolumeClaim: + claimName: server-keytab \ No newline at end of file diff --git a/resource-managers/kubernetes/integration-tests/kerberos-yml/nn-deployment.yml b/resource-managers/kubernetes/integration-tests/kerberos-yml/nn-deployment.yml new file mode 100644 index 0000000000000..d2f473aa66c1a --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/nn-deployment.yml @@ -0,0 +1,35 @@ +apiVersion: extensions/v1beta1 +kind: Deployment +metadata: + name: nn +spec: + replicas: 1 + template: + metadata: + annotations: + pod.beta.kubernetes.io/hostname: nn + labels: + name: hdfs-nn + kerberosService: nn + job: kerberostest + spec: + containers: + - command: + - /start-namenode.sh + name: nn + ports: + - containerPort: 9000 + image: ifilonenko/hadoop-base:latest + imagePullPolicy: IfNotPresent + runAsNonRoot: false + volumeMounts: + - mountPath: /var/keytabs + name: nn-keytab + restartPolicy: Always + volumes: + - name: nn-keytab + persistentVolumeClaim: + claimName: server-keytab + - name: nn-hadoop + persistentVolumeClaim: + claimName: nn-hadoop diff --git a/resource-managers/kubernetes/integration-tests/kerberos-yml/nn-hadoop.yml b/resource-managers/kubernetes/integration-tests/kerberos-yml/nn-hadoop.yml new file mode 100644 index 0000000000000..3e72046cd1d8f --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/nn-hadoop.yml @@ -0,0 +1,12 @@ +apiVersion: v1 +kind: PersistentVolumeClaim +metadata: + name: nn-hadoop + labels: + job: kerberostest +spec: + accessModes: + - ReadWriteOnce + resources: + requests: + storage: 100Mi \ No newline at end of file diff --git a/resource-managers/kubernetes/integration-tests/kerberos-yml/nn-service.yml b/resource-managers/kubernetes/integration-tests/kerberos-yml/nn-service.yml new file mode 100644 index 0000000000000..649302150aa39 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/nn-service.yml @@ -0,0 +1,17 @@ +apiVersion: v1 +kind: Service +metadata: + annotations: + service.alpha.kubernetes.io/tolerate-unready-endpoints: "true" + labels: + kerberosService: nn + job: kerberostest + name: nn +spec: + clusterIP: None + ports: + - protocol: TCP + port: 9000 + targetPort: 9000 + selector: + kerberosService: nn diff --git a/resource-managers/kubernetes/integration-tests/kerberos-yml/server-keytab.yml b/resource-managers/kubernetes/integration-tests/kerberos-yml/server-keytab.yml new file mode 100644 index 0000000000000..6c6a1008e1441 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/server-keytab.yml @@ -0,0 +1,12 @@ +apiVersion: v1 +kind: PersistentVolumeClaim +metadata: + name: server-keytab + labels: + job: kerberostest +spec: + accessModes: + - ReadWriteOnce + resources: + requests: + storage: 100Mi \ No newline at end of file diff --git a/resource-managers/kubernetes/integration-tests/kerberos-yml/test-env.sh b/resource-managers/kubernetes/integration-tests/kerberos-yml/test-env.sh new file mode 100644 index 0000000000000..056f0c5936e98 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/kerberos-yml/test-env.sh @@ -0,0 +1,27 @@ +#!/usr/bin/env bash +sed -i -e 's/#//' -e 's/default_ccache_name/# default_ccache_name/' /etc/krb5.conf +export HADOOP_OPTS="-Djava.net.preferIPv4Stack=true -Dsun.security.krb5.debug=true" +export HADOOP_JAAS_DEBUG=true +export HADOOP_ROOT_LOGGER=DEBUG,console +cp ${TMP_KRB_LOC} /etc/krb5.conf +cp ${TMP_CORE_LOC} /opt/spark/hconf/core-site.xml +cp ${TMP_HDFS_LOC} /opt/spark/hconf/hdfs-site.xml +mkdir -p /etc/krb5.conf.d +until /usr/bin/kinit -kt /var/keytabs/hdfs.keytab hdfs/nn.${NAMESPACE}.svc.cluster.local; do sleep 15; done +/opt/spark/bin/spark-submit \ + --deploy-mode cluster \ + --class ${CLASS_NAME} \ + --master k8s://${MASTER_URL} \ + --kubernetes-namespace ${NAMESPACE} \ + --conf spark.executor.instances=1 \ + --conf spark.app.name=spark-hdfs \ + --conf spark.driver.extraClassPath=/opt/spark/hconf/core-site.xml:/opt/spark/hconf/hdfs-site.xml:/opt/spark/hconf/yarn-site.xml:/etc/krb5.conf \ + --conf spark.kubernetes.driver.docker.image=spark-driver:latest \ + --conf spark.kubernetes.executor.docker.image=spark-executor:latest \ + --conf spark.kubernetes.initcontainer.docker.image=spark-init:latest \ + --conf spark.kubernetes.kerberos.enabled=true \ + --conf spark.kubernetes.kerberos.keytab=/var/keytabs/hdfs.keytab \ + --conf spark.kubernetes.kerberos.principal=hdfs/nn.${NAMESPACE}.svc.cluster.local@CLUSTER.LOCAL \ + --conf spark.kubernetes.driver.label.spark-app-locator=${APP_LOCATOR_LABEL} \ + ${SUBMIT_RESOURCE} \ + hdfs://nn.${NAMESPACE}.svc.cluster.local:9000/user/ifilonenko/wordcount.txt \ No newline at end of file diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index fc28026de1e43..51eb884784a3c 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -92,6 +92,7 @@ com.spotify docker-client + 8.8.2 test NN --> DN1 --> Data-Populator + val podWatcherCache = new KerberosPodWatcherCache(kerberosUtils, LABELS) + podWatcherCache.start() + val dpNode = podWatcherCache.stop() + while (!podWatcherCache.hasInLogs(dpNode, "")) { + logInfo("Waiting for data-populator to be formatted") + Thread.sleep(500) + } + } +} diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberosTestPodLauncher.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberosTestPodLauncher.scala new file mode 100644 index 0000000000000..53d372faace41 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberosTestPodLauncher.scala @@ -0,0 +1,121 @@ +/* + * 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.kubernetes.integrationtest + +import java.io.{File, FileInputStream} + +import scala.collection.JavaConverters._ + +import io.fabric8.kubernetes.api.model.KeyToPathBuilder +import io.fabric8.kubernetes.api.model.extensions.{Deployment, DeploymentBuilder} +import io.fabric8.kubernetes.client.KubernetesClient + +import org.apache.spark.deploy.kubernetes.submit.ContainerNameEqualityPredicate + + /** + * This class is responsible for launching a pod that runs spark-submit to simulate + * the necessary global environmental variables and files expected for a Kerberos task. + * In this test we specify HADOOP_CONF_DIR and ensure that for any arbitrary namespace + * the krb5.conf, core-site.xml, and hdfs-site.xml are resolved accordingly. + */ +private[spark] class KerberosTestPodLauncher( + kubernetesClient: KubernetesClient, + namespace: String) { + private val kerberosFiles = Seq("krb5.conf", "core-site.xml", "hdfs-site.xml") + private val KRB_VOLUME = "krb5-conf" + private val KRB_FILE_DIR = "/tmp" + private val KRB_CONFIG_MAP_NAME = "krb-config-map" + private val HADOOP_CONF_DIR_PATH = "/opt/spark/hconf" + private val keyPaths = kerberosFiles.map(file => + new KeyToPathBuilder() + .withKey(file) + .withPath(file) + .build()).toList + def startKerberosTest( + resource: String, + className: String, + appLabel: String, + yamlLocation: String): Unit = { + kubernetesClient.load(new FileInputStream(new File(yamlLocation))) + .get().get(0) match { + case deployment: Deployment => + val deploymentWithEnv: Deployment = new DeploymentBuilder(deployment) + .editSpec() + .editTemplate() + .editSpec() + .addNewVolume() + .withName(KRB_VOLUME) + .withNewConfigMap() + .withName(KRB_CONFIG_MAP_NAME) + .withItems(keyPaths.asJava) + .endConfigMap() + .endVolume() + .editMatchingContainer(new ContainerNameEqualityPredicate( + deployment.getMetadata.getName)) + .addNewEnv() + .withName("NAMESPACE") + .withValue(namespace) + .endEnv() + .addNewEnv() + .withName("MASTER_URL") + .withValue(kubernetesClient.getMasterUrl.toString) + .endEnv() + .addNewEnv() + .withName("SUBMIT_RESOURCE") + .withValue(resource) + .endEnv() + .addNewEnv() + .withName("CLASS_NAME") + .withValue(className) + .endEnv() + .addNewEnv() + .withName("HADOOP_CONF_DIR") + .withValue(HADOOP_CONF_DIR_PATH) + .endEnv() + .addNewEnv() + .withName("APP_LOCATOR_LABEL") + .withValue(appLabel) + .endEnv() + .addNewEnv() + .withName("SPARK_PRINT_LAUNCH_COMMAND") + .withValue("true") + .endEnv() + .addNewEnv() + .withName("TMP_KRB_LOC") + .withValue(s"$KRB_FILE_DIR/${kerberosFiles.head}") + .endEnv() + .addNewEnv() + .withName("TMP_CORE_LOC") + .withValue(s"$KRB_FILE_DIR/${kerberosFiles(1)}") + .endEnv() + .addNewEnv() + .withName("TMP_HDFS_LOC") + .withValue(s"$KRB_FILE_DIR/${kerberosFiles(2)}") + .endEnv() + .addNewVolumeMount() + .withName(KRB_VOLUME) + .withMountPath(KRB_FILE_DIR) + .endVolumeMount() + .endContainer() + .endSpec() + .endTemplate() + .endSpec() + .build() + kubernetesClient.extensions().deployments() + .inNamespace(namespace).create(deploymentWithEnv)} + } +} diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala index e204d0173aff8..17d48b8b508dd 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala @@ -34,6 +34,7 @@ import org.apache.spark.deploy.kubernetes.config._ import org.apache.spark.deploy.kubernetes.integrationtest.backend.IntegrationTestBackendFactory import org.apache.spark.deploy.kubernetes.integrationtest.backend.minikube.Minikube import org.apache.spark.deploy.kubernetes.integrationtest.constants.MINIKUBE_TEST_BACKEND +import org.apache.spark.deploy.kubernetes.integrationtest.kerberos.KerberosDriverWatcherCache import org.apache.spark.deploy.kubernetes.submit.{Client, ClientArguments, JavaMainAppResource, KeyAndCertPem, MainAppResource, PythonMainAppResource} import org.apache.spark.launcher.SparkLauncher import org.apache.spark.util.Utils @@ -46,6 +47,8 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { private var sparkConf: SparkConf = _ private var resourceStagingServerLauncher: ResourceStagingServerLauncher = _ private var staticAssetServerLauncher: StaticAssetServerLauncher = _ + private var kerberizedHadoopClusterLauncher: KerberizedHadoopClusterLauncher = _ + private var kerberosTestLauncher: KerberosTestPodLauncher = _ override def beforeAll(): Unit = { testBackend.initialize() @@ -54,6 +57,12 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { kubernetesTestComponents.kubernetesClient.inNamespace(kubernetesTestComponents.namespace)) staticAssetServerLauncher = new StaticAssetServerLauncher( kubernetesTestComponents.kubernetesClient.inNamespace(kubernetesTestComponents.namespace)) + kerberizedHadoopClusterLauncher = new KerberizedHadoopClusterLauncher( + kubernetesTestComponents.kubernetesClient.inNamespace(kubernetesTestComponents.namespace), + kubernetesTestComponents.namespace) + kerberosTestLauncher = new KerberosTestPodLauncher( + kubernetesTestComponents.kubernetesClient.inNamespace(kubernetesTestComponents.namespace), + kubernetesTestComponents.namespace) } override def afterAll(): Unit = { @@ -69,12 +78,59 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { } after { + kubernetesTestComponents.deleteKubernetesPVs() kubernetesTestComponents.deleteNamespace() } - test("Run PySpark Job on file from SUBMITTER with --py-files") { + test("Include HADOOP_CONF for HDFS based jobs") { assume(testBackend.name == MINIKUBE_TEST_BACKEND) + // Ensuring that HADOOP_CONF_DIR variable is set + sparkConf.setJars(Seq(CONTAINER_LOCAL_HELPER_JAR_PATH)) + runSparkApplicationAndVerifyCompletion( + JavaMainAppResource(CONTAINER_LOCAL_MAIN_APP_RESOURCE), + SPARK_PI_MAIN_CLASS, + Seq("HADOOP_CONF_DIR defined. Mounting HDFS specific .xml files", "Pi is roughly 3"), + Array("5"), + Seq.empty[String], + Some("test-data/simple-hadoop-conf")) + } + test("Secure HDFS test with HDFS keytab") { + assume(testBackend.name == MINIKUBE_TEST_BACKEND) + launchKerberizedCluster() + createKerberosTestPod( + CONTAINER_LOCAL_MAIN_APP_RESOURCE, + HDFS_TEST_CLASS, + APP_LOCATOR_LABEL, + "kerberos-yml/kerberos-test.yml") + val kubernetesClient = kubernetesTestComponents.kubernetesClient + val driverWatcherCache = new KerberosDriverWatcherCache( + kubernetesClient, + Map("spark-app-locator" -> APP_LOCATOR_LABEL)) + driverWatcherCache.start() + driverWatcherCache.stop() + val expectedLogOnCompletion = Seq( + "Returned length(s) of: 1", + "File contents: [This is an awesome word count file]") + val driverPod = kubernetesClient + .pods() + .withLabel("spark-app-locator", APP_LOCATOR_LABEL) + .list() + .getItems + .get(0) + Eventually.eventually(TIMEOUT, INTERVAL) { + expectedLogOnCompletion.foreach { e => + assert(kubernetesClient + .pods() + .withName(driverPod.getMetadata.getName) + .getLog + .contains(e), "The application did not complete.") + } + } + } + + test("Run PySpark Job on file from SUBMITTER with --py-files") { + assume(testBackend.name == MINIKUBE_TEST_BACKEND) launchStagingServer(SSLOptions(), None) sparkConf .set(DRIVER_DOCKER_IMAGE, @@ -103,7 +159,6 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { test("Simple submission test with the resource staging server.") { assume(testBackend.name == MINIKUBE_TEST_BACKEND) - launchStagingServer(SSLOptions(), None) runSparkPiAndVerifyCompletion(SUBMITTER_LOCAL_MAIN_APP_RESOURCE) } @@ -159,7 +214,8 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { GROUP_BY_MAIN_CLASS, Seq("The Result is"), Array.empty[String], - Seq.empty[String]) + Seq.empty[String], + None) } test("Use remote resources without the resource staging server.") { @@ -226,7 +282,8 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { s"File found on the executors at the relative path ${testExistenceFile.getName} with" + s" the correct contents."), Array(testExistenceFile.getName, TEST_EXISTENCE_FILE_CONTENTS), - Seq.empty[String]) + Seq.empty[String], + None) } test("Setting JVM options on the driver and executors with spaces.") { @@ -256,7 +313,8 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { JAVA_OPTIONS_MAIN_CLASS, Seq(s"All expected JVM options were present on the driver and executors."), Array(driverJvmOptionsFile.getName, executorJvmOptionsFile.getName), - Seq.empty[String]) + Seq.empty[String], + None) } test("Submit small local files without the resource staging server.") { @@ -274,7 +332,8 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { s"File found on the executors at the relative path ${testExistenceFile.getName} with" + s" the correct contents."), Array(testExistenceFile.getName, TEST_EXISTENCE_FILE_CONTENTS), - Seq.empty[String]) + Seq.empty[String], + None) } test("Use a very long application name.") { @@ -297,16 +356,33 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { } sparkConf.set(RESOURCE_STAGING_SERVER_URI, s"$resourceStagingServerUriScheme://" + - s"${Minikube.getMinikubeIp}:$resourceStagingServerPort") + s"${Minikube.getMinikubeIp}:$resourceStagingServerPort").set(KERBEROS_CONF, "simple") + } + + private def launchKerberizedCluster(): Unit = { + assume(testBackend.name == MINIKUBE_TEST_BACKEND) + kerberizedHadoopClusterLauncher.launchKerberizedCluster() + } + + private def createKerberosTestPod( + resource: String, + className: String, + appLabel: String, + yamlLocation: String): Unit = { + assume(testBackend.name == MINIKUBE_TEST_BACKEND) + kerberosTestLauncher.startKerberosTest(resource, className, appLabel, yamlLocation) } private def runSparkPiAndVerifyCompletion(appResource: String): Unit = { runSparkApplicationAndVerifyCompletion( JavaMainAppResource(appResource), SPARK_PI_MAIN_CLASS, - Seq("Pi is roughly 3"), + Seq( + "hadoop config map key was not specified", + "Pi is roughly 3"), Array.empty[String], - Seq.empty[String]) + Seq.empty[String], + None) } private def runPySparkPiAndVerifyCompletion( @@ -316,7 +392,8 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { PYSPARK_PI_MAIN_CLASS, Seq("Submitting 5 missing tasks from ResultStage", "Pi is roughly 3"), Array("5"), - otherPyFiles) + otherPyFiles, + None) } private def runSparkApplicationAndVerifyCompletion( @@ -324,13 +401,14 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { mainClass: String, expectedLogOnCompletion: Seq[String], appArgs: Array[String], - otherPyFiles: Seq[String]): Unit = { + otherPyFiles: Seq[String], + hadoopConfDir: Option[String]): Unit = { val clientArguments = ClientArguments( mainAppResource = appResource, mainClass = mainClass, driverArgs = appArgs, otherPyFiles = otherPyFiles) - Client.run(sparkConf, clientArguments) + Client.run(sparkConf, clientArguments, hadoopConfDir) val driverPod = kubernetesTestComponents.kubernetesClient .pods() .withLabel("spark-app-locator", APP_LOCATOR_LABEL) @@ -419,8 +497,8 @@ private[spark] object KubernetesSuite { s"integration-tests-jars/${EXAMPLES_JAR_FILE.getName}" val CONTAINER_LOCAL_HELPER_JAR_PATH = s"local:///opt/spark/examples/" + s"integration-tests-jars/${HELPER_JAR_FILE.getName}" - val TIMEOUT = PatienceConfiguration.Timeout(Span(2, Minutes)) - val INTERVAL = PatienceConfiguration.Interval(Span(2, Seconds)) + val TIMEOUT = PatienceConfiguration.Timeout(Span(10, Minutes)) + val INTERVAL = PatienceConfiguration.Interval(Span(10, Seconds)) val SPARK_PI_MAIN_CLASS = "org.apache.spark.deploy.kubernetes" + ".integrationtest.jobs.SparkPiWithInfiniteWait" val PYSPARK_PI_MAIN_CLASS = "org.apache.spark.deploy.PythonRunner" @@ -433,9 +511,13 @@ private[spark] object KubernetesSuite { ".integrationtest.jobs.FileExistenceTest" val GROUP_BY_MAIN_CLASS = "org.apache.spark.deploy.kubernetes" + ".integrationtest.jobs.GroupByTest" + val HDFS_TEST_CLASS = "org.apache.spark.deploy.kubernetes" + + ".integrationtest.jobs.HDFSTest" val JAVA_OPTIONS_MAIN_CLASS = "org.apache.spark.deploy.kubernetes" + ".integrationtest.jobs.JavaOptionsTest" val TEST_EXISTENCE_FILE_CONTENTS = "contents" + val KERBEROS_CONF = "spark.hadoop.hadoop.security.authentication" + case object ShuffleNotReadyException extends Exception } diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesTestComponents.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesTestComponents.scala index 0ca1f482269db..c04bb0c525d86 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesTestComponents.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesTestComponents.scala @@ -38,7 +38,21 @@ private[spark] class KubernetesTestComponents(defaultClient: DefaultKubernetesCl .endMetadata() .done() } - + def deleteKubernetesPVs(): Unit = { + // Temporary hack until client library for fabric8 is updated to get around + // the NPE that comes about when I do .list().getItems().asScala + try { + val pvList = kubernetesClient.persistentVolumes().list().getItems().asScala + if (pvList.nonEmpty) { + kubernetesClient.persistentVolumes().delete() + Eventually.eventually(KubernetesSuite.TIMEOUT, KubernetesSuite.INTERVAL) { + require(!pvList.exists(_.getMetadata.getNamespace == namespace)) + } + } + } catch { + case ex: java.lang.NullPointerException => + } + } def deleteNamespace(): Unit = { defaultClient.namespaces.withName(namespace).delete() Eventually.eventually(KubernetesSuite.TIMEOUT, KubernetesSuite.INTERVAL) { diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/docker/SparkDockerImageBuilder.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/docker/SparkDockerImageBuilder.scala index e240fcf953f8c..a210aab4b78fe 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/docker/SparkDockerImageBuilder.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/docker/SparkDockerImageBuilder.scala @@ -31,7 +31,6 @@ import org.apache.spark.internal.Logging import org.apache.spark.util.RedirectThread - private[spark] class SparkDockerImageBuilder (private val dockerEnv: Map[String, String]) extends Logging{ @@ -47,6 +46,7 @@ private[spark] class SparkDockerImageBuilder private val STAGING_SERVER_DOCKER_FILE = "dockerfiles/resource-staging-server/Dockerfile" private val STATIC_ASSET_SERVER_DOCKER_FILE = "dockerfiles/integration-test-asset-server/Dockerfile" + private val KERBEROS_DOCKER_FILE = "dockerfiles/kerberos-test/Dockerfile" private val TIMEOUT = PatienceConfiguration.Timeout(Span(2, Minutes)) private val INTERVAL = PatienceConfiguration.Interval(Span(2, Seconds)) private val dockerHost = dockerEnv.getOrElse("DOCKER_HOST", @@ -95,6 +95,7 @@ private[spark] class SparkDockerImageBuilder buildImage("spark-resource-staging-server", STAGING_SERVER_DOCKER_FILE) buildImage("spark-init", INIT_CONTAINER_DOCKER_FILE) buildImage("spark-integration-test-asset-server", STATIC_ASSET_SERVER_DOCKER_FILE) + buildImage("kerberos-test", KERBEROS_DOCKER_FILE) } private def buildImage(name: String, dockerFile: String): Unit = { @@ -103,5 +104,6 @@ private[spark] class SparkDockerImageBuilder name, dockerFile, new LoggingBuildHandler()) + logInfo(s"Built docker image for $name") } } diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosCMWatcherCache.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosCMWatcherCache.scala new file mode 100644 index 0000000000000..59968534c8312 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosCMWatcherCache.scala @@ -0,0 +1,104 @@ +/* + * 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.kubernetes.integrationtest.kerberos + +import java.util.concurrent.locks.{Condition, Lock, ReentrantLock} + +import scala.collection.JavaConverters._ + +import io.fabric8.kubernetes.api.model.ConfigMap +import io.fabric8.kubernetes.client.{KubernetesClientException, Watch, Watcher} +import io.fabric8.kubernetes.client.Watcher.Action + +import org.apache.spark.internal.Logging + + /** + * This class is responsible for ensuring that no logic progresses in the cluster launcher + * until a configmap with the HADOOP_CONF_DIR specifications has been created. + */ +private[spark] class KerberosCMWatcherCache(kerberosUtils: KerberosUtils) extends Logging { + private val kubernetesClient = kerberosUtils.getClient + private val namespace = kerberosUtils.getNamespace + private val requiredFiles = Seq("core-site.xml", "hdfs-site.xml", "krb5.conf") + private var watcher: Watch = _ + private var cmCache = scala.collection.mutable.Map[String, Map[String, String]]() + private var lock: Lock = new ReentrantLock() + private var cmCreated: Condition = lock.newCondition() + private val configMap = kerberosUtils.getConfigMap + private val configMapName = configMap.getMetadata.getName + private val blockingThread = new Thread(new Runnable { + override def run(): Unit = { + logInfo("Beginning of ConfigMap lock") + lock.lock() + try { + while (!created()) cmCreated.await() + } finally { + logInfo("Ending the ConfigMap lock") + lock.unlock() + stop() + } + }}) + + private val watcherThread = new Thread(new Runnable { + override def run(): Unit = { + logInfo("Beginning the watch of the Kerberos Config Map") + watcher = kubernetesClient + .configMaps() + .withName(configMapName) + .watch(new Watcher[ConfigMap] { + override def onClose(cause: KubernetesClientException): Unit = + logInfo("Ending the watch of Kerberos Config Map") + override def eventReceived(action: Watcher.Action, resource: ConfigMap): Unit = { + val name = resource.getMetadata.getName + action match { + case Action.DELETED | Action.ERROR => + logInfo(s"$name either deleted or error") + cmCache.remove(name) + case Action.ADDED | Action.MODIFIED => + val data = resource.getData.asScala.toMap + logInfo(s"$name includes ${data.keys.mkString(",")}") + cmCache(name) = data + if (created()) { + lock.lock() + try { + cmCreated.signalAll() + } finally { + lock.unlock() + } + } + }}} + ) + logInfo("Launching the Config Map") + kerberosUtils.getClient.configMaps().inNamespace(namespace).createOrReplace(configMap) + }}) + + def start(): Unit = { + blockingThread.start() + watcherThread.start() + blockingThread.join() + watcherThread.join()} + + def stop(): Unit = { + watcher.close() + } + + def created(): Boolean = { + cmCache.get(configMapName).exists{ data => + requiredFiles.forall(data.keys.toSeq.contains)} + } +} diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosDeployment.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosDeployment.scala new file mode 100644 index 0000000000000..a60011153f5d5 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosDeployment.scala @@ -0,0 +1,24 @@ +/* + * 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.kubernetes.integrationtest.kerberos + +import io.fabric8.kubernetes.api.model.Service +import io.fabric8.kubernetes.api.model.extensions.Deployment + +private[spark] case class KerberosDeployment( + podDeployment: Deployment, + service: Service) diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosDriverWatcherCache.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosDriverWatcherCache.scala new file mode 100644 index 0000000000000..b7a2176194e2c --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosDriverWatcherCache.scala @@ -0,0 +1,99 @@ +/* + * 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.kubernetes.integrationtest.kerberos + +import java.util.concurrent.locks.{Condition, Lock, ReentrantLock} + +import scala.collection.JavaConverters._ + +import io.fabric8.kubernetes.api.model.Pod +import io.fabric8.kubernetes.client.{KubernetesClient, KubernetesClientException, Watch, Watcher} +import io.fabric8.kubernetes.client.Watcher.Action + +import org.apache.spark.internal.Logging + + /** + * This class is responsible for ensuring that the driver-pod launched by the KerberosTestPod + * is running before trying to grab its logs for the sake of monitoring success of completition. + */ +private[spark] class KerberosDriverWatcherCache( + kubernetesClient: KubernetesClient, + labels: Map[String, String]) extends Logging { + private var podWatcher: Watch = _ + private var podCache = + scala.collection.mutable.Map[String, String]() + private var lock: Lock = new ReentrantLock() + private var driverRunning: Condition = lock.newCondition() + private var driverIsUp: Boolean = false + private val blockingThread = new Thread(new Runnable { + override def run(): Unit = { + logInfo("Beginning of Driver lock") + lock.lock() + try { + while (!driverIsUp) driverRunning.await() + } finally { + logInfo("Ending the Driver lock") + lock.unlock() + stop() + } + } + }) + + private val podWatcherThread = new Thread(new Runnable { + override def run(): Unit = { + logInfo("Beginning the watch of Driver pod") + podWatcher = kubernetesClient + .pods() + .withLabels(labels.asJava) + .watch(new Watcher[Pod] { + override def onClose(cause: KubernetesClientException): Unit = + logInfo("Ending the watch of Driver pod") + override def eventReceived(action: Watcher.Action, resource: Pod): Unit = { + val name = resource.getMetadata.getName + action match { + case Action.DELETED | Action.ERROR => + logInfo(s"$name either deleted or error") + podCache.remove(name) + case Action.ADDED | Action.MODIFIED => + val phase = resource.getStatus.getPhase + logInfo(s"$name is as $phase") + podCache(name) = phase + if (maybeDriverDone(name)) { + lock.lock() + try { + driverIsUp = true + driverRunning.signalAll() + } finally { + lock.unlock() + } + }}}}) + }}) + + def start(): Unit = { + blockingThread.start() + podWatcherThread.start() + blockingThread.join() + podWatcherThread.join() + } + + def stop(): Unit = { + podWatcher.close() + } + + private def maybeDriverDone(name: String): Boolean = podCache.get(name).contains("Running") +} diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosPVWatcherCache.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosPVWatcherCache.scala new file mode 100644 index 0000000000000..b58aa2e0e532e --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosPVWatcherCache.scala @@ -0,0 +1,184 @@ +/* + * 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.kubernetes.integrationtest.kerberos + +import java.util.concurrent.locks.{Condition, Lock, ReentrantLock} + +import scala.collection.JavaConverters._ + +import io.fabric8.kubernetes.api.model.{PersistentVolume, PersistentVolumeClaim} +import io.fabric8.kubernetes.client.{KubernetesClientException, Watch, Watcher} +import io.fabric8.kubernetes.client.Watcher.Action + +import org.apache.spark.internal.Logging + + /** + * This class is responsible for ensuring that the persistent volume claims are bounded + * to the correct persistent volume and that they are both created before launching the + * pods which expect to use them. + */ +private[spark] class KerberosPVWatcherCache( + kerberosUtils: KerberosUtils, + labels: Map[String, String]) extends Logging { + private val kubernetesClient = kerberosUtils.getClient + private val namespace = kerberosUtils.getNamespace + private var pvWatcher: Watch = _ + private var pvcWatcher: Watch = _ + private var pvCache = + scala.collection.mutable.Map[String, String]() + private var pvcCache = + scala.collection.mutable.Map[String, String]() + private var lock: Lock = new ReentrantLock() + private var nnBounded: Condition = lock.newCondition() + private var ktBounded: Condition = lock.newCondition() + private var nnIsUp: Boolean = false + private var ktIsUp: Boolean = false + private var nnSpawned: Boolean = false + private var ktSpawned: Boolean = false + private val blockingThread = new Thread(new Runnable { + override def run(): Unit = { + logInfo("Beginning of Persistent Storage Lock") + lock.lock() + try { + while (!nnIsUp) nnBounded.await() + while (!ktIsUp) ktBounded.await() + } finally { + logInfo("Ending the Persistent Storage lock") + lock.unlock() + stop() + } + } + }) + private val pvWatcherThread = new Thread(new Runnable { + override def run(): Unit = { + logInfo("Beginning the watch of Persistent Volumes") + pvWatcher = kubernetesClient + .persistentVolumes() + .withLabels(labels.asJava) + .watch(new Watcher[PersistentVolume] { + override def onClose(cause: KubernetesClientException): Unit = + logInfo("Ending the watch of Persistent Volumes", cause) + override def eventReceived(action: Watcher.Action, resource: PersistentVolume): Unit = { + val name = resource.getMetadata.getName + action match { + case Action.DELETED | Action.ERROR => + logInfo(s"$name either deleted or error") + pvCache.remove(name) + case Action.ADDED | Action.MODIFIED => + val phase = resource.getStatus.getPhase + logInfo(s"$name is at stage: $phase") + pvCache(name) = phase + if (maybeDeploymentAndServiceDone(name)) { + val modifyAndSignal: Runnable = new MSThread(name) + new Thread(modifyAndSignal).start() + }}}}) + }}) + private val pvcWatcherThread = new Thread(new Runnable { + override def run(): Unit = { + logInfo("Beginning the watch of Persistent Volume Claims") + pvcWatcher = kubernetesClient + .persistentVolumeClaims() + .withLabels(labels.asJava) + .watch(new Watcher[PersistentVolumeClaim] { + override def onClose(cause: KubernetesClientException): Unit = + logInfo("Ending the watch of Persistent Volume Claims") + override def eventReceived( + action: Watcher.Action, + resource: PersistentVolumeClaim): Unit = { + val name = resource.getMetadata.getName + action match { + case Action.DELETED | Action.ERROR => + logInfo(s"$name either deleted or error") + pvcCache.remove(name) + case Action.ADDED | Action.MODIFIED => + val volumeName = resource.getSpec.getVolumeName + logInfo(s"$name claims itself to $volumeName") + pvcCache(name) = volumeName + if (maybeDeploymentAndServiceDone(name)) { + val modifyAndSignal: Runnable = new MSThread(name) + new Thread(modifyAndSignal).start() + }}}}) + logInfo("Launching the Persistent Storage") + if (!nnSpawned) { + logInfo("Launching the NN Hadoop PV+PVC") + nnSpawned = true + deploy(kerberosUtils.getNNStorage) + } + }}) + + def start(): Unit = { + blockingThread.start() + pvWatcherThread.start() + pvcWatcherThread.start() + blockingThread.join() + pvWatcherThread.join() + pvcWatcherThread.join() + } + def stop(): Unit = { + pvWatcher.close() + pvcWatcher.close() + } + + private def maybeDeploymentAndServiceDone(name: String): Boolean = { + val finished = pvCache.get(name).contains("Bound") && + pvcCache.get(name).contains(name) + if (!finished) { + logInfo(s"$name is not bounded") + if (name == "nn-hadoop") nnIsUp = false + else if (name == "server-keytab") ktIsUp = false + } + finished + } + + private def deploy(kbs: KerberosStorage) : Unit = { + kubernetesClient + .persistentVolumeClaims().inNamespace(namespace).create(kbs.persistentVolumeClaim) + kubernetesClient + .persistentVolumes().create(kbs.persistentVolume) + } + + private class MSThread(name: String) extends Runnable { + override def run(): Unit = { + logInfo(s"$name PV and PVC are bounded") + lock.lock() + if (name == "nn-hadoop") { + nnIsUp = true + logInfo(s"nn-hadoop is bounded") + try { + nnBounded.signalAll() + } finally { + lock.unlock() + } + if (!ktSpawned) { + logInfo("Launching the KT Hadoop PV+PVC") + ktSpawned = true + deploy(kerberosUtils.getKTStorage) + } + } + else if (name == "server-keytab") { + while (!nnIsUp) ktBounded.await() + ktIsUp = true + logInfo(s"server-keytab is bounded") + try { + ktBounded.signalAll() + } finally { + lock.unlock() + } + }} + } +} diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosPodWatcherCache.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosPodWatcherCache.scala new file mode 100644 index 0000000000000..60cc9735212fa --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosPodWatcherCache.scala @@ -0,0 +1,256 @@ +/* + * 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.kubernetes.integrationtest.kerberos + +import java.util.concurrent.locks.{Condition, Lock, ReentrantLock} + +import scala.collection.JavaConverters._ + +import io.fabric8.kubernetes.api.model.{Pod, Service} +import io.fabric8.kubernetes.client.{KubernetesClientException, Watch, Watcher} +import io.fabric8.kubernetes.client.Watcher.Action + +import org.apache.spark.internal.Logging + + /** + * This class is used to ensure that the Hadoop cluster that is launched is executed + * in this order: KDC --> NN --> DN --> Data-Populator and that each one of these nodes + * is running before launching the Kerberos test. + */ +private[spark] class KerberosPodWatcherCache( + kerberosUtils: KerberosUtils, + labels: Map[String, String]) extends Logging { + private val kubernetesClient = kerberosUtils.getClient + private val namespace = kerberosUtils.getNamespace + private var podWatcher: Watch = _ + private var serviceWatcher: Watch = _ + private var podCache = + scala.collection.mutable.Map[String, String]() + private var serviceCache = + scala.collection.mutable.Map[String, String]() + private var lock: Lock = new ReentrantLock() + private var kdcRunning: Condition = lock.newCondition() + private var nnRunning: Condition = lock.newCondition() + private var dnRunning: Condition = lock.newCondition() + private var dpRunning: Condition = lock.newCondition() + private var kdcIsUp: Boolean = false + private var nnIsUp: Boolean = false + private var dnIsUp: Boolean = false + private var dpIsUp: Boolean = false + private var kdcSpawned: Boolean = false + private var nnSpawned: Boolean = false + private var dnSpawned: Boolean = false + private var dpSpawned: Boolean = false + private var dnName: String = _ + private var dpName: String = _ + + private val blockingThread = new Thread(new Runnable { + override def run(): Unit = { + logInfo("Beginning of Cluster lock") + lock.lock() + try { + while (!kdcIsUp) kdcRunning.await() + while (!nnIsUp) nnRunning.await() + while (!dnIsUp) dnRunning.await() + while (!dpIsUp) dpRunning.await() + } finally { + logInfo("Ending the Cluster lock") + lock.unlock() + stop() + } + } + }) + + private val podWatcherThread = new Thread(new Runnable { + override def run(): Unit = { + logInfo("Beginning the watch of Pods") + podWatcher = kubernetesClient + .pods() + .withLabels(labels.asJava) + .watch(new Watcher[Pod] { + override def onClose(cause: KubernetesClientException): Unit = + logInfo("Ending the watch of Pods") + override def eventReceived(action: Watcher.Action, resource: Pod): Unit = { + val name = resource.getMetadata.getName + val keyName = podNameParse(name) + action match { + case Action.DELETED | Action.ERROR => + logInfo(s"$name either deleted or error") + podCache.remove(keyName) + case Action.ADDED | Action.MODIFIED => + val phase = resource.getStatus.getPhase + logInfo(s"$name is as $phase") + if (name.startsWith("dn1")) { dnName = name } + if (name.startsWith("data-populator")) { dpName = name } + podCache(keyName) = phase + if (maybeDeploymentAndServiceDone(keyName)) { + val modifyAndSignal: Runnable = new MSThread(keyName) + new Thread(modifyAndSignal).start() + }}}}) + }}) + + private val serviceWatcherThread = new Thread(new Runnable { + override def run(): Unit = { + logInfo("Beginning the watch of Services") + serviceWatcher = kubernetesClient + .services() + .withLabels(labels.asJava) + .watch(new Watcher[Service] { + override def onClose(cause: KubernetesClientException): Unit = + logInfo("Ending the watch of Services") + override def eventReceived(action: Watcher.Action, resource: Service): Unit = { + val name = resource.getMetadata.getName + action match { + case Action.DELETED | Action.ERROR => + logInfo(s"$name either deleted or error") + serviceCache.remove(name) + case Action.ADDED | Action.MODIFIED => + val bound = resource.getSpec.getSelector.get("kerberosService") + logInfo(s"$name is bounded to $bound") + serviceCache(name) = bound + if (maybeDeploymentAndServiceDone(name)) { + val modifyAndSignal: Runnable = new MSThread(name) + new Thread(modifyAndSignal).start() + }}}}) + logInfo("Launching the Cluster") + if (!kdcSpawned) { + logInfo("Launching the KDC Node") + kdcSpawned = true + deploy(kerberosUtils.getKDC) + } + }}) + + def start(): Unit = { + blockingThread.start() + podWatcherThread.start() + serviceWatcherThread.start() + blockingThread.join() + podWatcherThread.join() + serviceWatcherThread.join() + } + + def stop(): String = { + podWatcher.close() + serviceWatcher.close() + dpName + } + + private def maybeDeploymentAndServiceDone(name: String): Boolean = { + val finished = podCache.get(name).contains("Running") && + serviceCache.get(name).contains(name) + if (!finished) { + logInfo(s"$name is not up with a service") + if (name == "kerberos") kdcIsUp = false + else if (name == "nn") nnIsUp = false + else if (name == "dn1") dnIsUp = false + else if (name == "data-populator") dpIsUp = false + } + finished + } + + private def deploy(kdc: KerberosDeployment) : Unit = { + kubernetesClient + .extensions().deployments().inNamespace(namespace).create(kdc.podDeployment) + kubernetesClient + .services().inNamespace(namespace).create(kdc.service) + } + + private class MSThread(name: String) extends Runnable { + override def run(): Unit = { + logInfo(s"$name Node and Service is up") + lock.lock() + if (name == "kerberos") { + kdcIsUp = true + logInfo(s"kdc has signaled") + try { + kdcRunning.signalAll() + } finally { + lock.unlock() + } + if (!nnSpawned) { + logInfo("Launching the NN Node") + nnSpawned = true + deploy(kerberosUtils.getNN) + } + } + else if (name == "nn") { + while (!kdcIsUp) kdcRunning.await() + nnIsUp = true + logInfo(s"nn has signaled") + try { + nnRunning.signalAll() + } finally { + lock.unlock() + } + if (!dnSpawned) { + logInfo("Launching the DN Node") + dnSpawned = true + deploy(kerberosUtils.getDN) + } + } + else if (name == "dn1") { + while (!kdcIsUp) kdcRunning.await() + while (!nnIsUp) nnRunning.await() + dnIsUp = true + logInfo(s"dn1 has signaled") + try { + dnRunning.signalAll() + } finally { + lock.unlock() + } + if (!dpSpawned) { + logInfo("Launching the DP Node") + dpSpawned = true + deploy(kerberosUtils.getDP) + } + } + else if (name == "data-populator") { + while (!kdcIsUp) kdcRunning.await() + while (!nnIsUp) nnRunning.await() + while (!dnIsUp) dnRunning.await() + while (!hasInLogs(dnName, "Got finalize command for block pool")) { + logInfo("Waiting on DN to be formatted") + Thread.sleep(500) + } + dpIsUp = true + logInfo(s"data-populator has signaled") + try { + dpRunning.signalAll() + } finally { + lock.unlock() + } + } + } + } + + private def podNameParse(name: String) : String = { + name match { + case _ if name.startsWith("kerberos") => "kerberos" + case _ if name.startsWith("nn") => "nn" + case _ if name.startsWith("dn1") => "dn1" + case _ if name.startsWith("data-populator") => "data-populator" + } + } + + def hasInLogs(name: String, expectation: String): Boolean = { + kubernetesClient + .pods() + .withName(name) + .getLog().contains(expectation) + } +} diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosStorage.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosStorage.scala new file mode 100644 index 0000000000000..28fe77c274ea1 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosStorage.scala @@ -0,0 +1,23 @@ +/* + * 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.kubernetes.integrationtest.kerberos + +import io.fabric8.kubernetes.api.model.{PersistentVolume, PersistentVolumeClaim} + +private[spark] case class KerberosStorage( + persistentVolumeClaim: PersistentVolumeClaim, + persistentVolume: PersistentVolume) diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosUtils.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosUtils.scala new file mode 100644 index 0000000000000..145683fdc5eb2 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosUtils.scala @@ -0,0 +1,154 @@ +/* + * 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.kubernetes.integrationtest.kerberos + +import java.io.{File, FileInputStream} + +import scala.collection.JavaConverters._ + +import io.fabric8.kubernetes.api.model._ +import io.fabric8.kubernetes.api.model.extensions.{Deployment, DeploymentBuilder} +import io.fabric8.kubernetes.client.KubernetesClient +import org.apache.commons.io.FileUtils.readFileToString + +import org.apache.spark.deploy.kubernetes.submit.ContainerNameEqualityPredicate + + /** + * This class is responsible for handling all Utils and Constants necessary for testing + */ +private[spark] class KerberosUtils( + kubernetesClient: KubernetesClient, + namespace: String) { + def getClient: KubernetesClient = kubernetesClient + def getNamespace: String = namespace + def yamlLocation(loc: String): String = s"kerberos-yml/$loc.yml" + def loadFromYaml(resource: String): FileInputStream = + new FileInputStream(new File(yamlLocation(resource))) + private val regex = "REPLACE_ME".r + private val regexDP = "# default_ccache_name = MEMORY".r + private val defaultCacheDP = "default_ccache_name = KRBCONF" + private def locationResolver(loc: String) = s"test-data/hadoop-conf/$loc" + private val kerberosFiles = Seq("krb5.conf", "core-site.xml", "hdfs-site.xml") + private val kerberosConfTupList = + kerberosFiles.map { file => + (file, regex.replaceAllIn(readFileToString(new File(locationResolver(file))), namespace))} ++ + Seq(("krb5-dp.conf", regexDP.replaceAllIn(regex.replaceAllIn(readFileToString( + new File(locationResolver("krb5.conf"))), namespace), defaultCacheDP))) + private val KRB_VOLUME = "krb5-conf" + private val KRB_FILE_DIR = "/tmp" + private val KRB_CONFIG_MAP_NAME = "krb-config-map" + private val PV_LABELS = Map("job" -> "kerberostest") + private val keyPaths: Seq[KeyToPath] = (kerberosFiles ++ Seq("krb5-dp.conf")) + .map(file => + new KeyToPathBuilder() + .withKey(file) + .withPath(file) + .build()).toList + private def createPVTemplate(name: String, pathType: String) : PersistentVolume = + new PersistentVolumeBuilder() + .withNewMetadata() + .withName(name) + .withLabels(Map( + "type" -> "local", + "job" -> "kerberostest").asJava) + .endMetadata() + .withNewSpec() + .withCapacity(Map("storage" -> new Quantity("1Gi")).asJava) + .withAccessModes("ReadWriteOnce") + .withHostPath( + new HostPathVolumeSource(s"/tmp/$namespace/$pathType")) + .endSpec() + .build() + private val pvNN = "nn-hadoop" + private val pvKT = "server-keytab" + private val persistentVolumeMap: Map[String, PersistentVolume] = Map( + pvNN -> createPVTemplate(pvNN, "nn"), + pvKT -> createPVTemplate(pvKT, "keytab")) + private def buildKerberosPV(pvType: String) = { + KerberosStorage( + kubernetesClient.load(loadFromYaml(pvType)) + .get().get(0).asInstanceOf[PersistentVolumeClaim], + persistentVolumeMap(pvType)) + } + def getNNStorage: KerberosStorage = buildKerberosPV(pvNN) + def getKTStorage: KerberosStorage = buildKerberosPV(pvKT) + def getLabels: Map[String, String] = PV_LABELS + def getKeyPaths: Seq[KeyToPath] = keyPaths + def getConfigMap: ConfigMap = new ConfigMapBuilder() + .withNewMetadata() + .withName(KRB_CONFIG_MAP_NAME) + .endMetadata() + .addToData(kerberosConfTupList.toMap.asJava) + .build() + private val kdcNode = Seq("kerberos-deployment", "kerberos-service") + private val nnNode = Seq("nn-deployment", "nn-service") + private val dnNode = Seq("dn1-deployment", "dn1-service") + private val dataPopulator = Seq("data-populator-deployment", "data-populator-service") + private def buildKerberosDeployment(seqPair: Seq[String]) = { + val deployment = + kubernetesClient.load(loadFromYaml(seqPair.head)).get().get(0).asInstanceOf[Deployment] + KerberosDeployment( + new DeploymentBuilder(deployment) + .editSpec() + .editTemplate() + .editSpec() + .addNewVolume() + .withName(KRB_VOLUME) + .withNewConfigMap() + .withName(KRB_CONFIG_MAP_NAME) + .withItems(keyPaths.asJava) + .endConfigMap() + .endVolume() + .editMatchingContainer(new ContainerNameEqualityPredicate( + deployment.getMetadata.getName)) + .addNewEnv() + .withName("NAMESPACE") + .withValue(namespace) + .endEnv() + .addNewEnv() + .withName("TMP_KRB_LOC") + .withValue(s"$KRB_FILE_DIR/${kerberosFiles.head}") + .endEnv() + .addNewEnv() + .withName("TMP_KRB_DP_LOC") + .withValue(s"$KRB_FILE_DIR/krb5-dp.conf") + .endEnv() + .addNewEnv() + .withName("TMP_CORE_LOC") + .withValue(s"$KRB_FILE_DIR/${kerberosFiles(1)}") + .endEnv() + .addNewEnv() + .withName("TMP_HDFS_LOC") + .withValue(s"$KRB_FILE_DIR/${kerberosFiles(2)}") + .endEnv() + .addNewVolumeMount() + .withName(KRB_VOLUME) + .withMountPath(KRB_FILE_DIR) + .endVolumeMount() + .endContainer() + .endSpec() + .endTemplate() + .endSpec() + .build(), + kubernetesClient.load(loadFromYaml(seqPair(1))).get().get(0).asInstanceOf[Service] + ) + } + def getKDC: KerberosDeployment = buildKerberosDeployment(kdcNode) + def getNN: KerberosDeployment = buildKerberosDeployment(nnNode) + def getDN: KerberosDeployment = buildKerberosDeployment(dnNode) + def getDP: KerberosDeployment = buildKerberosDeployment(dataPopulator) +} diff --git a/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf/core-site.xml b/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf/core-site.xml new file mode 100644 index 0000000000000..9e168812052ea --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf/core-site.xml @@ -0,0 +1,38 @@ + + + + + + + + + hadoop.security.authentication + kerberos + + + + hadoop.security.authorization + true + + + + fs.defaultFS + hdfs://nn.REPLACE_ME.svc.cluster.local:9000 + + + hadoop.rpc.protection + authentication + + \ No newline at end of file diff --git a/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf/hdfs-site.xml b/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf/hdfs-site.xml new file mode 100644 index 0000000000000..66dc969c46b63 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf/hdfs-site.xml @@ -0,0 +1,157 @@ + + + + + + + + + + dfs.replication + 1 + + + + + dfs.permissions + true + + + dfs.block.access.token.enable + true + + + + + dfs.namenode.keytab.file + /var/keytabs/hdfs.keytab + + + dfs.namenode.kerberos.principal + hdfs/nn.REPLACE_ME.svc.cluster.local@CLUSTER.LOCAL + + + dfs.namenode.kerberos.internal.spnego.principal + HTTP/nn.REPLACE_ME.svc.cluster.local@CLUSTER.LOCAL + + + dfs.namenode.rpc-address + nn.REPLACE_ME.svc.cluster.local:9000 + + + + + + dfs.namenode.delegation.token.max-lifetime + 3600000 + + + dfs.namenode.delegation.token.renew-interval + 3600000 + + + + + + + dfs.data.transfer.protection + integrity + + + dfs.datanode.address + 0.0.0.0:10019 + + + + dfs.datanode.http.address + 0.0.0.0:10022 + + + + dfs.http.policy + HTTPS_ONLY + + + + + dfs.namenode.keytab.file + /var/keytabs/hdfs.keytab + + + dfs.namenode.kerberos.principal + hdfs/nn.REPLACE_ME.svc.cluster.local@CLUSTER.LOCAL + + + dfs.namenode.kerberos.internal.spnego.principal + HTTP/nn.REPLACE_ME.svc.cluster.local@CLUSTER.LOCAL + + + + + dfs.namenode.datanode.registration.ip-hostname-check + false + + + dfs.datanode.data.dir.perm + 700 + + + dfs.namenode.name.dir + file:///hadoop/etc/data + + + dfs.datanode.name.dir + file:///hadoop/etc/data + + + dfs.data.dir + file:///hadoop/etc/data + + + dfs.datanode.keytab.file + /var/keytabs/hdfs.keytab + + + dfs.datanode.kerberos.principal + hdfs/dn1.REPLACE_ME.svc.cluster.local@CLUSTER.LOCAL + + + dfs.encrypt.data.transfer + true + + + dfs.encrypt.data.transfer.cipher.suites + AES/CTR/NoPadding + + + dfs.encrypt.data.transfer.cipher.key.bitlength + 256 + + + + + dfs.webhdfs.enabled + true + + + dfs.web.authentication.kerberos.principal + HTTP/dn1.REPLACE_ME.svc.cluster.local@CLUSTER.LOCAL + + + dfs.web.authentication.kerberos.keytab + /var/keytabs/hdfs.keytab + + + diff --git a/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf/krb5.conf b/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf/krb5.conf new file mode 100644 index 0000000000000..144f77d8995df --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf/krb5.conf @@ -0,0 +1,25 @@ +includedir /etc/krb5.conf.d/ + +[logging] +default = FILE:/var/log/krb5libs.log +kdc = FILE:/var/log/krb5kdc.log +admin_server = FILE:/var/log/kadmind.log + +[libdefaults] +dns_lookup_realm = false +ticket_lifetime = 24h +renew_lifetime = 7d +forwardable = true +rdns = false +default_realm = CLUSTER.LOCAL +# default_ccache_name = MEMORY + +[realms] +CLUSTER.LOCAL = { + kdc = kerberos.REPLACE_ME.svc.cluster.local + admin_server = kerberos.REPLACE_ME.svc.cluster.local +} + +[domain_realm] +.cluster.local = CLUSTER.LOCAL +cluster.local = CLUSTER.LOCAL diff --git a/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf/yarn-site.xml b/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf/yarn-site.xml new file mode 100644 index 0000000000000..92d9346232c76 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/test-data/hadoop-conf/yarn-site.xml @@ -0,0 +1,26 @@ + + + + + + + + + + + yarn.resourcemanager.principal + yarn/_HOST@CLUSTER.LOCAL + + \ No newline at end of file diff --git a/resource-managers/kubernetes/integration-tests/test-data/simple-hadoop-conf/core-site.xml b/resource-managers/kubernetes/integration-tests/test-data/simple-hadoop-conf/core-site.xml new file mode 100644 index 0000000000000..67975bd180c26 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/test-data/simple-hadoop-conf/core-site.xml @@ -0,0 +1,24 @@ + + + + + + + + + fs.defaultFS + hdfs://nn.REPLACE_ME.svc.cluster.local:9000 + + \ No newline at end of file diff --git a/resource-managers/kubernetes/integration-tests/test-data/simple-hadoop-conf/hdfs-site.xml b/resource-managers/kubernetes/integration-tests/test-data/simple-hadoop-conf/hdfs-site.xml new file mode 100644 index 0000000000000..76fc9c68fa375 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/test-data/simple-hadoop-conf/hdfs-site.xml @@ -0,0 +1,24 @@ + + + + + + + + + dfs.replication + 1 + + From 1b4b0eba5ac543fa53ff4045885b2122b4a4c4ad Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Thu, 28 Sep 2017 00:44:11 -0400 Subject: [PATCH 7/7] rename --- ...apache.spark.scheduler.ExternalClusterManager | 2 +- .../{kubernetes => k8s}/CompressionUtils.scala | 2 +- .../{kubernetes => k8s}/ConfigurationUtils.scala | 2 +- .../HadoopConfBootstrap.scala | 4 ++-- .../{kubernetes => k8s}/HadoopUGIUtil.scala | 2 +- ...tainerResourceStagingServerSecretPlugin.scala | 4 ++-- .../KerberosTokenConfBootstrap.scala | 4 ++-- .../KubernetesCredentials.scala | 2 +- .../KubernetesExternalShuffleService.scala | 6 +++--- .../{kubernetes => k8s}/OptionRequirements.scala | 2 +- .../PodWithDetachedInitContainer.scala | 2 +- .../PodWithMainContainer.scala | 2 +- .../SparkKubernetesClientFactory.scala | 4 ++-- .../SparkPodInitContainerBootstrap.scala | 4 ++-- .../deploy/{kubernetes => k8s}/config.scala | 4 ++-- .../deploy/{kubernetes => k8s}/constants.scala | 2 +- .../{kubernetes => k8s}/submit/Client.scala | 10 +++++----- .../submit/ContainerNameEqualityPredicate.scala | 2 +- .../DriverConfigurationStepsOrchestrator.scala | 14 +++++++------- .../submit/InitContainerUtil.scala | 4 ++-- .../submit/KubernetesFileUtils.scala | 2 +- .../submit/LoggingPodStatusWatcher.scala | 6 +++--- .../submit/MainAppResource.scala | 2 +- .../submit/MountSmallFilesBootstrap.scala | 4 ++-- .../PropertiesConfigMapFromScalaMapBuilder.scala | 2 +- .../submit/SubmittedDependencyUploaderImpl.scala | 6 +++--- .../submit/SubmittedResources.scala | 2 +- .../BaseDriverConfigurationStep.scala | 8 ++++---- .../submitsteps/DependencyResolutionStep.scala | 8 ++++---- .../submitsteps/DriverConfigurationStep.scala | 2 +- .../DriverKubernetesCredentialsStep.scala | 6 +++--- .../submitsteps/HadoopConfigBootstrapStep.scala | 6 +++--- .../submitsteps/InitContainerBootstrapStep.scala | 8 ++++---- .../submitsteps/KubernetesDriverSpec.scala | 2 +- .../submitsteps/MountSmallLocalFilesStep.scala | 6 +++--- .../submit/submitsteps/PythonStep.scala | 6 +++--- .../hadoopsteps/HadoopConfMounterStep.scala | 6 +++--- .../hadoopsteps/HadoopConfigSpec.scala | 2 +- .../hadoopsteps/HadoopConfigurationStep.scala | 2 +- .../HadoopKerberosKeytabResolverStep.scala | 6 +++--- .../HadoopKerberosSecretResolverStep.scala | 6 +++--- .../hadoopsteps/HadoopStepsOrchestrator.scala | 6 +++--- .../BaseInitContainerConfigurationStep.scala | 8 ++++---- .../InitContainerConfigurationStep.scala | 2 +- ...ContainerConfigurationStepsOrchestrator.scala | 12 ++++++------ .../initcontainer/InitContainerSpec.scala | 2 +- ...ResourcesInitContainerConfigurationStep.scala | 10 +++++----- .../rest/{kubernetes => k8s}/FileFetcher.scala | 2 +- ...tesSparkDependencyDownloadInitContainer.scala | 6 +++--- .../PemsToKeyStoreConverter.scala | 2 +- .../ResourceStagingServer.scala | 6 +++--- ...ResourceStagingServerSslOptionsProvider.scala | 6 +++--- .../ResourceStagingService.scala | 6 +++--- .../ResourceStagingServiceImpl.scala | 6 +++--- .../ResourceStagingServiceRetrofit.scala | 4 ++-- .../RetrofitClientFactory.scala | 2 +- .../SparkConfPropertiesParser.scala | 2 +- .../{kubernetes => k8s}/StagedResources.scala | 2 +- .../StagedResourcesCleaner.scala | 2 +- .../StagedResourcesOwner.scala | 2 +- .../StagedResourcesStore.scala | 2 +- .../{kubernetes => k8s}/InetAddressUtil.scala | 6 +++--- .../KubernetesClusterManager.scala | 10 +++++----- .../KubernetesClusterSchedulerBackend.scala | 10 +++++----- .../KubernetesTaskSchedulerImpl.scala | 4 ++-- .../KubernetesTaskSetManager.scala | 4 ++-- .../{kubernetes => k8s}/RackResolverUtil.scala | 6 +++--- .../{kubernetes => k8s}/ShufflePodCache.scala | 2 +- ...rResourceStagingServerSecretPluginSuite.scala | 4 ++-- .../deploy/{kubernetes => k8s}/SSLUtils.scala | 4 ++-- .../SparkPodInitContainerBootstrapSuite.scala | 4 ++-- .../{kubernetes => k8s}/submit/ClientSuite.scala | 8 ++++---- ...iverConfigurationStepsOrchestratorSuite.scala | 6 +++--- .../submit/HadoopConfBootstrapSuite.scala | 6 +++--- .../submit/KerberosTokenConfBootstrapSuite.scala | 6 +++--- .../KubernetesExternalShuffleServiceSuite.scala | 6 +++--- .../submit/SSLFilePairs.scala | 2 +- .../SubmittedDependencyUploaderSuite.scala | 6 +++--- .../BaseDriverConfigurationStepSuite.scala | 6 +++--- .../DependencyResolutionStepSuite.scala | 4 ++-- .../DriverKubernetesCredentialsStepSuite.scala | 6 +++--- .../HadoopConfigBootstrapStepSuite.scala | 6 +++--- .../InitContainerBootstrapStepSuite.scala | 8 ++++---- .../MountSmallLocalFilesStepTest.scala | 8 ++++---- .../submit/submitsteps/PythonStepSuite.scala | 2 +- .../hadoopsteps/HadoopConfMounterStepSuite.scala | 6 +++--- .../HadoopKerberosKeytabResolverStepSuite.scala | 6 +++--- .../HadoopKerberosSecretResolverStepSuite.scala | 4 ++-- .../HadoopStepsOrchestratorSuite.scala | 4 ++-- ...BaseInitContainerConfigurationStepSuite.scala | 6 +++--- ...inerConfigurationStepsOrchestratorSuite.scala | 6 +++--- ...ubmittedResourcesInitContainerStepSuite.scala | 10 +++++----- ...arkDependencyDownloadInitContainerSuite.scala | 6 +++--- ...rceStagingServerSslOptionsProviderSuite.scala | 4 ++-- .../ResourceStagingServerSuite.scala | 4 ++-- .../StagedResourcesCleanerSuite.scala | 2 +- .../StagedResourcesStoreSuite.scala | 2 +- .../KubernetesTaskSchedulerImplSuite.scala | 4 ++-- .../KubernetesTaskSetManagerSuite.scala | 4 ++-- .../RackResolverUtilImplSuite.scala | 2 +- .../integrationtest/PiHelper.java | 2 +- .../integrationtest/jobs/FileExistenceTest.scala | 2 +- .../integrationtest/jobs/GroupByTest.scala | 2 +- .../integrationtest/jobs/HDFSTest.scala | 2 +- .../integrationtest/jobs/JavaOptionsTest.scala | 2 +- .../jobs/SparkPiWithInfiniteWait.scala | 4 ++-- .../KerberizedHadoopClusterLauncher.scala | 4 ++-- .../KerberosTestPodLauncher.scala | 4 ++-- .../integrationtest/KubernetesSuite.scala | 16 ++++++++-------- .../KubernetesTestComponents.scala | 4 ++-- .../integrationtest/ProcessUtils.scala | 2 +- .../ResourceStagingServerLauncher.scala | 6 +++--- .../integrationtest/SparkReadinessWatcher.scala | 2 +- .../StaticAssetServerLauncher.scala | 2 +- .../backend/GCE/GCETestBackend.scala | 8 ++++---- .../backend/IntegrationTestBackend.scala | 8 ++++---- .../backend/minikube/Minikube.scala | 4 ++-- .../backend/minikube/MinikubeTestBackend.scala | 8 ++++---- .../integrationtest/constants.scala | 2 +- .../docker/SparkDockerImageBuilder.scala | 2 +- .../kerberos/KerberosCMWatcherCache.scala | 2 +- .../kerberos/KerberosDeployment.scala | 2 +- .../kerberos/KerberosDriverWatcherCache.scala | 2 +- .../kerberos/KerberosPVWatcherCache.scala | 2 +- .../kerberos/KerberosPodWatcherCache.scala | 2 +- .../kerberos/KerberosStorage.scala | 2 +- .../integrationtest/kerberos/KerberosUtils.scala | 4 ++-- .../restapis/SparkRestApiV1.scala | 2 +- 128 files changed, 287 insertions(+), 287 deletions(-) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/CompressionUtils.scala (98%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/ConfigurationUtils.scala (98%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/HadoopConfBootstrap.scala (96%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/HadoopUGIUtil.scala (98%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/InitContainerResourceStagingServerSecretPlugin.scala (96%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/KerberosTokenConfBootstrap.scala (96%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/KubernetesCredentials.scala (95%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/KubernetesExternalShuffleService.scala (97%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/OptionRequirements.scala (97%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/PodWithDetachedInitContainer.scala (95%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/PodWithMainContainer.scala (96%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/SparkKubernetesClientFactory.scala (97%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/SparkPodInitContainerBootstrap.scala (97%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/config.scala (99%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/constants.scala (99%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/Client.scala (96%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/ContainerNameEqualityPredicate.scala (95%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/DriverConfigurationStepsOrchestrator.scala (94%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/InitContainerUtil.scala (95%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/KubernetesFileUtils.scala (98%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/LoggingPodStatusWatcher.scala (97%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/MainAppResource.scala (95%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/MountSmallFilesBootstrap.scala (94%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/PropertiesConfigMapFromScalaMapBuilder.scala (97%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/SubmittedDependencyUploaderImpl.scala (94%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/SubmittedResources.scala (96%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/BaseDriverConfigurationStep.scala (96%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/DependencyResolutionStep.scala (90%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/DriverConfigurationStep.scala (94%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/DriverKubernetesCredentialsStep.scala (98%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/HadoopConfigBootstrapStep.scala (92%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/InitContainerBootstrapStep.scala (89%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/KubernetesDriverSpec.scala (97%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/MountSmallLocalFilesStep.scala (93%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/PythonStep.scala (90%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/hadoopsteps/HadoopConfMounterStep.scala (91%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/hadoopsteps/HadoopConfigSpec.scala (96%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/hadoopsteps/HadoopConfigurationStep.scala (93%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStep.scala (96%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/hadoopsteps/HadoopKerberosSecretResolverStep.scala (91%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/hadoopsteps/HadoopStepsOrchestrator.scala (94%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStep.scala (89%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/initcontainer/InitContainerConfigurationStep.scala (93%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestrator.scala (91%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/initcontainer/InitContainerSpec.scala (96%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerConfigurationStep.scala (95%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/{kubernetes => k8s}/FileFetcher.scala (95%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/{kubernetes => k8s}/KubernetesSparkDependencyDownloadInitContainer.scala (98%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/{kubernetes => k8s}/PemsToKeyStoreConverter.scala (99%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/{kubernetes => k8s}/ResourceStagingServer.scala (97%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/{kubernetes => k8s}/ResourceStagingServerSslOptionsProvider.scala (97%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/{kubernetes => k8s}/ResourceStagingService.scala (95%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/{kubernetes => k8s}/ResourceStagingServiceImpl.scala (93%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/{kubernetes => k8s}/ResourceStagingServiceRetrofit.scala (93%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/{kubernetes => k8s}/RetrofitClientFactory.scala (99%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/{kubernetes => k8s}/SparkConfPropertiesParser.scala (97%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/{kubernetes => k8s}/StagedResources.scala (95%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/{kubernetes => k8s}/StagedResourcesCleaner.scala (99%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/{kubernetes => k8s}/StagedResourcesOwner.scala (96%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/{kubernetes => k8s}/StagedResourcesStore.scala (98%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/{kubernetes => k8s}/InetAddressUtil.scala (87%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/{kubernetes => k8s}/KubernetesClusterManager.scala (95%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/{kubernetes => k8s}/KubernetesClusterSchedulerBackend.scala (99%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/{kubernetes => k8s}/KubernetesTaskSchedulerImpl.scala (96%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/{kubernetes => k8s}/KubernetesTaskSetManager.scala (97%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/{kubernetes => k8s}/RackResolverUtil.scala (93%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/{kubernetes => k8s}/ShufflePodCache.scala (98%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/InitContainerResourceStagingServerSecretPluginSuite.scala (96%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/SSLUtils.scala (97%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/SparkPodInitContainerBootstrapSuite.scala (98%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/ClientSuite.scala (97%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/DriverConfigurationStepsOrchestratorSuite.scala (93%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/HadoopConfBootstrapSuite.scala (94%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/KerberosTokenConfBootstrapSuite.scala (91%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/KubernetesExternalShuffleServiceSuite.scala (91%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/SSLFilePairs.scala (94%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/SubmittedDependencyUploaderSuite.scala (96%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/BaseDriverConfigurationStepSuite.scala (97%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/DependencyResolutionStepSuite.scala (96%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/DriverKubernetesCredentialsStepSuite.scala (97%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/HadoopConfigBootstrapStepSuite.scala (93%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/InitContainerBootstrapStepSuite.scala (96%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/MountSmallLocalFilesStepTest.scala (96%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/PythonStepSuite.scala (97%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/hadoopsteps/HadoopConfMounterStepSuite.scala (94%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStepSuite.scala (96%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/hadoopsteps/HadoopKerberosSecretResolverStepSuite.scala (95%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/hadoopsteps/HadoopStepsOrchestratorSuite.scala (96%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStepSuite.scala (94%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestratorSuite.scala (95%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerStepSuite.scala (96%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/{kubernetes => k8s}/KubernetesSparkDependencyDownloadInitContainerSuite.scala (98%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/{kubernetes => k8s}/ResourceStagingServerSslOptionsProviderSuite.scala (98%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/{kubernetes => k8s}/ResourceStagingServerSuite.scala (98%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/{kubernetes => k8s}/StagedResourcesCleanerSuite.scala (99%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/{kubernetes => k8s}/StagedResourcesStoreSuite.scala (98%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/{kubernetes => k8s}/KubernetesTaskSchedulerImplSuite.scala (98%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/{kubernetes => k8s}/KubernetesTaskSetManagerSuite.scala (98%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/{kubernetes => k8s}/RackResolverUtilImplSuite.scala (97%) rename resource-managers/kubernetes/integration-tests-spark-jobs-helpers/src/main/java/org/apache/spark/deploy/{kubernetes => k8s}/integrationtest/PiHelper.java (94%) rename resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/integrationtest/jobs/FileExistenceTest.scala (97%) rename resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/integrationtest/jobs/GroupByTest.scala (96%) rename resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/integrationtest/jobs/HDFSTest.scala (96%) rename resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/integrationtest/jobs/JavaOptionsTest.scala (98%) rename resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/integrationtest/jobs/SparkPiWithInfiniteWait.scala (92%) rename resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/integrationtest/KerberizedHadoopClusterLauncher.scala (95%) rename resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/integrationtest/KerberosTestPodLauncher.scala (97%) rename resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/integrationtest/KubernetesSuite.scala (96%) rename resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/integrationtest/KubernetesTestComponents.scala (96%) rename resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/integrationtest/ProcessUtils.scala (97%) rename resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/integrationtest/ResourceStagingServerLauncher.scala (97%) rename resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/integrationtest/SparkReadinessWatcher.scala (96%) rename resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/integrationtest/StaticAssetServerLauncher.scala (97%) rename resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/integrationtest/backend/GCE/GCETestBackend.scala (81%) rename resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/integrationtest/backend/IntegrationTestBackend.scala (78%) rename resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/integrationtest/backend/minikube/Minikube.scala (97%) rename resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/integrationtest/backend/minikube/MinikubeTestBackend.scala (81%) rename resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/integrationtest/constants.scala (93%) rename resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/integrationtest/docker/SparkDockerImageBuilder.scala (98%) rename resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/integrationtest/kerberos/KerberosCMWatcherCache.scala (98%) rename resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/integrationtest/kerberos/KerberosDeployment.scala (93%) rename resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/integrationtest/kerberos/KerberosDriverWatcherCache.scala (98%) rename resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/integrationtest/kerberos/KerberosPVWatcherCache.scala (99%) rename resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/integrationtest/kerberos/KerberosPodWatcherCache.scala (99%) rename resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/integrationtest/kerberos/KerberosStorage.scala (93%) rename resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/integrationtest/kerberos/KerberosUtils.scala (97%) rename resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/integrationtest/restapis/SparkRestApiV1.scala (96%) diff --git a/resource-managers/kubernetes/core/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager b/resource-managers/kubernetes/core/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager index 55e7e38b28a08..81d14766ffb8d 100644 --- a/resource-managers/kubernetes/core/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager +++ b/resource-managers/kubernetes/core/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager @@ -1 +1 @@ -org.apache.spark.scheduler.cluster.kubernetes.KubernetesClusterManager +org.apache.spark.scheduler.cluster.k8s.KubernetesClusterManager diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/CompressionUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/CompressionUtils.scala similarity index 98% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/CompressionUtils.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/CompressionUtils.scala index a6f0ca502f6f0..364882d542fa6 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/CompressionUtils.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/CompressionUtils.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes +package org.apache.spark.deploy.k8s import java.io.{File, FileInputStream, FileOutputStream, InputStream, OutputStream} import java.util.zip.{GZIPInputStream, GZIPOutputStream} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/ConfigurationUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala similarity index 98% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/ConfigurationUtils.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala index 1a008c236d00f..7645e54e46280 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/ConfigurationUtils.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.deploy.kubernetes +package org.apache.spark.deploy.k8s import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.internal.Logging diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/HadoopConfBootstrap.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopConfBootstrap.scala similarity index 96% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/HadoopConfBootstrap.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopConfBootstrap.scala index 17efe19b8f206..7c259e140e049 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/HadoopConfBootstrap.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopConfBootstrap.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes +package org.apache.spark.deploy.k8s import java.io.File @@ -22,7 +22,7 @@ import scala.collection.JavaConverters._ import io.fabric8.kubernetes.api.model.{ContainerBuilder, KeyToPathBuilder, PodBuilder} -import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.k8s.constants._ import org.apache.spark.internal.Logging /** diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/HadoopUGIUtil.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopUGIUtil.scala similarity index 98% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/HadoopUGIUtil.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopUGIUtil.scala index 8a004c0e5414f..dfc56260d9553 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/HadoopUGIUtil.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/HadoopUGIUtil.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes +package org.apache.spark.deploy.k8s import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataInputStream, DataOutputStream} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/InitContainerResourceStagingServerSecretPlugin.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerResourceStagingServerSecretPlugin.scala similarity index 96% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/InitContainerResourceStagingServerSecretPlugin.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerResourceStagingServerSecretPlugin.scala index 265b8f197a102..99947498d6a89 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/InitContainerResourceStagingServerSecretPlugin.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerResourceStagingServerSecretPlugin.scala @@ -14,11 +14,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes +package org.apache.spark.deploy.k8s import io.fabric8.kubernetes.api.model.{Container, ContainerBuilder, Pod, PodBuilder, Secret} -import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.k8s.constants._ private[spark] trait InitContainerResourceStagingServerSecretPlugin { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KerberosTokenConfBootstrap.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KerberosTokenConfBootstrap.scala similarity index 96% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KerberosTokenConfBootstrap.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KerberosTokenConfBootstrap.scala index 6225142fa4b52..9b8f83dc3f891 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KerberosTokenConfBootstrap.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KerberosTokenConfBootstrap.scala @@ -14,11 +14,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes +package org.apache.spark.deploy.k8s import io.fabric8.kubernetes.api.model.{ContainerBuilder, PodBuilder} -import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.k8s.constants._ import org.apache.spark.internal.Logging diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KubernetesCredentials.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesCredentials.scala similarity index 95% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KubernetesCredentials.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesCredentials.scala index aba94e6969529..833af9b3b5638 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KubernetesCredentials.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesCredentials.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes +package org.apache.spark.deploy.k8s case class KubernetesCredentials( oauthTokenBase64: Option[String], diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KubernetesExternalShuffleService.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesExternalShuffleService.scala similarity index 97% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KubernetesExternalShuffleService.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesExternalShuffleService.scala index c61f4f1d44acf..661c3a40a15a6 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KubernetesExternalShuffleService.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesExternalShuffleService.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.deploy.kubernetes +package org.apache.spark.deploy.k8s import java.io.File import java.nio.ByteBuffer @@ -29,8 +29,8 @@ import scala.collection.mutable import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.ExternalShuffleService -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ import org.apache.spark.internal.Logging import org.apache.spark.network.client.{RpcResponseCallback, TransportClient} import org.apache.spark.network.shuffle.ExternalShuffleBlockHandler diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/OptionRequirements.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/OptionRequirements.scala similarity index 97% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/OptionRequirements.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/OptionRequirements.scala index eda43de0a9a5b..89053de5b9558 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/OptionRequirements.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/OptionRequirements.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes +package org.apache.spark.deploy.k8s private[spark] object OptionRequirements { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/PodWithDetachedInitContainer.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/PodWithDetachedInitContainer.scala similarity index 95% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/PodWithDetachedInitContainer.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/PodWithDetachedInitContainer.scala index 36b1b07dc6bc4..cdec3a155a597 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/PodWithDetachedInitContainer.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/PodWithDetachedInitContainer.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes +package org.apache.spark.deploy.k8s import io.fabric8.kubernetes.api.model.{Container, Pod} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/PodWithMainContainer.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/PodWithMainContainer.scala similarity index 96% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/PodWithMainContainer.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/PodWithMainContainer.scala index 664eb41bd68d5..42f3343acee7d 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/PodWithMainContainer.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/PodWithMainContainer.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes +package org.apache.spark.deploy.k8s import io.fabric8.kubernetes.api.model.{Container, Pod} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/SparkKubernetesClientFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala similarity index 97% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/SparkKubernetesClientFactory.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala index d2729a2db2fa0..c90303b6ba506 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/SparkKubernetesClientFactory.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes +package org.apache.spark.deploy.k8s import java.io.File @@ -25,7 +25,7 @@ import io.fabric8.kubernetes.client.utils.HttpClientUtils import okhttp3.Dispatcher import org.apache.spark.SparkConf -import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.k8s.config._ import org.apache.spark.util.ThreadUtils /** diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/SparkPodInitContainerBootstrap.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkPodInitContainerBootstrap.scala similarity index 97% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/SparkPodInitContainerBootstrap.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkPodInitContainerBootstrap.scala index 2df7ac7a204c4..35246506ecb33 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/SparkPodInitContainerBootstrap.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkPodInitContainerBootstrap.scala @@ -14,11 +14,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes +package org.apache.spark.deploy.k8s import io.fabric8.kubernetes.api.model.{ContainerBuilder, EmptyDirVolumeSource, PodBuilder, VolumeMount, VolumeMountBuilder} -import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.k8s.constants._ /** * This is separated out from the init-container steps API because this component can be reused to diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/config.scala similarity index 99% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/config.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/config.scala index 442bdb01b979b..ebee7bd5b9181 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/config.scala @@ -14,12 +14,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes +package org.apache.spark.deploy.k8s import java.util.concurrent.TimeUnit import org.apache.spark.{SPARK_VERSION => sparkVersion} -import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.k8s.constants._ import org.apache.spark.internal.Logging import org.apache.spark.internal.config.ConfigBuilder import org.apache.spark.network.util.ByteUnit diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/constants.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/constants.scala similarity index 99% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/constants.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/constants.scala index dfb4e0838113f..df1e9f2e1ae2e 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/constants.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/constants.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes +package org.apache.spark.deploy.k8s package object constants { // Labels diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala similarity index 96% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala index 15f6c9d7ce0d8..e1d4b620c3ca9 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit +package org.apache.spark.deploy.k8s.submit import java.util.{Collections, UUID} @@ -24,10 +24,10 @@ import scala.collection.mutable import scala.collection.JavaConverters._ import org.apache.spark.SparkConf -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.submitsteps.{DriverConfigurationStep, KubernetesDriverSpec} -import org.apache.spark.deploy.kubernetes.SparkKubernetesClientFactory +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.deploy.k8s.submit.submitsteps.{DriverConfigurationStep, KubernetesDriverSpec} +import org.apache.spark.deploy.k8s.SparkKubernetesClientFactory import org.apache.spark.internal.Logging import org.apache.spark.util.Utils diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/ContainerNameEqualityPredicate.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/ContainerNameEqualityPredicate.scala similarity index 95% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/ContainerNameEqualityPredicate.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/ContainerNameEqualityPredicate.scala index 434919208ba2e..94adcc5857d2a 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/ContainerNameEqualityPredicate.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/ContainerNameEqualityPredicate.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit +package org.apache.spark.deploy.k8s.submit import java.lang.Boolean diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala similarity index 94% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestrator.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala index bc0e29ec6980d..6aac91a350d66 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestrator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala @@ -14,15 +14,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit +package org.apache.spark.deploy.k8s.submit import org.apache.spark.SparkConf -import org.apache.spark.deploy.kubernetes.ConfigurationUtils -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.submitsteps._ -import org.apache.spark.deploy.kubernetes.submit.submitsteps.hadoopsteps.HadoopStepsOrchestrator -import org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer.InitContainerConfigurationStepsOrchestrator +import org.apache.spark.deploy.k8s.ConfigurationUtils +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.deploy.k8s.submit.submitsteps._ +import org.apache.spark.deploy.k8s.submit.submitsteps.hadoopsteps.HadoopStepsOrchestrator +import org.apache.spark.deploy.k8s.submit.submitsteps.initcontainer.InitContainerConfigurationStepsOrchestrator import org.apache.spark.launcher.SparkLauncher import org.apache.spark.util.Utils diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/InitContainerUtil.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/InitContainerUtil.scala similarity index 95% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/InitContainerUtil.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/InitContainerUtil.scala index 837ec0e8c867e..55ed06d7ca415 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/InitContainerUtil.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/InitContainerUtil.scala @@ -14,14 +14,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit +package org.apache.spark.deploy.k8s.submit import com.fasterxml.jackson.databind.ObjectMapper import com.fasterxml.jackson.module.scala.DefaultScalaModule import io.fabric8.kubernetes.api.model.{Container, Pod, PodBuilder} import scala.collection.JavaConverters._ -import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.k8s.constants._ private[spark] object InitContainerUtil { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesFileUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesFileUtils.scala similarity index 98% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesFileUtils.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesFileUtils.scala index ec591923f1472..00ff72ec0efb1 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesFileUtils.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesFileUtils.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit +package org.apache.spark.deploy.k8s.submit import java.io.File diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/LoggingPodStatusWatcher.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/LoggingPodStatusWatcher.scala similarity index 97% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/LoggingPodStatusWatcher.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/LoggingPodStatusWatcher.scala index 4a8a7308b9fe4..c645b008d736d 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/LoggingPodStatusWatcher.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/LoggingPodStatusWatcher.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit +package org.apache.spark.deploy.k8s.submit import java.util.concurrent.{CountDownLatch, TimeUnit} @@ -27,7 +27,7 @@ import org.apache.spark.SparkException import org.apache.spark.internal.Logging import org.apache.spark.util.ThreadUtils -private[kubernetes] trait LoggingPodStatusWatcher extends Watcher[Pod] { +private[k8s] trait LoggingPodStatusWatcher extends Watcher[Pod] { def awaitCompletion(): Unit } @@ -39,7 +39,7 @@ private[kubernetes] trait LoggingPodStatusWatcher extends Watcher[Pod] { * @param maybeLoggingInterval ms between each state request. If provided, must be a positive * number. */ -private[kubernetes] class LoggingPodStatusWatcherImpl( +private[k8s] class LoggingPodStatusWatcherImpl( appId: String, maybeLoggingInterval: Option[Long]) extends LoggingPodStatusWatcher with Logging { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/MainAppResource.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/MainAppResource.scala similarity index 95% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/MainAppResource.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/MainAppResource.scala index 436d531a850ff..f806e65974fcc 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/MainAppResource.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/MainAppResource.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit +package org.apache.spark.deploy.k8s.submit private[spark] sealed trait MainAppResource diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/MountSmallFilesBootstrap.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/MountSmallFilesBootstrap.scala similarity index 94% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/MountSmallFilesBootstrap.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/MountSmallFilesBootstrap.scala index 79919b511ec37..2ace625b4d6a6 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/MountSmallFilesBootstrap.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/MountSmallFilesBootstrap.scala @@ -14,11 +14,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit +package org.apache.spark.deploy.k8s.submit import io.fabric8.kubernetes.api.model.{Container, ContainerBuilder, Pod, PodBuilder} -import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.k8s.constants._ private[spark] trait MountSmallFilesBootstrap { def mountSmallFilesSecret(pod: Pod, container: Container): (Pod, Container) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/PropertiesConfigMapFromScalaMapBuilder.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/PropertiesConfigMapFromScalaMapBuilder.scala similarity index 97% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/PropertiesConfigMapFromScalaMapBuilder.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/PropertiesConfigMapFromScalaMapBuilder.scala index 8103272c27518..d1d20711abffe 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/PropertiesConfigMapFromScalaMapBuilder.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/PropertiesConfigMapFromScalaMapBuilder.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit +package org.apache.spark.deploy.k8s.submit import java.io.StringWriter import java.util.Properties diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedDependencyUploaderImpl.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/SubmittedDependencyUploaderImpl.scala similarity index 94% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedDependencyUploaderImpl.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/SubmittedDependencyUploaderImpl.scala index 83d7a28f5ca10..7b829351a28ba 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedDependencyUploaderImpl.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/SubmittedDependencyUploaderImpl.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit +package org.apache.spark.deploy.k8s.submit import java.io.{File, FileOutputStream} import javax.ws.rs.core.MediaType @@ -27,8 +27,8 @@ import okhttp3.RequestBody import retrofit2.Call import org.apache.spark.{SparkException, SSLOptions} -import org.apache.spark.deploy.kubernetes.CompressionUtils -import org.apache.spark.deploy.rest.kubernetes.{ResourceStagingServiceRetrofit, RetrofitClientFactory, StagedResourcesOwner, StagedResourcesOwnerType} +import org.apache.spark.deploy.k8s.CompressionUtils +import org.apache.spark.deploy.rest.k8s.{ResourceStagingServiceRetrofit, RetrofitClientFactory, StagedResourcesOwner, StagedResourcesOwnerType} import org.apache.spark.util.Utils private[spark] trait SubmittedDependencyUploader { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedResources.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/SubmittedResources.scala similarity index 96% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedResources.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/SubmittedResources.scala index 225972c1057f2..b02cb21372678 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedResources.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/SubmittedResources.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit +package org.apache.spark.deploy.k8s.submit case class SubmittedResourceIdAndSecret(resourceId: String, resourceSecret: String) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/BaseDriverConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/BaseDriverConfigurationStep.scala similarity index 96% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/BaseDriverConfigurationStep.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/BaseDriverConfigurationStep.scala index 19784e8a8074f..4a2b558f9297b 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/BaseDriverConfigurationStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/BaseDriverConfigurationStep.scala @@ -14,15 +14,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps +package org.apache.spark.deploy.k8s.submit.submitsteps import io.fabric8.kubernetes.api.model.{ContainerBuilder, EnvVarBuilder, PodBuilder, QuantityBuilder} import scala.collection.JavaConverters._ import org.apache.spark.SparkConf -import org.apache.spark.deploy.kubernetes.ConfigurationUtils -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.k8s.ConfigurationUtils +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ /** * Represents the initial setup required for the driver. diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DependencyResolutionStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DependencyResolutionStep.scala similarity index 90% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DependencyResolutionStep.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DependencyResolutionStep.scala index 090240420119e..c2c76c2ba1960 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DependencyResolutionStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DependencyResolutionStep.scala @@ -14,16 +14,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps +package org.apache.spark.deploy.k8s.submit.submitsteps import java.io.File import io.fabric8.kubernetes.api.model.ContainerBuilder -import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.KubernetesFileUtils -import org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer.InitContainerConfigurationStep +import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.deploy.k8s.submit.KubernetesFileUtils +import org.apache.spark.deploy.k8s.submit.submitsteps.initcontainer.InitContainerConfigurationStep import org.apache.spark.util.Utils /** diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DriverConfigurationStep.scala similarity index 94% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverConfigurationStep.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DriverConfigurationStep.scala index 8070e32371f94..706b02db57672 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverConfigurationStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DriverConfigurationStep.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps +package org.apache.spark.deploy.k8s.submit.submitsteps /** * Represents a step in preparing the Kubernetes driver. diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverKubernetesCredentialsStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DriverKubernetesCredentialsStep.scala similarity index 98% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverKubernetesCredentialsStep.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DriverKubernetesCredentialsStep.scala index 74df2de480bf4..da5686099fbbb 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverKubernetesCredentialsStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DriverKubernetesCredentialsStep.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps +package org.apache.spark.deploy.k8s.submit.submitsteps import java.io.File import java.nio.charset.StandardCharsets @@ -25,8 +25,8 @@ import scala.collection.JavaConverters._ import scala.language.implicitConversions import org.apache.spark.SparkConf -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ /** * Mounts Kubernetes credentials into the driver pod. The driver will use such mounted credentials diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/HadoopConfigBootstrapStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/HadoopConfigBootstrapStep.scala similarity index 92% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/HadoopConfigBootstrapStep.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/HadoopConfigBootstrapStep.scala index a9c66aed7075e..66c513be985b7 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/HadoopConfigBootstrapStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/HadoopConfigBootstrapStep.scala @@ -14,14 +14,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps +package org.apache.spark.deploy.k8s.submit.submitsteps import scala.collection.JavaConverters._ import io.fabric8.kubernetes.api.model.ConfigMapBuilder -import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.submitsteps.hadoopsteps.{HadoopConfigSpec, HadoopConfigurationStep} +import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.deploy.k8s.submit.submitsteps.hadoopsteps.{HadoopConfigSpec, HadoopConfigurationStep} /** * This class configures the driverSpec with hadoop configuration logic which includes diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/InitContainerBootstrapStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/InitContainerBootstrapStep.scala similarity index 89% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/InitContainerBootstrapStep.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/InitContainerBootstrapStep.scala index b4248338cc8de..f8d1c909ac794 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/InitContainerBootstrapStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/InitContainerBootstrapStep.scala @@ -14,13 +14,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps +package org.apache.spark.deploy.k8s.submit.submitsteps import io.fabric8.kubernetes.api.model.{ContainerBuilder, HasMetadata} -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.submit.{InitContainerUtil, PropertiesConfigMapFromScalaMapBuilder} -import org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer.{InitContainerConfigurationStep, InitContainerSpec} +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.submit.{InitContainerUtil, PropertiesConfigMapFromScalaMapBuilder} +import org.apache.spark.deploy.k8s.submit.submitsteps.initcontainer.{InitContainerConfigurationStep, InitContainerSpec} /** * Configures the init-container that bootstraps dependencies into the driver pod. diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/KubernetesDriverSpec.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/KubernetesDriverSpec.scala similarity index 97% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/KubernetesDriverSpec.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/KubernetesDriverSpec.scala index 3ec4b6c4df10f..f76c0a8c9c6bd 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/KubernetesDriverSpec.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/KubernetesDriverSpec.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps +package org.apache.spark.deploy.k8s.submit.submitsteps import io.fabric8.kubernetes.api.model.{Container, ContainerBuilder, HasMetadata, Pod, PodBuilder} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/MountSmallLocalFilesStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/MountSmallLocalFilesStep.scala similarity index 93% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/MountSmallLocalFilesStep.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/MountSmallLocalFilesStep.scala index cd1b7f6b7eb7e..5a280467f35be 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/MountSmallLocalFilesStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/MountSmallLocalFilesStep.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps +package org.apache.spark.deploy.k8s.submit.submitsteps import java.io.File @@ -22,8 +22,8 @@ import com.google.common.io.{BaseEncoding, Files} import io.fabric8.kubernetes.api.model.SecretBuilder import scala.collection.JavaConverters._ -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.submit.{KubernetesFileUtils, MountSmallFilesBootstrap} +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.submit.{KubernetesFileUtils, MountSmallFilesBootstrap} import org.apache.spark.util.Utils private[spark] class MountSmallLocalFilesStep( diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/PythonStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/PythonStep.scala similarity index 90% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/PythonStep.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/PythonStep.scala index 024d643ddf9fd..f0003924e9c6d 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/PythonStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/PythonStep.scala @@ -14,12 +14,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps +package org.apache.spark.deploy.k8s.submit.submitsteps import io.fabric8.kubernetes.api.model.ContainerBuilder -import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.KubernetesFileUtils +import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.deploy.k8s.submit.KubernetesFileUtils private[spark] class PythonStep( primaryPyFile: String, diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopConfMounterStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopConfMounterStep.scala similarity index 91% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopConfMounterStep.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopConfMounterStep.scala index 53b979df326c2..037ea0ddfe156 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopConfMounterStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopConfMounterStep.scala @@ -14,14 +14,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps.hadoopsteps +package org.apache.spark.deploy.k8s.submit.submitsteps.hadoopsteps import java.io.File import org.apache.commons.io.FileUtils.readFileToString -import org.apache.spark.deploy.kubernetes.{HadoopConfBootstrap, PodWithMainContainer} -import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.k8s.{HadoopConfBootstrap, PodWithMainContainer} +import org.apache.spark.deploy.k8s.constants._ /** * This step is responsible for taking the contents from each file in diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopConfigSpec.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopConfigSpec.scala similarity index 96% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopConfigSpec.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopConfigSpec.scala index 51e9e506bcc22..b38cae25dca26 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopConfigSpec.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopConfigSpec.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps.hadoopsteps +package org.apache.spark.deploy.k8s.submit.submitsteps.hadoopsteps import io.fabric8.kubernetes.api.model.{Container, Pod, Secret} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopConfigurationStep.scala similarity index 93% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopConfigurationStep.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopConfigurationStep.scala index 54f46b9a17fc9..b08b180ce8531 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopConfigurationStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopConfigurationStep.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps.hadoopsteps +package org.apache.spark.deploy.k8s.submit.submitsteps.hadoopsteps /** * Represents a step in preparing the driver with Hadoop Configuration logic. diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStep.scala similarity index 96% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStep.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStep.scala index 4e0a0cfd8fe9b..1bf7f36821478 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStep.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps.hadoopsteps +package org.apache.spark.deploy.k8s.submit.submitsteps.hadoopsteps import java.io._ import java.security.PrivilegedExceptionAction @@ -29,8 +29,8 @@ import org.apache.hadoop.security.token.{Token, TokenIdentifier} import org.apache.spark.SparkConf import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.deploy.kubernetes.{HadoopUGIUtil, KerberosTokenConfBootstrapImpl, PodWithMainContainer} -import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.k8s.{HadoopUGIUtil, KerberosTokenConfBootstrapImpl, PodWithMainContainer} +import org.apache.spark.deploy.k8s.constants._ import org.apache.spark.internal.Logging /** diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosSecretResolverStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosSecretResolverStep.scala similarity index 91% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosSecretResolverStep.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosSecretResolverStep.scala index 3eeec1374ef96..9d60f932bc736 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosSecretResolverStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosSecretResolverStep.scala @@ -14,13 +14,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps.hadoopsteps +package org.apache.spark.deploy.k8s.submit.submitsteps.hadoopsteps import org.apache.hadoop.security.UserGroupInformation import org.apache.spark.SparkConf -import org.apache.spark.deploy.kubernetes.{KerberosTokenConfBootstrapImpl, PodWithMainContainer} -import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.k8s.{KerberosTokenConfBootstrapImpl, PodWithMainContainer} +import org.apache.spark.deploy.k8s.constants._ /** * This step assumes that you have already done all the heavy lifting in retrieving a diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopStepsOrchestrator.scala similarity index 94% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopStepsOrchestrator.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopStepsOrchestrator.scala index 93d184fedbcf1..8f052662ba600 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopStepsOrchestrator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopStepsOrchestrator.scala @@ -14,13 +14,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps.hadoopsteps +package org.apache.spark.deploy.k8s.submit.submitsteps.hadoopsteps import java.io.File import org.apache.spark.SparkConf -import org.apache.spark.deploy.kubernetes.{HadoopConfBootstrapImpl, HadoopUGIUtil, OptionRequirements} -import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.k8s.{HadoopConfBootstrapImpl, HadoopUGIUtil, OptionRequirements} +import org.apache.spark.deploy.k8s.config._ import org.apache.spark.internal.Logging /** diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStep.scala similarity index 89% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStep.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStep.scala index 60bf27beacaaf..dc335ac8f076e 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStep.scala @@ -14,11 +14,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer +package org.apache.spark.deploy.k8s.submit.submitsteps.initcontainer -import org.apache.spark.deploy.kubernetes.{PodWithDetachedInitContainer, SparkPodInitContainerBootstrap} -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.submit.KubernetesFileUtils +import org.apache.spark.deploy.k8s.{PodWithDetachedInitContainer, SparkPodInitContainerBootstrap} +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.submit.KubernetesFileUtils private[spark] class BaseInitContainerConfigurationStep( sparkJars: Seq[String], diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/InitContainerConfigurationStep.scala similarity index 93% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStep.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/InitContainerConfigurationStep.scala index 7b7622c3d4f8b..a3bdaa8e93344 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/InitContainerConfigurationStep.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer +package org.apache.spark.deploy.k8s.submit.submitsteps.initcontainer /** * Represents a step in preparing the init-container for the driver and executors. diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestrator.scala similarity index 91% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestrator.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestrator.scala index e4ea5235af18f..1f7dbf331f465 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestrator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestrator.scala @@ -14,14 +14,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer +package org.apache.spark.deploy.k8s.submit.submitsteps.initcontainer import org.apache.spark.SparkConf -import org.apache.spark.deploy.kubernetes.{InitContainerResourceStagingServerSecretPluginImpl, OptionRequirements, SparkPodInitContainerBootstrapImpl} -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.SubmittedDependencyUploaderImpl -import org.apache.spark.deploy.rest.kubernetes.{ResourceStagingServerSslOptionsProviderImpl, RetrofitClientFactoryImpl} +import org.apache.spark.deploy.k8s.{InitContainerResourceStagingServerSecretPluginImpl, OptionRequirements, SparkPodInitContainerBootstrapImpl} +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.deploy.k8s.submit.SubmittedDependencyUploaderImpl +import org.apache.spark.deploy.rest.k8s.{ResourceStagingServerSslOptionsProviderImpl, RetrofitClientFactoryImpl} import org.apache.spark.util.Utils /** diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerSpec.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/InitContainerSpec.scala similarity index 96% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerSpec.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/InitContainerSpec.scala index 5b5ac3c1f17c2..5fcb0495f0b19 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerSpec.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/InitContainerSpec.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer +package org.apache.spark.deploy.k8s.submit.submitsteps.initcontainer import io.fabric8.kubernetes.api.model.{Container, HasMetadata, Pod} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerConfigurationStep.scala similarity index 95% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerConfigurationStep.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerConfigurationStep.scala index 7aa27a1de6811..b803b67978397 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerConfigurationStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerConfigurationStep.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer +package org.apache.spark.deploy.k8s.submit.submitsteps.initcontainer import java.io.File @@ -24,10 +24,10 @@ import io.fabric8.kubernetes.api.model.{Secret, SecretBuilder} import scala.collection.JavaConverters._ import org.apache.spark.SparkException -import org.apache.spark.deploy.kubernetes.InitContainerResourceStagingServerSecretPlugin -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.SubmittedDependencyUploader +import org.apache.spark.deploy.k8s.InitContainerResourceStagingServerSecretPlugin +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.deploy.k8s.submit.SubmittedDependencyUploader import org.apache.spark.internal.config.OptionalConfigEntry import org.apache.spark.util.Utils diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/FileFetcher.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/FileFetcher.scala similarity index 95% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/FileFetcher.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/FileFetcher.scala index d050e0a41a15a..a40ced5a55be7 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/FileFetcher.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/FileFetcher.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import java.io.File diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkDependencyDownloadInitContainer.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/KubernetesSparkDependencyDownloadInitContainer.scala similarity index 98% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkDependencyDownloadInitContainer.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/KubernetesSparkDependencyDownloadInitContainer.scala index 0e274678ad6f0..1c78b307c2de5 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkDependencyDownloadInitContainer.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/KubernetesSparkDependencyDownloadInitContainer.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import java.io.File import java.util.concurrent.TimeUnit @@ -29,8 +29,8 @@ import scala.concurrent.duration.Duration import org.apache.spark.{SecurityManager => SparkSecurityManager, SparkConf, SSLOptions} import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.deploy.kubernetes.{CompressionUtils, KubernetesCredentials} -import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.k8s.{CompressionUtils, KubernetesCredentials} +import org.apache.spark.deploy.k8s.config._ import org.apache.spark.internal.Logging import org.apache.spark.util.{ThreadUtils, Utils} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/PemsToKeyStoreConverter.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/PemsToKeyStoreConverter.scala similarity index 99% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/PemsToKeyStoreConverter.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/PemsToKeyStoreConverter.scala index 17f90118e150d..777ed70d82c22 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/PemsToKeyStoreConverter.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/PemsToKeyStoreConverter.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import java.io.{File, FileInputStream, FileOutputStream, InputStreamReader} import java.security.{KeyStore, PrivateKey} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServer.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingServer.scala similarity index 97% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServer.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingServer.scala index 0b97317eba8b1..9a63830966dbc 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServer.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingServer.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import java.io.File @@ -31,8 +31,8 @@ import org.glassfish.jersey.server.ResourceConfig import org.glassfish.jersey.servlet.ServletContainer import org.apache.spark.SparkConf -import org.apache.spark.deploy.kubernetes.SparkKubernetesClientFactory -import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.k8s.SparkKubernetesClientFactory +import org.apache.spark.deploy.k8s.config._ import org.apache.spark.internal.Logging import org.apache.spark.util.{SystemClock, ThreadUtils, Utils} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServerSslOptionsProvider.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingServerSslOptionsProvider.scala similarity index 97% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServerSslOptionsProvider.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingServerSslOptionsProvider.scala index cb1e65421c013..30c5d3917d74a 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServerSslOptionsProvider.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingServerSslOptionsProvider.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import java.io.File import java.security.SecureRandom @@ -24,8 +24,8 @@ import com.google.common.io.Files import org.apache.commons.lang3.RandomStringUtils import org.apache.spark.{SecurityManager, SparkConf, SparkException, SSLOptions} -import org.apache.spark.deploy.kubernetes.OptionRequirements -import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.k8s.OptionRequirements +import org.apache.spark.deploy.k8s.config._ import org.apache.spark.internal.Logging private[spark] trait ResourceStagingServerSslOptionsProvider { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingService.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingService.scala similarity index 95% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingService.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingService.scala index b9d283a99ade9..b48b2ca4859d6 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingService.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingService.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import java.io.InputStream import javax.ws.rs.{Consumes, GET, HeaderParam, Path, PathParam, POST, Produces} @@ -22,8 +22,8 @@ import javax.ws.rs.core.{MediaType, StreamingOutput} import org.glassfish.jersey.media.multipart.FormDataParam -import org.apache.spark.deploy.kubernetes.KubernetesCredentials -import org.apache.spark.deploy.kubernetes.submit.SubmittedResourceIdAndSecret +import org.apache.spark.deploy.k8s.KubernetesCredentials +import org.apache.spark.deploy.k8s.submit.SubmittedResourceIdAndSecret /** * Service that receives application data that can be retrieved later on. This is primarily used diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServiceImpl.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingServiceImpl.scala similarity index 93% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServiceImpl.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingServiceImpl.scala index 7bc21c21619e1..ebf37fd277cd6 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServiceImpl.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingServiceImpl.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import java.io.{File, FileOutputStream, InputStream, OutputStream} import java.security.SecureRandom @@ -26,8 +26,8 @@ import com.google.common.io.{BaseEncoding, ByteStreams, Files} import scala.collection.concurrent.TrieMap import org.apache.spark.SparkException -import org.apache.spark.deploy.kubernetes.KubernetesCredentials -import org.apache.spark.deploy.kubernetes.submit.SubmittedResourceIdAndSecret +import org.apache.spark.deploy.k8s.KubernetesCredentials +import org.apache.spark.deploy.k8s.submit.SubmittedResourceIdAndSecret import org.apache.spark.internal.Logging import org.apache.spark.util.Utils diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServiceRetrofit.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingServiceRetrofit.scala similarity index 93% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServiceRetrofit.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingServiceRetrofit.scala index 5fbf0f9c43970..53803f46163e6 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServiceRetrofit.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingServiceRetrofit.scala @@ -14,13 +14,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import okhttp3.{RequestBody, ResponseBody} import retrofit2.Call import retrofit2.http.{Multipart, Path, Streaming} -import org.apache.spark.deploy.kubernetes.submit.SubmittedResourceIdAndSecret +import org.apache.spark.deploy.k8s.submit.SubmittedResourceIdAndSecret /** * Retrofit-compatible variant of {@link ResourceStagingService}. For documentation on diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/RetrofitClientFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/RetrofitClientFactory.scala similarity index 99% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/RetrofitClientFactory.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/RetrofitClientFactory.scala index 5046cb479054c..f5d41a5012bc1 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/RetrofitClientFactory.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/RetrofitClientFactory.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import java.io.FileInputStream import java.net.{InetSocketAddress, URI} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/SparkConfPropertiesParser.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkConfPropertiesParser.scala similarity index 97% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/SparkConfPropertiesParser.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkConfPropertiesParser.scala index 9e2b8a780df29..4376de42bd751 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/SparkConfPropertiesParser.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkConfPropertiesParser.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import java.io.{File, FileInputStream} import java.util.Properties diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/StagedResources.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/StagedResources.scala similarity index 95% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/StagedResources.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/StagedResources.scala index 81f394800f803..f4038ae05fd99 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/StagedResources.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/StagedResources.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import java.io.File diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesCleaner.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/StagedResourcesCleaner.scala similarity index 99% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesCleaner.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/StagedResourcesCleaner.scala index 5d9db728483fa..38636ac7d992d 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesCleaner.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/StagedResourcesCleaner.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import java.util.concurrent.{ScheduledExecutorService, TimeUnit} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesOwner.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/StagedResourcesOwner.scala similarity index 96% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesOwner.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/StagedResourcesOwner.scala index 4061bc36764d7..9a9691371b3f5 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesOwner.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/StagedResourcesOwner.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import com.fasterxml.jackson.core.`type`.TypeReference import com.fasterxml.jackson.module.scala.JsonScalaEnumeration diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesStore.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/StagedResourcesStore.scala similarity index 98% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesStore.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/StagedResourcesStore.scala index 0c0d428e035dc..7cb819dc90804 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesStore.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/StagedResourcesStore.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import java.io.{File, FileOutputStream, InputStream, IOException} import java.security.SecureRandom diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/InetAddressUtil.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/InetAddressUtil.scala similarity index 87% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/InetAddressUtil.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/InetAddressUtil.scala index e04ab9e541963..8d6d92ac7b412 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/InetAddressUtil.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/InetAddressUtil.scala @@ -14,19 +14,19 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.scheduler.cluster.kubernetes +package org.apache.spark.scheduler.cluster.k8s import java.net.InetAddress /** * Gets full host names of given IP addresses from DNS. */ -private[kubernetes] trait InetAddressUtil { +private[k8s] trait InetAddressUtil { def getFullHostName(ipAddress: String): String } -private[kubernetes] object InetAddressUtilImpl extends InetAddressUtil { +private[k8s] object InetAddressUtilImpl extends InetAddressUtil { // NOTE: This does issue a network call to DNS. Caching is done internally by the InetAddress // class for both hits and misses. diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala similarity index 95% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala index ff02c08ee2dce..eae9f92ed0d6f 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala @@ -14,16 +14,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.scheduler.cluster.kubernetes +package org.apache.spark.scheduler.cluster.k8s import java.io.File import io.fabric8.kubernetes.client.Config -import org.apache.spark.deploy.kubernetes._ -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.MountSmallFilesBootstrapImpl +import org.apache.spark.deploy.k8s._ +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.deploy.k8s.submit.MountSmallFilesBootstrapImpl import org.apache.spark.internal.Logging import org.apache.spark.scheduler.{ExternalClusterManager, SchedulerBackend, TaskScheduler, TaskSchedulerImpl} import org.apache.spark.util.Utils diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala similarity index 99% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index 3738e1b8d3991..c320ac779a151 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.scheduler.cluster.kubernetes +package org.apache.spark.scheduler.cluster.k8s import java.io.Closeable import java.net.InetAddress @@ -33,10 +33,10 @@ import scala.collection.JavaConverters._ import scala.concurrent.{ExecutionContext, Future} import org.apache.spark.{SparkContext, SparkEnv, SparkException} -import org.apache.spark.deploy.kubernetes._ -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.{InitContainerUtil, MountSmallFilesBootstrap} +import org.apache.spark.deploy.k8s._ +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.deploy.k8s.submit.{InitContainerUtil, MountSmallFilesBootstrap} import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.network.shuffle.kubernetes.KubernetesExternalShuffleClient import org.apache.spark.rpc.{RpcAddress, RpcCallContext, RpcEndpointAddress, RpcEnv} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesTaskSchedulerImpl.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesTaskSchedulerImpl.scala similarity index 96% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesTaskSchedulerImpl.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesTaskSchedulerImpl.scala index ca40d46400ea2..8781213d8f792 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesTaskSchedulerImpl.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesTaskSchedulerImpl.scala @@ -14,9 +14,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.scheduler.cluster.kubernetes +package org.apache.spark.scheduler.cluster.k8s -import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.k8s.config._ import org.apache.spark.scheduler.{SchedulerBackend, TaskSchedulerImpl, TaskSet, TaskSetManager} import org.apache.spark.util.Utils import org.apache.spark.SparkContext diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesTaskSetManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesTaskSetManager.scala similarity index 97% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesTaskSetManager.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesTaskSetManager.scala index 44d01a5d76b01..39c521978c5b1 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesTaskSetManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesTaskSetManager.scala @@ -14,11 +14,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.scheduler.cluster.kubernetes +package org.apache.spark.scheduler.cluster.k8s import scala.collection.mutable.ArrayBuffer -import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.k8s.config._ import org.apache.spark.scheduler.{TaskSchedulerImpl, TaskSet, TaskSetManager} private[spark] class KubernetesTaskSetManager( diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/RackResolverUtil.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/RackResolverUtil.scala similarity index 93% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/RackResolverUtil.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/RackResolverUtil.scala index 29a7dc982a5af..0b45aa0c117cc 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/RackResolverUtil.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/RackResolverUtil.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.scheduler.cluster.kubernetes +package org.apache.spark.scheduler.cluster.k8s import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.CommonConfigurationKeysPublic @@ -25,14 +25,14 @@ import org.apache.log4j.{Level, Logger} /** * Finds rack names that cluster nodes belong to in order to support HDFS rack locality. */ -private[kubernetes] trait RackResolverUtil { +private[k8s] trait RackResolverUtil { def isConfigured() : Boolean def resolveRack(hadoopConfiguration: Configuration, host: String): Option[String] } -private[kubernetes] class RackResolverUtilImpl(hadoopConfiguration: Configuration) +private[k8s] class RackResolverUtilImpl(hadoopConfiguration: Configuration) extends RackResolverUtil { val scriptPlugin : String = classOf[ScriptBasedMapping].getCanonicalName diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/ShufflePodCache.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ShufflePodCache.scala similarity index 98% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/ShufflePodCache.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ShufflePodCache.scala index 15e02664589eb..9efeee71dca5c 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/ShufflePodCache.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ShufflePodCache.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.scheduler.cluster.kubernetes +package org.apache.spark.scheduler.cluster.k8s import scala.collection.JavaConverters._ diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/InitContainerResourceStagingServerSecretPluginSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/InitContainerResourceStagingServerSecretPluginSuite.scala similarity index 96% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/InitContainerResourceStagingServerSecretPluginSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/InitContainerResourceStagingServerSecretPluginSuite.scala index 51e695fc08a36..2117a25e27fd4 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/InitContainerResourceStagingServerSecretPluginSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/InitContainerResourceStagingServerSecretPluginSuite.scala @@ -14,13 +14,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes +package org.apache.spark.deploy.k8s import io.fabric8.kubernetes.api.model._ import org.scalatest.BeforeAndAfter import scala.collection.JavaConverters._ -import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.k8s.constants._ import org.apache.spark.SparkFunSuite class InitContainerResourceStagingServerSecretPluginSuite diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/SSLUtils.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/SSLUtils.scala similarity index 97% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/SSLUtils.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/SSLUtils.scala index 8de0f56f007dc..0a41cd5d41d0f 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/SSLUtils.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/SSLUtils.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes +package org.apache.spark.deploy.k8s import java.io.{File, FileOutputStream, OutputStreamWriter} import java.math.BigInteger @@ -30,7 +30,7 @@ import org.bouncycastle.cert.jcajce.{JcaX509CertificateConverter, JcaX509v3Certi import org.bouncycastle.openssl.jcajce.JcaPEMWriter import org.bouncycastle.operator.jcajce.JcaContentSignerBuilder -import org.apache.spark.deploy.kubernetes.submit.{KeyAndCertPem, KeyStoreAndTrustStore} +import org.apache.spark.deploy.k8s.submit.{KeyAndCertPem, KeyStoreAndTrustStore} import org.apache.spark.util.Utils private[spark] object SSLUtils { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/SparkPodInitContainerBootstrapSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/SparkPodInitContainerBootstrapSuite.scala similarity index 98% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/SparkPodInitContainerBootstrapSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/SparkPodInitContainerBootstrapSuite.scala index d5f25983f5080..946fde95ca80b 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/SparkPodInitContainerBootstrapSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/SparkPodInitContainerBootstrapSuite.scala @@ -14,13 +14,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes +package org.apache.spark.deploy.k8s import io.fabric8.kubernetes.api.model._ import org.scalatest.BeforeAndAfter import scala.collection.JavaConverters._ -import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.k8s.constants._ import org.apache.spark.SparkFunSuite class SparkPodInitContainerBootstrapSuite extends SparkFunSuite with BeforeAndAfter { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/ClientSuite.scala similarity index 97% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/ClientSuite.scala index 35fd5c1f206fa..b36f1e4ff6fe4 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/ClientSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit +package org.apache.spark.deploy.k8s.submit import scala.collection.JavaConverters._ @@ -30,9 +30,9 @@ import org.scalatest.BeforeAndAfter import org.scalatest.mock.MockitoSugar._ import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.submitsteps.{DriverConfigurationStep, KubernetesDriverSpec} +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.deploy.k8s.submit.submitsteps.{DriverConfigurationStep, KubernetesDriverSpec} private[spark] class ClientSuite extends SparkFunSuite with BeforeAndAfter { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestratorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala similarity index 93% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestratorSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala index b87fa48291a8c..d64a6205174fd 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestratorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala @@ -14,11 +14,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit +package org.apache.spark.deploy.k8s.submit import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.submit.submitsteps.{BaseDriverConfigurationStep, DependencyResolutionStep, DriverConfigurationStep, DriverKubernetesCredentialsStep, HadoopConfigBootstrapStep, InitContainerBootstrapStep, MountSmallLocalFilesStep, PythonStep} +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.submit.submitsteps.{BaseDriverConfigurationStep, DependencyResolutionStep, DriverConfigurationStep, DriverKubernetesCredentialsStep, HadoopConfigBootstrapStep, InitContainerBootstrapStep, MountSmallLocalFilesStep, PythonStep} private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunSuite { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/HadoopConfBootstrapSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/HadoopConfBootstrapSuite.scala similarity index 94% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/HadoopConfBootstrapSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/HadoopConfBootstrapSuite.scala index 64426f0deb15e..28f6c9e53a4cc 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/HadoopConfBootstrapSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/HadoopConfBootstrapSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit +package org.apache.spark.deploy.k8s.submit import java.io.File import java.util.UUID @@ -28,8 +28,8 @@ import org.mockito.Mockito.when import org.scalatest.BeforeAndAfter import org.apache.spark.SparkFunSuite -import org.apache.spark.deploy.kubernetes.{HadoopConfBootstrapImpl, HadoopUGIUtil, PodWithMainContainer} -import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.k8s.{HadoopConfBootstrapImpl, HadoopUGIUtil, PodWithMainContainer} +import org.apache.spark.deploy.k8s.constants._ import org.apache.spark.util.Utils private[spark] class HadoopConfBootstrapSuite extends SparkFunSuite with BeforeAndAfter{ diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/KerberosTokenConfBootstrapSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KerberosTokenConfBootstrapSuite.scala similarity index 91% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/KerberosTokenConfBootstrapSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KerberosTokenConfBootstrapSuite.scala index 95d7159736213..13ee6790aa4e1 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/KerberosTokenConfBootstrapSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KerberosTokenConfBootstrapSuite.scala @@ -14,15 +14,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit +package org.apache.spark.deploy.k8s.submit import scala.collection.JavaConverters._ import io.fabric8.kubernetes.api.model._ import org.apache.spark.SparkFunSuite -import org.apache.spark.deploy.kubernetes.{KerberosTokenConfBootstrapImpl, PodWithMainContainer} -import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.k8s.{KerberosTokenConfBootstrapImpl, PodWithMainContainer} +import org.apache.spark.deploy.k8s.constants._ private[spark] class KerberosTokenConfBootstrapSuite extends SparkFunSuite { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesExternalShuffleServiceSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesExternalShuffleServiceSuite.scala similarity index 91% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesExternalShuffleServiceSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesExternalShuffleServiceSuite.scala index 425ba58a65d19..33ddf6f494dc3 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesExternalShuffleServiceSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesExternalShuffleServiceSuite.scala @@ -14,11 +14,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit +package org.apache.spark.deploy.k8s.submit import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.KubernetesExternalShuffleService +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.KubernetesExternalShuffleService import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.network.shuffle.kubernetes.KubernetesExternalShuffleClient diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SSLFilePairs.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/SSLFilePairs.scala similarity index 94% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SSLFilePairs.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/SSLFilePairs.scala index 5240128743b76..83181c1fe3c4e 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SSLFilePairs.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/SSLFilePairs.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit +package org.apache.spark.deploy.k8s.submit import java.io.File diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedDependencyUploaderSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/SubmittedDependencyUploaderSuite.scala similarity index 96% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedDependencyUploaderSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/SubmittedDependencyUploaderSuite.scala index 96fa92c254297..42114f1a6f82b 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedDependencyUploaderSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/SubmittedDependencyUploaderSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit +package org.apache.spark.deploy.k8s.submit import java.io.{ByteArrayInputStream, ByteArrayOutputStream, File} import java.util.UUID @@ -31,8 +31,8 @@ import org.scalatest.mock.MockitoSugar._ import retrofit2.{Call, Response} import org.apache.spark.{SparkFunSuite, SSLOptions} -import org.apache.spark.deploy.kubernetes.CompressionUtils -import org.apache.spark.deploy.rest.kubernetes.{ResourceStagingServiceRetrofit, RetrofitClientFactory, StagedResourcesOwner} +import org.apache.spark.deploy.k8s.CompressionUtils +import org.apache.spark.deploy.rest.k8s.{ResourceStagingServiceRetrofit, RetrofitClientFactory, StagedResourcesOwner} import org.apache.spark.util.Utils private[spark] class SubmittedDependencyUploaderSuite extends SparkFunSuite with BeforeAndAfter { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/BaseDriverConfigurationStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/BaseDriverConfigurationStepSuite.scala similarity index 97% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/BaseDriverConfigurationStepSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/BaseDriverConfigurationStepSuite.scala index 53c4ea60afacd..0dea10528179e 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/BaseDriverConfigurationStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/BaseDriverConfigurationStepSuite.scala @@ -14,14 +14,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps +package org.apache.spark.deploy.k8s.submit.submitsteps import io.fabric8.kubernetes.api.model.{ContainerBuilder, HasMetadata, PodBuilder} import scala.collection.JavaConverters._ import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ private[spark] class BaseDriverConfigurationStepSuite extends SparkFunSuite { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DependencyResolutionStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DependencyResolutionStepSuite.scala similarity index 96% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DependencyResolutionStepSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DependencyResolutionStepSuite.scala index 3f7ec61074b0c..02b2820acaf3f 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DependencyResolutionStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DependencyResolutionStepSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps +package org.apache.spark.deploy.k8s.submit.submitsteps import java.io.File @@ -22,7 +22,7 @@ import io.fabric8.kubernetes.api.model.{ContainerBuilder, HasMetadata, PodBuilde import scala.collection.JavaConverters._ import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.k8s.constants._ private[spark] class DependencyResolutionStepSuite extends SparkFunSuite { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverKubernetesCredentialsStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DriverKubernetesCredentialsStepSuite.scala similarity index 97% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverKubernetesCredentialsStepSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DriverKubernetesCredentialsStepSuite.scala index 3d5664713a2b8..b29edca05f4a0 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverKubernetesCredentialsStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DriverKubernetesCredentialsStepSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps +package org.apache.spark.deploy.k8s.submit.submitsteps import java.io.File @@ -25,8 +25,8 @@ import org.scalatest.BeforeAndAfter import scala.collection.JavaConverters._ import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ import org.apache.spark.util.Utils private[spark] class DriverKubernetesCredentialsStepSuite diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/HadoopConfigBootstrapStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/HadoopConfigBootstrapStepSuite.scala similarity index 93% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/HadoopConfigBootstrapStepSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/HadoopConfigBootstrapStepSuite.scala index 074a02c339d95..12de82600f3b9 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/HadoopConfigBootstrapStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/HadoopConfigBootstrapStepSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps +package org.apache.spark.deploy.k8s.submit.submitsteps import scala.collection.JavaConverters._ @@ -25,8 +25,8 @@ import org.mockito.Mockito.when import org.scalatest.BeforeAndAfter import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.submitsteps.hadoopsteps.{HadoopConfigSpec, HadoopConfigurationStep} +import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.deploy.k8s.submit.submitsteps.hadoopsteps.{HadoopConfigSpec, HadoopConfigurationStep} private[spark] class HadoopConfigBootstrapStepSuite extends SparkFunSuite with BeforeAndAfter{ diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/InitContainerBootstrapStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/InitContainerBootstrapStepSuite.scala similarity index 96% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/InitContainerBootstrapStepSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/InitContainerBootstrapStepSuite.scala index 1c9a1ac50229e..ee3fe126b40f7 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/InitContainerBootstrapStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/InitContainerBootstrapStepSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps +package org.apache.spark.deploy.k8s.submit.submitsteps import java.io.StringReader import java.util.Properties @@ -26,9 +26,9 @@ import io.fabric8.kubernetes.api.model.{ConfigMap, Container, ContainerBuilder, import scala.collection.JavaConverters._ import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer.{InitContainerConfigurationStep, InitContainerSpec} +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.deploy.k8s.submit.submitsteps.initcontainer.{InitContainerConfigurationStep, InitContainerSpec} import org.apache.spark.util.Utils class InitContainerBootstrapStepSuite extends SparkFunSuite { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/MountSmallLocalFilesStepTest.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/MountSmallLocalFilesStepTest.scala similarity index 96% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/MountSmallLocalFilesStepTest.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/MountSmallLocalFilesStepTest.scala index 11744ef409c20..f67ddcf4e2010 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/MountSmallLocalFilesStepTest.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/MountSmallLocalFilesStepTest.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps +package org.apache.spark.deploy.k8s.submit.submitsteps import java.io.{File, RandomAccessFile} @@ -27,9 +27,9 @@ import org.scalatest.BeforeAndAfter import scala.collection.JavaConverters._ import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.MountSmallFilesBootstrap +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.deploy.k8s.submit.MountSmallFilesBootstrap import org.apache.spark.util.Utils private[spark] class MountSmallLocalFilesStepTest extends SparkFunSuite with BeforeAndAfter { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/PythonStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/PythonStepSuite.scala similarity index 97% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/PythonStepSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/PythonStepSuite.scala index ce0dcee6acc46..66218f130b9c4 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/PythonStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/PythonStepSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps +package org.apache.spark.deploy.k8s.submit.submitsteps import io.fabric8.kubernetes.api.model._ import org.scalatest.BeforeAndAfter diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopConfMounterStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopConfMounterStepSuite.scala similarity index 94% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopConfMounterStepSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopConfMounterStepSuite.scala index 876675d5a4d96..41a889dce7ccb 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopConfMounterStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopConfMounterStepSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps.hadoopsteps +package org.apache.spark.deploy.k8s.submit.submitsteps.hadoopsteps import java.io.File import java.util.UUID @@ -32,8 +32,8 @@ import org.mockito.stubbing.Answer import org.scalatest.BeforeAndAfter import org.apache.spark.SparkFunSuite -import org.apache.spark.deploy.kubernetes.{HadoopConfBootstrap, PodWithMainContainer} -import org.apache.spark.deploy.kubernetes.constants.HADOOP_CONF_DIR_LOC +import org.apache.spark.deploy.k8s.{HadoopConfBootstrap, PodWithMainContainer} +import org.apache.spark.deploy.k8s.constants.HADOOP_CONF_DIR_LOC import org.apache.spark.util.Utils diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStepSuite.scala similarity index 96% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStepSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStepSuite.scala index 997556d1508a8..f5228112a0453 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosKeytabResolverStepSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps.hadoopsteps +package org.apache.spark.deploy.k8s.submit.submitsteps.hadoopsteps import java.io.File import java.util.UUID @@ -36,8 +36,8 @@ import org.mockito.stubbing.Answer import org.scalatest.BeforeAndAfter import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.kubernetes.HadoopUGIUtil -import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.k8s.HadoopUGIUtil +import org.apache.spark.deploy.k8s.constants._ import org.apache.spark.util.Utils diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosSecretResolverStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosSecretResolverStepSuite.scala similarity index 95% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosSecretResolverStepSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosSecretResolverStepSuite.scala index bb0fa237a2d5e..2b5a8c57e109c 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopKerberosSecretResolverStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopKerberosSecretResolverStepSuite.scala @@ -14,14 +14,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps.hadoopsteps +package org.apache.spark.deploy.k8s.submit.submitsteps.hadoopsteps import scala.collection.JavaConverters._ import io.fabric8.kubernetes.api.model._ import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.k8s.constants._ private[spark] class HadoopKerberosSecretResolverStepSuite extends SparkFunSuite { private val CONFIG_MAP_NAME = "config-map" diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopStepsOrchestratorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopStepsOrchestratorSuite.scala similarity index 96% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopStepsOrchestratorSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopStepsOrchestratorSuite.scala index 1f946666f53ce..539b682d175dd 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/hadoopsteps/HadoopStepsOrchestratorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/hadoopsteps/HadoopStepsOrchestratorSuite.scala @@ -14,10 +14,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps.hadoopsteps +package org.apache.spark.deploy.k8s.submit.submitsteps.hadoopsteps import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.k8s.config._ private[spark] class HadoopStepsOrchestratorSuite extends SparkFunSuite { private val NAMESPACE = "testNamespace" diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStepSuite.scala similarity index 94% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStepSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStepSuite.scala index 79545a55f33b2..53e768c0cb898 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStepSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer +package org.apache.spark.deploy.k8s.submit.submitsteps.initcontainer import io.fabric8.kubernetes.api.model._ import org.mockito.{Mock, MockitoAnnotations} @@ -26,8 +26,8 @@ import org.scalatest.BeforeAndAfter import scala.collection.JavaConverters._ import org.apache.spark.SparkFunSuite -import org.apache.spark.deploy.kubernetes.{PodWithDetachedInitContainer, SparkPodInitContainerBootstrap} -import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.k8s.{PodWithDetachedInitContainer, SparkPodInitContainerBootstrap} +import org.apache.spark.deploy.k8s.config._ private[spark] class BaseInitContainerConfigurationStepSuite extends SparkFunSuite with BeforeAndAfter{ diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestratorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestratorSuite.scala similarity index 95% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestratorSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestratorSuite.scala index 1cc8007803457..a5495e393ea33 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestratorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestratorSuite.scala @@ -14,11 +14,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer +package org.apache.spark.deploy.k8s.submit.submitsteps.initcontainer import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ class InitContainerConfigurationStepsOrchestratorSuite extends SparkFunSuite { private val NAMESPACE = "namespace" diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerStepSuite.scala similarity index 96% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerStepSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerStepSuite.scala index e9c828fdb397c..7894d6fc191ac 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerStepSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer +package org.apache.spark.deploy.k8s.submit.submitsteps.initcontainer import java.io.File import java.util.UUID @@ -31,10 +31,10 @@ import org.scalatest.BeforeAndAfter import scala.collection.JavaConverters._ import org.apache.spark.SparkFunSuite -import org.apache.spark.deploy.kubernetes.InitContainerResourceStagingServerSecretPlugin -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.{SubmittedDependencyUploader, SubmittedResourceIdAndSecret} +import org.apache.spark.deploy.k8s.InitContainerResourceStagingServerSecretPlugin +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.deploy.k8s.submit.{SubmittedDependencyUploader, SubmittedResourceIdAndSecret} import org.apache.spark.util.Utils private[spark] class SubmittedResourcesInitContainerStepSuite diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkDependencyDownloadInitContainerSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/KubernetesSparkDependencyDownloadInitContainerSuite.scala similarity index 98% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkDependencyDownloadInitContainerSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/KubernetesSparkDependencyDownloadInitContainerSuite.scala index f2fdf026390cd..4c80637da7843 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkDependencyDownloadInitContainerSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/KubernetesSparkDependencyDownloadInitContainerSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import java.io.{ByteArrayOutputStream, File} import java.util.UUID @@ -33,8 +33,8 @@ import org.scalatest.mock.MockitoSugar._ import retrofit2.{Call, Callback, Response} import org.apache.spark.{SparkConf, SparkFunSuite, SSLOptions} -import org.apache.spark.deploy.kubernetes.CompressionUtils -import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.k8s.CompressionUtils +import org.apache.spark.deploy.k8s.config._ import org.apache.spark.util.Utils class KubernetesSparkDependencyDownloadInitContainerSuite diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServerSslOptionsProviderSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingServerSslOptionsProviderSuite.scala similarity index 98% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServerSslOptionsProviderSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingServerSslOptionsProviderSuite.scala index 3bb318d713a54..1ab7a7f4f966d 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServerSslOptionsProviderSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingServerSslOptionsProviderSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import java.io.{File, FileInputStream, StringWriter} import java.security.KeyStore @@ -25,7 +25,7 @@ import org.bouncycastle.openssl.jcajce.JcaPEMWriter import org.scalatest.BeforeAndAfter import org.apache.spark.{SparkConf, SparkFunSuite, SSLOptions} -import org.apache.spark.deploy.kubernetes.SSLUtils +import org.apache.spark.deploy.k8s.SSLUtils import org.apache.spark.util.Utils class ResourceStagingServerSslOptionsProviderSuite extends SparkFunSuite with BeforeAndAfter { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServerSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingServerSuite.scala similarity index 98% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServerSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingServerSuite.scala index 1bcd85a611e00..4e6b4abfb6874 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServerSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingServerSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import java.net.ServerSocket import javax.ws.rs.core.MediaType @@ -29,7 +29,7 @@ import org.scalatest.mock.MockitoSugar.mock import retrofit2.Call import org.apache.spark.{SparkFunSuite, SSLOptions} -import org.apache.spark.deploy.kubernetes.SSLUtils +import org.apache.spark.deploy.k8s.SSLUtils import org.apache.spark.internal.Logging import org.apache.spark.util.Utils diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesCleanerSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/StagedResourcesCleanerSuite.scala similarity index 99% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesCleanerSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/StagedResourcesCleanerSuite.scala index 8b398a9891f34..d9b5c4ac5b9c5 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesCleanerSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/StagedResourcesCleanerSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import java.util.concurrent.{ScheduledExecutorService, TimeUnit} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesStoreSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/StagedResourcesStoreSuite.scala similarity index 98% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesStoreSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/StagedResourcesStoreSuite.scala index 6b5737ebf2e23..b58046da67800 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesStoreSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/StagedResourcesStoreSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import java.io.{ByteArrayInputStream, File} import java.nio.file.Paths diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesTaskSchedulerImplSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesTaskSchedulerImplSuite.scala similarity index 98% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesTaskSchedulerImplSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesTaskSchedulerImplSuite.scala index e10b9508530f7..4924b67480048 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesTaskSchedulerImplSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesTaskSchedulerImplSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.scheduler.cluster.kubernetes +package org.apache.spark.scheduler.cluster.k8s import io.fabric8.kubernetes.api.model.{Pod, PodSpec, PodStatus} import org.mockito.Matchers._ @@ -22,7 +22,7 @@ import org.mockito.Mockito._ import org.scalatest.BeforeAndAfter import org.apache.spark.{SparkContext, SparkFunSuite} -import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.k8s.config._ import org.apache.spark.scheduler.FakeTask class KubernetesTaskSchedulerImplSuite extends SparkFunSuite with BeforeAndAfter { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesTaskSetManagerSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesTaskSetManagerSuite.scala similarity index 98% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesTaskSetManagerSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesTaskSetManagerSuite.scala index 889758731a6c1..56c576a7dd449 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesTaskSetManagerSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesTaskSetManagerSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.scheduler.cluster.kubernetes +package org.apache.spark.scheduler.cluster.k8s import scala.collection.mutable.ArrayBuffer @@ -23,7 +23,7 @@ import org.mockito.Mockito._ import org.scalatest.BeforeAndAfter import org.apache.spark.{SparkContext, SparkFunSuite} -import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.k8s.config._ import org.apache.spark.scheduler.{FakeTask, FakeTaskScheduler, HostTaskLocation, TaskLocation} class KubernetesTaskSetManagerSuite extends SparkFunSuite with BeforeAndAfter { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/kubernetes/RackResolverUtilImplSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/RackResolverUtilImplSuite.scala similarity index 97% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/kubernetes/RackResolverUtilImplSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/RackResolverUtilImplSuite.scala index ee671ce7d6590..ec1428117280f 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/kubernetes/RackResolverUtilImplSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/RackResolverUtilImplSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.scheduler.cluster.kubernetes +package org.apache.spark.scheduler.cluster.k8s import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.CommonConfigurationKeysPublic diff --git a/resource-managers/kubernetes/integration-tests-spark-jobs-helpers/src/main/java/org/apache/spark/deploy/kubernetes/integrationtest/PiHelper.java b/resource-managers/kubernetes/integration-tests-spark-jobs-helpers/src/main/java/org/apache/spark/deploy/k8s/integrationtest/PiHelper.java similarity index 94% rename from resource-managers/kubernetes/integration-tests-spark-jobs-helpers/src/main/java/org/apache/spark/deploy/kubernetes/integrationtest/PiHelper.java rename to resource-managers/kubernetes/integration-tests-spark-jobs-helpers/src/main/java/org/apache/spark/deploy/k8s/integrationtest/PiHelper.java index 99d982397bb6e..b18f8deea6a20 100644 --- a/resource-managers/kubernetes/integration-tests-spark-jobs-helpers/src/main/java/org/apache/spark/deploy/kubernetes/integrationtest/PiHelper.java +++ b/resource-managers/kubernetes/integration-tests-spark-jobs-helpers/src/main/java/org/apache/spark/deploy/k8s/integrationtest/PiHelper.java @@ -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; +package org.apache.spark.deploy.k8s.integrationtest; /** * Primarily extracted so that a separate jar can be added as a dependency for the diff --git a/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/FileExistenceTest.scala b/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/k8s/integrationtest/jobs/FileExistenceTest.scala similarity index 97% rename from resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/FileExistenceTest.scala rename to resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/k8s/integrationtest/jobs/FileExistenceTest.scala index a9e328f4ff248..be1c3f6dc3af4 100644 --- a/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/FileExistenceTest.scala +++ b/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/k8s/integrationtest/jobs/FileExistenceTest.scala @@ -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.jobs +package org.apache.spark.deploy.k8s.integrationtest.jobs import java.io.File import java.nio.file.Paths diff --git a/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/GroupByTest.scala b/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/k8s/integrationtest/jobs/GroupByTest.scala similarity index 96% rename from resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/GroupByTest.scala rename to resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/k8s/integrationtest/jobs/GroupByTest.scala index fe47d42485b24..7bae66cbf71f7 100644 --- a/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/GroupByTest.scala +++ b/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/k8s/integrationtest/jobs/GroupByTest.scala @@ -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.jobs +package org.apache.spark.deploy.k8s.integrationtest.jobs import java.util.Random diff --git a/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/HDFSTest.scala b/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/k8s/integrationtest/jobs/HDFSTest.scala similarity index 96% rename from resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/HDFSTest.scala rename to resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/k8s/integrationtest/jobs/HDFSTest.scala index 463dd25d14a1f..75536739e3426 100644 --- a/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/HDFSTest.scala +++ b/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/k8s/integrationtest/jobs/HDFSTest.scala @@ -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.jobs +package org.apache.spark.deploy.k8s.integrationtest.jobs import org.apache.spark.sql.SparkSession diff --git a/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/JavaOptionsTest.scala b/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/k8s/integrationtest/jobs/JavaOptionsTest.scala similarity index 98% rename from resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/JavaOptionsTest.scala rename to resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/k8s/integrationtest/jobs/JavaOptionsTest.scala index 7d457fc044227..d0cd7a5329c9b 100644 --- a/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/JavaOptionsTest.scala +++ b/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/k8s/integrationtest/jobs/JavaOptionsTest.scala @@ -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.jobs +package org.apache.spark.deploy.k8s.integrationtest.jobs import java.io.{File, FileInputStream} import java.util.Properties diff --git a/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/SparkPiWithInfiniteWait.scala b/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/k8s/integrationtest/jobs/SparkPiWithInfiniteWait.scala similarity index 92% rename from resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/SparkPiWithInfiniteWait.scala rename to resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/k8s/integrationtest/jobs/SparkPiWithInfiniteWait.scala index d3372749f999e..5ae22671e41f7 100644 --- a/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/SparkPiWithInfiniteWait.scala +++ b/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/k8s/integrationtest/jobs/SparkPiWithInfiniteWait.scala @@ -14,9 +14,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.integrationtest.jobs +package org.apache.spark.deploy.k8s.integrationtest.jobs -import org.apache.spark.deploy.kubernetes.integrationtest.PiHelper +import org.apache.spark.deploy.k8s.integrationtest.PiHelper import org.apache.spark.sql.SparkSession // Equivalent to SparkPi except does not stop the Spark Context diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberizedHadoopClusterLauncher.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KerberizedHadoopClusterLauncher.scala similarity index 95% rename from resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberizedHadoopClusterLauncher.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KerberizedHadoopClusterLauncher.scala index 687c125eb583b..37567f5ab8323 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberizedHadoopClusterLauncher.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KerberizedHadoopClusterLauncher.scala @@ -14,11 +14,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.integrationtest +package org.apache.spark.deploy.k8s.integrationtest import io.fabric8.kubernetes.client.KubernetesClient -import org.apache.spark.deploy.kubernetes.integrationtest.kerberos._ +import org.apache.spark.deploy.k8s.integrationtest.kerberos._ import org.apache.spark.internal.Logging /** diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberosTestPodLauncher.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KerberosTestPodLauncher.scala similarity index 97% rename from resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberosTestPodLauncher.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KerberosTestPodLauncher.scala index 53d372faace41..2add192218543 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KerberosTestPodLauncher.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KerberosTestPodLauncher.scala @@ -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 +package org.apache.spark.deploy.k8s.integrationtest import java.io.{File, FileInputStream} @@ -24,7 +24,7 @@ import io.fabric8.kubernetes.api.model.KeyToPathBuilder import io.fabric8.kubernetes.api.model.extensions.{Deployment, DeploymentBuilder} import io.fabric8.kubernetes.client.KubernetesClient -import org.apache.spark.deploy.kubernetes.submit.ContainerNameEqualityPredicate +import org.apache.spark.deploy.k8s.submit.ContainerNameEqualityPredicate /** * This class is responsible for launching a pod that runs spark-submit to simulate diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala similarity index 96% rename from resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala index 17d48b8b508dd..c8235845e28df 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala @@ -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 +package org.apache.spark.deploy.k8s.integrationtest import java.io.{File, FileOutputStream} import java.nio.file.Paths @@ -29,13 +29,13 @@ import org.scalatest.time.{Minutes, Seconds, Span} import scala.collection.JavaConverters._ import org.apache.spark.{SparkConf, SparkFunSuite, SSLOptions} -import org.apache.spark.deploy.kubernetes.SSLUtils -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.integrationtest.backend.IntegrationTestBackendFactory -import org.apache.spark.deploy.kubernetes.integrationtest.backend.minikube.Minikube -import org.apache.spark.deploy.kubernetes.integrationtest.constants.MINIKUBE_TEST_BACKEND -import org.apache.spark.deploy.kubernetes.integrationtest.kerberos.KerberosDriverWatcherCache -import org.apache.spark.deploy.kubernetes.submit.{Client, ClientArguments, JavaMainAppResource, KeyAndCertPem, MainAppResource, PythonMainAppResource} +import org.apache.spark.deploy.k8s.SSLUtils +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.integrationtest.backend.IntegrationTestBackendFactory +import org.apache.spark.deploy.k8s.integrationtest.backend.minikube.Minikube +import org.apache.spark.deploy.k8s.integrationtest.constants.MINIKUBE_TEST_BACKEND +import org.apache.spark.deploy.k8s.integrationtest.kerberos.KerberosDriverWatcherCache +import org.apache.spark.deploy.k8s.submit.{Client, ClientArguments, JavaMainAppResource, KeyAndCertPem, MainAppResource, PythonMainAppResource} import org.apache.spark.launcher.SparkLauncher import org.apache.spark.util.Utils diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesTestComponents.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesTestComponents.scala similarity index 96% rename from resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesTestComponents.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesTestComponents.scala index c04bb0c525d86..0fef59f3585b6 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesTestComponents.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesTestComponents.scala @@ -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 +package org.apache.spark.deploy.k8s.integrationtest import java.util.UUID @@ -23,7 +23,7 @@ import org.scalatest.concurrent.Eventually import scala.collection.JavaConverters._ import org.apache.spark.SparkConf -import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.k8s.config._ private[spark] class KubernetesTestComponents(defaultClient: DefaultKubernetesClient) { diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/ProcessUtils.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ProcessUtils.scala similarity index 97% rename from resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/ProcessUtils.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ProcessUtils.scala index 4008007b72fc4..ed15f66f53458 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/ProcessUtils.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ProcessUtils.scala @@ -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 +package org.apache.spark.deploy.k8s.integrationtest import java.io.{BufferedReader, InputStreamReader} import java.util.concurrent.TimeUnit diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/ResourceStagingServerLauncher.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ResourceStagingServerLauncher.scala similarity index 97% rename from resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/ResourceStagingServerLauncher.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ResourceStagingServerLauncher.scala index e5e1b1f085f9f..1aa90f4ce691d 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/ResourceStagingServerLauncher.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ResourceStagingServerLauncher.scala @@ -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 +package org.apache.spark.deploy.k8s.integrationtest import java.io.{File, StringWriter} import java.util.Properties @@ -25,8 +25,8 @@ import io.fabric8.kubernetes.client.KubernetesClient import scala.collection.JavaConverters._ import org.apache.spark.SSLOptions -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.submit.{ContainerNameEqualityPredicate, KeyAndCertPem} +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.submit.{ContainerNameEqualityPredicate, KeyAndCertPem} import org.apache.spark.util.Utils /** diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/SparkReadinessWatcher.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/SparkReadinessWatcher.scala similarity index 96% rename from resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/SparkReadinessWatcher.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/SparkReadinessWatcher.scala index bd604ab94b936..f1fd6dc19ce54 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/SparkReadinessWatcher.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/SparkReadinessWatcher.scala @@ -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 +package org.apache.spark.deploy.k8s.integrationtest import java.util.concurrent.TimeUnit diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/StaticAssetServerLauncher.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/StaticAssetServerLauncher.scala similarity index 97% rename from resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/StaticAssetServerLauncher.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/StaticAssetServerLauncher.scala index 6b483769f5254..69a9df63bc4af 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/StaticAssetServerLauncher.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/StaticAssetServerLauncher.scala @@ -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 +package org.apache.spark.deploy.k8s.integrationtest import io.fabric8.kubernetes.api.model.{HTTPGetActionBuilder, Pod} import io.fabric8.kubernetes.client.KubernetesClient diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/GCE/GCETestBackend.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/GCE/GCETestBackend.scala similarity index 81% rename from resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/GCE/GCETestBackend.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/GCE/GCETestBackend.scala index 1ef096be4af02..09a3118a202eb 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/GCE/GCETestBackend.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/GCE/GCETestBackend.scala @@ -14,13 +14,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.integrationtest.backend.GCE +package org.apache.spark.deploy.k8s.integrationtest.backend.GCE import io.fabric8.kubernetes.client.{ConfigBuilder, DefaultKubernetesClient} -import org.apache.spark.deploy.kubernetes.config.resolveK8sMaster -import org.apache.spark.deploy.kubernetes.integrationtest.backend.IntegrationTestBackend -import org.apache.spark.deploy.kubernetes.integrationtest.constants.GCE_TEST_BACKEND +import org.apache.spark.deploy.k8s.config.resolveK8sMaster +import org.apache.spark.deploy.k8s.integrationtest.backend.IntegrationTestBackend +import org.apache.spark.deploy.k8s.integrationtest.constants.GCE_TEST_BACKEND private[spark] class GCETestBackend(val master: String) extends IntegrationTestBackend { private var defaultClient: DefaultKubernetesClient = _ diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/IntegrationTestBackend.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/IntegrationTestBackend.scala similarity index 78% rename from resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/IntegrationTestBackend.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/IntegrationTestBackend.scala index c5bc923dd51a6..b61daf65a4093 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/IntegrationTestBackend.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/IntegrationTestBackend.scala @@ -15,13 +15,13 @@ * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.integrationtest.backend +package org.apache.spark.deploy.k8s.integrationtest.backend import io.fabric8.kubernetes.client.DefaultKubernetesClient -import org.apache.spark.deploy.kubernetes.integrationtest.backend.GCE.GCETestBackend -import org.apache.spark.deploy.kubernetes.integrationtest.backend.minikube.{Minikube, MinikubeTestBackend} -import org.apache.spark.deploy.kubernetes.integrationtest.docker.SparkDockerImageBuilder +import org.apache.spark.deploy.k8s.integrationtest.backend.GCE.GCETestBackend +import org.apache.spark.deploy.k8s.integrationtest.backend.minikube.{Minikube, MinikubeTestBackend} +import org.apache.spark.deploy.k8s.integrationtest.docker.SparkDockerImageBuilder private[spark] trait IntegrationTestBackend { def name(): String diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/minikube/Minikube.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/minikube/Minikube.scala similarity index 97% rename from resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/minikube/Minikube.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/minikube/Minikube.scala index 7c4b344e8f72b..bd31bde6dabf4 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/minikube/Minikube.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/minikube/Minikube.scala @@ -14,13 +14,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.integrationtest.backend.minikube +package org.apache.spark.deploy.k8s.integrationtest.backend.minikube import java.nio.file.Paths import io.fabric8.kubernetes.client.{ConfigBuilder, DefaultKubernetesClient} -import org.apache.spark.deploy.kubernetes.integrationtest.ProcessUtils +import org.apache.spark.deploy.k8s.integrationtest.ProcessUtils import org.apache.spark.internal.Logging import org.apache.spark.util.Utils diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/minikube/MinikubeTestBackend.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/minikube/MinikubeTestBackend.scala similarity index 81% rename from resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/minikube/MinikubeTestBackend.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/minikube/MinikubeTestBackend.scala index 461264877edc2..8e94f13360536 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/minikube/MinikubeTestBackend.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/minikube/MinikubeTestBackend.scala @@ -14,13 +14,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.integrationtest.backend.minikube +package org.apache.spark.deploy.k8s.integrationtest.backend.minikube import io.fabric8.kubernetes.client.DefaultKubernetesClient -import org.apache.spark.deploy.kubernetes.integrationtest.backend.IntegrationTestBackend -import org.apache.spark.deploy.kubernetes.integrationtest.constants.MINIKUBE_TEST_BACKEND -import org.apache.spark.deploy.kubernetes.integrationtest.docker.SparkDockerImageBuilder +import org.apache.spark.deploy.k8s.integrationtest.backend.IntegrationTestBackend +import org.apache.spark.deploy.k8s.integrationtest.constants.MINIKUBE_TEST_BACKEND +import org.apache.spark.deploy.k8s.integrationtest.docker.SparkDockerImageBuilder private[spark] class MinikubeTestBackend extends IntegrationTestBackend { private var defaultClient: DefaultKubernetesClient = _ diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/constants.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/constants.scala similarity index 93% rename from resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/constants.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/constants.scala index bfded1003fc25..0807a68cd823c 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/constants.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/constants.scala @@ -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 +package org.apache.spark.deploy.k8s.integrationtest package object constants { val MINIKUBE_TEST_BACKEND = "minikube" diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/docker/SparkDockerImageBuilder.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/docker/SparkDockerImageBuilder.scala similarity index 98% rename from resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/docker/SparkDockerImageBuilder.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/docker/SparkDockerImageBuilder.scala index a210aab4b78fe..d8243bd1850e0 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/docker/SparkDockerImageBuilder.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/docker/SparkDockerImageBuilder.scala @@ -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.docker +package org.apache.spark.deploy.k8s.integrationtest.docker import java.io.File import java.net.URI diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosCMWatcherCache.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/kerberos/KerberosCMWatcherCache.scala similarity index 98% rename from resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosCMWatcherCache.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/kerberos/KerberosCMWatcherCache.scala index 59968534c8312..953ccbc9a3775 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosCMWatcherCache.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/kerberos/KerberosCMWatcherCache.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.integrationtest.kerberos +package org.apache.spark.deploy.k8s.integrationtest.kerberos import java.util.concurrent.locks.{Condition, Lock, ReentrantLock} diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosDeployment.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/kerberos/KerberosDeployment.scala similarity index 93% rename from resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosDeployment.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/kerberos/KerberosDeployment.scala index a60011153f5d5..0f2719575bf58 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosDeployment.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/kerberos/KerberosDeployment.scala @@ -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.kerberos +package org.apache.spark.deploy.k8s.integrationtest.kerberos import io.fabric8.kubernetes.api.model.Service import io.fabric8.kubernetes.api.model.extensions.Deployment diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosDriverWatcherCache.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/kerberos/KerberosDriverWatcherCache.scala similarity index 98% rename from resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosDriverWatcherCache.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/kerberos/KerberosDriverWatcherCache.scala index b7a2176194e2c..cf87572e9b4d4 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosDriverWatcherCache.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/kerberos/KerberosDriverWatcherCache.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.integrationtest.kerberos +package org.apache.spark.deploy.k8s.integrationtest.kerberos import java.util.concurrent.locks.{Condition, Lock, ReentrantLock} diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosPVWatcherCache.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/kerberos/KerberosPVWatcherCache.scala similarity index 99% rename from resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosPVWatcherCache.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/kerberos/KerberosPVWatcherCache.scala index b58aa2e0e532e..2ef0fb9a5b510 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosPVWatcherCache.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/kerberos/KerberosPVWatcherCache.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.integrationtest.kerberos +package org.apache.spark.deploy.k8s.integrationtest.kerberos import java.util.concurrent.locks.{Condition, Lock, ReentrantLock} diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosPodWatcherCache.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/kerberos/KerberosPodWatcherCache.scala similarity index 99% rename from resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosPodWatcherCache.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/kerberos/KerberosPodWatcherCache.scala index 60cc9735212fa..11f28587be3e1 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosPodWatcherCache.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/kerberos/KerberosPodWatcherCache.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.integrationtest.kerberos +package org.apache.spark.deploy.k8s.integrationtest.kerberos import java.util.concurrent.locks.{Condition, Lock, ReentrantLock} diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosStorage.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/kerberos/KerberosStorage.scala similarity index 93% rename from resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosStorage.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/kerberos/KerberosStorage.scala index 28fe77c274ea1..15d0ffdeeaa2d 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosStorage.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/kerberos/KerberosStorage.scala @@ -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.kerberos +package org.apache.spark.deploy.k8s.integrationtest.kerberos import io.fabric8.kubernetes.api.model.{PersistentVolume, PersistentVolumeClaim} diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosUtils.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/kerberos/KerberosUtils.scala similarity index 97% rename from resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosUtils.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/kerberos/KerberosUtils.scala index 145683fdc5eb2..1ff0af87165c7 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/kerberos/KerberosUtils.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/kerberos/KerberosUtils.scala @@ -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.kerberos +package org.apache.spark.deploy.k8s.integrationtest.kerberos import java.io.{File, FileInputStream} @@ -25,7 +25,7 @@ import io.fabric8.kubernetes.api.model.extensions.{Deployment, DeploymentBuilder import io.fabric8.kubernetes.client.KubernetesClient import org.apache.commons.io.FileUtils.readFileToString -import org.apache.spark.deploy.kubernetes.submit.ContainerNameEqualityPredicate +import org.apache.spark.deploy.k8s.submit.ContainerNameEqualityPredicate /** * This class is responsible for handling all Utils and Constants necessary for testing diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/restapis/SparkRestApiV1.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/restapis/SparkRestApiV1.scala similarity index 96% rename from resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/restapis/SparkRestApiV1.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/restapis/SparkRestApiV1.scala index 7a3b06b1b5e58..6c7ef3c592431 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/restapis/SparkRestApiV1.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/restapis/SparkRestApiV1.scala @@ -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.restapis +package org.apache.spark.deploy.k8s.integrationtest.restapis import java.util.{List => JList} import javax.ws.rs._