Skip to content

Commit c461cf4

Browse files
committed
Merge remote-tracking branch 'upstream/master'
2 parents ceb43da + ba5bcad commit c461cf4

File tree

17 files changed

+177
-61
lines changed

17 files changed

+177
-61
lines changed

README.md

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -4,8 +4,8 @@ Spark is a fast and general cluster computing system for Big Data. It provides
44
high-level APIs in Scala, Java, and Python, and an optimized engine that
55
supports general computation graphs for data analysis. It also supports a
66
rich set of higher-level tools including Spark SQL for SQL and structured
7-
data processing, MLLib for machine learning, GraphX for graph processing,
8-
and Spark Streaming.
7+
data processing, MLlib for machine learning, GraphX for graph processing,
8+
and Spark Streaming for stream processing.
99

1010
<http://spark.apache.org/>
1111

bin/pyspark

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -85,6 +85,8 @@ export PYSPARK_SUBMIT_ARGS
8585

8686
# For pyspark tests
8787
if [[ -n "$SPARK_TESTING" ]]; then
88+
unset YARN_CONF_DIR
89+
unset HADOOP_CONF_DIR
8890
if [[ -n "$PYSPARK_DOC_TEST" ]]; then
8991
exec "$PYSPARK_PYTHON" -m doctest $1
9092
else

core/src/main/scala/org/apache/spark/rdd/RDD.scala

Lines changed: 3 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -1064,11 +1064,10 @@ abstract class RDD[T: ClassTag](
10641064
// greater than totalParts because we actually cap it at totalParts in runJob.
10651065
var numPartsToTry = 1
10661066
if (partsScanned > 0) {
1067-
// If we didn't find any rows after the first iteration, just try all partitions next.
1068-
// Otherwise, interpolate the number of partitions we need to try, but overestimate it
1069-
// by 50%.
1067+
// If we didn't find any rows after the previous iteration, quadruple and retry. Otherwise,
1068+
// interpolate the number of partitions we need to try, but overestimate it by 50%.
10701069
if (buf.size == 0) {
1071-
numPartsToTry = totalParts - 1
1070+
numPartsToTry = partsScanned * 4
10721071
} else {
10731072
numPartsToTry = (1.5 * num * partsScanned / buf.size).toInt
10741073
}

ec2/spark_ec2.py

Lines changed: 25 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -102,9 +102,17 @@ def parse_args():
102102
"(for debugging)")
103103
parser.add_option(
104104
"--ebs-vol-size", metavar="SIZE", type="int", default=0,
105-
help="Attach a new EBS volume of size SIZE (in GB) to each node as " +
106-
"/vol. The volumes will be deleted when the instances terminate. " +
107-
"Only possible on EBS-backed AMIs.")
105+
help="Size (in GB) of each EBS volume.")
106+
parser.add_option(
107+
"--ebs-vol-type", default="standard",
108+
help="EBS volume type (e.g. 'gp2', 'standard').")
109+
parser.add_option(
110+
"--ebs-vol-num", type="int", default=1,
111+
help="Number of EBS volumes to attach to each node as /vol[x]. " +
112+
"The volumes will be deleted when the instances terminate. " +
113+
"Only possible on EBS-backed AMIs. " +
114+
"EBS volumes are only attached if --ebs-vol-size > 0." +
115+
"Only support up to 8 EBS volumes.")
108116
parser.add_option(
109117
"--swap", metavar="SWAP", type="int", default=1024,
110118
help="Swap space to set up per node, in MB (default: 1024)")
@@ -348,13 +356,16 @@ def launch_cluster(conn, opts, cluster_name):
348356
print >> stderr, "Could not find AMI " + opts.ami
349357
sys.exit(1)
350358

351-
# Create block device mapping so that we can add an EBS volume if asked to
359+
# Create block device mapping so that we can add EBS volumes if asked to.
360+
# The first drive is attached as /dev/sds, 2nd as /dev/sdt, ... /dev/sdz
352361
block_map = BlockDeviceMapping()
353362
if opts.ebs_vol_size > 0:
354-
device = EBSBlockDeviceType()
355-
device.size = opts.ebs_vol_size
356-
device.delete_on_termination = True
357-
block_map["/dev/sdv"] = device
363+
for i in range(opts.ebs_vol_num):
364+
device = EBSBlockDeviceType()
365+
device.size = opts.ebs_vol_size
366+
device.volume_type=opts.ebs_vol_type
367+
device.delete_on_termination = True
368+
block_map["/dev/sd" + chr(ord('s') + i)] = device
358369

