Skip to content
This repository was archived by the owner on Jan 9, 2020. It is now read-only.
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Next Next commit
dynamic allocation: shuffle service docker, yaml and test fixture
  • Loading branch information
foxish committed May 14, 2017
commit 3fa9257d4d6a5f42952131ca22381ae471dd9a66
36 changes: 36 additions & 0 deletions conf/kubernetes-shuffle-service.yaml
Original file line number Diff line number Diff line change
@@ -0,0 +1,36 @@
apiVersion: extensions/v1beta1
kind: DaemonSet
metadata:
labels:
app: spark-shuffle-service
spark-version: 2.1.0
name: shuffle
spec:
template:
metadata:
labels:
app: spark-shuffle-service
spark-version: 2.1.0
spec:
volumes:
- name: temp-volume
hostPath:
path: '/var/tmp' # change this path according to your cluster configuration.
containers:
- name: shuffle
# This is an official image that is built
# from the dockerfiles/shuffle directory
# in the spark distribution.
image: kubespark/spark-shuffle:v2.1.0-kubernetes-0.1.0-alpha.3
volumeMounts:
- mountPath: '/tmp'
name: temp-volume
# more volumes can be mounted here.
# The spark job must be configured to use these
# mounts using the configuration:
# spark.kubernetes.shuffle.dir=<mount-1>,<mount-2>,...
resources:
requests:
cpu: "1"
limits:
cpu: "1"
Original file line number Diff line number Diff line change
@@ -0,0 +1,39 @@
#
# 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 openjdk:8-alpine

# 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 spark-shuffle:latest -f dockerfiles/shuffle/Dockerfile .

RUN apk upgrade --update
RUN apk add --update bash
RUN mkdir -p /opt/spark
RUN touch /opt/spark/RELEASE

ADD jars /opt/spark/jars
ADD examples /opt/spark/examples
ADD bin /opt/spark/bin
ADD sbin /opt/spark/sbin
ADD conf /opt/spark/conf

ENV SPARK_HOME /opt/spark

WORKDIR /opt/spark

CMD ["/bin/sh","-c","/opt/spark/bin/spark-class org.apache.spark.deploy.ExternalShuffleService 1"]
Original file line number Diff line number Diff line change
@@ -0,0 +1,54 @@
/*
* 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 java.util.Random

import org.apache.spark.sql.SparkSession

object GroupByTest {
def main(args: Array[String]) {
val spark = SparkSession
.builder
.appName("GroupBy Test")
.getOrCreate()

val numMappers = if (args.length > 0) args(0).toInt else 5
val numKVPairs = if (args.length > 1) args(1).toInt else 200000
val valSize = if (args.length > 2) args(2).toInt else 2
val numReducers = if (args.length > 3) args(3).toInt else numMappers

val pairs1 = spark.sparkContext.parallelize(0 until numMappers, numMappers).flatMap { p =>
val ranGen = new Random
val arr1 = new Array[(Int, Array[Byte])](numKVPairs)
for (i <- 0 until numKVPairs) {
val byteArr = new Array[Byte](valSize)
ranGen.nextBytes(byteArr)
arr1(i) = (ranGen.nextInt(Int.MaxValue), byteArr)
}
arr1
}.cache()
// Enforce that everything has been calculated and in cache
pairs1.count()

// scalastyle:off println
println(pairs1.groupByKey(numReducers).count())
// scalastyle:on println
spark.stop()
}
}

Original file line number Diff line number Diff line change
Expand Up @@ -67,4 +67,8 @@ private[spark] object KubernetesSuite {
".integrationtest.jobs.SparkPiWithInfiniteWait"
val FILE_EXISTENCE_MAIN_CLASS = "org.apache.spark.deploy.kubernetes" +
".integrationtest.jobs.FileExistenceTest"
val GROUP_BY_MAIN_CLASS = "org.apache.spark.deploy.kubernetes" +
".integrationtest.jobs.GroupByTest"

case class ShuffleNotReadyException() extends Exception
}