-
Notifications
You must be signed in to change notification settings - Fork 28.5k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[SPARK-12084][Core]Fix codes that uses ByteBuffer.array incorrectly #10083
Changes from 3 commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -307,7 +307,7 @@ private[spark] class KryoSerializerInstance(ks: KryoSerializer) extends Serializ | |
override def deserialize[T: ClassTag](bytes: ByteBuffer): T = { | ||
val kryo = borrowKryo() | ||
try { | ||
input.setBuffer(bytes.array) | ||
input.setBuffer(bytes.array(), bytes.arrayOffset() + bytes.position(), bytes.remaining()) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Not necessary for this change, but at some point it might be worth it to change this to use Kryo's There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Kryo will use the array as an internal buffer. Why it's not necessary? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I'm saying that the change I proposed is not necessary, not that your change is not necessary. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Got it. Sorry for my misunderstanding. |
||
kryo.readClassAndObject(input).asInstanceOf[T] | ||
} finally { | ||
releaseKryo(kryo) | ||
|
@@ -319,7 +319,7 @@ private[spark] class KryoSerializerInstance(ks: KryoSerializer) extends Serializ | |
val oldClassLoader = kryo.getClassLoader | ||
try { | ||
kryo.setClassLoader(loader) | ||
input.setBuffer(bytes.array) | ||
input.setBuffer(bytes.array(), bytes.arrayOffset() + bytes.position(), bytes.remaining()) | ||
kryo.readClassAndObject(input).asInstanceOf[T] | ||
} finally { | ||
kryo.setClassLoader(oldClassLoader) | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -79,7 +79,10 @@ object AvroConversionUtil extends Serializable { | |
|
||
def unpackBytes(obj: Any): Array[Byte] = { | ||
val bytes: Array[Byte] = obj match { | ||
case buf: java.nio.ByteBuffer => buf.array() | ||
case buf: java.nio.ByteBuffer => | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. You can't use There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This is in |
||
val arr = new Array[Byte](buf.remaining()) | ||
buf.get(arr) | ||
arr | ||
case arr: Array[Byte] => arr | ||
case other => throw new SparkException( | ||
s"Unknown BYTES type ${other.getClass.getName}") | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -27,6 +27,7 @@ import scala.util.control.NonFatal | |
import org.apache.hadoop.conf.Configuration | ||
import org.apache.hadoop.fs.Path | ||
|
||
import org.apache.spark.network.util.JavaUtils | ||
import org.apache.spark.streaming.Time | ||
import org.apache.spark.streaming.util.{BatchedWriteAheadLog, WriteAheadLog, WriteAheadLogUtils} | ||
import org.apache.spark.util.{Clock, Utils} | ||
|
@@ -212,7 +213,7 @@ private[streaming] class ReceivedBlockTracker( | |
writeAheadLog.readAll().asScala.foreach { byteBuffer => | ||
logTrace("Recovering record " + byteBuffer) | ||
Utils.deserialize[ReceivedBlockTrackerLogEvent]( | ||
byteBuffer.array, Thread.currentThread().getContextClassLoader) match { | ||
JavaUtils.bufferToArray(byteBuffer), Thread.currentThread().getContextClassLoader) match { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Might be worth it to have a version of |
||
case BlockAdditionEvent(receivedBlockInfo) => | ||
insertAddedBlock(receivedBlockInfo) | ||
case BatchAllocationEvent(time, allocatedBlocks) => | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -24,6 +24,8 @@ import scala.util.Try | |
import org.apache.hadoop.conf.Configuration | ||
import org.apache.hadoop.fs.FSDataOutputStream | ||
|
||
import org.apache.spark.util.Utils | ||
|
||
/** | ||
* A writer for writing byte-buffers to a write ahead log file. | ||
*/ | ||
|
@@ -48,17 +50,7 @@ private[streaming] class FileBasedWriteAheadLogWriter(path: String, hadoopConf: | |
val lengthToWrite = data.remaining() | ||
val segment = new FileBasedWriteAheadLogSegment(path, nextOffset, lengthToWrite) | ||
stream.writeInt(lengthToWrite) | ||
if (data.hasArray) { | ||
stream.write(data.array()) | ||
} else { | ||
// If the buffer is not backed by an array, we transfer using temp array | ||
// Note that despite the extra array copy, this should be faster than byte-by-byte copy | ||
while (data.hasRemaining) { | ||
val array = new Array[Byte](data.remaining) | ||
data.get(array) | ||
stream.write(array) | ||
} | ||
} | ||
Utils.writeByteBuffer(data, stream: OutputStream) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Weird, was the compiler complaining about something here? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yes. Because |
||
flush() | ||
nextOffset = stream.getPos() | ||
segment | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Because
decompressCache
putsByteBuffer
as a key, here should not change theschemaBytes
'sposition
, so cannot useByteBufferInputStream
here.