[GitHub] spark pull request #13704: [SPARK-15985][SQL] Reduce runtime overhead of a p...

2016-07-11 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/13704#discussion_r70381491
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyCastsSuite.scala
 ---
@@ -0,0 +1,78 @@
+/*
+ * 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.optimizer
+
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.dsl.plans._
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.PlanTest
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.RuleExecutor
+import org.apache.spark.sql.types._
+
+class SimplifyCastsSuite extends PlanTest {
+
+  object Optimize extends RuleExecutor[LogicalPlan] {
+val batches = Batch("SimplifyCasts", FixedPoint(50), SimplifyCasts) :: 
Nil
+  }
+
+  test("non-nullable to non-nullable array cast") {
+val input = LocalRelation('a.array(ArrayType(IntegerType, false)))
+val array_intPrimitive = Literal.create(
+  Seq(1, 2, 3, 4, 5), ArrayType(IntegerType, false))
+val plan = input.select(array_intPrimitive
--- End diff --

ah, i see. in `'a.array(dt)`, the `dt` is element type, so you are creating 
an array of array. However, the `array` method doesn't take `nullable`, maybe 
we should fix 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 #13704: [SPARK-15985][SQL] Reduce runtime overhead of a p...

2016-07-11 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/13704#discussion_r70368410
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyCastsSuite.scala
 ---
@@ -0,0 +1,78 @@
+/*
+ * 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.optimizer
+
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.dsl.plans._
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.PlanTest
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.RuleExecutor
+import org.apache.spark.sql.types._
+
+class SimplifyCastsSuite extends PlanTest {
+
+  object Optimize extends RuleExecutor[LogicalPlan] {
+val batches = Batch("SimplifyCasts", FixedPoint(50), SimplifyCasts) :: 
Nil
+  }
+
+  test("non-nullable to non-nullable array cast") {
+val input = LocalRelation('a.array(ArrayType(IntegerType, false)))
+val array_intPrimitive = Literal.create(
+  Seq(1, 2, 3, 4, 5), ArrayType(IntegerType, false))
+val plan = input.select(array_intPrimitive
--- End diff --

@cloud-fan This statement cannot be optimized, probably, due to of missing 
information on `a`. As a result, it causes assertion an error. Did I make some 
mistakes?
```
  test("non-nullable to non-nullable array cast") {
val input = LocalRelation('a.array(ArrayType(IntegerType, false)))
val plan = input.select(
  'a.cast(ArrayType(IntegerType, false)).as('casted)).analyze
val optimized = Optimize.execute(plan)
val expected = input.select('a.as('casted)).analyze
print(s"optimized: $plan")
print(s"optimized: $optimized")
comparePlans(optimized, expected)
  }

optimized: 'Project [cast(a#0 as array) AS casted#1]
+- LocalRelation , [a#0]
optimized: 'Project [cast(a#0 as array) AS casted#1]
+- LocalRelation , [a#0]


== FAIL: Plans do not match ===
!'Project [cast(a#0 as array) AS casted#0]   Project [a#0 AS casted#0]
 +- LocalRelation , [a#0]  +- LocalRelation , 
[a#0]
```

The original one
```
  test("non-nullable to non-nullable array cast") {
val input = LocalRelation('a.array(ArrayType(IntegerType, false)))
val array_intPrimitive = Literal.create(
  Seq(1, 2, 3, 4, 5), ArrayType(IntegerType, false))
val plan = input.select(array_intPrimitive
  .cast(ArrayType(IntegerType, false)).as('a)).analyze
val optimized = Optimize.execute(plan)
val expected = input.select(array_intPrimitive.as('a)).analyze
print(s"optimized: $plan")
print(s"optimized: $optimized")
comparePlans(optimized, expected)
  }

optimized: Project [cast([1,2,3,4,5] as array) AS a#1]
+- LocalRelation , [a#0]
optimized: Project [[1,2,3,4,5] AS a#1]
+- LocalRelation , [a#0]
```


---
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 #13704: [SPARK-15985][SQL] Reduce runtime overhead of a p...

2016-07-11 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/13704#discussion_r70367470
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/PrimitiveArrayBenchmark.scala
 ---
@@ -0,0 +1,84 @@
+/*
+ * 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.benchmark
+
+import java.util.{Arrays, Comparator, Random}
+
+import org.apache.spark.sql.SQLContext
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.unsafe.array.LongArray
+import org.apache.spark.unsafe.memory.MemoryBlock
+import org.apache.spark.util.Benchmark
+import org.apache.spark.util.collection.Sorter
+import org.apache.spark.util.collection.unsafe.sort._
+
+/**
+ * Benchmark to measure performance for accessing primitive arrays
+ * To run this:
+ *  1. Replace ignore(...) with test(...)
+ *  2. build/sbt "sql/test-only *benchmark.PrimitiveArrayBenchmark"
+ *
+ * Benchmarks in this file are skipped in normal builds.
+ */
+class PrimitiveArrayBenchmark extends BenchmarkBase {
--- End diff --

Got it. In the future, it would be good to prepare criteria to require 
benchmark results in Wiki if anyone create a PR for optimizers.


---
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 #13704: [SPARK-15985][SQL] Reduce runtime overhead of a p...

2016-07-11 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/13704#discussion_r70364606
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyCastsSuite.scala
 ---
@@ -0,0 +1,78 @@
+/*
+ * 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.optimizer
+
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.dsl.plans._
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.PlanTest
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.RuleExecutor
+import org.apache.spark.sql.types._
+
+class SimplifyCastsSuite extends PlanTest {
+
+  object Optimize extends RuleExecutor[LogicalPlan] {
+val batches = Batch("SimplifyCasts", FixedPoint(50), SimplifyCasts) :: 
Nil
+  }
+
+  test("non-nullable to non-nullable array cast") {
+val input = LocalRelation('a.array(ArrayType(IntegerType, false)))
+val array_intPrimitive = Literal.create(
+  Seq(1, 2, 3, 4, 5), ArrayType(IntegerType, false))
+val plan = input.select(array_intPrimitive
+  .cast(ArrayType(IntegerType, false)).as('a)).analyze
+val optimized = Optimize.execute(plan)
+val expected = input.select(array_intPrimitive.as('a)).analyze
+comparePlans(optimized, expected)
+  }
+
+  test("non-nullable to nullable array cast") {
--- End diff --

we should also test `nullable to non-nullable`, to make sure we don't 
optimize 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 #13704: [SPARK-15985][SQL] Reduce runtime overhead of a p...

2016-07-11 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/13704#discussion_r70364539
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/PrimitiveArrayBenchmark.scala
 ---
@@ -0,0 +1,84 @@
+/*
+ * 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.benchmark
+
+import java.util.{Arrays, Comparator, Random}
+
+import org.apache.spark.sql.SQLContext
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.unsafe.array.LongArray
+import org.apache.spark.unsafe.memory.MemoryBlock
+import org.apache.spark.util.Benchmark
+import org.apache.spark.util.collection.Sorter
+import org.apache.spark.util.collection.unsafe.sort._
+
+/**
+ * Benchmark to measure performance for accessing primitive arrays
+ * To run this:
+ *  1. Replace ignore(...) with test(...)
+ *  2. build/sbt "sql/test-only *benchmark.PrimitiveArrayBenchmark"
+ *
+ * Benchmarks in this file are skipped in normal builds.
+ */
+class PrimitiveArrayBenchmark extends BenchmarkBase {
--- End diff --

I'm still a little against creating benchmark for this case. It will be 
really messy if anyone who send a patch for optimizer, also create a benchmark 
for 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 #13704: [SPARK-15985][SQL] Reduce runtime overhead of a p...

2016-07-11 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/13704#discussion_r70364232
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyCastsSuite.scala
 ---
@@ -0,0 +1,78 @@
+/*
+ * 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.optimizer
+
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.dsl.plans._
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.PlanTest
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.RuleExecutor
+import org.apache.spark.sql.types._
+
+class SimplifyCastsSuite extends PlanTest {
+
+  object Optimize extends RuleExecutor[LogicalPlan] {
+val batches = Batch("SimplifyCasts", FixedPoint(50), SimplifyCasts) :: 
Nil
+  }
+
+  test("non-nullable to non-nullable array cast") {
+val input = LocalRelation('a.array(ArrayType(IntegerType, false)))
+val array_intPrimitive = Literal.create(
+  Seq(1, 2, 3, 4, 5), ArrayType(IntegerType, false))
+val plan = input.select(array_intPrimitive
--- End diff --

`input.select('a.cast(ArrayType(IntegerType, false)).as('casted))`, then we 
don't need to create the literal


---
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 #13704: [SPARK-15985][SQL] Reduce runtime overhead of a p...

2016-07-11 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/13704#discussion_r70234587
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 ---
@@ -1441,6 +1441,26 @@ object PushPredicateThroughJoin extends 
Rule[LogicalPlan] with PredicateHelper {
 object SimplifyCasts extends Rule[LogicalPlan] {
   def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions 
{
 case Cast(e, dataType) if e.dataType == dataType => e
+case Cast(e, dataType) =>
+  (e.dataType, dataType) match {
+case (fromDt: ArrayType, toDt: ArrayType) =>
--- End diff --

thanks, I like this simple one


---
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 #13704: [SPARK-15985][SQL] Reduce runtime overhead of a p...

2016-07-11 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/13704#discussion_r70231367
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 ---
@@ -1441,6 +1441,26 @@ object PushPredicateThroughJoin extends 
Rule[LogicalPlan] with PredicateHelper {
 object SimplifyCasts extends Rule[LogicalPlan] {
   def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions 
{
 case Cast(e, dataType) if e.dataType == dataType => e
+case Cast(e, dataType) =>
+  (e.dataType, dataType) match {
+case (fromDt: ArrayType, toDt: ArrayType) =>
--- End diff --

how about
```
case c @ Cast(e, dataType) => (e.dataType, dataType) match {
  case (ArrayType(from, false), ArrayType(to, true)) if from == to => e
  case (MapType(fromKey, fromValue, false), ArrayType(toKey, toValue, 
true)) if fromKey == toKey && fromValue == toValue => e
  case _ => c
}
```


---
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 #13704: [SPARK-15985][SQL] Reduce runtime overhead of a p...

2016-07-11 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/13704#discussion_r70230473
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -2018,6 +2018,8 @@ class Analyzer(
 fail(child, DateType, walkedTypePath)
   case (StringType, to: NumericType) =>
 fail(child, to, walkedTypePath)
+  case (from: ArrayType, to: ArrayType) if !from.containsNull =>
--- End diff --

Got it. I will add unit tests later. They require a combination of `Cast` 
and `SimplifyCasts`.


---
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 #13704: [SPARK-15985][SQL] Reduce runtime overhead of a p...

2016-07-10 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/13704#discussion_r70202042
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -2018,6 +2018,8 @@ class Analyzer(
 fail(child, DateType, walkedTypePath)
   case (StringType, to: NumericType) =>
 fail(child, to, walkedTypePath)
+  case (from: ArrayType, to: ArrayType) if !from.containsNull =>
--- End diff --

I mean MapType. It's similar to ArrayType, the value of it can be nullable 
or non-nullable.


---
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 #13704: [SPARK-15985][SQL] Reduce runtime overhead of a p...

2016-07-10 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/13704#discussion_r70199583
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -2018,6 +2018,8 @@ class Analyzer(
 fail(child, DateType, walkedTypePath)
   case (StringType, to: NumericType) =>
 fail(child, to, walkedTypePath)
+  case (from: ArrayType, to: ArrayType) if !from.containsNull =>
--- End diff --

I will try improving the `SimplifyCasts` rule to force to eliminate the 
cast from non-element-nullable array to nullable ones. I do not understand the 
following. Will it be automatically done by improving the `SimplifyCasts` or do 
we need to improve another rule?
> "we can handle map too"

I will add unit tests for it. I think that it is good to add a benchmark to 
show degree of improvements.


---
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 #13704: [SPARK-15985][SQL] Reduce runtime overhead of a p...

2016-07-10 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/13704#discussion_r70194231
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -2018,6 +2018,8 @@ class Analyzer(
 fail(child, DateType, walkedTypePath)
   case (StringType, to: NumericType) =>
 fail(child, to, walkedTypePath)
+  case (from: ArrayType, to: ArrayType) if !from.containsNull =>
--- End diff --

How about we improve the `SimplifyCasts` rule, to eliminate the cast from 
non-element-nullable array to nullable ones? And we can handle map too.

Then we can add unit tests for it, instead of adding a benchmark. I think 
benchmark is unnecessary for this case, removing a `Cast` is definitely faster.


---
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 #13704: [SPARK-15985][SQL] Reduce runtime overhead of a p...

2016-07-10 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/13704#discussion_r70193764
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
 ---
@@ -479,7 +479,7 @@ case class Cast(child: Expression, dataType: DataType) 
extends UnaryExpression w
 case DoubleType => castToDoubleCode(from)
 
 case array: ArrayType =>
-  castArrayCode(from.asInstanceOf[ArrayType].elementType, 
array.elementType, ctx)
+  castArrayCode(from.asInstanceOf[ArrayType], array, ctx)
--- End diff --

Sure, I will create another PR and update the description.


---
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 #13704: [SPARK-15985][SQL] Reduce runtime overhead of a p...

2016-07-10 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/13704#discussion_r70193070
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
 ---
@@ -479,7 +479,7 @@ case class Cast(child: Expression, dataType: DataType) 
extends UnaryExpression w
 case DoubleType => castToDoubleCode(from)
 
 case array: ArrayType =>
-  castArrayCode(from.asInstanceOf[ArrayType].elementType, 
array.elementType, ctx)
+  castArrayCode(from.asInstanceOf[ArrayType], array, ctx)
--- End diff --

how about we move this into another PR? I think the main purpose of this PR 
is to eliminate the unnecessary Cast


---
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 #13704: [SPARK-15985][SQL] Reduce runtime overhead of a p...

2016-07-03 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/13704#discussion_r69391625
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
 ---
@@ -837,8 +837,36 @@ case class Cast(child: Expression, dataType: DataType) 
extends UnaryExpression w
 val j = ctx.freshName("j")
 val values = ctx.freshName("values")
 
+val isPrimitiveFrom = ctx.isPrimitiveType(fromType)
--- End diff --

Yes, you are right. The latest code also checks 
```ArrayType.containsNull``` of ```from``` and ```to```.


---
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 #13704: [SPARK-15985][SQL] Reduce runtime overhead of a p...

2016-06-28 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/13704#discussion_r68885567
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
 ---
@@ -837,8 +837,36 @@ case class Cast(child: Expression, dataType: DataType) 
extends UnaryExpression w
 val j = ctx.freshName("j")
 val values = ctx.freshName("values")
 
+val isPrimitiveFrom = ctx.isPrimitiveType(fromType)
--- End diff --

we need to make sure the input array's element nullability is false, but 
primitive type array doesn't guarantee it. e.g. we can have 
`ArrayType(ByteType, true)`


---
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 #13704: [SPARK-15985][SQL] Reduce runtime overhead of a p...

2016-06-27 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/13704#discussion_r68548441
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
 ---
@@ -837,8 +837,36 @@ case class Cast(child: Expression, dataType: DataType) 
extends UnaryExpression w
 val j = ctx.freshName("j")
 val values = ctx.freshName("values")
 
+val isPrimitiveFrom = ctx.isPrimitiveType(fromType)
+val isPrimitiveTo = ctx.isPrimitiveType(toType)
+
 (c, evPrim, evNull) =>
-  s"""
+  if (isPrimitiveFrom && isPrimitiveTo) {
+// ensure no null in input and output arrays
--- End diff --

I assume that here is a part to generate code for Java primitive arrays 
regarding ```from``` and ```to```. Since Java primitive array (e.g. int[]) 
cannot have ```null``` value unlike SQL, I said "ensure not null in input and 
output arrays."
What 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 #13704: [SPARK-15985][SQL] Reduce runtime overhead of a p...

2016-06-27 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/13704#discussion_r68544526
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
 ---
@@ -837,8 +837,36 @@ case class Cast(child: Expression, dataType: DataType) 
extends UnaryExpression w
 val j = ctx.freshName("j")
 val values = ctx.freshName("values")
 
+val isPrimitiveFrom = ctx.isPrimitiveType(fromType)
+val isPrimitiveTo = ctx.isPrimitiveType(toType)
+
 (c, evPrim, evNull) =>
-  s"""
+  if (isPrimitiveFrom && isPrimitiveTo) {
+// ensure no null in input and output arrays
--- End diff --

where do we ensure 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 #13704: [SPARK-15985][SQL] Reduce runtime overhead of a p...

2016-06-16 Thread kiszk
GitHub user kiszk opened a pull request:

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

[SPARK-15985][SQL] Reduce runtime overhead of a program that reads an 
primitive array in Dataset

## What changes were proposed in this pull request?

This PR reduces runtime overhead of a program the reads an primitive array 
in Dataset. Generated code copies array elements from Dataset to a temporary 
array. If we know that types of source and destination are primitive array, we 
apply one of the following optimization:

1. Eliminate an allocation of ```Object[]``` and call 
```ArrayData.toArray()``` method if types of source and destination are 
the same
2. Eliminate a pair of ```isNullAt()``` and a ```null``` assignment and 
allocate an primitive array instead of ```Object[]``` if types of source and 
destination are different


An example program
```
val ds = Seq(Array(1.0, 2.0, 3.0), Array(4.0, 5.0, 6.0)).toDS()
ds.map(p => {
 var s = 0.0
 for (i <- 0 to 2) { s += p(i) }
 s
   }).show
```

Generated code before applying this PR
```
/* 036 */   protected void processNext() throws java.io.IOException {
/* 037 */ while (inputadapter_input.hasNext()) {
/* 038 */   InternalRow inputadapter_row = (InternalRow) 
inputadapter_input.next();
/* 039 */   boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 040 */   ArrayData inputadapter_value = inputadapter_isNull ? null : 
(inputadapter_row.getArray(0));
/* 041 */
/* 042 */   boolean deserializetoobject_isNull1 = inputadapter_isNull;
/* 043 */   ArrayData deserializetoobject_value1 = null;
/* 044 */   if (!inputadapter_isNull) {
/* 045 */ final int deserializetoobject_n = 
inputadapter_value.numElements();
/* 046 */ final Object[] deserializetoobject_values = new 
Object[deserializetoobject_n];
/* 047 */ for (int deserializetoobject_j = 0; deserializetoobject_j 
< deserializetoobject_n; deserializetoobject_j ++) {
/* 048 */   if (inputadapter_value.isNullAt(deserializetoobject_j)) 
{
/* 049 */ deserializetoobject_values[deserializetoobject_j] = 
null;
/* 050 */   } else {
/* 051 */ boolean deserializetoobject_feNull = false;
/* 052 */ double deserializetoobject_fePrim =
/* 053 */ inputadapter_value.getDouble(deserializetoobject_j);
/* 054 */
/* 055 */ boolean deserializetoobject_teNull = 
deserializetoobject_feNull;
/* 056 */ double deserializetoobject_tePrim = -1.0;
/* 057 */ if (!deserializetoobject_feNull) {
/* 058 */   deserializetoobject_tePrim = 
deserializetoobject_fePrim;
/* 059 */ }
/* 060 */
/* 061 */ if (deserializetoobject_teNull) {
/* 062 */   deserializetoobject_values[deserializetoobject_j] = 
null;
/* 063 */ } else {
/* 064 */   deserializetoobject_values[deserializetoobject_j] = 
deserializetoobject_tePrim;
/* 065 */ }
/* 066 */   }
/* 067 */ }
/* 068 */ deserializetoobject_value1 = new 
org.apache.spark.sql.catalyst.util.GenericArrayData(deserializetoobject_values);
/* 069 */
/* 070 */   }
/* 071 */
/* 072 */   boolean deserializetoobject_isNull = 
deserializetoobject_isNull1;
/* 073 */   final double[] deserializetoobject_value = 
deserializetoobject_isNull ? null : (double[]) 
deserializetoobject_value1.toDoubleArray();
/* 074 */   deserializetoobject_isNull = deserializetoobject_value == 
null;
/* 075 */
/* 076 */   Object mapelements_obj = ((Expression) 
references[0]).eval(null);
/* 077 */   scala.Function1 mapelements_value1 = (scala.Function1) 
mapelements_obj;
/* 078 */
/* 079 */   boolean mapelements_isNull = false || 
deserializetoobject_isNull;
/* 080 */   final double mapelements_value = mapelements_isNull ? -1.0 
: (Double) mapelements_value1.apply(deserializetoobject_value);
/* 081 */
/* 082 */   serializefromobject_rowWriter.zeroOutNullBytes();
/* 083 */
/* 084 */   if (mapelements_isNull) {
/* 085 */ serializefromobject_rowWriter.setNullAt(0);
/* 086 */   } else {
/* 087 */ serializefromobject_rowWriter.write(0, mapelements_value);
/* 088 */   }
/* 089 */   append(serializefromobject_result);
/* 090 */   if (shouldStop()) return;
/* 091 */ }
/* 092 */   }
```

Generated code after applying this PR
```
/* 036 */   protected void processNext() throws java.io.IOException {
/* 037 */ while (inputadapter_input.hasNext()) {
/* 038 */   InternalRow inputadapter_row = (InternalRow) 
inputadapter_input.next();
/* 039 */   boolean inputadapter_is