[GitHub] spark pull request: [SPARK-9022] [SQL] Generated projections for U...

2015-07-17 Thread asfgit
Github user asfgit closed the pull request at:

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


---
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-9022] [SQL] Generated projections for U...

2015-07-17 Thread rxin
Github user rxin commented on the pull request:

https://github.com/apache/spark/pull/7437#issuecomment-122214389
  
Thanks - I've merged this.



---
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-9022] [SQL] Generated projections for U...

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

https://github.com/apache/spark/pull/7437#issuecomment-122213676
  
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-9022] [SQL] Generated projections for U...

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

https://github.com/apache/spark/pull/7437#issuecomment-122213574
  
  [Test build #37594 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/37594/console)
 for   PR 7437 at commit 
[`dbf538e`](https://github.com/apache/spark/commit/dbf538e748650753f764ac35b9b98b33db50663a).
 * This patch **passes all tests**.
 * This patch merges cleanly.
 * This patch adds the following public classes _(experimental)_:
  * `abstract class UnsafeProjection extends Projection `
  * `case class FromUnsafeProjection(fields: Seq[DataType]) extends 
Projection `
  * `abstract class BaseProjection extends Projection `
  * `class SpecificProjection extends $`
  * `class SpecificProjection extends $`



---
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-9022] [SQL] Generated projections for U...

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

https://github.com/apache/spark/pull/7437#issuecomment-122197492
  
  [Test build #37594 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/37594/consoleFull)
 for   PR 7437 at commit 
[`dbf538e`](https://github.com/apache/spark/commit/dbf538e748650753f764ac35b9b98b33db50663a).


---
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-9022] [SQL] Generated projections for U...

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

https://github.com/apache/spark/pull/7437#issuecomment-122196934
  
 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-9022] [SQL] Generated projections for U...

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

https://github.com/apache/spark/pull/7437#issuecomment-122196956
  
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-9022] [SQL] Generated projections for U...

2015-07-16 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/7437#discussion_r34865356
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala
 ---
@@ -0,0 +1,110 @@
+/*
+ * 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.types.{BinaryType, StringType}
+
+
+/**
+ * Generates a [[Projection]] that returns an [[UnsafeRow]].
+ *
+ * Note: The returned UnsafeRow will be pointed to a scratch buffer inside 
the projection.
+ */
+object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], 
UnsafeProjection] {
+
+  protected def canonicalize(in: Seq[Expression]): Seq[Expression] =
+in.map(ExpressionCanonicalizer.execute)
+
+  protected def bind(in: Seq[Expression], inputSchema: Seq[Attribute]): 
Seq[Expression] =
+in.map(BindReferences.bindReference(_, inputSchema))
+
+  protected def create(expressions: Seq[Expression]): UnsafeProjection = {
+val ctx = newCodeGenContext()
+val codes = expressions.map(_.gen(ctx))
+val all_exprs = codes.map(_.code).mkString("\n")
+val fixedSize = 8 * codes.length + 
UnsafeRow.calculateBitSetWidthInBytes(codes.length)
+val stringWriter = 
"org.apache.spark.sql.catalyst.expressions.StringUnsafeColumnWriter"
+val binaryWriter = 
"org.apache.spark.sql.catalyst.expressions.BinaryUnsafeColumnWriter"
+val additionalSize = expressions.zipWithIndex.map { case (e, i) =>
+  e.dataType match {
+case StringType =>
+  s" + (${codes(i).isNull} ? 0 : 
$stringWriter.getSize(${codes(i).primitive}))"
+case BinaryType =>
+  s" + (${codes(i).isNull} ? 0 : 
$binaryWriter.getSize(${codes(i).primitive}))"
+case _ => ""
+  }
+}.mkString("")
+
+val writers = expressions.zipWithIndex.map { case (e, i) =>
+  val update = e.dataType match {
+case dt if ctx.isPrimitiveType(dt) =>
+  s"${ctx.setColumn("target", dt, i, codes(i).primitive)}"
+case StringType =>
+  s"cursor += $stringWriter.write(target, ${codes(i).primitive}, 
$i, cursor)"
+case BinaryType =>
+  s"cursor += $binaryWriter.write(target, ${codes(i).primitive}, 
$i, cursor)"
+case _ =>
+  throw new Exception(s"Not supported DataType: ${e.dataType}")
--- End diff --

need to support nulltype here.

BTW let's throw UnsupportedOperationException rather than Exception.



---
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-9022] [SQL] Generated projections for U...

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

https://github.com/apache/spark/pull/7437#issuecomment-122191875
  
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-9022] [SQL] Generated projections for U...

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

