Skip to content

Commit 39f3810

Browse files
author
Ilya Ganelin
committed
resolved merge issues
2 parents e446287 + e8422c5 commit 39f3810

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

44 files changed

+854
-241
lines changed
Lines changed: 97 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,97 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.spark;
19+
20+
import org.apache.spark.scheduler.SparkListener;
21+
import org.apache.spark.scheduler.SparkListenerApplicationEnd;
22+
import org.apache.spark.scheduler.SparkListenerApplicationStart;
23+
import org.apache.spark.scheduler.SparkListenerBlockManagerAdded;
24+
import org.apache.spark.scheduler.SparkListenerBlockManagerRemoved;
25+
import org.apache.spark.scheduler.SparkListenerEnvironmentUpdate;
26+
import org.apache.spark.scheduler.SparkListenerExecutorAdded;
27+
import org.apache.spark.scheduler.SparkListenerExecutorMetricsUpdate;
28+
import org.apache.spark.scheduler.SparkListenerExecutorRemoved;
29+
import org.apache.spark.scheduler.SparkListenerJobEnd;
30+
import org.apache.spark.scheduler.SparkListenerJobStart;
31+
import org.apache.spark.scheduler.SparkListenerStageCompleted;
32+
import org.apache.spark.scheduler.SparkListenerStageSubmitted;
33+
import org.apache.spark.scheduler.SparkListenerTaskEnd;
34+
import org.apache.spark.scheduler.SparkListenerTaskGettingResult;
35+
import org.apache.spark.scheduler.SparkListenerTaskStart;
36+
import org.apache.spark.scheduler.SparkListenerUnpersistRDD;
37+
38+
/**
39+
* Java clients should extend this class instead of implementing
40+
* SparkListener directly. This is to prevent java clients
41+
* from breaking when new events are added to the SparkListener
42+
* trait.
43+
*
44+
* This is a concrete class instead of abstract to enforce
45+
* new events get added to both the SparkListener and this adapter
46+
* in lockstep.
47+
*/
48+
public class JavaSparkListener implements SparkListener {
49+
50+
@Override
51+
public void onStageCompleted(SparkListenerStageCompleted stageCompleted) { }
52+
53+
@Override
54+
public void onStageSubmitted(SparkListenerStageSubmitted stageSubmitted) { }
55+
56+
@Override
57+
public void onTaskStart(SparkListenerTaskStart taskStart) { }
58+
59+
@Override
60+
public void onTaskGettingResult(SparkListenerTaskGettingResult taskGettingResult) { }
61+
62+
@Override
63+
public void onTaskEnd(SparkListenerTaskEnd taskEnd) { }
64+
65+
@Override
66+
public void onJobStart(SparkListenerJobStart jobStart) { }
67+
68+
@Override
69+
public void onJobEnd(SparkListenerJobEnd jobEnd) { }
70+
71+
@Override
72+
public void onEnvironmentUpdate(SparkListenerEnvironmentUpdate environmentUpdate) { }
73+
74+
@Override
75+
public void onBlockManagerAdded(SparkListenerBlockManagerAdded blockManagerAdded) { }
76+
77+
@Override
78+
public void onBlockManagerRemoved(SparkListenerBlockManagerRemoved blockManagerRemoved) { }
79+
80+
@Override
81+
public void onUnpersistRDD(SparkListenerUnpersistRDD unpersistRDD) { }
82+
83+
@Override
84+
public void onApplicationStart(SparkListenerApplicationStart applicationStart) { }
85+
86+
@Override
87+
public void onApplicationEnd(SparkListenerApplicationEnd applicationEnd) { }
88+
89+
@Override
90+
public void onExecutorMetricsUpdate(SparkListenerExecutorMetricsUpdate executorMetricsUpdate) { }
91+
92+
@Override
93+
public void onExecutorAdded(SparkListenerExecutorAdded executorAdded) { }
94+
95+
@Override
96+
public void onExecutorRemoved(SparkListenerExecutorRemoved executorRemoved) { }
97+
}

core/src/main/scala/org/apache/spark/CacheManager.scala

