Skip to content

Commit f41c0a9

Browse files
kellyzlysrowen
authored andcommitted
[SPARK-22660][BUILD] Use position() and limit() to fix ambiguity issue in scala-2.12
…a-2.12 and JDK9 ## What changes were proposed in this pull request? Some compile error after upgrading to scala-2.12 ```javascript spark_source/core/src/main/scala/org/apache/spark/executor/Executor.scala:455: ambiguous reference to overloaded definition, method limit in class ByteBuffer of type (x$1: Int)java.nio.ByteBuffer method limit in class Buffer of type ()Int match expected type ? val resultSize = serializedDirectResult.limit error ``` The limit method was moved from ByteBuffer to the superclass Buffer and it can no longer be called without (). The same reason for position method. ```javascript /home/zly/prj/oss/jdk9_HOS_SOURCE/spark_source/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformationExec.scala:427: ambiguous reference to overloaded definition, [error] both method putAll in class Properties of type (x$1: java.util.Map[_, _])Unit [error] and method putAll in class Hashtable of type (x$1: java.util.Map[_ <: Object, _ <: Object])Unit [error] match argument types (java.util.Map[String,String]) [error] props.putAll(outputSerdeProps.toMap.asJava) [error] ^ ``` This is because the key type is Object instead of String which is unsafe. ## How was this patch tested? running tests Please review http://spark.apache.org/contributing.html before opening a pull request. Author: kellyzly <[email protected]> Closes #19854 from kellyzly/SPARK-22660.
1 parent b790719 commit f41c0a9

File tree

12 files changed

+22
-15
lines changed

12 files changed

+22
-15
lines changed