https://github.com/apache/spark/pull/7437#issuecomment-122191825
  
  [Test build #37591 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/37591/console)
 for   PR 7437 at commit 
[`dc737b2`](https://github.com/apache/spark/commit/dc737b20d47979184441890063ecc1ba336ac67b).
 * This patch **fails Spark unit tests**.
 * This patch merges cleanly.
 * This patch adds the following public classes _(experimental)_:
  * `abstract class UnsafeProjection extends Projection `
  * `case class FromUnsafeProjection(fields: Seq[DataType]) extends 
Projection `
  * `abstract class BaseProjection extends Projection `
  * `class SpecificProjection extends $`
  * `class SpecificProjection extends $`



---
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-9022] [SQL] Generated projections for U...

2015-07-16 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/7437#discussion_r34864786
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala
 ---
@@ -0,0 +1,113 @@
+/*
+ * 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.types.{BinaryType, StringType}
+
+
+/**
+ * Generates a [[Projection]] that returns an [[UnsafeRow]].
+ *
+ * Note: The returned UnsafeRow will be pointed to a scratch buffer inside 
the projection.
+ */
+object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], 
UnsafeProjection] {
+
+  protected def canonicalize(in: Seq[Expression]): Seq[Expression] =
+in.map(ExpressionCanonicalizer.execute)
+
+  protected def bind(in: Seq[Expression], inputSchema: Seq[Attribute]): 
Seq[Expression] =
+in.map(BindReferences.bindReference(_, inputSchema))
+
+  protected def create(expressions: Seq[Expression]): UnsafeProjection = {
+val ctx = newCodeGenContext()
+val codes = expressions.map(_.gen(ctx))
--- End diff --

maybe even just `exprs`? I think `codes` just means something else rather 
than the plural for `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.
---

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



[GitHub] spark pull request: [SPARK-9022] [SQL] Generated projections for U...

2015-07-16 Thread davies
Github user davies commented on a diff in the pull request:

https://github.com/apache/spark/pull/7437#discussion_r34864633
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala
 ---
@@ -0,0 +1,113 @@
+/*
+ * 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.types.{BinaryType, StringType}
+
+
+/**
+ * Generates a [[Projection]] that returns an [[UnsafeRow]].
+ *
+ * Note: The returned UnsafeRow will be pointed to a scratch buffer inside 
the projection.
+ */
+object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], 
UnsafeProjection] {
+
+  protected def canonicalize(in: Seq[Expression]): Seq[Expression] =
+in.map(ExpressionCanonicalizer.execute)
+
+  protected def bind(in: Seq[Expression], inputSchema: Seq[Attribute]): 
Seq[Expression] =
+in.map(BindReferences.bindReference(_, inputSchema))
+
+  protected def create(expressions: Seq[Expression]): UnsafeProjection = {
+val ctx = newCodeGenContext()
+val codes = expressions.map(_.gen(ctx))
--- End diff --

With these long name, then many lines will be wrapped into two lines, make 
it hard to read. I'd prefer short name. How do you 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-9022] [SQL] Generated projections for U...

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

https://github.com/apache/spark/pull/7437#issuecomment-122188665
  
  [Test build #37591 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/37591/consoleFull)
 for   PR 7437 at commit 
[`dc737b2`](https://github.com/apache/spark/commit/dc737b20d47979184441890063ecc1ba336ac67b).


---
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-9022] [SQL] Generated projections for U...

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

https://github.com/apache/spark/pull/7437#issuecomment-122188112
  
 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-9022] [SQL] Generated projections for U...

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

https://github.com/apache/spark/pull/7437#issuecomment-122188198
  
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-9022] [SQL] Generated projections for U...

2015-07-16 Thread davies
Github user davies commented on a diff in the pull request:

https://github.com/apache/spark/pull/7437#discussion_r34864461
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverter.scala
 ---
@@ -261,6 +265,13 @@ private class StringUnsafeColumnWriter private() 
extends BytesUnsafeColumnWriter
   def getBytes(source: InternalRow, column: Int): Array[Byte] = {
 source.getAs[UTF8String](column).getBytes
   }
+  // TODO(davies): refactor this
+  // specialized for codegen
+  def getSize(value: UTF8String): Int =
--- End diff --

This a new overloaded function for `UTF8String`


---
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-9022] [SQL] Generated projections for U...

2015-07-16 Thread JoshRosen
Github user JoshRosen commented on the pull request:

https://github.com/apache/spark/pull/7437#issuecomment-122150311
  
This looks pretty good to me overall, modulo a few minor code clarity 
comments.


---
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-9022] [SQL] Generated projections for U...

2015-07-16 Thread JoshRosen
Github user JoshRosen commented on a diff in the pull request:

https://github.com/apache/spark/pull/7437#discussion_r34857207
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala
 ---
@@ -0,0 +1,113 @@
+/*
+ * 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.types.{BinaryType, StringType}
+
+
+/**
+ * Generates a [[Projection]] that returns an [[UnsafeRow]].
+ *
+ * Note: The returned UnsafeRow will be pointed to a scratch buffer inside 
the projection.
+ */
+object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], 
UnsafeProjection] {
+
+  protected def canonicalize(in: Seq[Expression]): Seq[Expression] =
+in.map(ExpressionCanonicalizer.execute)
+
+  protected def bind(in: Seq[Expression], inputSchema: Seq[Attribute]): 
Seq[Expression] =
+in.map(BindReferences.bindReference(_, inputSchema))
+
+  protected def create(expressions: Seq[Expression]): UnsafeProjection = {
+val ctx = newCodeGenContext()
+val codes = expressions.map(_.gen(ctx))
+val all_exprs = codes.map(_.code).mkString("\n")
+
+val additionalSize = expressions.zipWithIndex.map { case (e, i) =>
+  e.dataType match {
+case StringType =>
+  s" + (${codes(i).isNull} ? 0 : 
stringWriter.getSize(${codes(i).primitive}))"
+case BinaryType =>
+  s" + (${codes(i).isNull} ? 0 : 
binaryWriter.getSize(${codes(i).primitive}))"
+case _ => ""
+  }
+}.mkString("")
+
+val writers = expressions.zipWithIndex.map { case (e, i) =>
+  val update = e.dataType match {
+case dt if ctx.isPrimitiveType(dt) =>
+  s"${ctx.setColumn("target", dt, i, codes(i).primitive)}"
+case StringType =>
+  s"cursor += stringWriter.write(target, ${codes(i).primitive}, 
$i, cursor)"
+case BinaryType =>
+  s"cursor += binaryWriter.write(target, ${codes(i).primitive}, 
$i, cursor)"
+  }
+  s"""if (${codes(i).isNull}) {
+target.setNullAt($i);
+  } else {
+$update;
+  }"""
+}.mkString("\n  ")
+val fixedSize = (8 * expressions.length) + 
UnsafeRow.calculateBitSetWidthInBytes(expressions.length)
+
+val code = s"""
+public Object generate($exprType[] expr) {
+  return new SpecificProjection();
+}
+
+class SpecificProjection extends ${classOf[UnsafeProjection].getName} {
+
+  private final 
org.apache.spark.sql.catalyst.expressions.StringUnsafeColumnWriter
+stringWriter = new 
org.apache.spark.sql.catalyst.expressions.StringUnsafeColumnWriter();
+  private final 
org.apache.spark.sql.catalyst.expressions.BinaryUnsafeColumnWriter
+binaryWriter = new 
org.apache.spark.sql.catalyst.expressions.BinaryUnsafeColumnWriter();
+  private final 
org.apache.spark.sql.catalyst.expressions.ObjectUnsafeColumnWriter
+objectWriter = new 
org.apache.spark.sql.catalyst.expressions.ObjectUnsafeColumnWriter();
+
+  private UnsafeRow target = new UnsafeRow();
+  private byte[] buffer = new byte[64];
+
+  public SpecificProjection() {}
+
+  public Object apply(Object row) {
+return apply((InternalRow) i);
+  }
+
+  public UnsafeRow apply(InternalRow i) {
+${all_exprs}
+
+int numBytes = $fixedSize $additionalSize;
--- End diff --

Ah, I see that Reynold also got confused by this 
(https://github.com/apache/spark/pull/7437/files#r34856369), so we should 
probably take his suggestion to help clarify things 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 fe

[GitHub] spark pull request: [SPARK-9022] [SQL] Generated projections for U...

2015-07-16 Thread JoshRosen
Github user JoshRosen commented on a diff in the pull request:

https://github.com/apache/spark/pull/7437#discussion_r34857092
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala
 ---
@@ -0,0 +1,113 @@
+/*
+ * 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.types.{BinaryType, StringType}
+
+
+/**
+ * Generates a [[Projection]] that returns an [[UnsafeRow]].
+ *
+ * Note: The returned UnsafeRow will be pointed to a scratch buffer inside 
the projection.
+ */
+object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], 
UnsafeProjection] {
+
+  protected def canonicalize(in: Seq[Expression]): Seq[Expression] =
+in.map(ExpressionCanonicalizer.execute)
+
+  protected def bind(in: Seq[Expression], inputSchema: Seq[Attribute]): 
Seq[Expression] =
+in.map(BindReferences.bindReference(_, inputSchema))
+
+  protected def create(expressions: Seq[Expression]): UnsafeProjection = {
+val ctx = newCodeGenContext()
+val codes = expressions.map(_.gen(ctx))
+val all_exprs = codes.map(_.code).mkString("\n")
+
+val additionalSize = expressions.zipWithIndex.map { case (e, i) =>
+  e.dataType match {
+case StringType =>
+  s" + (${codes(i).isNull} ? 0 : 
stringWriter.getSize(${codes(i).primitive}))"
+case BinaryType =>
+  s" + (${codes(i).isNull} ? 0 : 
binaryWriter.getSize(${codes(i).primitive}))"
+case _ => ""
+  }
+}.mkString("")
+
+val writers = expressions.zipWithIndex.map { case (e, i) =>
+  val update = e.dataType match {
+case dt if ctx.isPrimitiveType(dt) =>
+  s"${ctx.setColumn("target", dt, i, codes(i).primitive)}"
+case StringType =>
+  s"cursor += stringWriter.write(target, ${codes(i).primitive}, 
$i, cursor)"
+case BinaryType =>
+  s"cursor += binaryWriter.write(target, ${codes(i).primitive}, 
$i, cursor)"
+  }
+  s"""if (${codes(i).isNull}) {
+target.setNullAt($i);
+  } else {
+$update;
+  }"""
+}.mkString("\n  ")
+val fixedSize = (8 * expressions.length) + 
UnsafeRow.calculateBitSetWidthInBytes(expressions.length)
+
+val code = s"""
+public Object generate($exprType[] expr) {
+  return new SpecificProjection();
+}
+
+class SpecificProjection extends ${classOf[UnsafeProjection].getName} {
+
+  private final 
org.apache.spark.sql.catalyst.expressions.StringUnsafeColumnWriter
+stringWriter = new 
org.apache.spark.sql.catalyst.expressions.StringUnsafeColumnWriter();
+  private final 
org.apache.spark.sql.catalyst.expressions.BinaryUnsafeColumnWriter
+binaryWriter = new 
org.apache.spark.sql.catalyst.expressions.BinaryUnsafeColumnWriter();
+  private final 
org.apache.spark.sql.catalyst.expressions.ObjectUnsafeColumnWriter
+objectWriter = new 
org.apache.spark.sql.catalyst.expressions.ObjectUnsafeColumnWriter();
+
+  private UnsafeRow target = new UnsafeRow();
+  private byte[] buffer = new byte[64];
+
+  public SpecificProjection() {}
+
+  public Object apply(Object row) {
+return apply((InternalRow) i);
+  }
+
+  public UnsafeRow apply(InternalRow i) {
+${all_exprs}
+
+int numBytes = $fixedSize $additionalSize;
--- End diff --

Ah, nevermind: I see that `$additionalSize` has a leading `+`.


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

[GitHub] spark pull request: [SPARK-9022] [SQL] Generated projections for U...

2015-07-16 Thread JoshRosen
Github user JoshRosen commented on a diff in the pull request:

https://github.com/apache/spark/pull/7437#discussion_r34857045
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala
 ---
@@ -0,0 +1,113 @@
+/*
+ * 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.types.{BinaryType, StringType}
+
+
+/**
+ * Generates a [[Projection]] that returns an [[UnsafeRow]].
+ *
+ * Note: The returned UnsafeRow will be pointed to a scratch buffer inside 
the projection.
+ */
+object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], 
UnsafeProjection] {
+
+  protected def canonicalize(in: Seq[Expression]): Seq[Expression] =
+in.map(ExpressionCanonicalizer.execute)
+
+  protected def bind(in: Seq[Expression], inputSchema: Seq[Attribute]): 
Seq[Expression] =
+in.map(BindReferences.bindReference(_, inputSchema))
+
+  protected def create(expressions: Seq[Expression]): UnsafeProjection = {
+val ctx = newCodeGenContext()
+val codes = expressions.map(_.gen(ctx))
+val all_exprs = codes.map(_.code).mkString("\n")
+
+val additionalSize = expressions.zipWithIndex.map { case (e, i) =>
+  e.dataType match {
+case StringType =>
+  s" + (${codes(i).isNull} ? 0 : 
stringWriter.getSize(${codes(i).primitive}))"
+case BinaryType =>
+  s" + (${codes(i).isNull} ? 0 : 
binaryWriter.getSize(${codes(i).primitive}))"
+case _ => ""
+  }
+}.mkString("")
+
+val writers = expressions.zipWithIndex.map { case (e, i) =>
+  val update = e.dataType match {
+case dt if ctx.isPrimitiveType(dt) =>
+  s"${ctx.setColumn("target", dt, i, codes(i).primitive)}"
+case StringType =>
+  s"cursor += stringWriter.write(target, ${codes(i).primitive}, 
$i, cursor)"
+case BinaryType =>
+  s"cursor += binaryWriter.write(target, ${codes(i).primitive}, 
$i, cursor)"
+  }
+  s"""if (${codes(i).isNull}) {
+target.setNullAt($i);
+  } else {
+$update;
+  }"""
+}.mkString("\n  ")
+val fixedSize = (8 * expressions.length) + 
UnsafeRow.calculateBitSetWidthInBytes(expressions.length)
+
+val code = s"""
+public Object generate($exprType[] expr) {
+  return new SpecificProjection();
+}
+
+class SpecificProjection extends ${classOf[UnsafeProjection].getName} {
+
+  private final 
org.apache.spark.sql.catalyst.expressions.StringUnsafeColumnWriter
+stringWriter = new 
org.apache.spark.sql.catalyst.expressions.StringUnsafeColumnWriter();
+  private final 
org.apache.spark.sql.catalyst.expressions.BinaryUnsafeColumnWriter
+binaryWriter = new 
org.apache.spark.sql.catalyst.expressions.BinaryUnsafeColumnWriter();
+  private final 
org.apache.spark.sql.catalyst.expressions.ObjectUnsafeColumnWriter
+objectWriter = new 
org.apache.spark.sql.catalyst.expressions.ObjectUnsafeColumnWriter();
+
+  private UnsafeRow target = new UnsafeRow();
+  private byte[] buffer = new byte[64];
+
+  public SpecificProjection() {}
+
+  public Object apply(Object row) {
+return apply((InternalRow) i);
+  }
+
+  public UnsafeRow apply(InternalRow i) {
+${all_exprs}
+
+int numBytes = $fixedSize $additionalSize;
--- End diff --

Should this be `$fixedSize + $additionalSize;`?


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

[GitHub] spark pull request: [SPARK-9022] [SQL] Generated projections for U...

2015-07-16 Thread JoshRosen
Github user JoshRosen commented on a diff in the pull request:

https://github.com/apache/spark/pull/7437#discussion_r34856976
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala
 ---
@@ -0,0 +1,113 @@
+/*
+ * 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.types.{BinaryType, StringType}
+
+
+/**
+ * Generates a [[Projection]] that returns an [[UnsafeRow]].
+ *
+ * Note: The returned UnsafeRow will be pointed to a scratch buffer inside 
the projection.
+ */
+object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], 
UnsafeProjection] {
+
+  protected def canonicalize(in: Seq[Expression]): Seq[Expression] =
+in.map(ExpressionCanonicalizer.execute)
+
+  protected def bind(in: Seq[Expression], inputSchema: Seq[Attribute]): 
Seq[Expression] =
+in.map(BindReferences.bindReference(_, inputSchema))
+
+  protected def create(expressions: Seq[Expression]): UnsafeProjection = {
+val ctx = newCodeGenContext()
+val codes = expressions.map(_.gen(ctx))
+val all_exprs = codes.map(_.code).mkString("\n")
+
+val additionalSize = expressions.zipWithIndex.map { case (e, i) =>
+  e.dataType match {
+case StringType =>
+  s" + (${codes(i).isNull} ? 0 : 
stringWriter.getSize(${codes(i).primitive}))"
+case BinaryType =>
+  s" + (${codes(i).isNull} ? 0 : 
binaryWriter.getSize(${codes(i).primitive}))"
+case _ => ""
+  }
+}.mkString("")
+
+val writers = expressions.zipWithIndex.map { case (e, i) =>
+  val update = e.dataType match {
+case dt if ctx.isPrimitiveType(dt) =>
+  s"${ctx.setColumn("target", dt, i, codes(i).primitive)}"
+case StringType =>
+  s"cursor += stringWriter.write(target, ${codes(i).primitive}, 
$i, cursor)"
+case BinaryType =>
+  s"cursor += binaryWriter.write(target, ${codes(i).primitive}, 
$i, cursor)"
+  }
+  s"""if (${codes(i).isNull}) {
+target.setNullAt($i);
+  } else {
+$update;
+  }"""
+}.mkString("\n  ")
+val fixedSize = (8 * expressions.length) + 
UnsafeRow.calculateBitSetWidthInBytes(expressions.length)
+
+val code = s"""
+public Object generate($exprType[] expr) {
+  return new SpecificProjection();
+}
+
+class SpecificProjection extends ${classOf[UnsafeProjection].getName} {
+
+  private final 
org.apache.spark.sql.catalyst.expressions.StringUnsafeColumnWriter
+stringWriter = new 
org.apache.spark.sql.catalyst.expressions.StringUnsafeColumnWriter();
+  private final 
org.apache.spark.sql.catalyst.expressions.BinaryUnsafeColumnWriter
+binaryWriter = new 
org.apache.spark.sql.catalyst.expressions.BinaryUnsafeColumnWriter();
+  private final 
org.apache.spark.sql.catalyst.expressions.ObjectUnsafeColumnWriter
+objectWriter = new 
org.apache.spark.sql.catalyst.expressions.ObjectUnsafeColumnWriter();
+
+  private UnsafeRow target = new UnsafeRow();
+  private byte[] buffer = new byte[64];
+
+  public SpecificProjection() {}
+
+  public Object apply(Object row) {
--- End diff --

Why do we need this `apply` method?


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

[GitHub] spark pull request: [SPARK-9022] [SQL] Generated projections for U...

2015-07-16 Thread JoshRosen
Github user JoshRosen commented on a diff in the pull request:

https://github.com/apache/spark/pull/7437#discussion_r34856887
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverter.scala
 ---
@@ -261,6 +265,13 @@ private class StringUnsafeColumnWriter private() 
extends BytesUnsafeColumnWriter
   def getBytes(source: InternalRow, column: Int): Array[Byte] = {
 source.getAs[UTF8String](column).getBytes
   }
+  // TODO(davies): refactor this
+  // specialized for codegen
+  def getSize(value: UTF8String): Int =
+ByteArrayMethods.roundNumberOfBytesToNearestWord(value.numBytes())
+  def write(target: UnsafeRow, value: UTF8String, column: Int, cursor: 
Int): Int = {
+write(target, value.getBytes, column, cursor)
+  }
 }
 
 private class BinaryUnsafeColumnWriter private() extends 
BytesUnsafeColumnWriter {
--- End diff --

I've noticed that we have both objects and classes for each of these 
ColumnWriters, but forget why I needed to do this.  I tried it out in IntelliJ 
and I think that we can just convert these classes into `object`s and drop the 
`private object` lines near line 154.


---
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-9022] [SQL] Generated projections for U...

2015-07-16 Thread JoshRosen
Github user JoshRosen commented on a diff in the pull request:

https://github.com/apache/spark/pull/7437#discussion_r34856687
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala
 ---
@@ -0,0 +1,113 @@
+/*
+ * 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.types.{BinaryType, StringType}
+
+
+/**
+ * Generates a [[Projection]] that returns an [[UnsafeRow]].
+ *
+ * Note: The returned UnsafeRow will be pointed to a scratch buffer inside 
the projection.
+ */
+object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], 
UnsafeProjection] {
+
+  protected def canonicalize(in: Seq[Expression]): Seq[Expression] =
+in.map(ExpressionCanonicalizer.execute)
+
+  protected def bind(in: Seq[Expression], inputSchema: Seq[Attribute]): 
Seq[Expression] =
+in.map(BindReferences.bindReference(_, inputSchema))
+
+  protected def create(expressions: Seq[Expression]): UnsafeProjection = {
+val ctx = newCodeGenContext()
+val codes = expressions.map(_.gen(ctx))
--- End diff --

`expressionCode`?


---
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-9022] [SQL] Generated projections for U...

2015-07-16 Thread JoshRosen
Github user JoshRosen commented on a diff in the pull request:

https://github.com/apache/spark/pull/7437#discussion_r34856669
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala
 ---
@@ -0,0 +1,113 @@
+/*
+ * 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.types.{BinaryType, StringType}
+
+
+/**
+ * Generates a [[Projection]] that returns an [[UnsafeRow]].
+ *
+ * Note: The returned UnsafeRow will be pointed to a scratch buffer inside 
the projection.
+ */
+object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], 
UnsafeProjection] {
+
+  protected def canonicalize(in: Seq[Expression]): Seq[Expression] =
+in.map(ExpressionCanonicalizer.execute)
+
+  protected def bind(in: Seq[Expression], inputSchema: Seq[Attribute]): 
Seq[Expression] =
+in.map(BindReferences.bindReference(_, inputSchema))
+
+  protected def create(expressions: Seq[Expression]): UnsafeProjection = {
+val ctx = newCodeGenContext()
+val codes = expressions.map(_.gen(ctx))
+val all_exprs = codes.map(_.code).mkString("\n")
+
+val additionalSize = expressions.zipWithIndex.map { case (e, i) =>
+  e.dataType match {
+case StringType =>
+  s" + (${codes(i).isNull} ? 0 : 
stringWriter.getSize(${codes(i).primitive}))"
+case BinaryType =>
+  s" + (${codes(i).isNull} ? 0 : 
binaryWriter.getSize(${codes(i).primitive}))"
+case _ => ""
+  }
+}.mkString("")
+
+val writers = expressions.zipWithIndex.map { case (e, i) =>
+  val update = e.dataType match {
+case dt if ctx.isPrimitiveType(dt) =>
+  s"${ctx.setColumn("target", dt, i, codes(i).primitive)}"
+case StringType =>
+  s"cursor += stringWriter.write(target, ${codes(i).primitive}, 
$i, cursor)"
+case BinaryType =>
+  s"cursor += binaryWriter.write(target, ${codes(i).primitive}, 
$i, cursor)"
+  }
+  s"""if (${codes(i).isNull}) {
+target.setNullAt($i);
+  } else {
+$update;
+  }"""
+}.mkString("\n  ")
+val fixedSize = (8 * expressions.length) + 
UnsafeRow.calculateBitSetWidthInBytes(expressions.length)
+
+val code = s"""
+public Object generate($exprType[] expr) {
+  return new SpecificProjection();
+}
+
+class SpecificProjection extends ${classOf[UnsafeProjection].getName} {
+
+  private final 
org.apache.spark.sql.catalyst.expressions.StringUnsafeColumnWriter
+stringWriter = new 
org.apache.spark.sql.catalyst.expressions.StringUnsafeColumnWriter();
--- End diff --

Aren't there singletons that you can use instead?


---
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-9022] [SQL] Generated projections for U...

2015-07-16 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/7437#discussion_r34856661
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala
 ---
@@ -0,0 +1,113 @@
+/*
+ * 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.types.{BinaryType, StringType}
+
+
+/**
+ * Generates a [[Projection]] that returns an [[UnsafeRow]].
+ *
+ * Note: The returned UnsafeRow will be pointed to a scratch buffer inside 
the projection.
+ */
+object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], 
UnsafeProjection] {
+
+  protected def canonicalize(in: Seq[Expression]): Seq[Expression] =
+in.map(ExpressionCanonicalizer.execute)
+
+  protected def bind(in: Seq[Expression], inputSchema: Seq[Attribute]): 
Seq[Expression] =
+in.map(BindReferences.bindReference(_, inputSchema))
+
+  protected def create(expressions: Seq[Expression]): UnsafeProjection = {
+val ctx = newCodeGenContext()
+val codes = expressions.map(_.gen(ctx))
--- End diff --

maybe expressionJavaCode?


---
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-9022] [SQL] Generated projections for U...

2015-07-16 Thread rxin
Github user rxin commented on the pull request:

https://github.com/apache/spark/pull/7437#issuecomment-122147230
  
LGTM. Would be great to improve documentation a bit so people that don't 
have a lot of background can also pick up the 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.
---

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



[GitHub] spark pull request: [SPARK-9022] [SQL] Generated projections for U...

2015-07-16 Thread JoshRosen
Github user JoshRosen commented on a diff in the pull request:

https://github.com/apache/spark/pull/7437#discussion_r34856595
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala
 ---
@@ -0,0 +1,113 @@
+/*
+ * 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.types.{BinaryType, StringType}
+
+
+/**
+ * Generates a [[Projection]] that returns an [[UnsafeRow]].
+ *
+ * Note: The returned UnsafeRow will be pointed to a scratch buffer inside 
the projection.
+ */
+object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], 
UnsafeProjection] {
+
+  protected def canonicalize(in: Seq[Expression]): Seq[Expression] =
+in.map(ExpressionCanonicalizer.execute)
+
+  protected def bind(in: Seq[Expression], inputSchema: Seq[Attribute]): 
Seq[Expression] =
+in.map(BindReferences.bindReference(_, inputSchema))
+
+  protected def create(expressions: Seq[Expression]): UnsafeProjection = {
+val ctx = newCodeGenContext()
+val codes = expressions.map(_.gen(ctx))
+val all_exprs = codes.map(_.code).mkString("\n")
--- End diff --

Should this be a Java-style variable name instead?


---
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-9022] [SQL] Generated projections for U...

2015-07-16 Thread JoshRosen
Github user JoshRosen commented on a diff in the pull request:

https://github.com/apache/spark/pull/7437#discussion_r34856606
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala
 ---
@@ -0,0 +1,113 @@
+/*
+ * 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.types.{BinaryType, StringType}
+
+
+/**
+ * Generates a [[Projection]] that returns an [[UnsafeRow]].
+ *
+ * Note: The returned UnsafeRow will be pointed to a scratch buffer inside 
the projection.
+ */
+object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], 
UnsafeProjection] {
+
+  protected def canonicalize(in: Seq[Expression]): Seq[Expression] =
+in.map(ExpressionCanonicalizer.execute)
+
+  protected def bind(in: Seq[Expression], inputSchema: Seq[Attribute]): 
Seq[Expression] =
+in.map(BindReferences.bindReference(_, inputSchema))
+
+  protected def create(expressions: Seq[Expression]): UnsafeProjection = {
+val ctx = newCodeGenContext()
+val codes = expressions.map(_.gen(ctx))
--- End diff --

This is a confusing variable name.


---
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-9022] [SQL] Generated projections for U...

2015-07-16 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/7437#discussion_r34856568
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala
 ---
@@ -0,0 +1,113 @@
+/*
+ * 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.types.{BinaryType, StringType}
+
+
+/**
+ * Generates a [[Projection]] that returns an [[UnsafeRow]].
+ *
+ * Note: The returned UnsafeRow will be pointed to a scratch buffer inside 
the projection.
--- End diff --

one thing that would be great to add is the high level comment on how this 
works. Due to a mix of codegen and scala code, it is definitely non-obvious for 
somebody that doesn't understand the background.

Saying something that we generate the code to compute the length, and then 
copy the data into a scratch buffer space in the form of UnsafeRow.


---
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-9022] [SQL] Generated projections for U...

2015-07-16 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/7437#discussion_r34856452
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala
 ---
@@ -0,0 +1,113 @@
+/*
+ * 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.types.{BinaryType, StringType}
+
+
+/**
+ * Generates a [[Projection]] that returns an [[UnsafeRow]].
+ *
+ * Note: The returned UnsafeRow will be pointed to a scratch buffer inside 
the projection.
+ */
+object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], 
UnsafeProjection] {
+
+  protected def canonicalize(in: Seq[Expression]): Seq[Expression] =
+in.map(ExpressionCanonicalizer.execute)
+
+  protected def bind(in: Seq[Expression], inputSchema: Seq[Attribute]): 
Seq[Expression] =
+in.map(BindReferences.bindReference(_, inputSchema))
+
+  protected def create(expressions: Seq[Expression]): UnsafeProjection = {
+val ctx = newCodeGenContext()
+val codes = expressions.map(_.gen(ctx))
+val all_exprs = codes.map(_.code).mkString("\n")
+
+val additionalSize = expressions.zipWithIndex.map { case (e, i) =>
+  e.dataType match {
+case StringType =>
+  s" + (${codes(i).isNull} ? 0 : 
stringWriter.getSize(${codes(i).primitive}))"
--- End diff --

ah actually rename additionalSize -> variableSize


---
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-9022] [SQL] Generated projections for U...

2015-07-16 Thread JoshRosen
Github user JoshRosen commented on a diff in the pull request:

https://github.com/apache/spark/pull/7437#discussion_r34856464
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverter.scala
 ---
@@ -261,6 +265,13 @@ private class StringUnsafeColumnWriter private() 
extends BytesUnsafeColumnWriter
   def getBytes(source: InternalRow, column: Int): Array[Byte] = {
 source.getAs[UTF8String](column).getBytes
   }
+  // TODO(davies): refactor this
+  // specialized for codegen
+  def getSize(value: UTF8String): Int =
--- End diff --

`override`?


---
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-9022] [SQL] Generated projections for U...

2015-07-16 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/7437#discussion_r34856369
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala
 ---
@@ -0,0 +1,113 @@
+/*
+ * 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.types.{BinaryType, StringType}
+
+
+/**
+ * Generates a [[Projection]] that returns an [[UnsafeRow]].
+ *
+ * Note: The returned UnsafeRow will be pointed to a scratch buffer inside 
the projection.
+ */
+object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], 
UnsafeProjection] {
+
+  protected def canonicalize(in: Seq[Expression]): Seq[Expression] =
+in.map(ExpressionCanonicalizer.execute)
+
+  protected def bind(in: Seq[Expression], inputSchema: Seq[Attribute]): 
Seq[Expression] =
+in.map(BindReferences.bindReference(_, inputSchema))
+
+  protected def create(expressions: Seq[Expression]): UnsafeProjection = {
+val ctx = newCodeGenContext()
+val codes = expressions.map(_.gen(ctx))
+val all_exprs = codes.map(_.code).mkString("\n")
+
+val additionalSize = expressions.zipWithIndex.map { case (e, i) =>
+  e.dataType match {
+case StringType =>
+  s" + (${codes(i).isNull} ? 0 : 
stringWriter.getSize(${codes(i).primitive}))"
--- End diff --

I think it'd be more clear to leave out the +, and add a 0 in the else 
case. then 
```scala
$fixedSize $additionalSize;
```
can become
```scala
$fixedSize + $additionalSize;
```

When I was looking at that I got slightly confused by what those two 
variables are


---
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-9022] [SQL] Generated projections for U...

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

https://github.com/apache/spark/pull/7437#issuecomment-122031320
  
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-9022] [SQL] Generated projections for U...

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

https://github.com/apache/spark/pull/7437#issuecomment-122031100
  
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-9022] [SQL] Generated projections for U...

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

https://github.com/apache/spark/pull/7437#issuecomment-122031079
  
 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-9022] [SQL] Generated projections for U...

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

https://github.com/apache/spark/pull/7437#issuecomment-121825299
  
  [Test build #1078 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/1078/console)
 for   PR 7437 at commit 
[`729138d`](https://github.com/apache/spark/commit/729138d6c91db87ebcfd7b969e7d4bd0feb77a14).
 * This patch **fails Scala style tests**.
 * This patch merges cleanly.
 * This patch adds the following public classes _(experimental)_:
  * `abstract class StandaloneRecoveryModeFactory(conf: SparkConf, 
serializer: Serializer) `
  * `class RFormula(override val uid: String)`
  * `abstract class LeafExpression extends Expression `
  * `abstract class UnaryExpression extends Expression `
  * `abstract class BinaryExpression extends Expression `
  * `case class FromUnsafeProjection(fields: Seq[DataType]) extends 
Projection `
  * `abstract class BaseProjection extends Projection `
  * `class SpecificProjection extends $`
  * `class SpecificProjection extends $`
  * `abstract class BinaryNode extends LogicalPlan `



---
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-9022] [SQL] Generated projections for U...

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

https://github.com/apache/spark/pull/7437#issuecomment-121825136
  
  [Test build #1078 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/1078/consoleFull)
 for   PR 7437 at commit 
[`729138d`](https://github.com/apache/spark/commit/729138d6c91db87ebcfd7b969e7d4bd0feb77a14).


---
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-9022] [SQL] Generated projections for U...

2015-07-15 Thread rxin
Github user rxin commented on the pull request:

https://github.com/apache/spark/pull/7437#issuecomment-121825079
  
@davies doesn't need to be part of this pr, but can you think about how we 
can do codegen testing with this new Unsafe project?


---
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-9022] [SQL] Generated projections for U...

2015-07-15 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/7437#discussion_r34754494
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala
 ---
@@ -0,0 +1,113 @@
+/*
+ * 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.types.{BinaryType, DataType, StringType, 
StructType}
+
+
+/**
+ * Generates byte code that convert a InternalRow to an [[UnsafeRow]].
+ */
+object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], 
Projection] {
+
+  def generate(schema: StructType): Projection = 
generate(schema.fields.map(_.dataType))
+
+  def generate(fields: Seq[DataType]): Projection =
--- End diff --

Better to give it a different name -- since generate is very general.



---
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-9022] [SQL] Generated projections for U...

2015-07-15 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/7437#discussion_r34754466
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala
 ---
@@ -0,0 +1,113 @@
+/*
+ * 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.types.{BinaryType, DataType, StringType, 
StructType}
+
+
+/**
+ * Generates byte code that convert a InternalRow to an [[UnsafeRow]].
+ */
+object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], 
Projection] {
+
+  def generate(schema: StructType): Projection = 
generate(schema.fields.map(_.dataType))
--- End diff --

this is not used at all, is 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-9022] [SQL] Generated projections for U...

2015-07-15 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/7437#discussion_r34754457
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala
 ---
@@ -0,0 +1,113 @@
+/*
+ * 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.types.{BinaryType, DataType, StringType, 
StructType}
+
+
+/**
+ * Generates byte code that convert a InternalRow to an [[UnsafeRow]].
--- End diff --

update the documentation 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 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-9022] [SQL] Generated projections for U...

2015-07-15 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/7437#discussion_r34754330
  
--- Diff: 
sql/catalyst/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java
 ---
@@ -19,11 +19,11 @@
 
 import java.io.IOException;
 
+import com.google.common.annotations.VisibleForTesting;
--- End diff --

import order


---
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-9022] [SQL] Generated projections for U...

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

https://github.com/apache/spark/pull/7437#issuecomment-121817885
  
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-9022] [SQL] Generated projections for U...

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

https://github.com/apache/spark/pull/7437#issuecomment-121817758
  
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-9022] [SQL] Generated projections for U...

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

https://github.com/apache/spark/pull/7437#issuecomment-121817749
  
 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-9022] [SQL] Generated projections for U...

2015-07-15 Thread davies
GitHub user davies opened a pull request:

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

[SPARK-9022] [SQL] Generated projections for UnsafeRow

Added two projections: GenerateUnsafeProjection and FromUnsafeProjection, 
which could be used to convert UnsafeRow from/to GenericInternalRow.

They will re-use the buffer during projection, similar to MutableProjection 
(without all the interface MutableProjection has).

cc @rxin @JoshRosen

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

$ git pull https://github.com/davies/spark unsafe_proj2

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

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


commit 5a2637347a2f96d55a17b4c866bccfc40b654ffc
Author: Davies Liu 
Date:   2015-07-16T03:30:19Z

unsafe projections




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