[GitHub] spark pull request: [WIP][SPARK-2054][SQL] Code Generation for Exp...

2014-07-23 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/993#issuecomment-49837566
  
QA tests have started for PR 993. This patch merges cleanly. brView 
progress: 
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17019/consoleFull


---
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.
---


[GitHub] spark pull request: [WIP][SPARK-2054][SQL] Code Generation for Exp...

2014-07-23 Thread marmbrus
Github user marmbrus commented on the pull request:

https://github.com/apache/spark/pull/993#issuecomment-49912228
  
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.
---


[GitHub] spark pull request: [WIP][SPARK-2054][SQL] Code Generation for Exp...

2014-07-23 Thread concretevitamin
Github user concretevitamin commented on a diff in the pull request:

https://github.com/apache/spark/pull/993#discussion_r15311508
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala
 ---
@@ -0,0 +1,218 @@
+/*
+ * 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.catalyst.expressions.codegen
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.types._
+
+
+/**
+ * Generates bytecode that produces a new [[Row]] object based on a fixed 
set of input
+ * [[Expression Expressions]] and a given input [[Row]].  The returned 
[[Row]] object is custom
+ * generated based on the output types of the [[Expression]] to avoid 
boxing of primitive values.
+ */
+object GenerateProjection extends CodeGenerator[Seq[Expression], 
Projection] {
+  import scala.reflect.runtime.{universe = ru}
+  import scala.reflect.runtime.universe._
+
+  protected def canonicalize(in: Seq[Expression]): Seq[Expression] =
+in.map(ExpressionCanonicalizer(_))
+
+  protected def bind(in: Seq[Expression], inputSchema: Seq[Attribute]): 
Seq[Expression] =
+in.map(BindReferences.bindReference(_, inputSchema))
+
+  // Make Mutablility optional...
+  protected def create(expressions: Seq[Expression]): Projection = {
+val tupleLength = ru.Literal(Constant(expressions.length))
+val lengthDef = qfinal val length = $tupleLength
+
+/* TODO: Configurable...
+val nullFunctions =
+  q
+private final val nullSet = new 
org.apache.spark.util.collection.BitSet(length)
+final def setNullAt(i: Int) = nullSet.set(i)
--- End diff --

Could inlining help these defs?


---
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.
---


[GitHub] spark pull request: [WIP][SPARK-2054][SQL] Code Generation for Exp...

2014-07-23 Thread marmbrus
Github user marmbrus commented on a diff in the pull request:

https://github.com/apache/spark/pull/993#discussion_r15321958
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala 
---
@@ -33,6 +33,13 @@ abstract class DataType {
 
 case object NullType extends DataType
 
+object NativeType {
+  def all = Seq(
+IntegerType, BooleanType, LongType, DoubleType, FloatType, ShortType, 
ByteType, StringType)
+
+  def unapply(dt: DataType): Boolean = all.contains(dt)
--- End diff --

It is, but the syntax would be `t @ NativeType()` which is pretty similar 
to the standard matching of `_: IntegerType`


---
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.
---


[GitHub] spark pull request: [WIP][SPARK-2054][SQL] Code Generation for Exp...

2014-07-22 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/993#issuecomment-49776325
  
QA tests have started for PR 993. This patch merges cleanly. brView 
progress: 
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/16976/consoleFull


---
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.
---


[GitHub] spark pull request: [WIP][SPARK-2054][SQL] Code Generation for Exp...

2014-07-11 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/993#issuecomment-48697796
  
QA tests have started for PR 993. This patch merges cleanly. brView 
progress: 
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/16557/consoleFull


---
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.
---


[GitHub] spark pull request: [WIP][SPARK-2054][SQL] Code Generation for Exp...

2014-07-11 Thread markhamstra
Github user markhamstra commented on a diff in the pull request:

https://github.com/apache/spark/pull/993#discussion_r14845258
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
 ---
@@ -0,0 +1,421 @@
+/*
+ * 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.catalyst.expressions.codegen
+
+import scala.language.existentials
+
+import org.apache.spark.Logging
+import org.apache.spark.sql.catalyst.expressions
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.types._
+
+/**
+ * A base class for generators of byte code that performs expression 
evaluation.  Includes helpers
+ * for refering to Catalyst types and building trees that perform 
evaluation of individual
+ * expressions.
+ */
+abstract class CodeGenerator extends Logging {
+  import scala.reflect.runtime.{universe = ru}
+  import scala.reflect.runtime.universe._
+
+  import scala.tools.reflect.ToolBox
+
+  val toolBox = runtimeMirror(getClass.getClassLoader).mkToolBox()
+
+  val rowType = typeOf[Row]
+  val mutableRowType = typeOf[MutableRow]
+  val genericRowType = typeOf[GenericRow]
+  val genericMutableRowType = typeOf[GenericMutableRow]
+
+  val projectionType = typeOf[Projection]
+  val mutableProjectionType = typeOf[MutableProjection]
+
+  private val curId = new java.util.concurrent.atomic.AtomicInteger()
+  private val javaSeperator = $
--- End diff --

sp. separator


---
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.
---


[GitHub] spark pull request: [WIP][SPARK-2054][SQL] Code Generation for Exp...

2014-07-11 Thread markhamstra
Github user markhamstra commented on a diff in the pull request:

https://github.com/apache/spark/pull/993#discussion_r14845309
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
 ---
@@ -0,0 +1,421 @@
+/*
+ * 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.catalyst.expressions.codegen
+
+import scala.language.existentials
+
+import org.apache.spark.Logging
+import org.apache.spark.sql.catalyst.expressions
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.types._
+
+/**
+ * A base class for generators of byte code that performs expression 
evaluation.  Includes helpers
+ * for refering to Catalyst types and building trees that perform 
evaluation of individual
--- End diff --

sp. referring


---
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.
---


[GitHub] spark pull request: [WIP][SPARK-2054][SQL] Code Generation for Exp...

2014-07-11 Thread markhamstra
Github user markhamstra commented on a diff in the pull request:

https://github.com/apache/spark/pull/993#discussion_r14846043
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
 ---
@@ -0,0 +1,421 @@
+/*
+ * 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.catalyst.expressions.codegen
+
+import scala.language.existentials
+
+import org.apache.spark.Logging
+import org.apache.spark.sql.catalyst.expressions
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.types._
+
+/**
+ * A base class for generators of byte code that performs expression 
evaluation.  Includes helpers
+ * for refering to Catalyst types and building trees that perform 
evaluation of individual
+ * expressions.
+ */
+abstract class CodeGenerator extends Logging {
+  import scala.reflect.runtime.{universe = ru}
+  import scala.reflect.runtime.universe._
+
+  import scala.tools.reflect.ToolBox
+
+  val toolBox = runtimeMirror(getClass.getClassLoader).mkToolBox()
+
+  val rowType = typeOf[Row]
+  val mutableRowType = typeOf[MutableRow]
+  val genericRowType = typeOf[GenericRow]
+  val genericMutableRowType = typeOf[GenericMutableRow]
+
+  val projectionType = typeOf[Projection]
+  val mutableProjectionType = typeOf[MutableProjection]
+
+  private val curId = new java.util.concurrent.atomic.AtomicInteger()
+  private val javaSeperator = $
+
+  /**
+   * Returns a term name that is unique within this instance of a 
`CodeGenerator`.
+   *
+   * (Since we aren't in a macro context we do not seem to have access to 
the built in `freshName`
+   * function.)
+   */
+  protected def freshName(prefix: String): TermName = {
+newTermName(s$prefix$javaSeperator${curId.getAndIncrement})
+  }
+
+  /**
+   * Scala ASTs for evaluating an [[Expression]] given a [[Row]] of input.
+   *
+   * @param code The sequence of statements required to evaluate the 
expression.
+   * @param nullTerm A term that holds a boolean value representing 
whether the expression evaluated
+   * to null.
+   * @param primitiveTerm A term for a possible primitive value of the 
result of the evaluation. Not
+   *  valid if `nullTerm` is set to `false`.
+   * @param objectTerm An possibly boxed version of the result of 
evaluating this expression.
--- End diff --

s/A/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.
---


[GitHub] spark pull request: [WIP][SPARK-2054][SQL] Code Generation for Exp...

2014-07-11 Thread markhamstra
Github user markhamstra commented on a diff in the pull request:

https://github.com/apache/spark/pull/993#discussion_r14846216
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
 ---
@@ -0,0 +1,421 @@
+/*
+ * 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.catalyst.expressions.codegen
+
+import scala.language.existentials
+
+import org.apache.spark.Logging
+import org.apache.spark.sql.catalyst.expressions
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.types._
+
+/**
+ * A base class for generators of byte code that performs expression 
evaluation.  Includes helpers
+ * for refering to Catalyst types and building trees that perform 
evaluation of individual
+ * expressions.
+ */
+abstract class CodeGenerator extends Logging {
+  import scala.reflect.runtime.{universe = ru}
+  import scala.reflect.runtime.universe._
+
+  import scala.tools.reflect.ToolBox
+
+  val toolBox = runtimeMirror(getClass.getClassLoader).mkToolBox()
+
+  val rowType = typeOf[Row]
+  val mutableRowType = typeOf[MutableRow]
+  val genericRowType = typeOf[GenericRow]
+  val genericMutableRowType = typeOf[GenericMutableRow]
+
+  val projectionType = typeOf[Projection]
+  val mutableProjectionType = typeOf[MutableProjection]
+
+  private val curId = new java.util.concurrent.atomic.AtomicInteger()
+  private val javaSeperator = $
+
+  /**
+   * Returns a term name that is unique within this instance of a 
`CodeGenerator`.
+   *
+   * (Since we aren't in a macro context we do not seem to have access to 
the built in `freshName`
+   * function.)
+   */
+  protected def freshName(prefix: String): TermName = {
+newTermName(s$prefix$javaSeperator${curId.getAndIncrement})
+  }
+
+  /**
+   * Scala ASTs for evaluating an [[Expression]] given a [[Row]] of input.
+   *
+   * @param code The sequence of statements required to evaluate the 
expression.
+   * @param nullTerm A term that holds a boolean value representing 
whether the expression evaluated
+   * to null.
+   * @param primitiveTerm A term for a possible primitive value of the 
result of the evaluation. Not
+   *  valid if `nullTerm` is set to `false`.
+   * @param objectTerm An possibly boxed version of the result of 
evaluating this expression.
+   */
+  protected case class EvaluatedExpression(
+  code: Seq[Tree],
+  nullTerm: TermName,
+  primitiveTerm: TermName,
+  objectTerm: TermName)
+
+  /**
+   * Given an expression tree returns the code required to determine both 
if the result is NULL
+   * as well as the code required to compute the value.
--- End diff --

Description of the output seems less than precise.


---
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.
---


[GitHub] spark pull request: [WIP][SPARK-2054][SQL] Code Generation for Exp...

2014-07-11 Thread markhamstra
Github user markhamstra commented on a diff in the pull request:

https://github.com/apache/spark/pull/993#discussion_r14847035
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
 ---
@@ -0,0 +1,421 @@
+/*
+ * 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.catalyst.expressions.codegen
+
+import scala.language.existentials
+
+import org.apache.spark.Logging
+import org.apache.spark.sql.catalyst.expressions
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.types._
+
+/**
+ * A base class for generators of byte code that performs expression 
evaluation.  Includes helpers
+ * for refering to Catalyst types and building trees that perform 
evaluation of individual
+ * expressions.
+ */
+abstract class CodeGenerator extends Logging {
+  import scala.reflect.runtime.{universe = ru}
+  import scala.reflect.runtime.universe._
+
+  import scala.tools.reflect.ToolBox
+
+  val toolBox = runtimeMirror(getClass.getClassLoader).mkToolBox()
+
+  val rowType = typeOf[Row]
+  val mutableRowType = typeOf[MutableRow]
+  val genericRowType = typeOf[GenericRow]
+  val genericMutableRowType = typeOf[GenericMutableRow]
+
+  val projectionType = typeOf[Projection]
+  val mutableProjectionType = typeOf[MutableProjection]
+
+  private val curId = new java.util.concurrent.atomic.AtomicInteger()
+  private val javaSeperator = $
+
+  /**
+   * Returns a term name that is unique within this instance of a 
`CodeGenerator`.
+   *
+   * (Since we aren't in a macro context we do not seem to have access to 
the built in `freshName`
+   * function.)
+   */
+  protected def freshName(prefix: String): TermName = {
+newTermName(s$prefix$javaSeperator${curId.getAndIncrement})
+  }
+
+  /**
+   * Scala ASTs for evaluating an [[Expression]] given a [[Row]] of input.
+   *
+   * @param code The sequence of statements required to evaluate the 
expression.
+   * @param nullTerm A term that holds a boolean value representing 
whether the expression evaluated
+   * to null.
+   * @param primitiveTerm A term for a possible primitive value of the 
result of the evaluation. Not
+   *  valid if `nullTerm` is set to `false`.
+   * @param objectTerm An possibly boxed version of the result of 
evaluating this expression.
+   */
+  protected case class EvaluatedExpression(
+  code: Seq[Tree],
+  nullTerm: TermName,
+  primitiveTerm: TermName,
+  objectTerm: TermName)
+
+  /**
+   * Given an expression tree returns the code required to determine both 
if the result is NULL
+   * as well as the code required to compute the value.
+   */
+  def expressionEvaluator(e: Expression): EvaluatedExpression = {
+val primitiveTerm = freshName(primitiveTerm)
+val nullTerm = freshName(nullTerm)
+val objectTerm = freshName(objectTerm)
+
+implicit class Evaluate1(e: Expression) {
+  def castOrNull(f: TermName = Tree, dataType: DataType): Seq[Tree] = 
{
+val eval = expressionEvaluator(e)
+eval.code ++
+  q
+  val $nullTerm = ${eval.nullTerm}
+  val $primitiveTerm =
+if($nullTerm)
+  ${defaultPrimitive(dataType)}
+else
+  ${f(eval.primitiveTerm)}
+.children
--- End diff --

Is this really the way we want to format `q`?  To my eye, it would be 
much more readable as:
```scala
q
  val $nullTerm = ${eval1.nullTerm} || ${eval2.nullTerm}
  val $primitiveTerm: ${termForType(resultType)} =
if($nullTerm) {
  ${defaultPrimitive(resultType)}
} else {
  $resultCode.asInstanceOf[${termForType(resultType)}]
}
 .children : Seq[Tree]
```



---
If your project is set up for it, you 

[GitHub] spark pull request: [WIP][SPARK-2054][SQL] Code Generation for Exp...

2014-07-11 Thread markhamstra
Github user markhamstra commented on a diff in the pull request:

https://github.com/apache/spark/pull/993#discussion_r14848366
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
 ---
@@ -0,0 +1,421 @@
+/*
+ * 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.catalyst.expressions.codegen
+
+import scala.language.existentials
+
+import org.apache.spark.Logging
+import org.apache.spark.sql.catalyst.expressions
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.types._
+
+/**
+ * A base class for generators of byte code that performs expression 
evaluation.  Includes helpers
+ * for refering to Catalyst types and building trees that perform 
evaluation of individual
+ * expressions.
+ */
+abstract class CodeGenerator extends Logging {
+  import scala.reflect.runtime.{universe = ru}
+  import scala.reflect.runtime.universe._
+
+  import scala.tools.reflect.ToolBox
+
+  val toolBox = runtimeMirror(getClass.getClassLoader).mkToolBox()
+
+  val rowType = typeOf[Row]
+  val mutableRowType = typeOf[MutableRow]
+  val genericRowType = typeOf[GenericRow]
+  val genericMutableRowType = typeOf[GenericMutableRow]
+
+  val projectionType = typeOf[Projection]
+  val mutableProjectionType = typeOf[MutableProjection]
+
+  private val curId = new java.util.concurrent.atomic.AtomicInteger()
+  private val javaSeperator = $
+
+  /**
+   * Returns a term name that is unique within this instance of a 
`CodeGenerator`.
+   *
+   * (Since we aren't in a macro context we do not seem to have access to 
the built in `freshName`
+   * function.)
+   */
+  protected def freshName(prefix: String): TermName = {
+newTermName(s$prefix$javaSeperator${curId.getAndIncrement})
+  }
+
+  /**
+   * Scala ASTs for evaluating an [[Expression]] given a [[Row]] of input.
+   *
+   * @param code The sequence of statements required to evaluate the 
expression.
+   * @param nullTerm A term that holds a boolean value representing 
whether the expression evaluated
+   * to null.
+   * @param primitiveTerm A term for a possible primitive value of the 
result of the evaluation. Not
+   *  valid if `nullTerm` is set to `false`.
+   * @param objectTerm An possibly boxed version of the result of 
evaluating this expression.
+   */
+  protected case class EvaluatedExpression(
+  code: Seq[Tree],
+  nullTerm: TermName,
+  primitiveTerm: TermName,
+  objectTerm: TermName)
+
+  /**
+   * Given an expression tree returns the code required to determine both 
if the result is NULL
+   * as well as the code required to compute the value.
+   */
+  def expressionEvaluator(e: Expression): EvaluatedExpression = {
+val primitiveTerm = freshName(primitiveTerm)
+val nullTerm = freshName(nullTerm)
+val objectTerm = freshName(objectTerm)
+
+implicit class Evaluate1(e: Expression) {
+  def castOrNull(f: TermName = Tree, dataType: DataType): Seq[Tree] = 
{
+val eval = expressionEvaluator(e)
+eval.code ++
+  q
+  val $nullTerm = ${eval.nullTerm}
+  val $primitiveTerm =
+if($nullTerm)
+  ${defaultPrimitive(dataType)}
+else
+  ${f(eval.primitiveTerm)}
+.children
+  }
+}
+
+implicit class Evaluate2(expressions: (Expression, Expression)) {
+
+  /**
+   * Short hand for generating binary evaluation code, which depends 
on two sub-evaluations of
+   * the same type.  If either of the sub-expressions is null, the 
results of this computation
+   * is assumed to be null.
+   *
+   * @param f a function from two primitive term names to a tree that 
evaluates them.
+   

[GitHub] spark pull request: [WIP][SPARK-2054][SQL] Code Generation for Exp...

2014-07-11 Thread markhamstra
Github user markhamstra commented on a diff in the pull request:

https://github.com/apache/spark/pull/993#discussion_r14849312
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala
 ---
@@ -0,0 +1,83 @@
+/*
+ * 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.catalyst.expressions.codegen
+
+import org.apache.spark.sql.catalyst.expressions._
+
+/**
+ * Generates byte code that produces a [[MutableRow]] object that can 
update itself based on a new
+ * input [[Row]] for a fixed set of [[Expression Expressions]].
+ */
+object GenerateMutableProjection extends CodeGenerator {
+  import scala.reflect.runtime.{universe = ru}
+  import scala.reflect.runtime.universe._
+
+  // TODO: Should be weak references... bounded in size.
+  val projectionCache = new collection.mutable.HashMap[Seq[Expression], () 
= MutableProjection]
+
+  def apply(expressions: Seq[Expression], inputSchema: Seq[Attribute]): 
(() = MutableProjection) =
+apply(expressions.map(BindReferences.bindReference(_, inputSchema)))
+
+  // TODO: Safe to fire up multiple instances of the compiler?
+  def apply(expressions: Seq[Expression]): () = MutableProjection =
+globalLock.synchronized {
+  val cleanedExpressions = expressions.map(ExpressionCanonicalizer(_))
+  projectionCache.getOrElseUpdate(cleanedExpressions, 
createProjection(cleanedExpressions))
+}
+
+  val mutableRowName = newTermName(mutableRow)
+
+  def createProjection(expressions: Seq[Expression]): (() = 
MutableProjection) = {
--- End diff --

Should these `create*` functions be public?


---
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.
---


[GitHub] spark pull request: [WIP][SPARK-2054][SQL] Code Generation for Exp...

2014-07-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/993#issuecomment-48433215
  
 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.
---


[GitHub] spark pull request: [WIP][SPARK-2054][SQL] Code Generation for Exp...

2014-07-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/993#issuecomment-48433227
  
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.
---


[GitHub] spark pull request: [WIP][SPARK-2054][SQL] Code Generation for Exp...

2014-07-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/993#issuecomment-48433291
  

Refer to this link for build results: 
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/16447/


---
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.
---


[GitHub] spark pull request: [WIP][SPARK-2054][SQL] Code Generation for Exp...

2014-07-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/993#issuecomment-48433290
  
Merged build finished. 


---
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.
---


[GitHub] spark pull request: [WIP][SPARK-2054][SQL] Code Generation for Exp...

2014-07-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/993#issuecomment-48434077
  
 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.
---


[GitHub] spark pull request: [WIP][SPARK-2054][SQL] Code Generation for Exp...

2014-07-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/993#issuecomment-48434083
  
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.
---


[GitHub] spark pull request: [WIP][SPARK-2054][SQL] Code Generation for Exp...

2014-07-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/993#issuecomment-48434137
  
Merged build finished. 


---
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.
---


[GitHub] spark pull request: [WIP][SPARK-2054][SQL] Code Generation for Exp...

2014-07-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/993#issuecomment-48434138
  

Refer to this link for build results: 
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/16448/


---
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.
---


[GitHub] spark pull request: [WIP][SPARK-2054][SQL] Code Generation for Exp...

2014-07-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/993#issuecomment-48437951
  
 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.
---


[GitHub] spark pull request: [WIP][SPARK-2054][SQL] Code Generation for Exp...

2014-07-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/993#issuecomment-48437961
  
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.
---


[GitHub] spark pull request: [WIP][SPARK-2054][SQL] Code Generation for Exp...

2014-07-09 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/993#issuecomment-48438100
  

Refer to this link for build results: 
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/16450/


---
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.
---


[GitHub] spark pull request: [WIP][SPARK-2054][SQL] Code Generation for Exp...

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

https://github.com/apache/spark/pull/993#issuecomment-45585458
  
 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.
---


[GitHub] spark pull request: [WIP][SPARK-2054][SQL] Code Generation for Exp...

2014-06-10 Thread marmbrus
Github user marmbrus commented on a diff in the pull request:

https://github.com/apache/spark/pull/993#discussion_r13634526
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala
 ---
@@ -72,7 +72,9 @@ abstract class RuleExecutor[TreeType : TreeNode[_]] 
extends Logging {
 }
 iteration += 1
 if (iteration  batch.strategy.maxIterations) {
-  logger.info(sMax iterations ($iteration) reached for batch 
${batch.name})
+  if (iteration != 2) {
--- End diff --

It doesn't makes sense to print out the max iteration reached warning 
when the rule is only supposed to run once.


---
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.
---


[GitHub] spark pull request: [WIP][SPARK-2054][SQL] Code Generation for Exp...

2014-06-10 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/993#discussion_r13634545
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala
 ---
@@ -72,7 +72,9 @@ abstract class RuleExecutor[TreeType : TreeNode[_]] 
extends Logging {
 }
 iteration += 1
 if (iteration  batch.strategy.maxIterations) {
-  logger.info(sMax iterations ($iteration) reached for batch 
${batch.name})
+  if (iteration != 2) {
--- End diff --

maybe add that to the comment :)


---
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.
---


[GitHub] spark pull request: [WIP][SPARK-2054][SQL] Code Generation for Exp...

2014-06-06 Thread marmbrus
GitHub user marmbrus opened a pull request:

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

[WIP][SPARK-2054][SQL] Code Generation for Expression Evaluation

This PR drops the aggregation code generation, which will be added back in 
a follow-up PR.  The following remains to be done:
 - [ ] Remove remaining spurious changes.
 - [ ] Make configurable with #956

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

$ git pull https://github.com/marmbrus/spark newCodeGen

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

https://github.com/apache/spark/pull/993.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 #993


commit 1972e2d5ecf919824cb47ee6b6295242c785e064
Author: Michael Armbrust mich...@databricks.com
Date:   2014-05-11T18:30:56Z

Split out generic logic for hash joins and create two concrete physical 
operators: BroadcastHashJoin and ShuffledHashJoin.

commit f27119cf9dc46d18ea869e0e49a69f16dd0b7886
Author: Michael Armbrust mich...@databricks.com
Date:   2014-05-11T18:31:20Z

A simple strategy that broadcasts tables only when they are found in a 
configuration hint.

commit f7c750b7cf6473188a8f06d97e6532d8041660ed
Author: Michael Armbrust mich...@databricks.com
Date:   2014-05-12T22:09:39Z

Formatting.

commit 2904d29e5a0d9197f301bac82c40682619f6b508
Author: Michael Armbrust mich...@databricks.com
Date:   2014-05-29T22:34:24Z

WIP: Code generation and more

commit fa6939c552623fa05f14cc06ce80b20fa7b7d549
Author: Michael Armbrust mich...@databricks.com
Date:   2014-05-29T22:46:30Z

Widen datatypes

commit 8b1b4a955895f2046384be5f5ca254b7e2971ab2
Author: Michael Armbrust mich...@databricks.com
Date:   2014-05-29T23:52:53Z

Simple draft of IN generation.

commit c742978b817762a802c8c18a72e13bb723b518c0
Author: Michael Armbrust mich...@databricks.com
Date:   2014-05-30T01:01:29Z

Make IN case insensitive.

commit 0557d4c8799a8b5384c1410544912f9359b01977
Author: Michael Armbrust mich...@databricks.com
Date:   2014-05-30T01:14:14Z

Add byte code dumper.

commit c6f4835c67f07d199e2e79defbe716cc2f47293b
Author: Michael Armbrust mich...@databricks.com
Date:   2014-05-30T01:22:59Z

Unique names for isIn functions.

commit 764673b7ab2d4de48bf3880256940f66ad06f662
Author: Michael Armbrust mich...@databricks.com
Date:   2014-05-30T02:27:50Z

Global lock on code generation, cache orderings.

commit 6ebe45f7142f600f72aac73a4cac32f56ab684d4
Author: Zongheng Yang zonghen...@gmail.com
Date:   2014-06-01T20:31:20Z

WIP: Case code gen hack.

commit 760ce3cdc16db449068e1f9cf3e15384e3cf305f
Author: Zongheng Yang zonghen...@gmail.com
Date:   2014-06-01T21:30:12Z

Fix parsing CASE?

commit b54d3bc7a9fcd8d8134b2c4eb1765482872d9c9e
Author: Michael Armbrust mich...@databricks.com
Date:   2014-06-01T22:51:32Z

Hacky draft of case statements with code gen.

commit 7c5405f45b7b546d6e3b3f4f5667ce515910e50c
Author: Michael Armbrust mich...@databricks.com
Date:   2014-06-02T02:36:53Z

Correctly type nulls in the case when.

commit 5d0db90ae86e7fb38de67a93960c074810993a77
Author: Michael Armbrust mich...@databricks.com
Date:   2014-06-06T05:12:03Z

Quiet logging.

commit 82cbd15483775a62fddd446a333401ff7640e8e2
Author: Michael Armbrust mich...@databricks.com
Date:   2014-06-06T05:12:42Z

Turn off HashAggregation

commit 1209daaf49b0a87e7f68f89c79d02b446e624db3
Author: Michael Armbrust mich...@databricks.com
Date:   2014-06-06T05:16:37Z

Drop broken CASE WHEN.

commit 9c8872b57c5b120ae841c4525f032f7d5cb5712c
Author: Michael Armbrust mich...@databricks.com
Date:   2014-06-06T05:37:37Z

Drop anon SchemaRDD.

commit 0748abd9adc171257548df142bf63ab73159e5ac
Author: Michael Armbrust mich...@databricks.com
Date:   2014-06-06T05:55:32Z

Fix style. Drop test code.




---
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.
---


[GitHub] spark pull request: [WIP][SPARK-2054][SQL] Code Generation for Exp...

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

https://github.com/apache/spark/pull/993#issuecomment-45305818
  
 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.
---


[GitHub] spark pull request: [WIP][SPARK-2054][SQL] Code Generation for Exp...

2014-06-06 Thread rxin
Github user rxin commented on the pull request:

https://github.com/apache/spark/pull/993#issuecomment-45306506
  
One more to do is maven build ...


---
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.
---


[GitHub] spark pull request: [WIP][SPARK-2054][SQL] Code Generation for Exp...

2014-06-06 Thread concretevitamin
Github user concretevitamin commented on the pull request:

https://github.com/apache/spark/pull/993#issuecomment-45307236
  
Another TODO might be to beef up IN's code gen semantics (recall NULL in 
NULL and the alike cases).


---
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.
---


[GitHub] spark pull request: [WIP][SPARK-2054][SQL] Code Generation for Exp...

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

https://github.com/apache/spark/pull/993#issuecomment-45311891
  
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.
---


[GitHub] spark pull request: [WIP][SPARK-2054][SQL] Code Generation for Exp...

2014-06-06 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/993#discussion_r13479600
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala
 ---
@@ -48,14 +48,19 @@ class Projection(expressions: Seq[Expression]) extends 
(Row = Row) {
  * each time an input row is added.  This significatly reduces the cost of 
calcuating the
  * projection, but means that it is not safe
  */
-case class MutableProjection(expressions: Seq[Expression]) extends (Row = 
Row) {
+case class InterpretedMutableProjection(expressions: Seq[Expression]) 
extends MutableProjection {
   def this(expressions: Seq[Expression], inputSchema: Seq[Attribute]) =
 this(expressions.map(BindReferences.bindReference(_, inputSchema)))
 
   private[this] val exprArray = expressions.toArray
-  private[this] val mutableRow = new GenericMutableRow(exprArray.size)
+  private[this] var mutableRow: MutableRow = new 
GenericMutableRow(exprArray.size)
   def currentValue: Row = mutableRow
 
+  def target(row: MutableRow): MutableProjection = {
--- End diff --

maybe add some scaladoc to explain how this is 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.
---


[GitHub] spark pull request: [WIP][SPARK-2054][SQL] Code Generation for Exp...

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

https://github.com/apache/spark/pull/993#issuecomment-45311970
  
Merged build finished. 


---
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.
---


[GitHub] spark pull request: [WIP][SPARK-2054][SQL] Code Generation for Exp...

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

https://github.com/apache/spark/pull/993#issuecomment-45311971
  

Refer to this link for build results: 
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/15499/


---
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.
---


[GitHub] spark pull request: [WIP][SPARK-2054][SQL] Code Generation for Exp...

2014-06-06 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/993#discussion_r13479648
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala
 ---
@@ -23,6 +23,16 @@ import org.apache.spark.sql.catalyst.types.DataType
 import org.apache.spark.sql.catalyst.types.StringType
 import org.apache.spark.sql.catalyst.types.BooleanType
 
+case class SubString(string: Expression, start: Expression, end: 
Expression) extends Expression {
+  def children = string :: start :: end :: Nil
+  def references = children.flatMap(_.references).toSet
+  def dataType = StringType
+  def nullable = string.nullable
+
+  override def eval(input: Row) = ???
--- End diff --

should this be filled in?


---
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.
---


[GitHub] spark pull request: [WIP][SPARK-2054][SQL] Code Generation for Exp...

2014-06-06 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/993#discussion_r13479655
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala
 ---
@@ -23,6 +23,16 @@ import org.apache.spark.sql.catalyst.types.DataType
 import org.apache.spark.sql.catalyst.types.StringType
 import org.apache.spark.sql.catalyst.types.BooleanType
 
+case class SubString(string: Expression, start: Expression, end: 
Expression) extends Expression {
+  def children = string :: start :: end :: Nil
+  def references = children.flatMap(_.references).toSet
+  def dataType = StringType
+  def nullable = string.nullable
+
+  override def eval(input: Row) = ???
+
+  override def toString = ssubstr($string, $start, $end
--- End diff --

missing closing )


---
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.
---


[GitHub] spark pull request: [WIP][SPARK-2054][SQL] Code Generation for Exp...

2014-06-06 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/993#discussion_r13479686
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala
 ---
@@ -72,7 +72,9 @@ abstract class RuleExecutor[TreeType : TreeNode[_]] 
extends Logging {
 }
 iteration += 1
 if (iteration  batch.strategy.maxIterations) {
-  logger.info(sMax iterations ($iteration) reached for batch 
${batch.name})
+  if (iteration != 2) {
--- End diff --

what does this do?


---
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.
---


[GitHub] spark pull request: [WIP][SPARK-2054][SQL] Code Generation for Exp...

2014-06-06 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/993#discussion_r13479742
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala 
---
@@ -72,6 +79,13 @@ abstract class NumericType extends NativeType {
   val numeric: Numeric[JvmType]
 }
 
+object NumericType {
+  def unapply(a: Expression): Boolean = a match {
--- End diff --

ditto


---
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.
---


[GitHub] spark pull request: [WIP][SPARK-2054][SQL] Code Generation for Exp...

2014-06-06 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/993#discussion_r13479735
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala 
---
@@ -33,6 +33,13 @@ abstract class DataType {
 
 case object NullType extends DataType
 
+object NativeType {
+  def all = Seq(
+IntegerType, BooleanType, LongType, DoubleType, FloatType, ShortType, 
ByteType, StringType)
+
+  def unapply(dt: DataType): Boolean = all.contains(dt)
--- End diff --

Is this used for pattern matching? I think it'd be easier to understand to 
readers if it has a name like isNativeType(tpe) rather than NativeType(tpe)


---
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.
---


[GitHub] spark pull request: [WIP][SPARK-2054][SQL] Code Generation for Exp...

2014-06-06 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/993#discussion_r13479764
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala 
---
@@ -72,6 +79,13 @@ abstract class NumericType extends NativeType {
   val numeric: Numeric[JvmType]
 }
 
+object NumericType {
+  def unapply(a: Expression): Boolean = a match {
--- End diff --

also this is really just
```scala
def unapply(a: Expression): Boolean = e.dataType.isInstanceOf[NumericType]
```


---
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.
---


[GitHub] spark pull request: [WIP][SPARK-2054][SQL] Code Generation for Exp...

2014-06-06 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/993#discussion_r13479850
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/aggregates.scala ---
@@ -0,0 +1,175 @@
+/*
+ * 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
+
+import org.apache.spark.SparkContext
+import org.apache.spark.sql.catalyst.errors._
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.physical._
+import org.apache.spark.sql.catalyst.types._
+
+/**
+ * Attempt to rewrite aggregate to be more efficient.
+ *
+ * @param partial if true then aggregation is done partially on local data 
without shuffling to
+ *ensure all values where `groupingExpressions` are equal 
are present.
+ * @param groupingExpressions expressions that are evaluated to determine 
grouping.
+ * @param aggregateExpressions expressions that are computed for each 
group.
+ * @param child the input data source.
+ */
+case class HashAggregate(
--- End diff --

Is this just a rewrite of Aggregate?


---
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.
---


[GitHub] spark pull request: [WIP][SPARK-2054][SQL] Code Generation for Exp...

2014-06-06 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/993#discussion_r13479917
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/AnonymousRowSuite.scala ---
@@ -0,0 +1,32 @@
+package org.apache.spark.sql.catalyst.expressions
+
+import org.scalatest.FunSuite
+
+import org.apache.spark.sql.catalyst.expressions.AnonymousRow._
+
+case class Data(a: Int, b: Int)
+
+class AnonymousRowSuite extends FunSuite {
--- End diff --

what is AnonymousRow?


---
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.
---


[GitHub] spark pull request: [WIP][SPARK-2054][SQL] Code Generation for Exp...

2014-06-06 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/993#discussion_r13480009
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/GeneratedRow.scala
 ---
@@ -0,0 +1,833 @@
+/*
+ * 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
+package catalyst
+package expressions
+
+import types._
+
+object DumpByteCode {
+  import scala.sys.process._
+  val dumpDirectory = util.getTempFilePath(sparkSqlByteCode)
+  dumpDirectory.mkdir()
+
+  def apply(obj: Any): Unit = {
+val generatedClass = obj.getClass
+val classLoader =
+  generatedClass
+.getClassLoader
+.asInstanceOf[scala.tools.nsc.interpreter.AbstractFileClassLoader]
+val generatedBytes = classLoader.classBytes(generatedClass.getName)
+
+val packageDir = new java.io.File(dumpDirectory, 
generatedClass.getPackage.getName)
+if (!packageDir.exists()) { packageDir.mkdir() }
+
+val classFile =
+  new java.io.File(packageDir, 
generatedClass.getName.split(\\.).last + .class)
+
+val outfile = new java.io.FileOutputStream(classFile)
+outfile.write(generatedBytes)
+outfile.close()
+
+println(
+  sjavap -p -v -classpath ${dumpDirectory.getCanonicalPath} 
${generatedClass.getName}.!!)
+  }
+}
+
+object CodeGeneration
+
+class CodeGenerator extends Logging {
+  import scala.reflect.runtime.{universe = ru}
+  import scala.reflect.runtime.universe._
+
+  import scala.tools.reflect.ToolBox
+
+  val toolBox = runtimeMirror(getClass.getClassLoader).mkToolBox()
+
+  // TODO: Use typetags?
+  val rowType = tqorg.apache.spark.sql.catalyst.expressions.Row
+  val mutableRowType = 
tqorg.apache.spark.sql.catalyst.expressions.MutableRow
+  val genericRowType = 
tqorg.apache.spark.sql.catalyst.expressions.GenericRow
+  val genericMutableRowType = 
tqorg.apache.spark.sql.catalyst.expressions.GenericMutableRow
+
+  val projectionType = 
tqorg.apache.spark.sql.catalyst.expressions.Projection
+  val mutableProjectionType = 
tqorg.apache.spark.sql.catalyst.expressions.MutableProjection
+
+  private val curId = new java.util.concurrent.atomic.AtomicInteger()
+  private val javaSeperator = $
+
+  /**
+   * Returns a term name that is unique within this instance of a 
`CodeGenerator`.
+   *
+   * (Since we aren't in a macro context we do not seem to have access to 
the built in `freshName`
+   * function.)
+   */
+  protected def freshName(prefix: String): TermName = {
+newTermName(s$prefix$javaSeperator${curId.getAndIncrement})
+  }
+
+  /**
+   * Scala ASTs for evaluating an [[Expression]] given a [[Row]] of input.
+   *
+   * @param code The sequence of statements required to evaluate the 
expression.
+   * @param nullTerm A term that holds a boolean value representing 
whether the expression evaluated
+   * to null.
+   * @param primitiveTerm A term for a possible primitive value of the 
result of the evaluation. Not
+   *  valid if `nullTerm` is set to `false`.
+   * @param objectTerm An possibly boxed version of the result of 
evaluating this expression.
+   */
+  protected case class EvaluatedExpression(
+  code: Seq[Tree],
+  nullTerm: TermName,
+  primitiveTerm: TermName,
+  objectTerm: TermName) {
+
+def withObjectTerm = ???
+  }
+
+  /**
+   * Given an expression tree returns the code required to determine both 
if the result is NULL
+   * as well as the code required to compute the value.
+   */
+   def expressionEvaluator(e: Expression): EvaluatedExpression = {
--- End diff --

indent is off here


---
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 

[GitHub] spark pull request: [WIP][SPARK-2054][SQL] Code Generation for Exp...

2014-06-06 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/993#discussion_r13480043
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/GeneratedRow.scala
 ---
@@ -0,0 +1,833 @@
+/*
+ * 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
+package catalyst
+package expressions
+
+import types._
+
+object DumpByteCode {
+  import scala.sys.process._
+  val dumpDirectory = util.getTempFilePath(sparkSqlByteCode)
+  dumpDirectory.mkdir()
+
+  def apply(obj: Any): Unit = {
+val generatedClass = obj.getClass
+val classLoader =
+  generatedClass
+.getClassLoader
+.asInstanceOf[scala.tools.nsc.interpreter.AbstractFileClassLoader]
+val generatedBytes = classLoader.classBytes(generatedClass.getName)
+
+val packageDir = new java.io.File(dumpDirectory, 
generatedClass.getPackage.getName)
+if (!packageDir.exists()) { packageDir.mkdir() }
+
+val classFile =
+  new java.io.File(packageDir, 
generatedClass.getName.split(\\.).last + .class)
+
+val outfile = new java.io.FileOutputStream(classFile)
+outfile.write(generatedBytes)
+outfile.close()
+
+println(
+  sjavap -p -v -classpath ${dumpDirectory.getCanonicalPath} 
${generatedClass.getName}.!!)
+  }
+}
+
+object CodeGeneration
+
+class CodeGenerator extends Logging {
+  import scala.reflect.runtime.{universe = ru}
+  import scala.reflect.runtime.universe._
+
+  import scala.tools.reflect.ToolBox
+
+  val toolBox = runtimeMirror(getClass.getClassLoader).mkToolBox()
+
+  // TODO: Use typetags?
+  val rowType = tqorg.apache.spark.sql.catalyst.expressions.Row
+  val mutableRowType = 
tqorg.apache.spark.sql.catalyst.expressions.MutableRow
+  val genericRowType = 
tqorg.apache.spark.sql.catalyst.expressions.GenericRow
+  val genericMutableRowType = 
tqorg.apache.spark.sql.catalyst.expressions.GenericMutableRow
+
+  val projectionType = 
tqorg.apache.spark.sql.catalyst.expressions.Projection
+  val mutableProjectionType = 
tqorg.apache.spark.sql.catalyst.expressions.MutableProjection
+
+  private val curId = new java.util.concurrent.atomic.AtomicInteger()
+  private val javaSeperator = $
+
+  /**
+   * Returns a term name that is unique within this instance of a 
`CodeGenerator`.
+   *
+   * (Since we aren't in a macro context we do not seem to have access to 
the built in `freshName`
+   * function.)
+   */
+  protected def freshName(prefix: String): TermName = {
+newTermName(s$prefix$javaSeperator${curId.getAndIncrement})
+  }
+
+  /**
+   * Scala ASTs for evaluating an [[Expression]] given a [[Row]] of input.
+   *
+   * @param code The sequence of statements required to evaluate the 
expression.
+   * @param nullTerm A term that holds a boolean value representing 
whether the expression evaluated
+   * to null.
+   * @param primitiveTerm A term for a possible primitive value of the 
result of the evaluation. Not
+   *  valid if `nullTerm` is set to `false`.
+   * @param objectTerm An possibly boxed version of the result of 
evaluating this expression.
+   */
+  protected case class EvaluatedExpression(
+  code: Seq[Tree],
+  nullTerm: TermName,
+  primitiveTerm: TermName,
+  objectTerm: TermName) {
+
+def withObjectTerm = ???
+  }
+
+  /**
+   * Given an expression tree returns the code required to determine both 
if the result is NULL
+   * as well as the code required to compute the value.
+   */
+   def expressionEvaluator(e: Expression): EvaluatedExpression = {
+val primitiveTerm = freshName(primitiveTerm)
+val nullTerm = freshName(nullTerm)
+val objectTerm = freshName(objectTerm)
+
+implicit class Evaluate1(e: Expression) {
+  def 

[GitHub] spark pull request: [WIP][SPARK-2054][SQL] Code Generation for Exp...

2014-06-06 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/993#discussion_r13480098
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/GeneratedRow.scala
 ---
@@ -0,0 +1,833 @@
+/*
+ * 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
+package catalyst
+package expressions
+
+import types._
+
+object DumpByteCode {
+  import scala.sys.process._
+  val dumpDirectory = util.getTempFilePath(sparkSqlByteCode)
+  dumpDirectory.mkdir()
+
+  def apply(obj: Any): Unit = {
+val generatedClass = obj.getClass
+val classLoader =
+  generatedClass
+.getClassLoader
+.asInstanceOf[scala.tools.nsc.interpreter.AbstractFileClassLoader]
+val generatedBytes = classLoader.classBytes(generatedClass.getName)
+
+val packageDir = new java.io.File(dumpDirectory, 
generatedClass.getPackage.getName)
+if (!packageDir.exists()) { packageDir.mkdir() }
+
+val classFile =
+  new java.io.File(packageDir, 
generatedClass.getName.split(\\.).last + .class)
+
+val outfile = new java.io.FileOutputStream(classFile)
+outfile.write(generatedBytes)
+outfile.close()
+
+println(
+  sjavap -p -v -classpath ${dumpDirectory.getCanonicalPath} 
${generatedClass.getName}.!!)
+  }
+}
+
+object CodeGeneration
+
+class CodeGenerator extends Logging {
+  import scala.reflect.runtime.{universe = ru}
+  import scala.reflect.runtime.universe._
+
+  import scala.tools.reflect.ToolBox
+
+  val toolBox = runtimeMirror(getClass.getClassLoader).mkToolBox()
+
+  // TODO: Use typetags?
+  val rowType = tqorg.apache.spark.sql.catalyst.expressions.Row
+  val mutableRowType = 
tqorg.apache.spark.sql.catalyst.expressions.MutableRow
+  val genericRowType = 
tqorg.apache.spark.sql.catalyst.expressions.GenericRow
+  val genericMutableRowType = 
tqorg.apache.spark.sql.catalyst.expressions.GenericMutableRow
+
+  val projectionType = 
tqorg.apache.spark.sql.catalyst.expressions.Projection
+  val mutableProjectionType = 
tqorg.apache.spark.sql.catalyst.expressions.MutableProjection
+
+  private val curId = new java.util.concurrent.atomic.AtomicInteger()
+  private val javaSeperator = $
+
+  /**
+   * Returns a term name that is unique within this instance of a 
`CodeGenerator`.
+   *
+   * (Since we aren't in a macro context we do not seem to have access to 
the built in `freshName`
+   * function.)
+   */
+  protected def freshName(prefix: String): TermName = {
+newTermName(s$prefix$javaSeperator${curId.getAndIncrement})
+  }
+
+  /**
+   * Scala ASTs for evaluating an [[Expression]] given a [[Row]] of input.
+   *
+   * @param code The sequence of statements required to evaluate the 
expression.
+   * @param nullTerm A term that holds a boolean value representing 
whether the expression evaluated
+   * to null.
+   * @param primitiveTerm A term for a possible primitive value of the 
result of the evaluation. Not
+   *  valid if `nullTerm` is set to `false`.
+   * @param objectTerm An possibly boxed version of the result of 
evaluating this expression.
+   */
+  protected case class EvaluatedExpression(
+  code: Seq[Tree],
+  nullTerm: TermName,
+  primitiveTerm: TermName,
+  objectTerm: TermName) {
+
+def withObjectTerm = ???
+  }
+
+  /**
+   * Given an expression tree returns the code required to determine both 
if the result is NULL
+   * as well as the code required to compute the value.
+   */
+   def expressionEvaluator(e: Expression): EvaluatedExpression = {
+val primitiveTerm = freshName(primitiveTerm)
+val nullTerm = freshName(nullTerm)
+val objectTerm = freshName(objectTerm)
+
+implicit class Evaluate1(e: Expression) {
+  def 

[GitHub] spark pull request: [WIP][SPARK-2054][SQL] Code Generation for Exp...

2014-06-06 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/993#discussion_r13480084
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/GeneratedRow.scala
 ---
@@ -0,0 +1,833 @@
+/*
--- End diff --

given how large this file is, it might be worth it to create a cg package 
(or not) and break the file into multiple files.


---
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.
---


[GitHub] spark pull request: [WIP][SPARK-2054][SQL] Code Generation for Exp...

2014-06-06 Thread hsaputra
Github user hsaputra commented on the pull request:

https://github.com/apache/spark/pull/993#issuecomment-45349273
  
HI @marmbrus, one generic comment, could you add object or class header 
comment to describe why each of them needed and the context why they are used. 
It should be very useful for people trying to use and help to improve and fix 
issue in the module.


---
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.
---


[GitHub] spark pull request: [WIP][SPARK-2054][SQL] Code Generation for Exp...

2014-06-06 Thread hsaputra
Github user hsaputra commented on a diff in the pull request:

https://github.com/apache/spark/pull/993#discussion_r13495163
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/GeneratedRow.scala
 ---
@@ -0,0 +1,833 @@
+/*
+ * 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
+package catalyst
+package expressions
+
+import types._
+
+object DumpByteCode {
+  import scala.sys.process._
+  val dumpDirectory = util.getTempFilePath(sparkSqlByteCode)
+  dumpDirectory.mkdir()
+
+  def apply(obj: Any): Unit = {
+val generatedClass = obj.getClass
+val classLoader =
+  generatedClass
+.getClassLoader
+.asInstanceOf[scala.tools.nsc.interpreter.AbstractFileClassLoader]
+val generatedBytes = classLoader.classBytes(generatedClass.getName)
+
+val packageDir = new java.io.File(dumpDirectory, 
generatedClass.getPackage.getName)
+if (!packageDir.exists()) { packageDir.mkdir() }
+
+val classFile =
+  new java.io.File(packageDir, 
generatedClass.getName.split(\\.).last + .class)
+
+val outfile = new java.io.FileOutputStream(classFile)
+outfile.write(generatedBytes)
+outfile.close()
+
+println(
+  sjavap -p -v -classpath ${dumpDirectory.getCanonicalPath} 
${generatedClass.getName}.!!)
+  }
+}
+
+object CodeGeneration
+
+class CodeGenerator extends Logging {
--- End diff --

Would be helpful to add class header comment to describe the usage of this 
class in bigger 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.
---