core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -99,7 +99,8 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, id: Long)
9999
private def calcChecksum(block: ByteBuffer): Int = {
100100
val adler = new Adler32()
101101
if (block.hasArray) {
102-
adler.update(block.array, block.arrayOffset + block.position, block.limit - block.position)
102+
adler.update(block.array, block.arrayOffset + block.position(), block.limit()
103+
- block.position())
103104
} else {
104105
val bytes = new Array[Byte](block.remaining())
105106
block.duplicate.get(bytes)

core/src/main/scala/org/apache/spark/executor/Executor.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -452,7 +452,7 @@ private[spark] class Executor(
452452
// TODO: do not serialize value twice
453453
val directResult = new DirectTaskResult(valueBytes, accumUpdates)
454454
val serializedDirectResult = ser.serialize(directResult)
455-
val resultSize = serializedDirectResult.limit
455+
val resultSize = serializedDirectResult.limit()
456456

457457
// directSend = sending directly back to the driver
458458
val serializedResult: ByteBuffer = {

core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -287,13 +287,13 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
287287
private def launchTasks(tasks: Seq[Seq[TaskDescription]]) {
288288
for (task <- tasks.flatten) {
289289
val serializedTask = TaskDescription.encode(task)
290-
if (serializedTask.limit >= maxRpcMessageSize) {
290+
if (serializedTask.limit() >= maxRpcMessageSize) {
291291
scheduler.taskIdToTaskSetManager.get(task.taskId).foreach { taskSetMgr =>
292292
try {
293293
var msg = "Serialized task %s:%d was %d bytes, which exceeds max allowed: " +
294294
"spark.rpc.message.maxSize (%d bytes). Consider increasing " +
295295
"spark.rpc.message.maxSize or using broadcast variables for large values."
296-
msg = msg.format(task.taskId, task.index, serializedTask.limit, maxRpcMessageSize)
296+
msg = msg.format(task.taskId, task.index, serializedTask.limit(), maxRpcMessageSize)
297297
taskSetMgr.abort(msg)
298298
} catch {
299299
case e: Exception => logError("Exception in error callback", e)

core/src/main/scala/org/apache/spark/util/ByteBufferInputStream.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -54,7 +54,7 @@ class ByteBufferInputStream(private var buffer: ByteBuffer)
5454
override def skip(bytes: Long): Long = {
5555
if (buffer != null) {
5656
val amountToSkip = math.min(bytes, buffer.remaining).toInt
57-
buffer.position(buffer.position + amountToSkip)
57+
buffer.position(buffer.position() + amountToSkip)
5858
if (buffer.remaining() == 0) {
5959
cleanUp()
6060
}

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

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -65,7 +65,7 @@ private[spark] class ChunkedByteBuffer(var chunks: Array[ByteBuffer]) {
6565
for (bytes <- getChunks()) {
6666
while (bytes.remaining() > 0) {
6767
val ioSize = Math.min(bytes.remaining(), bufferWriteChunkSize)
68-
bytes.limit(bytes.position + ioSize)
68+
bytes.limit(bytes.position() + ioSize)
6969
channel.write(bytes)
7070
}
7171
}
@@ -206,7 +206,7 @@ private[spark] class ChunkedByteBufferInputStream(
206206
override def skip(bytes: Long): Long = {
207207
if (currentChunk != null) {
208208
val amountToSkip = math.min(bytes, currentChunk.remaining).toInt
209-
currentChunk.position(currentChunk.position + amountToSkip)
209+
currentChunk.position(currentChunk.position() + amountToSkip)
210210
if (currentChunk.remaining() == 0) {
211211
if (chunks.hasNext) {
212212
currentChunk = chunks.next()

core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -199,7 +199,7 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext {
199199
def check[T: ClassTag](t: T) {
200200
assert(ser.deserialize[T](ser.serialize(t)) === t)
201201
// Check that very long ranges don't get written one element at a time
202-
assert(ser.serialize(t).limit < 100)
202+
assert(ser.serialize(t).limit() < 100)
203203
}
204204
check(1 to 1000000)
205205
check(1 to 1000000 by 2)

core/src/test/scala/org/apache/spark/storage/DiskStoreSuite.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -118,7 +118,7 @@ class DiskStoreSuite extends SparkFunSuite {
118118
val chunks = chunkedByteBuffer.chunks
119119
assert(chunks.size === 2)
120120
for (chunk <- chunks) {
121-
assert(chunk.limit === 10 * 1024)
121+
assert(chunk.limit() === 10 * 1024)
122122
}
123123

124124
val e = intercept[IllegalArgumentException]{

external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -296,7 +296,9 @@ class KafkaTestUtils(withBrokerProps: Map[String, Object] = Map.empty) extends L
296296
props.put("replica.socket.timeout.ms", "1500")
297297
props.put("delete.topic.enable", "true")
298298
props.put("offsets.topic.num.partitions", "1")
299-
props.putAll(withBrokerProps.asJava)
299+
// Can not use properties.putAll(propsMap.asJava) in scala-2.12
300+
// See https://github.com/scala/bug/issues/10418
301+
withBrokerProps.foreach { case (k, v) => props.put(k, v) }
300302
props
301303
}
302304

sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/NullableColumnAccessor.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -35,7 +35,7 @@ private[columnar] trait NullableColumnAccessor extends ColumnAccessor {
3535
nextNullIndex = if (nullCount > 0) ByteBufferHelper.getInt(nullsBuffer) else -1
3636
pos = 0
3737

38-
underlyingBuffer.position(underlyingBuffer.position + 4 + nullCount * 4)
38+
underlyingBuffer.position(underlyingBuffer.position() + 4 + nullCount * 4)
3939
super.initialize()
4040
}
4141

sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/compressionSchemes.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -112,7 +112,7 @@ private[columnar] case object PassThrough extends CompressionScheme {
112112
var nextNullIndex = if (nullCount > 0) ByteBufferHelper.getInt(nullsBuffer) else capacity
113113
var pos = 0
114114
var seenNulls = 0
115-
var bufferPos = buffer.position
115+
var bufferPos = buffer.position()
116116
while (pos < capacity) {
117117
if (pos != nextNullIndex) {
118118
val len = nextNullIndex - pos

0 commit comments

Comments
 (0)