diff options
author | Davies Liu <davies@databricks.com> | 2015-08-12 21:26:00 -0700 |
---|---|---|
committer | Reynold Xin <rxin@databricks.com> | 2015-08-12 21:26:00 -0700 |
commit | a8ab2634c1eee143a4deaf309204df8add727f9e (patch) | |
tree | dccc4cfeed6be61aff6dbad37fdd2f088152d8c3 /sql | |
parent | 2278219054314f1d31ffc358a59aa5067f9f5de9 (diff) | |
download | spark-a8ab2634c1eee143a4deaf309204df8add727f9e.tar.gz spark-a8ab2634c1eee143a4deaf309204df8add727f9e.tar.bz2 spark-a8ab2634c1eee143a4deaf309204df8add727f9e.zip |
[SPARK-9832] [SQL] add a thread-safe lookup for BytesToBytseMap
This patch add a thread-safe lookup for BytesToBytseMap, and use that in broadcasted HashedRelation.
Author: Davies Liu <davies@databricks.com>
Closes #8151 from davies/safeLookup.
Diffstat (limited to 'sql')
-rw-r--r-- | sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala | 6 |
1 files changed, 4 insertions, 2 deletions
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala index bb333b4d5e..ea02076b41 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala @@ -215,8 +215,10 @@ private[joins] final class UnsafeHashedRelation( if (binaryMap != null) { // Used in Broadcast join - val loc = binaryMap.lookup(unsafeKey.getBaseObject, unsafeKey.getBaseOffset, - unsafeKey.getSizeInBytes) + val map = binaryMap // avoid the compiler error + val loc = new map.Location // this could be allocated in stack + binaryMap.safeLookup(unsafeKey.getBaseObject, unsafeKey.getBaseOffset, + unsafeKey.getSizeInBytes, loc) if (loc.isDefined) { val buffer = CompactBuffer[UnsafeRow]() |