Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,8 @@ import org.apache.spark.shuffle.ShuffleMemoryManager
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.execution.SparkSqlSerializer
import org.apache.spark.sql.execution.metric.LongSQLMetric
import org.apache.spark.sql.execution.local.LocalNode
import org.apache.spark.sql.execution.metric.{LongSQLMetric, SQLMetrics}
import org.apache.spark.unsafe.Platform
import org.apache.spark.unsafe.map.BytesToBytesMap
import org.apache.spark.unsafe.memory.{MemoryLocation, ExecutorMemoryManager, MemoryAllocator, TaskMemoryManager}
Expand Down Expand Up @@ -113,6 +114,10 @@ final class UniqueKeyHashedRelation(private var hashTable: JavaHashMap[InternalR

private[execution] object HashedRelation {

def apply(localNode: LocalNode, keyGenerator: Projection): HashedRelation = {
apply(localNode.asIterator, SQLMetrics.nullLongMetric, keyGenerator)
}

def apply(
input: Iterator[InternalRow],
numInputRows: LongSQLMetric,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -75,8 +75,7 @@ case class HashJoinNode(

override def open(): Unit = {
buildNode.open()
hashed = HashedRelation.apply(
new LocalNodeIterator(buildNode), SQLMetrics.nullLongMetric, buildSideKeyGenerator)
hashed = HashedRelation(buildNode, buildSideKeyGenerator)
streamedNode.open()
joinRow = new JoinedRow
resultProjection = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -69,10 +69,15 @@ abstract class LocalNode(conf: SQLConf) extends TreeNode[LocalNode] with Logging
*/
def close(): Unit

/**
* Returns the content through the [[Iterator]] interface.
*/
final def asIterator: Iterator[InternalRow] = new LocalNodeIterator(this)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is this on the critical path? If yes - it'd actually be faster to expose the LocalNodeIterator type since JIT can inline it.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this is called when a HashJoinNode is opened. Will it make much of a difference?


/**
* Returns the content of the iterator from the beginning to the end in the form of a Scala Seq.
*/
def collect(): Seq[Row] = {
final def collect(): Seq[Row] = {
val converter = CatalystTypeConverters.createToScalaConverter(StructType.fromAttributes(output))
val result = new scala.collection.mutable.ArrayBuffer[Row]
open()
Expand Down Expand Up @@ -135,7 +140,7 @@ abstract class BinaryLocalNode(conf: SQLConf) extends LocalNode(conf) {
/**
* An thin wrapper around a [[LocalNode]] that provides an `Iterator` interface.
*/
private[local] class LocalNodeIterator(localNode: LocalNode) extends Iterator[InternalRow] {
private class LocalNodeIterator(localNode: LocalNode) extends Iterator[InternalRow] {
private var nextRow: InternalRow = _

override def hasNext: Boolean = {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,116 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.spark.sql.execution.local

import org.apache.spark.SparkFunSuite
import org.apache.spark.sql.SQLConf
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.types.IntegerType

class LocalNodeSuite extends SparkFunSuite {
private val data = (1 to 100).toArray

test("basic open, next, fetch, close") {
val node = new DummyLocalNode(data)
assert(!node.isOpen)
node.open()
assert(node.isOpen)
data.foreach { i =>
assert(node.next())
// fetch should be idempotent
val fetched = node.fetch()
assert(node.fetch() === fetched)
assert(node.fetch() === fetched)
assert(node.fetch().numFields === 1)
assert(node.fetch().getInt(0) === i)
}
assert(!node.next())
node.close()
assert(!node.isOpen)
}

test("asIterator") {
val node = new DummyLocalNode(data)
val iter = node.asIterator
node.open()
data.foreach { i =>
// hasNext should be idempotent
assert(iter.hasNext)
assert(iter.hasNext)
val item = iter.next()
assert(item.numFields === 1)
assert(item.getInt(0) === i)
}
intercept[NoSuchElementException] {
iter.next()
}
node.close()
}

test("collect") {
val node = new DummyLocalNode(data)
node.open()
val collected = node.collect()
assert(collected.size === data.size)
assert(collected.forall(_.size === 1))
assert(collected.map(_.getInt(0)) === data)
node.close()
}

}

/**
* A dummy [[LocalNode]] that just returns one row per integer in the input.
*/
private case class DummyLocalNode(conf: SQLConf, input: Array[Int]) extends LocalNode(conf) {
private var index = Int.MinValue

def this(input: Array[Int]) {
this(new SQLConf, input)
}

def isOpen: Boolean = {
index != Int.MinValue
}

override def output: Seq[Attribute] = {
Seq(AttributeReference("something", IntegerType)())
}

override def children: Seq[LocalNode] = Seq.empty

override def open(): Unit = {
index = -1
}

override def next(): Boolean = {
index += 1
index < input.size
}

override def fetch(): InternalRow = {
assert(index >= 0 && index < input.size)
val values = Array(input(index).asInstanceOf[Any])
new GenericInternalRow(values)
}

override def close(): Unit = {
index = Int.MinValue
}
}