Skip to content

Commit fe52801

Browse files
authored
Merge pull request #22 from gatorsmile/SPARK-24865-xiao
resolve the conflicts
2 parents f2f1a97 + 7995272 commit fe52801

File tree

273 files changed

+4288
-1584
lines changed

Some content is hidden

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

273 files changed

+4288
-1584
lines changed

R/pkg/inst/tests/testthat/test_basic.R

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -18,9 +18,9 @@
1818
context("basic tests for CRAN")
1919

2020
test_that("create DataFrame from list or data.frame", {
21-
tryCatch( checkJavaVersion(),
21+
tryCatch(checkJavaVersion(),
2222
error = function(e) { skip("error on Java check") },
23-
warning = function(e) { skip("warning on Java check") } )
23+
warning = function(e) { skip("warning on Java check") })
2424

2525
sparkR.session(master = sparkRTestMaster, enableHiveSupport = FALSE,
2626
sparkConfig = sparkRTestConfig)
@@ -54,9 +54,9 @@ test_that("create DataFrame from list or data.frame", {
5454
})
5555

5656
test_that("spark.glm and predict", {
57-
tryCatch( checkJavaVersion(),
57+
tryCatch(checkJavaVersion(),
5858
error = function(e) { skip("error on Java check") },
59-
warning = function(e) { skip("warning on Java check") } )
59+
warning = function(e) { skip("warning on Java check") })
6060

6161
sparkR.session(master = sparkRTestMaster, enableHiveSupport = FALSE,
6262
sparkConfig = sparkRTestConfig)

bin/docker-image-tool.sh

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -135,7 +135,7 @@ BASEDOCKERFILE=
135135
PYDOCKERFILE=
136136
NOCACHEARG=
137137
BUILD_PARAMS=
138-
while getopts f:mr:t:n:b: option
138+
while getopts f:p:mr:t:n:b: option
139139
do
140140
case "${option}"
141141
in

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

Lines changed: 5 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1524,7 +1524,11 @@ class SparkContext(config: SparkConf) extends Logging {
15241524
def addFile(path: String, recursive: Boolean): Unit = {
15251525
val uri = new Path(path).toUri
15261526
val schemeCorrectedPath = uri.getScheme match {
1527-
case null | "local" => new File(path).getCanonicalFile.toURI.toString
1527+
case null => new File(path).getCanonicalFile.toURI.toString
1528+
case "local" =>
1529+
logWarning("File with 'local' scheme is not supported to add to file server, since " +
1530+
"it is already available on every node.")
1531+
return
15281532
case _ => path
15291533
}
15301534

core/src/main/scala/org/apache/spark/internal/config/package.scala

Lines changed: 5 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -432,7 +432,11 @@ package object config {
432432
"external shuffle service, this feature can only be worked when external shuffle" +
433433
"service is newer than Spark 2.2.")
434434
.bytesConf(ByteUnit.BYTE)
435-
.createWithDefault(Long.MaxValue)
435+
// fetch-to-mem is guaranteed to fail if the message is bigger than 2 GB, so we might
436+
// as well use fetch-to-disk in that case. The message includes some metadata in addition
437+
// to the block data itself (in particular UploadBlock has a lot of metadata), so we leave
438+
// extra room.
439+
.createWithDefault(Int.MaxValue - 512)
436440

437441
private[spark] val TASK_METRICS_TRACK_UPDATED_BLOCK_STATUSES =
438442
ConfigBuilder("spark.taskMetrics.trackUpdatedBlockStatuses")

core/src/main/scala/org/apache/spark/storage/BlockManager.scala

Lines changed: 9 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -130,6 +130,8 @@ private[spark] class BlockManager(
130130

131131
private[spark] val externalShuffleServiceEnabled =
132132
conf.getBoolean("spark.shuffle.service.enabled", false)
133+
private val chunkSize =
134+
conf.getSizeAsBytes("spark.storage.memoryMapLimitForTests", Int.MaxValue.toString).toInt
133135

134136
val diskBlockManager = {
135137
// Only perform cleanup if an external service is not serving our shuffle files.
@@ -660,6 +662,11 @@ private[spark] class BlockManager(
660662
* Get block from remote block managers as serialized bytes.
661663
*/
662664
def getRemoteBytes(blockId: BlockId): Option[ChunkedByteBuffer] = {
665+
// TODO if we change this method to return the ManagedBuffer, then getRemoteValues
666+
// could just use the inputStream on the temp file, rather than memory-mapping the file.
667+
// Until then, replication can cause the process to use too much memory and get killed
668+
// by the OS / cluster manager (not a java OOM, since its a memory-mapped file) even though
669+
// we've read the data to disk.
663670
logDebug(s"Getting remote block $blockId")
664671
require(blockId != null, "BlockId is null")
665672
var runningFailureCount = 0
@@ -690,7 +697,7 @@ private[spark] class BlockManager(
690697
logDebug(s"Getting remote block $blockId from $loc")
691698
val data = try {
692699
blockTransferService.fetchBlockSync(
693-
loc.host, loc.port, loc.executorId, blockId.toString, tempFileManager).nioByteBuffer()
700+
loc.host, loc.port, loc.executorId, blockId.toString, tempFileManager)
694701
} catch {
695702
case NonFatal(e) =>
696703
runningFailureCount += 1
@@ -724,7 +731,7 @@ private[spark] class BlockManager(
724731
}
725732

726733
if (data != null) {
727-
return Some(new ChunkedByteBuffer(data))
734+
return Some(ChunkedByteBuffer.fromManagedBuffer(data, chunkSize))
728735
}
729736
logDebug(s"The value of block $blockId is null")
730737
}

core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala

Lines changed: 38 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -17,17 +17,21 @@
1717

1818
package org.apache.spark.util.io
1919

20-
import java.io.InputStream
20+
import java.io.{File, FileInputStream, InputStream}
2121
import java.nio.ByteBuffer
22-
import java.nio.channels.WritableByteChannel
22+
import java.nio.channels.{FileChannel, WritableByteChannel}
23+
import java.nio.file.StandardOpenOption
24+
25+
import scala.collection.mutable.ListBuffer
2326

2427
import com.google.common.primitives.UnsignedBytes
25-
import io.netty.buffer.{ByteBuf, Unpooled}
2628

2729
import org.apache.spark.SparkEnv
2830
import org.apache.spark.internal.config
31+
import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer}
2932
import org.apache.spark.network.util.ByteArrayWritableChannel
3033
import org.apache.spark.storage.StorageUtils
34+
import org.apache.spark.util.Utils
3135

3236
/**
3337
* Read-only byte buffer which is physically stored as multiple chunks rather than a single
@@ -81,10 +85,10 @@ private[spark] class ChunkedByteBuffer(var chunks: Array[ByteBuffer]) {
8185
}
8286

8387
/**
84-
* Wrap this buffer to view it as a Netty ByteBuf.
88+
* Wrap this in a custom "FileRegion" which allows us to transfer over 2 GB.
8589
*/
86-
def toNetty: ByteBuf = {
87-
Unpooled.wrappedBuffer(chunks.length, getChunks(): _*)
90+
def toNetty: ChunkedByteBufferFileRegion = {
91+
new ChunkedByteBufferFileRegion(this, bufferWriteChunkSize)
8892
}
8993

9094
/**
@@ -166,6 +170,34 @@ private[spark] class ChunkedByteBuffer(var chunks: Array[ByteBuffer]) {
166170

167171
}
168172

173+
object ChunkedByteBuffer {
174+
// TODO eliminate this method if we switch BlockManager to getting InputStreams
175+
def fromManagedBuffer(data: ManagedBuffer, maxChunkSize: Int): ChunkedByteBuffer = {
176+
data match {
177+
case f: FileSegmentManagedBuffer =>
178+
map(f.getFile, maxChunkSize, f.getOffset, f.getLength)
179+
case other =>
180+
new ChunkedByteBuffer(other.nioByteBuffer())
181+
}
182+
}
183+
184+
def map(file: File, maxChunkSize: Int, offset: Long, length: Long): ChunkedByteBuffer = {
185+
Utils.tryWithResource(FileChannel.open(file.toPath, StandardOpenOption.READ)) { channel =>
186+
var remaining = length
187+
var pos = offset
188+
val chunks = new ListBuffer[ByteBuffer]()
189+
while (remaining > 0) {
190+
val chunkSize = math.min(remaining, maxChunkSize)
191+
val chunk = channel.map(FileChannel.MapMode.READ_ONLY, pos, chunkSize)
192+
pos += chunkSize
193+
remaining -= chunkSize
194+
chunks += chunk
195+
}
196+
new ChunkedByteBuffer(chunks.toArray)
197+
}
198+
}
199+
}
200+
169201
/**
170202
* Reads data from a ChunkedByteBuffer.
171203
*
Lines changed: 86 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,86 @@
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+
package org.apache.spark.util.io
18+
19+
import java.nio.channels.WritableByteChannel
20+
21+
import io.netty.channel.FileRegion
22+
import io.netty.util.AbstractReferenceCounted
23+
24+
import org.apache.spark.internal.Logging
25+
import org.apache.spark.network.util.AbstractFileRegion
26+
27+
28+
/**
29+
* This exposes a ChunkedByteBuffer as a netty FileRegion, just to allow sending > 2gb in one netty
30+
* message. This is because netty cannot send a ByteBuf > 2g, but it can send a large FileRegion,
31+
* even though the data is not backed by a file.
32+
*/
33+
private[io] class ChunkedByteBufferFileRegion(
34+
private val chunkedByteBuffer: ChunkedByteBuffer,
35+
private val ioChunkSize: Int) extends AbstractFileRegion {
36+
37+
private var _transferred: Long = 0
38+
// this duplicates the original chunks, so we're free to modify the position, limit, etc.
39+
private val chunks = chunkedByteBuffer.getChunks()
40+
private val size = chunks.foldLeft(0L) { _ + _.remaining() }
41+
42+
protected def deallocate: Unit = {}
43+
44+
override def count(): Long = size
45+
46+
// this is the "start position" of the overall Data in the backing file, not our current position
47+
override def position(): Long = 0
48+
49+
override def transferred(): Long = _transferred
50+
51+
private var currentChunkIdx = 0
52+
53+
def transferTo(target: WritableByteChannel, position: Long): Long = {
54+
assert(position == _transferred)
55+
if (position == size) return 0L
56+
var keepGoing = true
57+
var written = 0L
58+
var currentChunk = chunks(currentChunkIdx)
59+
while (keepGoing) {
60+
while (currentChunk.hasRemaining && keepGoing) {
61+
val ioSize = Math.min(currentChunk.remaining(), ioChunkSize)
62+
val originalLimit = currentChunk.limit()
63+
currentChunk.limit(currentChunk.position() + ioSize)
64+
val thisWriteSize = target.write(currentChunk)
65+
currentChunk.limit(originalLimit)
66+
written += thisWriteSize
67+
if (thisWriteSize < ioSize) {
68+
// the channel did not accept our entire write. We do *not* keep trying -- netty wants
69+
// us to just stop, and report how much we've written.
70+
keepGoing = false
71+
}
72+
}
73+
if (keepGoing) {
74+
// advance to the next chunk (if there are any more)
75+
currentChunkIdx += 1
76+
if (currentChunkIdx == chunks.size) {
77+
keepGoing = false
78+
} else {
79+
currentChunk = chunks(currentChunkIdx)
80+
}
81+
}
82+
}
83+
_transferred += written
84+
written
85+
}
86+
}

0 commit comments

Comments
 (0)