359370
# AWS ignores the AMI-specified block device mapping for M3 (see SPARK-3342).
360371
if opts.instance_type.startswith('m3.'):
@@ -828,6 +839,12 @@ def get_partition(total, num_partitions, current_partitions):
828839

829840
def real_main():
830841
(opts, action, cluster_name) = parse_args()
842+
843+
# Input parameter validation
844+
if opts.ebs_vol_num > 8:
845+
print >> stderr, "ebs-vol-num cannot be greater than 8"
846+
sys.exit(1)
847+
831848
try:
832849
conn = ec2.connect_to_region(opts.region)
833850
except Exception as e:

python/pyspark/rdd.py

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -1089,11 +1089,11 @@ def take(self, num):
10891089
# we actually cap it at totalParts in runJob.
10901090
numPartsToTry = 1
10911091
if partsScanned > 0:
1092-
# If we didn't find any rows after the first iteration, just
1093-
# try all partitions next. Otherwise, interpolate the number
1094-
# of partitions we need to try, but overestimate it by 50%.
1092+
# If we didn't find any rows after the previous iteration,
1093+
# quadruple and retry. Otherwise, interpolate the number of
1094+
# partitions we need to try, but overestimate it by 50%.
10951095
if len(items) == 0:
1096-
numPartsToTry = totalParts - 1
1096+
numPartsToTry = partsScanned * 4
10971097
else:
10981098
numPartsToTry = int(1.5 * num * partsScanned / len(items))
10991099

yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala

Lines changed: 5 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -35,7 +35,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration
3535
import org.apache.hadoop.yarn.ipc.YarnRPC
3636
import org.apache.hadoop.yarn.util.{Apps, ConverterUtils, Records, ProtoUtils}
3737

38-
import org.apache.spark.{SparkConf, Logging}
38+
import org.apache.spark.{SecurityManager, SparkConf, Logging}
3939

4040

