You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by sr...@apache.org on 2016/12/28 10:30:41 UTC

spark git commit: [SPARK-19010][CORE] Include Kryo exception in case of overflow

Repository: spark
Updated Branches:
  refs/heads/master 9cff67f34 -> 67fb33e7e


[SPARK-19010][CORE] Include Kryo exception in case of overflow

## What changes were proposed in this pull request?

This is to workaround an implicit result of #4947 which suppressed the
original Kryo exception if the overflow happened during serialization.

## How was this patch tested?

`KryoSerializerSuite` was augmented to reflect this change.

Author: Sergei Lebedev <su...@gmail.com>

Closes #16416 from superbobry/patch-1.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/67fb33e7
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/67fb33e7
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/67fb33e7

Branch: refs/heads/master
Commit: 67fb33e7e078eef3ecd5dcbfc26659b6fe2d054e
Parents: 9cff67f
Author: Sergei Lebedev <su...@gmail.com>
Authored: Wed Dec 28 10:30:38 2016 +0000
Committer: Sean Owen <so...@cloudera.com>
Committed: Wed Dec 28 10:30:38 2016 +0000

----------------------------------------------------------------------
 .../main/scala/org/apache/spark/serializer/KryoSerializer.scala   | 2 +-
 .../scala/org/apache/spark/serializer/KryoSerializerSuite.scala   | 3 ++-
 2 files changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/67fb33e7/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
index 7eb2da1..0381563 100644
--- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
+++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
@@ -313,7 +313,7 @@ private[spark] class KryoSerializerInstance(ks: KryoSerializer, useUnsafe: Boole
     } catch {
       case e: KryoException if e.getMessage.startsWith("Buffer overflow") =>
         throw new SparkException(s"Kryo serialization failed: ${e.getMessage}. To avoid this, " +
-          "increase spark.kryoserializer.buffer.max value.")
+          "increase spark.kryoserializer.buffer.max value.", e)
     } finally {
       releaseKryo(kryo)
     }

http://git-wip-us.apache.org/repos/asf/spark/blob/67fb33e7/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala
index 5040841..a30653b 100644
--- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala
@@ -23,7 +23,7 @@ import scala.collection.JavaConverters._
 import scala.collection.mutable
 import scala.reflect.ClassTag
 
-import com.esotericsoftware.kryo.Kryo
+import com.esotericsoftware.kryo.{Kryo, KryoException}
 import com.esotericsoftware.kryo.io.{Input => KryoInput, Output => KryoOutput}
 import org.roaringbitmap.RoaringBitmap
 
@@ -351,6 +351,7 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext {
     val ser = new KryoSerializer(conf).newInstance()
     val thrown = intercept[SparkException](ser.serialize(largeObject))
     assert(thrown.getMessage.contains(kryoBufferMaxProperty))
+    assert(thrown.getCause.isInstanceOf[KryoException])
   }
 
   test("SPARK-12222: deserialize RoaringBitmap throw Buffer underflow exception") {


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org