[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-08 Thread asfgit
Github user asfgit closed the pull request at:

https://github.com/apache/spark/pull/8953


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-07 Thread andrewor14
Github user andrewor14 commented on a diff in the pull request:

https://github.com/apache/spark/pull/8953#discussion_r41455915
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/local/LocalNode.scala ---
@@ -129,7 +129,7 @@ abstract class LocalNode(conf: SQLConf) extends 
QueryPlan[LocalNode] with Loggin
 }
   }
 
-  protected def newMutableProjection(
+  protected[sql] def newMutableProjection(
--- End diff --

these don't have to be `protected[sql]` anymore right?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-07 Thread andrewor14
Github user andrewor14 commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-146351913
  
LGTM will merge pending tasks. Thanks for addressing the comments so 
quickly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-07 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-146314715
  
 Merged build triggered.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-07 Thread andrewor14
Github user andrewor14 commented on a diff in the pull request:

https://github.com/apache/spark/pull/8953#discussion_r41454228
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala
 ---
@@ -26,7 +26,10 @@ abstract class BaseMutableProjection extends 
MutableProjection
 
 /**
  * Generates byte code that produces a [[MutableRow]] object that can 
update itself based on a new
- * input [[InternalRow]] for a fixed set of [[Expression Expressions]].
+ * input [[InternalRow]] for a fixed set of [[Expression Expressions]]. It 
exposes a `target`
+ * method. This method is used to set the row that will be updated. So, 
when `target` is used, the
+ * [[MutableRow]] object created internally will not be used. If `target` 
is not used, the
+ * [[MutableRow]] object created internally will be used.
--- End diff --

oh got it


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-07 Thread andrewor14
Github user andrewor14 commented on a diff in the pull request:

https://github.com/apache/spark/pull/8953#discussion_r41454426
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/local/BinaryHashJoinNode.scala
 ---
@@ -0,0 +1,77 @@
+/*
+* 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.sql.SQLConf
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.execution.joins.{HashedRelation, BuildLeft, 
BuildRight, BuildSide}
+
+/**
+ * A [[HashJoinNode]] that builds the [[HashedRelation]] according to the 
value of
+ * `buildSide`. The actual work of this node is defined in 
[[HashJoinNode]].
+ */
+case class BinaryHashJoinNode(
+conf: SQLConf,
+leftKeys: Seq[Expression],
+rightKeys: Seq[Expression],
+buildSide: BuildSide,
+left: LocalNode,
+right: LocalNode)
+  extends BinaryLocalNode(conf) with HashJoinNode {
+
+  protected override val (streamedNode, streamedKeys) = buildSide match {
+case BuildLeft => (right, rightKeys)
+case BuildRight => (left, leftKeys)
+  }
+
+  private val (buildNode, buildKeys) = buildSide match {
+case BuildLeft => (left, leftKeys)
+case BuildRight => (right, rightKeys)
+  }
+
+  override def output: Seq[Attribute] = left.output ++ right.output
+
+  private def buildSideKeyGenerator: Projection = {
+// We are expecting the data types of buildKeys and streamedKeys are 
the same.
+assert(buildKeys.map(_.dataType) == streamedKeys.map(_.dataType))
+if (isUnsafeMode) {
+  UnsafeProjection.create(buildKeys, buildNode.output)
+} else {
+  newMutableProjection(buildKeys, buildNode.output)()
+}
+  }
+
+  protected override def doOpen(): Unit = {
+// buildNode's prepare has been called in this.prepare.
--- End diff --

probably don't need this comment? We generally don't call `prepare` in 
`open` I don't think


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-07 Thread andrewor14
Github user andrewor14 commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-146350510
  
@yhuai thanks, this looks much simpler. My remaining comments are mainly 
concerned with comments and tests.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-07 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-146351574
  
 Merged build triggered.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-07 Thread yhuai
Github user yhuai commented on a diff in the pull request:

https://github.com/apache/spark/pull/8953#discussion_r41439778
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/execution/local/HashJoinNodeSuite.scala
 ---
@@ -51,20 +87,50 @@ class HashJoinNodeSuite extends LocalNodeTest {
   val rightInputMap = rightInput.toMap
   val leftNode = new DummyNode(joinNameAttributes, leftInput)
   val rightNode = new DummyNode(joinNicknameAttributes, rightInput)
-  val makeNode = (node1: LocalNode, node2: LocalNode) => {
-resolveExpressions(new HashJoinNode(
-  conf, Seq('id1), Seq('id2), buildSide, node1, node2))
+  val makeBinaryHashJoinNode = (node1: LocalNode, node2: LocalNode) => 
{
+val binaryHashJoinNode =
+  BinaryHashJoinNode(conf, Seq('id1), Seq('id2), buildSide, node1, 
node2)
+resolveExpressions(binaryHashJoinNode)
+  }
+  val makeBroadcastJoinNode = (node1: LocalNode, node2: LocalNode) => {
+val leftKeys = Seq('id1.attr)
+val rightKeys = Seq('id2.attr)
+// Figure out the build side and stream side.
+val (buildNode, buildKeys, streamedNode, streamedKeys) = buildSide 
match {
+  case BuildLeft => (node1, leftKeys, node2, rightKeys)
+  case BuildRight => (node2, rightKeys, node1, leftKeys)
+}
+// Resolve the expressions of the build side and then create a 
HashedRelation.
+val resolvedBuildNode = resolveExpressions(buildNode)
+val resolvedBuildKeys = resolveExpressions(buildKeys, 
resolvedBuildNode)
+val hashedRelation = buildHashedRelation(conf, resolvedBuildKeys, 
resolvedBuildNode)
+val broadcastHashedRelation = 
sqlContext.sparkContext.broadcast(hashedRelation)
--- End diff --

@andrewor14 This is the place I need sqlcontext.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-07 Thread andrewor14
Github user andrewor14 commented on a diff in the pull request:

https://github.com/apache/spark/pull/8953#discussion_r41454327
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala
 ---
@@ -23,8 +23,8 @@ import org.apache.spark.sql.types._
 
 
 /**
- * Generates byte code that produces a [[MutableRow]] object that can 
update itself based on a new
- * input [[InternalRow]] for a fixed set of [[Expression Expressions]].
+ * Generates byte code that produces a [[MutableRow]] object (not a 
[[UnsafeRow]]) that can update
--- End diff --

not a -> not an


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-07 Thread andrewor14
Github user andrewor14 commented on a diff in the pull request:

https://github.com/apache/spark/pull/8953#discussion_r41454970
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/local/HashJoinNode.scala 
---
@@ -73,10 +60,21 @@ case class HashJoinNode(
 }
   }
 
+  /** Sets the HashedRelation used by this node. */
--- End diff --

I would add a sentence to say this needs happen before the first `next` is 
called.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-07 Thread andrewor14
Github user andrewor14 commented on a diff in the pull request:

https://github.com/apache/spark/pull/8953#discussion_r41454914
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/local/HashJoinNode.scala 
---
@@ -73,10 +60,21 @@ case class HashJoinNode(
 }
   }
 
+  /** Sets the HashedRelation used by this node. */
+  protected def withHashedRelation(hashedRelation: HashedRelation): Unit = 
{
+hashed = hashedRelation
+  }
+
+  /**
+   * For nodes that extends this, they can use doOpen to add operations 
needed in the open method.
--- End diff --

I think just the following is good:
`Custom open implementation to be overridden by subclasses.`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-07 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-146314736
  
Merged build started.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-07 Thread yhuai
Github user yhuai commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-146314450
  
@andrewor14 I have made `HashJoinNode` a `trait`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-07 Thread yhuai
Github user yhuai commented on a diff in the pull request:

https://github.com/apache/spark/pull/8953#discussion_r41439747
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/execution/local/HashJoinNodeSuite.scala
 ---
@@ -19,10 +19,12 @@ package org.apache.spark.sql.execution.local
 
 import org.apache.spark.sql.SQLConf
 import org.apache.spark.sql.catalyst.dsl.expressions._
-import org.apache.spark.sql.execution.joins.{BuildLeft, BuildRight, 
BuildSide}
+import org.apache.spark.sql.catalyst.expressions.{UnsafeProjection, 
Expression}
+import org.apache.spark.sql.execution.joins.{HashedRelation, BuildLeft, 
BuildRight, BuildSide}
+import org.apache.spark.sql.test.SharedSQLContext
 
 
-class HashJoinNodeSuite extends LocalNodeTest {
+class HashJoinNodeSuite extends LocalNodeTest with SharedSQLContext {
--- End diff --

I need to create a broadcast variable. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-07 Thread yhuai
Github user yhuai commented on a diff in the pull request:

https://github.com/apache/spark/pull/8953#discussion_r41439686
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/local/BinaryHashJoinNode.scala
 ---
@@ -0,0 +1,102 @@
+/*
+* 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.sql.SQLConf
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.execution.joins.{HashedRelation, BuildLeft, 
BuildRight, BuildSide}
+
+/**
+ * A wrapper of [[HashJoinNode]]. It will build the [[HashedRelation]] 
according to the value of
+ * `buildSide`. The actual work of this node will be delegated to the 
[[HashJoinNode]]
+ * that is created in `open`.
+ */
+case class BinaryHashJoinNode(
+conf: SQLConf,
+leftKeys: Seq[Expression],
+rightKeys: Seq[Expression],
+buildSide: BuildSide,
+left: LocalNode,
+right: LocalNode) extends BinaryLocalNode(conf) {
+
+  private[this] lazy val (buildNode, buildKeys, streamedNode, 
streamedKeys) = buildSide match {
+case BuildLeft => (left, leftKeys, right, rightKeys)
+case BuildRight => (right, rightKeys, left, leftKeys)
+  }
+
+  private[this] val hashJoinNode: HashJoinNode = {
+HashJoinNode(
+  conf = conf,
+  streamedKeys = streamedKeys,
+  streamedNode = streamedNode,
+  buildSide = buildSide,
+  buildOutput = buildNode.output,
+  isWrapped = true)
+  }
+  override def output: Seq[Attribute] = left.output ++ right.output
+
+  private[this] def isUnsafeMode: Boolean = {
+(codegenEnabled && unsafeEnabled && 
UnsafeProjection.canSupport(buildKeys))
+  }
+
+  private[this] def buildSideKeyGenerator: Projection = {
+if (isUnsafeMode) {
+  UnsafeProjection.create(buildKeys, buildNode.output)
+} else {
+  newMutableProjection(buildKeys, buildNode.output)()
+}
+  }
+
+  override def open(): Unit = {
+// buildNode's prepare has been called in this.prepare.
+buildNode.open()
+val hashedRelation = HashedRelation(buildNode, buildSideKeyGenerator)
+// We have built the HashedRelation. So, close buildNode.
+buildNode.close()
+
+// Call the open of streamedNode.
--- End diff --

removed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-07 Thread andrewor14
Github user andrewor14 commented on a diff in the pull request:

https://github.com/apache/spark/pull/8953#discussion_r41454832
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/local/HashJoinNode.scala 
---
@@ -73,10 +60,21 @@ case class HashJoinNode(
 }
   }
 
+  /** Sets the HashedRelation used by this node. */
+  protected def withHashedRelation(hashedRelation: HashedRelation): Unit = 
{
+hashed = hashedRelation
+  }
+
+  /**
+   * For nodes that extends this, they can use doOpen to add operations 
needed in the open method.
+   * The implementation of this method should invoke its children's open 
methods.
+   */
+  protected def doOpen(): Unit
+
   override def open(): Unit = {
-buildNode.open()
-hashed = HashedRelation(buildNode, buildSideKeyGenerator)
-streamedNode.open()
+// First, call doOpen to invoke custom operations for a node.
+doOpen()
+// Second, initialize common internal states.
--- End diff --

These comments still seem unnecessary :). I would remove them but feel free 
to keep them if you prefer.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-07 Thread andrewor14
Github user andrewor14 commented on a diff in the pull request:

https://github.com/apache/spark/pull/8953#discussion_r41455801
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/execution/local/HashJoinNodeSuite.scala
 ---
@@ -34,6 +36,40 @@ class HashJoinNodeSuite extends LocalNodeTest {
   }
 
   /**
+   * Builds a [[HashedRelation]] based on a resolved `buildKeys`
+   * and a resolved `buildNode`.
+   */
+  private def buildHashedRelation(
+  conf: SQLConf,
+  buildKeys: Seq[Expression],
+  buildNode: LocalNode): HashedRelation = {
+
+// Check if we are in the Unsafe mode.
+val isUnsafeMode =
+  conf.codegenEnabled &&
+conf.unsafeEnabled &&
+UnsafeProjection.canSupport(buildKeys)
+
+// Create projection used for extracting keys
+val buildSideKeyGenerator =
+  if (isUnsafeMode) {
+UnsafeProjection.create(buildKeys, buildNode.output)
+  } else {
+buildNode.newMutableProjection(buildKeys, buildNode.output)()
+  }
+
+// Setup the node.
+buildNode.prepare()
+buildNode.open()
+// Build the HashedRelation
+val hashedRelation = HashedRelation(buildNode, buildSideKeyGenerator)
+// Close the node.
--- End diff --

can you remove some of these comments that don't add any value?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-07 Thread yhuai
Github user yhuai commented on a diff in the pull request:

https://github.com/apache/spark/pull/8953#discussion_r41439536
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala
 ---
@@ -26,7 +26,10 @@ abstract class BaseMutableProjection extends 
MutableProjection
 
 /**
  * Generates byte code that produces a [[MutableRow]] object that can 
update itself based on a new
- * input [[InternalRow]] for a fixed set of [[Expression Expressions]].
+ * input [[InternalRow]] for a fixed set of [[Expression Expressions]]. It 
exposes a `target`
+ * method. This method is used to set the row that will be updated. So, 
when `target` is used, the
+ * [[MutableRow]] object created internally will not be used. If `target` 
is not used, the
+ * [[MutableRow]] object created internally will be used.
--- End diff --

See `public ${classOf[BaseMutableProjection].getName} 
target($mutableRowType row) {`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-07 Thread andrewor14
Github user andrewor14 commented on a diff in the pull request:

https://github.com/apache/spark/pull/8953#discussion_r41454596
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/local/HashJoinNode.scala 
---
@@ -17,27 +17,23 @@
 
 package org.apache.spark.sql.execution.local
 
-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.execution.joins._
-import org.apache.spark.sql.execution.metric.SQLMetrics
 
 /**
+ * A node for inner hash equi-join. [[BinaryHashJoinNode]] and 
[[BroadcastHashJoinNode]]
--- End diff --

```
An abstract node for sharing common functionality among different 
implementations of
inner hash equi-join, notably [[BinaryHashJoinNode]] and 
[[BroadcastHashJoinNode]].
```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-07 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-146351831
  
  [Test build #43352 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/43352/consoleFull)
 for   PR 8953 at commit 
[`081e331`](https://github.com/apache/spark/commit/081e33118685f8bfa537ae954b1b889d90912f9c).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-07 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-146351602
  
Merged build started.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-07 Thread yhuai
Github user yhuai commented on a diff in the pull request:

https://github.com/apache/spark/pull/8953#discussion_r41439563
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/local/BinaryHashJoinNode.scala
 ---
@@ -0,0 +1,102 @@
+/*
+* 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.sql.SQLConf
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.execution.joins.{HashedRelation, BuildLeft, 
BuildRight, BuildSide}
+
+/**
+ * A wrapper of [[HashJoinNode]]. It will build the [[HashedRelation]] 
according to the value of
+ * `buildSide`. The actual work of this node will be delegated to the 
[[HashJoinNode]]
+ * that is created in `open`.
+ */
+case class BinaryHashJoinNode(
+conf: SQLConf,
+leftKeys: Seq[Expression],
+rightKeys: Seq[Expression],
+buildSide: BuildSide,
+left: LocalNode,
+right: LocalNode) extends BinaryLocalNode(conf) {
+
+  private[this] lazy val (buildNode, buildKeys, streamedNode, 
streamedKeys) = buildSide match {
--- End diff --

Removed `lazy`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-07 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-146315065
  
  [Test build #43340 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/43340/consoleFull)
 for   PR 8953 at commit 
[`ecbd1d0`](https://github.com/apache/spark/commit/ecbd1d01a26420e3129910a5e06b77a98598a703).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-07 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-146345115
  
Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-07 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-146344965
  
  [Test build #43340 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/43340/console)
 for   PR 8953 at commit 
[`ecbd1d0`](https://github.com/apache/spark/commit/ecbd1d01a26420e3129910a5e06b77a98598a703).
 * This patch **passes all tests**.
 * This patch merges cleanly.
 * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-07 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-146345117
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/43340/
Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-07 Thread andrewor14
Github user andrewor14 commented on a diff in the pull request:

https://github.com/apache/spark/pull/8953#discussion_r41454294
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala
 ---
@@ -26,7 +26,10 @@ abstract class BaseMutableProjection extends 
MutableProjection
 
 /**
  * Generates byte code that produces a [[MutableRow]] object that can 
update itself based on a new
- * input [[InternalRow]] for a fixed set of [[Expression Expressions]].
+ * input [[InternalRow]] for a fixed set of [[Expression Expressions]]. It 
exposes a `target`
+ * method. This method is used to set the row that will be updated. So, 
when `target` is used, the
+ * [[MutableRow]] object created internally will not be used. If `target` 
is not used, the
--- End diff --

This comment is quite verbose. I think you can just say
```
It exposes a `target` method, which is used to set the row that will be 
updated. The internal
[[MutableRow]] object created internally is used only when `target` is not 
used.
```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-07 Thread andrewor14
Github user andrewor14 commented on a diff in the pull request:

https://github.com/apache/spark/pull/8953#discussion_r41455352
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/execution/local/HashJoinNodeSuite.scala
 ---
@@ -19,10 +19,12 @@ package org.apache.spark.sql.execution.local
 
 import org.apache.spark.sql.SQLConf
 import org.apache.spark.sql.catalyst.dsl.expressions._
-import org.apache.spark.sql.execution.joins.{BuildLeft, BuildRight, 
BuildSide}
+import org.apache.spark.sql.catalyst.expressions.{UnsafeProjection, 
Expression}
+import org.apache.spark.sql.execution.joins.{HashedRelation, BuildLeft, 
BuildRight, BuildSide}
+import org.apache.spark.sql.test.SharedSQLContext
 
 
-class HashJoinNodeSuite extends LocalNodeTest {
+class HashJoinNodeSuite extends LocalNodeTest with SharedSQLContext {
--- End diff --

can't you just do
```
new TorrentBroadcast[HashedRelation](relation, id)
```
directly?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-07 Thread andrewor14
Github user andrewor14 commented on a diff in the pull request:

https://github.com/apache/spark/pull/8953#discussion_r41455395
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/execution/local/HashJoinNodeSuite.scala
 ---
@@ -51,20 +87,50 @@ class HashJoinNodeSuite extends LocalNodeTest {
   val rightInputMap = rightInput.toMap
   val leftNode = new DummyNode(joinNameAttributes, leftInput)
   val rightNode = new DummyNode(joinNicknameAttributes, rightInput)
-  val makeNode = (node1: LocalNode, node2: LocalNode) => {
-resolveExpressions(new HashJoinNode(
-  conf, Seq('id1), Seq('id2), buildSide, node1, node2))
+  val makeBinaryHashJoinNode = (node1: LocalNode, node2: LocalNode) => 
{
+val binaryHashJoinNode =
+  BinaryHashJoinNode(conf, Seq('id1), Seq('id2), buildSide, node1, 
node2)
+resolveExpressions(binaryHashJoinNode)
+  }
+  val makeBroadcastJoinNode = (node1: LocalNode, node2: LocalNode) => {
+val leftKeys = Seq('id1.attr)
+val rightKeys = Seq('id2.attr)
+// Figure out the build side and stream side.
+val (buildNode, buildKeys, streamedNode, streamedKeys) = buildSide 
match {
+  case BuildLeft => (node1, leftKeys, node2, rightKeys)
+  case BuildRight => (node2, rightKeys, node1, leftKeys)
+}
+// Resolve the expressions of the build side and then create a 
HashedRelation.
+val resolvedBuildNode = resolveExpressions(buildNode)
+val resolvedBuildKeys = resolveExpressions(buildKeys, 
resolvedBuildNode)
+val hashedRelation = buildHashedRelation(conf, resolvedBuildKeys, 
resolvedBuildNode)
+val broadcastHashedRelation = 
sqlContext.sparkContext.broadcast(hashedRelation)
--- End diff --

can't you just do
```
new TorrentBroadcast[HashedRelation](relation, id)
```
directly? If you do that do you still need the spark context?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-07 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-146379899
  
  [Test build #43352 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/43352/console)
 for   PR 8953 at commit 
[`081e331`](https://github.com/apache/spark/commit/081e33118685f8bfa537ae954b1b889d90912f9c).
 * This patch **fails PySpark unit tests**.
 * This patch merges cleanly.
 * This patch adds the following public classes _(experimental)_:
  * `case class BinaryHashJoinNode(`
  * `case class BroadcastHashJoinNode(`
  * `trait HashJoinNode `



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-07 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-146379946
  
Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-07 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-146381338
  
  [Test build #43366 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/43366/consoleFull)
 for   PR 8953 at commit 
[`081e331`](https://github.com/apache/spark/commit/081e33118685f8bfa537ae954b1b889d90912f9c).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-07 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-146380203
  
 Merged build triggered.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-07 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-146380220
  
Merged build started.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-07 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-146379948
  
Test FAILed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/43352/
Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-07 Thread andrewor14
Github user andrewor14 commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-146380034
  
retest this please


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-07 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-146399283
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/43366/
Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-07 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-146399185
  
  [Test build #43366 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/43366/console)
 for   PR 8953 at commit 
[`081e331`](https://github.com/apache/spark/commit/081e33118685f8bfa537ae954b1b889d90912f9c).
 * This patch **passes all tests**.
 * This patch merges cleanly.
 * This patch adds the following public classes _(experimental)_:
  * `case class BinaryHashJoinNode(`
  * `case class BroadcastHashJoinNode(`
  * `trait HashJoinNode `



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-07 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-146399281
  
Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-06 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-145943182
  
 Merged build triggered.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-06 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-145943213
  
Merged build started.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-06 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-145943458
  
  [Test build #43283 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/43283/consoleFull)
 for   PR 8953 at commit 
[`65c6a50`](https://github.com/apache/spark/commit/65c6a50cdb9e8ad242e35ee94dc5c6a68f81649a).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-06 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-145974452
  
  [Test build #43283 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/43283/console)
 for   PR 8953 at commit 
[`65c6a50`](https://github.com/apache/spark/commit/65c6a50cdb9e8ad242e35ee94dc5c6a68f81649a).
 * This patch **passes all tests**.
 * This patch merges cleanly.
 * This patch adds the following public classes _(experimental)_:
  * `case class BinaryHashJoinNode(`
  * `case class BroadcastHashJoinNode(`



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-06 Thread JoshRosen
Github user JoshRosen commented on a diff in the pull request:

https://github.com/apache/spark/pull/8953#discussion_r41307678
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/local/BinaryHashJoinNode.scala
 ---
@@ -0,0 +1,102 @@
+/*
+* 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.sql.SQLConf
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.execution.joins.{HashedRelation, BuildLeft, 
BuildRight, BuildSide}
+
+/**
+ * A wrapper of [[HashJoinNode]]. It will build the [[HashedRelation]] 
according to the value of
+ * `buildSide`. The actual work of this node will be delegated to the 
[[HashJoinNode]]
+ * that is created in `open`.
+ */
+case class BinaryHashJoinNode(
+conf: SQLConf,
+leftKeys: Seq[Expression],
+rightKeys: Seq[Expression],
+buildSide: BuildSide,
+left: LocalNode,
+right: LocalNode) extends BinaryLocalNode(conf) {
+
+  private[this] lazy val (buildNode, buildKeys, streamedNode, 
streamedKeys) = buildSide match {
--- End diff --

Why is this a lazy val?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-06 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-145974627
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/43283/
Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-06 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-145974624
  
Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-06 Thread andrewor14
Github user andrewor14 commented on a diff in the pull request:

https://github.com/apache/spark/pull/8953#discussion_r41317182
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/local/BinaryHashJoinNode.scala
 ---
@@ -0,0 +1,102 @@
+/*
+* 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.sql.SQLConf
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.execution.joins.{HashedRelation, BuildLeft, 
BuildRight, BuildSide}
+
+/**
+ * A wrapper of [[HashJoinNode]]. It will build the [[HashedRelation]] 
according to the value of
+ * `buildSide`. The actual work of this node will be delegated to the 
[[HashJoinNode]]
+ * that is created in `open`.
+ */
+case class BinaryHashJoinNode(
+conf: SQLConf,
+leftKeys: Seq[Expression],
+rightKeys: Seq[Expression],
+buildSide: BuildSide,
+left: LocalNode,
+right: LocalNode) extends BinaryLocalNode(conf) {
+
+  private[this] lazy val (buildNode, buildKeys, streamedNode, 
streamedKeys) = buildSide match {
--- End diff --

agreed, doesn't need to be


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-06 Thread andrewor14
Github user andrewor14 commented on a diff in the pull request:

https://github.com/apache/spark/pull/8953#discussion_r41345489
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/local/HashJoinNode.scala 
---
@@ -24,19 +24,33 @@ import org.apache.spark.sql.execution.joins._
 import org.apache.spark.sql.execution.metric.SQLMetrics
 
 /**
+ * A node for inner hash equi-join. It can be used individually or wrapped 
by other
+ * inner hash equi-join nodes such as [[BinaryHashJoinNode]]. This node 
takes a already
+ * built [[HashedRelation]] and a [[LocalNode]] representing the streamed 
side.
+ * If this node is used individually, `isWrapped` should be set to false.
+ * If this node is wrapped in another node, `isWrapped` should be set to 
true
--- End diff --

As discussed offline, it might be better to put the common code in a trait 
instead of wrapping local nodes. Right now it's kind of confusing because 
sometimes we call `prepare` or `close` but sometimes we don't.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-06 Thread andrewor14
Github user andrewor14 commented on a diff in the pull request:

https://github.com/apache/spark/pull/8953#discussion_r41345552
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/execution/local/HashJoinNodeSuite.scala
 ---
@@ -19,10 +19,12 @@ package org.apache.spark.sql.execution.local
 
 import org.apache.spark.sql.SQLConf
 import org.apache.spark.sql.catalyst.dsl.expressions._
-import org.apache.spark.sql.execution.joins.{BuildLeft, BuildRight, 
BuildSide}
+import org.apache.spark.sql.catalyst.expressions.{UnsafeProjection, 
Expression}
+import org.apache.spark.sql.execution.joins.{HashedRelation, BuildLeft, 
BuildRight, BuildSide}
+import org.apache.spark.sql.test.SharedSQLContext
 
 
-class HashJoinNodeSuite extends LocalNodeTest {
+class HashJoinNodeSuite extends LocalNodeTest with SharedSQLContext {
--- End diff --

where does this test use `SQLContext`? We made an effort to not introduce 
this dependency in an earlier patch so we should try to avoid the same here. 
#8764


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-06 Thread andrewor14
Github user andrewor14 commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-146062232
  
As discussed offline, we should explore alternatives where we don't have to 
wrap local nodes to avoid the wrapping logic. Right now we need to worry about 
the initialization / prepare / close order and it's easy to miss something 
there.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-06 Thread andrewor14
Github user andrewor14 commented on a diff in the pull request:

https://github.com/apache/spark/pull/8953#discussion_r41341938
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala
 ---
@@ -26,7 +26,10 @@ abstract class BaseMutableProjection extends 
MutableProjection
 
 /**
  * Generates byte code that produces a [[MutableRow]] object that can 
update itself based on a new
- * input [[InternalRow]] for a fixed set of [[Expression Expressions]].
+ * input [[InternalRow]] for a fixed set of [[Expression Expressions]]. It 
exposes a `target`
+ * method. This method is used to set the row that will be updated. So, 
when `target` is used, the
+ * [[MutableRow]] object created internally will not be used. If `target` 
is not used, the
+ * [[MutableRow]] object created internally will be used.
--- End diff --

I don't understand this comment. Where is `target`? I can't find it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-06 Thread andrewor14
Github user andrewor14 commented on a diff in the pull request:

https://github.com/apache/spark/pull/8953#discussion_r41342201
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/local/BinaryHashJoinNode.scala
 ---
@@ -0,0 +1,102 @@
+/*
+* 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.sql.SQLConf
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.execution.joins.{HashedRelation, BuildLeft, 
BuildRight, BuildSide}
+
+/**
+ * A wrapper of [[HashJoinNode]]. It will build the [[HashedRelation]] 
according to the value of
+ * `buildSide`. The actual work of this node will be delegated to the 
[[HashJoinNode]]
+ * that is created in `open`.
+ */
+case class BinaryHashJoinNode(
+conf: SQLConf,
+leftKeys: Seq[Expression],
+rightKeys: Seq[Expression],
+buildSide: BuildSide,
+left: LocalNode,
+right: LocalNode) extends BinaryLocalNode(conf) {
+
+  private[this] lazy val (buildNode, buildKeys, streamedNode, 
streamedKeys) = buildSide match {
+case BuildLeft => (left, leftKeys, right, rightKeys)
+case BuildRight => (right, rightKeys, left, leftKeys)
+  }
+
+  private[this] val hashJoinNode: HashJoinNode = {
+HashJoinNode(
+  conf = conf,
+  streamedKeys = streamedKeys,
+  streamedNode = streamedNode,
+  buildSide = buildSide,
+  buildOutput = buildNode.output,
+  isWrapped = true)
+  }
+  override def output: Seq[Attribute] = left.output ++ right.output
+
+  private[this] def isUnsafeMode: Boolean = {
+(codegenEnabled && unsafeEnabled && 
UnsafeProjection.canSupport(buildKeys))
+  }
+
+  private[this] def buildSideKeyGenerator: Projection = {
+if (isUnsafeMode) {
+  UnsafeProjection.create(buildKeys, buildNode.output)
+} else {
+  newMutableProjection(buildKeys, buildNode.output)()
+}
+  }
+
+  override def open(): Unit = {
+// buildNode's prepare has been called in this.prepare.
+buildNode.open()
+val hashedRelation = HashedRelation(buildNode, buildSideKeyGenerator)
+// We have built the HashedRelation. So, close buildNode.
+buildNode.close()
+
+// Call the open of streamedNode.
+streamedNode.open()
+// Set the HashedRelation used by the HashJoinNode.
+hashJoinNode.withHashedRelation(hashedRelation)
+// Setup this HashJoinNode. We still call these in case there is any 
setup work
+// that needs to be done in this HashJoinNode. Because isWrapped is 
true,
+// prepare and open will not propagate to the child of streamedNode.
+hashJoinNode.prepare()
--- End diff --

is it too late to call `prepare` here in `open`? Don't we need to call it 
in `prepare`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-06 Thread andrewor14
Github user andrewor14 commented on a diff in the pull request:

https://github.com/apache/spark/pull/8953#discussion_r41342262
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/local/BinaryHashJoinNode.scala
 ---
@@ -0,0 +1,102 @@
+/*
+* 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.sql.SQLConf
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.execution.joins.{HashedRelation, BuildLeft, 
BuildRight, BuildSide}
+
+/**
+ * A wrapper of [[HashJoinNode]]. It will build the [[HashedRelation]] 
according to the value of
+ * `buildSide`. The actual work of this node will be delegated to the 
[[HashJoinNode]]
+ * that is created in `open`.
+ */
+case class BinaryHashJoinNode(
+conf: SQLConf,
+leftKeys: Seq[Expression],
+rightKeys: Seq[Expression],
+buildSide: BuildSide,
+left: LocalNode,
+right: LocalNode) extends BinaryLocalNode(conf) {
+
+  private[this] lazy val (buildNode, buildKeys, streamedNode, 
streamedKeys) = buildSide match {
+case BuildLeft => (left, leftKeys, right, rightKeys)
+case BuildRight => (right, rightKeys, left, leftKeys)
+  }
+
+  private[this] val hashJoinNode: HashJoinNode = {
+HashJoinNode(
+  conf = conf,
+  streamedKeys = streamedKeys,
+  streamedNode = streamedNode,
+  buildSide = buildSide,
+  buildOutput = buildNode.output,
+  isWrapped = true)
+  }
+  override def output: Seq[Attribute] = left.output ++ right.output
+
+  private[this] def isUnsafeMode: Boolean = {
+(codegenEnabled && unsafeEnabled && 
UnsafeProjection.canSupport(buildKeys))
+  }
+
+  private[this] def buildSideKeyGenerator: Projection = {
+if (isUnsafeMode) {
+  UnsafeProjection.create(buildKeys, buildNode.output)
+} else {
+  newMutableProjection(buildKeys, buildNode.output)()
+}
+  }
+
+  override def open(): Unit = {
+// buildNode's prepare has been called in this.prepare.
+buildNode.open()
+val hashedRelation = HashedRelation(buildNode, buildSideKeyGenerator)
+// We have built the HashedRelation. So, close buildNode.
+buildNode.close()
+
+// Call the open of streamedNode.
--- End diff --

these comments are not very useful... :) I would just remove them since 
they don't add any value.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-06 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-145939334
  
 Merged build triggered.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-06 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-145939744
  
  [Test build #43282 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/43282/consoleFull)
 for   PR 8953 at commit 
[`e8a3d17`](https://github.com/apache/spark/commit/e8a3d174c290b80cf84ef32490a242592af68ae6).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-06 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-145939417
  
Merged build started.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-06 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-145940926
  
  [Test build #43282 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/43282/console)
 for   PR 8953 at commit 
[`e8a3d17`](https://github.com/apache/spark/commit/e8a3d174c290b80cf84ef32490a242592af68ae6).
 * This patch **fails Scala style tests**.
 * This patch merges cleanly.
 * This patch adds the following public classes _(experimental)_:
  * `case class BinaryHashJoinNode(`
  * `case class BroadcastHashJoinNode(`



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-06 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-145940939
  
Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-06 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-145940945
  
Test FAILed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/43282/
Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-04 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-145320578
  
Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-04 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-145320579
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/43223/
Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-04 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-145320558
  
  [Test build #43223 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/43223/console)
 for   PR 8953 at commit 
[`b9d5bc9`](https://github.com/apache/spark/commit/b9d5bc9efcbd3eae8ce6a114cf5fc16864ccbe0c).
 * This patch **passes all tests**.
 * This patch merges cleanly.
 * This patch adds the following public classes _(experimental)_:
  * `case class BinaryHashJoinNode(`
  * `case class BroadcastHashJoinNode(`



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-145315792
  
 Merged build triggered.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-03 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-145315793
  
Merged build started.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-03 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-145316435
  
  [Test build #43223 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/43223/consoleFull)
 for   PR 8953 at commit 
[`b9d5bc9`](https://github.com/apache/spark/commit/b9d5bc9efcbd3eae8ce6a114cf5fc16864ccbe0c).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-02 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-145110237
  
  [Test build #43187 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/43187/consoleFull)
 for   PR 8953 at commit 
[`f262b36`](https://github.com/apache/spark/commit/f262b367d530195f722caa1f3b0f86218a56f579).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-02 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-145116114
  
  [Test build #43189 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/43189/consoleFull)
 for   PR 8953 at commit 
[`f262b36`](https://github.com/apache/spark/commit/f262b367d530195f722caa1f3b0f86218a56f579).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-02 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-145105907
  
Merged build started.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-02 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-145115496
  
Merged build started.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-02 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-145111631
  
Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-02 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-145111615
  
  [Test build #43187 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/43187/console)
 for   PR 8953 at commit 
[`f262b36`](https://github.com/apache/spark/commit/f262b367d530195f722caa1f3b0f86218a56f579).
 * This patch **fails Scala style tests**.
 * This patch merges cleanly.
 * This patch adds the following public classes _(experimental)_:
  * `case class BinarydHashJoinNode(`
  * `case class BroadcastHashJoinNode(`



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-02 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-145111635
  
Test FAILed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/43187/
Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-02 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-145105845
  
 Merged build triggered.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-02 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-145115473
  
 Merged build triggered.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-02 Thread yhuai
Github user yhuai commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-145114438
  
test this please


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-02 Thread yhuai
Github user yhuai commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-145170338
  
test this please


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-02 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-145170971
  
 Merged build triggered.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-02 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-145170999
  
Merged build started.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-02 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-145172033
  
  [Test build #43201 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/43201/consoleFull)
 for   PR 8953 at commit 
[`f262b36`](https://github.com/apache/spark/commit/f262b367d530195f722caa1f3b0f86218a56f579).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-02 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-145187389
  
  [Test build #43201 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/43201/console)
 for   PR 8953 at commit 
[`f262b36`](https://github.com/apache/spark/commit/f262b367d530195f722caa1f3b0f86218a56f579).
 * This patch **passes all tests**.
 * This patch merges cleanly.
 * This patch adds the following public classes _(experimental)_:
  * `case class BinarydHashJoinNode(`
  * `case class BroadcastHashJoinNode(`



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-02 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-145187431
  
Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [SQL] Build HashedRelation outsi...

2015-10-02 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-145187432
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/43201/
Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [sql] Build HashedRelation outsi...

2015-09-30 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-144573079
  
 Merged build triggered.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [sql] Build HashedRelation outsi...

2015-09-30 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-144573095
  
Merged build started.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [sql] Build HashedRelation outsi...

2015-09-30 Thread yhuai
GitHub user yhuai opened a pull request:

https://github.com/apache/spark/pull/8953

[SPARK-10887] [sql] Build HashedRelation outside of HashJoinNode.

This PR refactors `HashJoinNode` to take a existing `HashedRelation`. So, 
we can reuse this node for both `ShuffledHashJoin` and `BroadcastHashJoin`.

https://issues.apache.org/jira/browse/SPARK-10887

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/yhuai/spark SPARK-10887

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/spark/pull/8953.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #8953


commit 0cf8d44634b43df280ebef43dcd4941e8195a656
Author: Yin Huai 
Date:   2015-09-30T23:20:53Z

Build HashedRelation outside of HashJoinNode.




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [sql] Build HashedRelation outsi...

2015-09-30 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-144574443
  
  [Test build #43140 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/43140/consoleFull)
 for   PR 8953 at commit 
[`0cf8d44`](https://github.com/apache/spark/commit/0cf8d44634b43df280ebef43dcd4941e8195a656).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [sql] Build HashedRelation outsi...

2015-09-30 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-144590367
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/43140/
Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [sql] Build HashedRelation outsi...

2015-09-30 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-144590300
  
  [Test build #43140 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/43140/console)
 for   PR 8953 at commit 
[`0cf8d44`](https://github.com/apache/spark/commit/0cf8d44634b43df280ebef43dcd4941e8195a656).
 * This patch **passes all tests**.
 * This patch merges cleanly.
 * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-10887] [sql] Build HashedRelation outsi...

2015-09-30 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/8953#issuecomment-144590365
  
Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org