4141
class ExecutorRunnable(
@@ -46,7 +46,8 @@ class ExecutorRunnable(
4646
slaveId: String,
4747
hostname: String,
4848
executorMemory: Int,
49-
executorCores: Int)
49+
executorCores: Int,
50+
securityMgr: SecurityManager)
5051
extends Runnable with ExecutorRunnableUtil with Logging {
5152

5253
var rpc: YarnRPC = YarnRPC.create(conf)
@@ -86,6 +87,8 @@ class ExecutorRunnable(
8687
logInfo("Setting up executor with commands: " + commands)
8788
ctx.setCommands(commands)
8889

90+
ctx.setApplicationACLs(YarnSparkHadoopUtil.getApplicationAclsForYarn(securityMgr))
91+
8992
// Send the start request to the ContainerManager
9093
val startReq = Records.newRecord(classOf[StartContainerRequest])
9194
.asInstanceOf[StartContainerRequest]

yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala

Lines changed: 11 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -23,7 +23,7 @@ import java.util.concurrent.atomic.AtomicInteger
2323
import scala.collection.JavaConversions._
2424
import scala.collection.mutable.{ArrayBuffer, HashMap}
2525

26-
import org.apache.spark.SparkConf
26+
import org.apache.spark.{SecurityManager, SparkConf}
2727
import org.apache.spark.scheduler.SplitInfo
2828

2929
import org.apache.hadoop.conf.Configuration
@@ -41,21 +41,23 @@ private[yarn] class YarnAllocationHandler(
4141
resourceManager: AMRMProtocol,
4242
appAttemptId: ApplicationAttemptId,
4343
args: ApplicationMasterArguments,
44-
preferredNodes: collection.Map[String, collection.Set[SplitInfo]])
45-
extends YarnAllocator(conf, sparkConf, args, preferredNodes) {
44+
preferredNodes: collection.Map[String, collection.Set[SplitInfo]],
45+
securityMgr: SecurityManager)
46+
extends YarnAllocator(conf, sparkConf, args, preferredNodes, securityMgr) {
4647

4748
private val lastResponseId = new AtomicInteger()
4849
private val releaseList: CopyOnWriteArrayList[ContainerId] = new CopyOnWriteArrayList()
4950

5051
override protected def allocateContainers(count: Int): YarnAllocateResponse = {
5152
var resourceRequests: List[ResourceRequest] = null
5253

53-
// default.
54-
if (count <= 0 || preferredHostToCount.isEmpty) {
55-
logDebug("numExecutors: " + count + ", host preferences: " +
56-
preferredHostToCount.isEmpty)
57-
resourceRequests = List(createResourceRequest(
58-
AllocationType.ANY, null, count, YarnSparkHadoopUtil.RM_REQUEST_PRIORITY))
54+
logDebug("numExecutors: " + count)
55+
if (count <= 0) {
56+
resourceRequests = List()
57+
} else if (preferredHostToCount.isEmpty) {
58+
logDebug("host preferences is empty")
59+
resourceRequests = List(createResourceRequest(
60+
AllocationType.ANY, null, count, YarnSparkHadoopUtil.RM_REQUEST_PRIORITY))
5961
} else {
6062
// request for all hosts in preferred nodes and for numExecutors -
6163
// candidates.size, request by default allocation policy.

yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala

Lines changed: 4 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -27,7 +27,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration
2727
import org.apache.hadoop.yarn.ipc.YarnRPC
2828
import org.apache.hadoop.yarn.util.{ConverterUtils, Records}
2929

30-
import org.apache.spark.{Logging, SparkConf}
30+
import org.apache.spark.{Logging, SecurityManager, SparkConf}
3131
import org.apache.spark.scheduler.SplitInfo
3232
import org.apache.spark.util.Utils
3333

@@ -45,15 +45,16 @@ private class YarnRMClientImpl(args: ApplicationMasterArguments) extends YarnRMC
4545
sparkConf: SparkConf,
4646
preferredNodeLocations: Map[String, Set[SplitInfo]],
4747
uiAddress: String,
48-
uiHistoryAddress: String) = {
48+
uiHistoryAddress: String,
49+
securityMgr: SecurityManager) = {
4950
this.rpc = YarnRPC.create(conf)
5051
this.uiHistoryAddress = uiHistoryAddress
5152

5253
resourceManager = registerWithResourceManager(conf)
5354
registerApplicationMaster(uiAddress)
5455

5556
new YarnAllocationHandler(conf, sparkConf, resourceManager, getAttemptId(), args,
56-
preferredNodeLocations)
57+
preferredNodeLocations, securityMgr)
5758
}
5859

5960
override def getAttemptId() = {

yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala

Lines changed: 7 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -116,7 +116,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments,
116116
val securityMgr = new SecurityManager(sparkConf)
117117

118118
if (isDriver) {
119-
runDriver()
119+
runDriver(securityMgr)
120120
} else {
121121
runExecutorLauncher(securityMgr)
122122
}
@@ -157,7 +157,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments,
157157
sparkContextRef.compareAndSet(sc, null)
158158
}
159159

160-
private def registerAM(uiAddress: String) = {
160+
private def registerAM(uiAddress: String, securityMgr: SecurityManager) = {
161161
val sc = sparkContextRef.get()
162162

163163
val appId = client.getAttemptId().getApplicationId().toString()
@@ -170,13 +170,14 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments,
170170
if (sc != null) sc.getConf else sparkConf,
171171
if (sc != null) sc.preferredNodeLocationData else Map(),
172172
uiAddress,
173-
historyAddress)
173+
historyAddress,
174+
securityMgr)
174175

175176
allocator.allocateResources()
176177
reporterThread = launchReporterThread()
177178
}
178179

179-
private def runDriver(): Unit = {
180+
private def runDriver(securityMgr: SecurityManager): Unit = {
180181
addAmIpFilter()
181182
val userThread = startUserClass()
182183

@@ -188,7 +189,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments,
188189
if (sc == null) {
189190
finish(FinalApplicationStatus.FAILED, "Timed out waiting for SparkContext.")
190191
} else {
191-
registerAM(sc.ui.appUIHostPort)
192+
registerAM(sc.ui.appUIHostPort, securityMgr)
192193
try {
193194
userThread.join()
194195
} finally {
@@ -203,7 +204,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments,
203204
conf = sparkConf, securityManager = securityMgr)._1
204205
actor = waitForSparkDriver()
205206
addAmIpFilter()
206-
registerAM(sparkConf.get("spark.driver.appUIAddress", ""))
207+
registerAM(sparkConf.get("spark.driver.appUIAddress", ""), securityMgr)
207208

208209
// In client mode the actor will stop the reporter thread.
209210
reporterThread.join()

yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala

Lines changed: 2 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -430,10 +430,8 @@ trait ClientBase extends Logging {
430430

431431
// send the acl settings into YARN to control who has access via YARN interfaces
432432
val securityManager = new SecurityManager(sparkConf)
433-
val acls = Map[ApplicationAccessType, String] (
434-
ApplicationAccessType.VIEW_APP -> securityManager.getViewAcls,
435-
ApplicationAccessType.MODIFY_APP -> securityManager.getModifyAcls)
436-
amContainer.setApplicationACLs(acls)
433+
amContainer.setApplicationACLs(YarnSparkHadoopUtil.getApplicationAclsForYarn(securityManager))
434+
437435
amContainer
438436
}
439437
}

0 commit comments

Comments
 (0)