Skip to content
GitLab
Explore
Sign in
Primary navigation
Search or go to…
Project
S
spark
Manage
Activity
Members
Labels
Plan
Issues
Issue boards
Milestones
Wiki
Code
Merge requests
Repository
Branches
Commits
Tags
Repository graph
Compare revisions
Snippets
Build
Pipelines
Jobs
Pipeline schedules
Artifacts
Deploy
Releases
Model registry
Operate
Environments
Monitor
Incidents
Analyze
Value stream analytics
Contributor analytics
CI/CD analytics
Repository analytics
Model experiments
Help
Help
Support
GitLab documentation
Compare GitLab plans
Community forum
Contribute to GitLab
Provide feedback
Keyboard shortcuts
?
Snippets
Groups
Projects
Show more breadcrumbs
cs525-sp18-g07
spark
Commits
940869df
Commit
940869df
authored
12 years ago
by
Reynold Xin
Browse files
Options
Downloads
Patches
Plain Diff
Disable running combiners on map tasks when mergeCombiners function is
not specified by the user.
parent
3a6a95dc
No related branches found
Branches containing commit
No related tags found
Tags containing commit
No related merge requests found
Changes
2
Hide whitespace changes
Inline
Side-by-side
Showing
2 changed files
core/src/main/scala/spark/ShuffledRDD.scala
+28
-8
28 additions, 8 deletions
core/src/main/scala/spark/ShuffledRDD.scala
core/src/main/scala/spark/scheduler/ShuffleMapTask.scala
+28
-11
28 additions, 11 deletions
core/src/main/scala/spark/scheduler/ShuffleMapTask.scala
with
56 additions
and
19 deletions
core/src/main/scala/spark/ShuffledRDD.scala
+
28
−
8
View file @
940869df
...
...
@@ -27,16 +27,36 @@ class ShuffledRDD[K, V, C](
override
def
compute
(
split
:
Split
)
:
Iterator
[(
K
,
C
)]
=
{
val
combiners
=
new
JHashMap
[
K
,
C
]
def
mergePair
(
k
:
K
,
c
:
C
)
{
val
oldC
=
combiners
.
get
(
k
)
if
(
oldC
==
null
)
{
combiners
.
put
(
k
,
c
)
}
else
{
combiners
.
put
(
k
,
aggregator
.
mergeCombiners
(
oldC
,
c
))
val
fetcher
=
SparkEnv
.
get
.
shuffleFetcher
if
(
aggregator
.
mergeCombiners
!=
null
)
{
// If mergeCombiners is specified, combiners are applied on the map
// partitions. In this case, post-shuffle we get a list of outputs from
// the combiners and merge them using mergeCombiners.
def
mergePairWithMapSideCombiners
(
k
:
K
,
c
:
C
)
{
val
oldC
=
combiners
.
get
(
k
)
if
(
oldC
==
null
)
{
combiners
.
put
(
k
,
c
)
}
else
{
combiners
.
put
(
k
,
aggregator
.
mergeCombiners
(
oldC
,
c
))
}
}
fetcher
.
fetch
[
K
,
C
](
dep
.
shuffleId
,
split
.
index
,
mergePairWithMapSideCombiners
)
}
else
{
// If mergeCombiners is not specified, no combiner is applied on the map
// partitions (i.e. map side aggregation is turned off). Post-shuffle we
// get a list of values and we use mergeValue to merge them.
def
mergePairWithoutMapSideCombiners
(
k
:
K
,
v
:
V
)
{
val
oldC
=
combiners
.
get
(
k
)
if
(
oldC
==
null
)
{
combiners
.
put
(
k
,
aggregator
.
createCombiner
(
v
))
}
else
{
combiners
.
put
(
k
,
aggregator
.
mergeValue
(
oldC
,
v
))
}
}
fetcher
.
fetch
[
K
,
V
](
dep
.
shuffleId
,
split
.
index
,
mergePairWithoutMapSideCombiners
)
}
val
fetcher
=
SparkEnv
.
get
.
shuffleFetcher
fetcher
.
fetch
[
K
,
C
](
dep
.
shuffleId
,
split
.
index
,
mergePair
)
return
new
Iterator
[(
K
,
C
)]
{
var
iter
=
combiners
.
entrySet
().
iterator
()
...
...
This diff is collapsed.
Click to expand it.
core/src/main/scala/spark/scheduler/ShuffleMapTask.scala
+
28
−
11
View file @
940869df
...
...
@@ -104,27 +104,44 @@ class ShuffleMapTask(
val
numOutputSplits
=
dep
.
partitioner
.
numPartitions
val
aggregator
=
dep
.
aggregator
.
asInstanceOf
[
Aggregator
[
Any
,
Any
,
Any
]]
val
partitioner
=
dep
.
partitioner
val
buckets
=
Array
.
tabulate
(
numOutputSplits
)(
_
=>
new
HashMap
[
Any
,
Any
])
for
(
elem
<-
rdd
.
iterator
(
split
))
{
val
(
k
,
v
)
=
elem
.
asInstanceOf
[(
Any
,
Any
)]
var
bucketId
=
partitioner
.
getPartition
(
k
)
val
bucket
=
buckets
(
bucketId
)
var
existing
=
bucket
.
get
(
k
)
if
(
existing
==
null
)
{
bucket
.
put
(
k
,
aggregator
.
createCombiner
(
v
))
val
bucketIterators
=
if
(
aggregator
.
mergeCombiners
!=
null
)
{
// Apply combiners (map-side aggregation) to the map output.
val
buckets
=
Array
.
tabulate
(
numOutputSplits
)(
_
=>
new
HashMap
[
Any
,
Any
])
for
(
elem
<-
rdd
.
iterator
(
split
))
{
val
(
k
,
v
)
=
elem
.
asInstanceOf
[(
Any
,
Any
)]
val
bucketId
=
partitioner
.
getPartition
(
k
)
val
bucket
=
buckets
(
bucketId
)
val
existing
=
bucket
.
get
(
k
)
if
(
existing
==
null
)
{
bucket
.
put
(
k
,
aggregator
.
createCombiner
(
v
))
}
else
{
bucket
.
put
(
k
,
aggregator
.
mergeValue
(
existing
,
v
))
}
}
buckets
.
map
(
_
.
iterator
)
}
else
{
bucket
.
put
(
k
,
aggregator
.
mergeValue
(
existing
,
v
))
// No combiners (no map-side aggregation). Simply partition the map output.
val
buckets
=
Array
.
tabulate
(
numOutputSplits
)(
_
=>
new
ArrayBuffer
[(
Any
,
Any
)])
for
(
elem
<-
rdd
.
iterator
(
split
))
{
val
pair
=
elem
.
asInstanceOf
[(
Any
,
Any
)]
val
bucketId
=
partitioner
.
getPartition
(
pair
.
_1
)
buckets
(
bucketId
)
+=
pair
}
buckets
.
map
(
_
.
iterator
)
}
}
val
ser
=
SparkEnv
.
get
.
serializer
.
newInstance
()
val
blockManager
=
SparkEnv
.
get
.
blockManager
for
(
i
<-
0
until
numOutputSplits
)
{
val
blockId
=
"shuffleid_"
+
dep
.
shuffleId
+
"_"
+
partition
+
"_"
+
i
// Get a scala iterator from java map
val
iter
:
Iterator
[(
Any
,
Any
)]
=
bucket
s
(
i
).
i
terator
val
iter
:
Iterator
[(
Any
,
Any
)]
=
bucket
I
terator
s
(
i
)
// TODO: This should probably be DISK_ONLY
blockManager
.
put
(
blockId
,
iter
,
StorageLevel
.
MEMORY_ONLY
,
false
)
}
return
SparkEnv
.
get
.
blockManager
.
blockManagerId
}
...
...
This diff is collapsed.
Click to expand it.
Preview
0%
Loading
Try again
or
attach a new file
.
Cancel
You are about to add
0
people
to the discussion. Proceed with caution.
Finish editing this message first!
Save comment
Cancel
Please
register
or
sign in
to comment