Lines changed: 5 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -44,7 +44,11 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging {
4444
blockManager.get(key) match {
4545
case Some(blockResult) =>
4646
// Partition is already materialized, so just return its values
47-
context.taskMetrics.inputMetrics = Some(blockResult.inputMetrics)
47+
val inputMetrics = blockResult.inputMetrics
48+
val existingMetrics = context.taskMetrics
49+
.getInputMetricsForReadMethod(inputMetrics.readMethod)
50+
existingMetrics.addBytesRead(inputMetrics.bytesRead)
51+
4852
new InterruptibleIterator(context, blockResult.data.asInstanceOf[Iterator[T]])
4953

5054
case None =>

core/src/main/scala/org/apache/spark/SparkContext.scala

Lines changed: 3 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -520,10 +520,9 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
520520

521521
/** Distribute a local Scala collection to form an RDD.
522522
*
523-
* @note Parallelize acts lazily. If `seq` is a mutable collection and is
524-
* altered after the call to parallelize and before the first action on the
525-
* RDD, the resultant RDD will reflect the modified collection. Pass a copy of
526-
* the argument to avoid this.
523+
* @note Parallelize acts lazily. If `seq` is a mutable collection and is altered after the call
524+
* to parallelize and before the first action on the RDD, the resultant RDD will reflect the
525+
* modified collection. Pass a copy of the argument to avoid this.
527526
*/
528527
def parallelize[T: ClassTag](seq: Seq[T], numSlices: Int = defaultParallelism): RDD[T] = {
529528
new ParallelCollectionRDD[T](this, seq, numSlices, Map[Int, Seq[String]]())

core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -23,7 +23,7 @@ import scala.collection.mutable.ListBuffer
2323

2424
import org.apache.log4j.Level
2525

26-
import org.apache.spark.util.MemoryParam
26+
import org.apache.spark.util.{IntParam, MemoryParam}
2727

2828
/**
2929
* Command-line parser for the driver client.
@@ -51,8 +51,8 @@ private[spark] class ClientArguments(args: Array[String]) {
5151
parse(args.toList)
5252

5353
def parse(args: List[String]): Unit = args match {
54-
case ("--cores" | "-c") :: value :: tail =>
55-
cores = value.toInt
54+
case ("--cores" | "-c") :: IntParam(value) :: tail =>
55+
cores = value
5656
parse(tail)
5757

5858
case ("--memory" | "-m") :: MemoryParam(value) :: tail =>

core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -200,6 +200,7 @@ object SparkSubmit {
200200
// Yarn cluster only
201201
OptionAssigner(args.name, YARN, CLUSTER, clOption = "--name"),
202202
OptionAssigner(args.driverMemory, YARN, CLUSTER, clOption = "--driver-memory"),
203+
OptionAssigner(args.driverCores, YARN, CLUSTER, clOption = "--driver-cores"),
203204
OptionAssigner(args.queue, YARN, CLUSTER, clOption = "--queue"),
204205
OptionAssigner(args.numExecutors, YARN, CLUSTER, clOption = "--num-executors"),
205206
OptionAssigner(args.executorMemory, YARN, CLUSTER, clOption = "--executor-memory"),

core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -108,6 +108,9 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St
108108
.orElse(sparkProperties.get("spark.driver.memory"))
109109
.orElse(env.get("SPARK_DRIVER_MEMORY"))
110110
.orNull
111+
driverCores = Option(driverCores)
112+
.orElse(sparkProperties.get("spark.driver.cores"))
113+
.orNull
111114
executorMemory = Option(executorMemory)
112115
.orElse(sparkProperties.get("spark.executor.memory"))
113116
.orElse(env.get("SPARK_EXECUTOR_MEMORY"))
@@ -406,6 +409,8 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St
406409
| --total-executor-cores NUM Total cores for all executors.
407410
|
408411
| YARN-only:
412+
| --driver-cores NUM Number of cores used by the driver, only in cluster mode
413+
| (Default: 1).
409414
| --executor-cores NUM Number of cores per executor (Default: 1).
410415
| --queue QUEUE_NAME The YARN queue to submit to (Default: "default").
411416
| --num-executors NUM Number of executors to launch (Default: 2).

core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala

Lines changed: 7 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -38,8 +38,8 @@ private[spark] class ApplicationInfo(
3838
extends Serializable {
3939

4040
@transient var state: ApplicationState.Value = _
41-
@transient var executors: mutable.HashMap[Int, ExecutorInfo] = _
42-
@transient var removedExecutors: ArrayBuffer[ExecutorInfo] = _
41+
@transient var executors: mutable.HashMap[Int, ExecutorDesc] = _
42+
@transient var removedExecutors: ArrayBuffer[ExecutorDesc] = _
4343
@transient var coresGranted: Int = _
4444
@transient var endTime: Long = _
4545
@transient var appSource: ApplicationSource = _
@@ -55,12 +55,12 @@ private[spark] class ApplicationInfo(
5555

5656
private def init() {
5757
state = ApplicationState.WAITING
58-
executors = new mutable.HashMap[Int, ExecutorInfo]
58+
executors = new mutable.HashMap[Int, ExecutorDesc]
5959
coresGranted = 0
6060
endTime = -1L
6161
appSource = new ApplicationSource(this)
6262
nextExecutorId = 0
63-
removedExecutors = new ArrayBuffer[ExecutorInfo]
63+
removedExecutors = new ArrayBuffer[ExecutorDesc]
6464
}
6565

6666
private def newExecutorId(useID: Option[Int] = None): Int = {
@@ -75,14 +75,14 @@ private[spark] class ApplicationInfo(
7575
}
7676
}
7777

78-
def addExecutor(worker: WorkerInfo, cores: Int, useID: Option[Int] = None): ExecutorInfo = {
79-
val exec = new ExecutorInfo(newExecutorId(useID), this, worker, cores, desc.memoryPerSlave)
78+
def addExecutor(worker: WorkerInfo, cores: Int, useID: Option[Int] = None): ExecutorDesc = {
79+
val exec = new ExecutorDesc(newExecutorId(useID), this, worker, cores, desc.memoryPerSlave)
8080
executors(exec.id) = exec
8181
coresGranted += cores
8282
exec
8383
}
8484

85-
def removeExecutor(exec: ExecutorInfo) {
85+
def removeExecutor(exec: ExecutorDesc) {
8686
if (executors.contains(exec.id)) {
8787
removedExecutors += executors(exec.id)
8888
executors -= exec.id

core/src/main/scala/org/apache/spark/deploy/master/ExecutorInfo.scala renamed to core/src/main/scala/org/apache/spark/deploy/master/ExecutorDesc.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -19,7 +19,7 @@ package org.apache.spark.deploy.master
1919

2020
import org.apache.spark.deploy.{ExecutorDescription, ExecutorState}
2121

22-
private[spark] class ExecutorInfo(
22+
private[spark] class ExecutorDesc(
2323
val id: Int,
2424
val application: ApplicationInfo,
2525
val worker: WorkerInfo,
@@ -37,7 +37,7 @@ private[spark] class ExecutorInfo(
3737

3838
override def equals(other: Any): Boolean = {
3939
other match {
40-
case info: ExecutorInfo =>
40+
case info: ExecutorDesc =>
4141
fullId == info.fullId &&
4242
worker.id == info.worker.id &&
4343
cores == info.cores &&

core/src/main/scala/org/apache/spark/deploy/master/Master.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -581,7 +581,7 @@ private[spark] class Master(
581581
}
582582
}
583583

584-
def launchExecutor(worker: WorkerInfo, exec: ExecutorInfo) {
584+
def launchExecutor(worker: WorkerInfo, exec: ExecutorDesc) {
585585
logInfo("Launching executor " + exec.fullId + " on worker " + worker.id)
586586
worker.addExecutor(exec)
587587
worker.actor ! LaunchExecutor(masterUrl,

core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -38,7 +38,7 @@ private[spark] class WorkerInfo(
3838
Utils.checkHost(host, "Expected hostname")
3939
assert (port > 0)
4040

41-
@transient var executors: mutable.HashMap[String, ExecutorInfo] = _ // executorId => info
41+
@transient var executors: mutable.HashMap[String, ExecutorDesc] = _ // executorId => info
4242
@transient var drivers: mutable.HashMap[String, DriverInfo] = _ // driverId => info
4343
@transient var state: WorkerState.Value = _
4444
@transient var coresUsed: Int = _
@@ -70,13 +70,13 @@ private[spark] class WorkerInfo(
7070
host + ":" + port
7171
}
7272

73-
def addExecutor(exec: ExecutorInfo) {
73+
def addExecutor(exec: ExecutorDesc) {
7474
executors(exec.fullId) = exec
7575
coresUsed += exec.cores
7676
memoryUsed += exec.memory
7777
}
7878

79-
def removeExecutor(exec: ExecutorInfo) {
79+
def removeExecutor(exec: ExecutorDesc) {
8080
if (executors.contains(exec.fullId)) {
8181
executors -= exec.fullId
8282
coresUsed -= exec.cores

0 commit comments

Comments
 (0)