[GitHub] spark pull request #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2017-02-07 Thread asfgit
Github user asfgit closed the pull request at:

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


---
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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2017-02-05 Thread hvanhovell
Github user hvanhovell commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r99489777
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala
 ---
@@ -0,0 +1,166 @@
+/*
+ * 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.expressions._
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+* push down operations into [[CreateNamedStructLike]].
+*/
+object SimplifyCreateStructOps extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp {
+  // push down field extraction
+  case GetStructField(createNamedStructLike: CreateNamedStructLike, 
ordinal, _) =>
+createNamedStructLike.valExprs(ordinal)
+}
+  }
+}
+
+/**
+* push down operations into [[CreateArray]].
+*/
+object SimplifyCreateArrayOps extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp {
+  // push down field selection (array of structs)
+  case GetArrayStructFields(CreateArray(elems), field, ordinal, 
numFields, containsNull) =>
+// instead f selecting the field on the entire array,
+// select it from each member of the array.
+// pushing down the operation this way open other optimizations 
opportunities
+// (i.e. struct(...,x,...).x)
+CreateArray(elems.map(GetStructField(_, ordinal, 
Some(field.name
+  // push down item selection.
+  case ga @ GetArrayItem(CreateArray(elems), IntegerLiteral(idx)) =>
+// instead of creating the array and then selecting one row,
+// remove array creation altgether.
+if (idx >= 0 && idx < elems.size) {
+  // valid index
+  elems(idx)
+} else {
+  // out of bounds, mimic the runtime behavior and return null
+  Literal(null, ga.dataType)
+}
+}
+  }
+}
+
+/**
+* push down operations into [[CreateMap]].
+*/
+object SimplifyCreateMapOps extends Rule[LogicalPlan] {
+  object ComparisonResult extends Enumeration {
--- End diff --

We can remove most of this code right?


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

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



[GitHub] spark pull request #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2017-02-04 Thread eyalfa
Github user eyalfa commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r99475332
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/complexTypesSuite.scala
 ---
@@ -0,0 +1,499 @@
+/*
+ * 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.scalatest.Matchers
+
+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.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.Range
+import org.apache.spark.sql.catalyst.rules.RuleExecutor
+import org.apache.spark.sql.types._
+
+/**
+* SPARK-18601 discusses simplification direct access to complex types 
creators.
+* i.e. {{{create_named_struct(square, `x` * `x`).square}}} can be 
simplified to {{{`x` * `x`}}}.
+* sam applies to create_array and create_map
+*/
+class ComplexTypesSuite extends PlanTest{
+
+  object Optimize extends RuleExecutor[LogicalPlan] {
+val batches =
+  Batch("collapse projections", FixedPoint(10),
+  CollapseProject) ::
+  Batch("Constant Folding", FixedPoint(10),
+  NullPropagation(conf),
+  ConstantFolding,
+  BooleanSimplification,
+  SimplifyConditionals,
+  SimplifyBinaryComparison,
+  SimplifyCreateStructOps,
+  SimplifyCreateArrayOps,
+  SimplifyCreateMapOps) :: Nil
+  }
+
+  val idAtt = ('id).long.notNull
+
+  lazy val baseOptimizedPlan = Range(1L, 1000L, 1, Some(2), idAtt :: Nil)
+
+  val idRef = baseOptimizedPlan.output.head
+
+  implicit class ComplexTypeDslSupport(e : Expression) {
+def getStructField(f : String): GetStructField = {
+  assert(e.resolved)
+  assert(e.dataType.isInstanceOf[StructType])
+  val structType = e.dataType.asInstanceOf[StructType]
+  val ord = structType.fieldNames.indexOf(f)
+  assert(-1 != ord)
+  GetStructField(e, ord, Some(f))
+}
+
+def getArrayStructField(f : String) : Expression = {
+  assert(e.resolved)
+  assert(e.dataType.isInstanceOf[ArrayType])
+  val arrType = e.dataType.asInstanceOf[ArrayType]
+  assert(arrType.elementType.isInstanceOf[StructType])
+  val structType = arrType.elementType.asInstanceOf[StructType]
+  val ord = structType.fieldNames.indexOf(f)
+  assert(-1 != ord)
+  GetArrayStructFields(e, structType(ord), ord, 1, 
arrType.containsNull)
+}
+
+def getArrayItem(i : Int) : GetArrayItem = {
+  assert(e.resolved)
+  assert(e.dataType.isInstanceOf[ArrayType])
+  GetArrayItem(e, Literal(i))
+}
+
+def getMapValue(k : Expression) : Expression = {
+  assert(e.resolved)
+  assert(e.dataType.isInstanceOf[MapType])
+  val mapType = e.dataType.asInstanceOf[MapType]
+  assert(k.dataType == mapType.keyType)
+  GetMapValue(e, k)
+}
+
+def addCaseWhen( cond : Expression, v : Expression) : Expression = {
+  assert(e.resolved)
+  assert(e.isInstanceOf[CaseWhen])
+  assert(cond.dataType == BooleanType)
+  assert(v.dataType == e.dataType)
+  val CaseWhen(branches, default) = e
+  CaseWhen( branches :+ (cond, v), default)
+}
+  }
+
+  test("explicit") {
+val rel = baseOptimizedPlan.select(
+  CreateNamedStruct("att" :: idRef :: Nil).getStructField("att") as 
"outerAtt"
+   )
+
+assertResult(StructType(StructField("outerAtt", LongType, nullable = 
false) :: Nil))(rel.schema)
+
+val optimized = Optimize execute rel
+
+val expected = baseOptimizedPlan.select(idRef as "outerAtt")
+
+

[GitHub] spark pull request #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2017-02-04 Thread eyalfa
Github user eyalfa commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r99475106
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/complexTypesSuite.scala
 ---
@@ -0,0 +1,499 @@
+/*
+ * 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.scalatest.Matchers
+
+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.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.Range
+import org.apache.spark.sql.catalyst.rules.RuleExecutor
+import org.apache.spark.sql.types._
+
+/**
+* SPARK-18601 discusses simplification direct access to complex types 
creators.
+* i.e. {{{create_named_struct(square, `x` * `x`).square}}} can be 
simplified to {{{`x` * `x`}}}.
+* sam applies to create_array and create_map
+*/
+class ComplexTypesSuite extends PlanTest{
+
+  object Optimize extends RuleExecutor[LogicalPlan] {
+val batches =
+  Batch("collapse projections", FixedPoint(10),
+  CollapseProject) ::
+  Batch("Constant Folding", FixedPoint(10),
+  NullPropagation(conf),
+  ConstantFolding,
+  BooleanSimplification,
+  SimplifyConditionals,
+  SimplifyBinaryComparison,
+  SimplifyCreateStructOps,
+  SimplifyCreateArrayOps,
+  SimplifyCreateMapOps) :: Nil
+  }
+
+  val idAtt = ('id).long.notNull
+
+  lazy val baseOptimizedPlan = Range(1L, 1000L, 1, Some(2), idAtt :: Nil)
+
+  val idRef = baseOptimizedPlan.output.head
+
+  implicit class ComplexTypeDslSupport(e : Expression) {
+def getStructField(f : String): GetStructField = {
+  assert(e.resolved)
+  assert(e.dataType.isInstanceOf[StructType])
+  val structType = e.dataType.asInstanceOf[StructType]
+  val ord = structType.fieldNames.indexOf(f)
+  assert(-1 != ord)
+  GetStructField(e, ord, Some(f))
+}
+
+def getArrayStructField(f : String) : Expression = {
+  assert(e.resolved)
+  assert(e.dataType.isInstanceOf[ArrayType])
+  val arrType = e.dataType.asInstanceOf[ArrayType]
+  assert(arrType.elementType.isInstanceOf[StructType])
+  val structType = arrType.elementType.asInstanceOf[StructType]
+  val ord = structType.fieldNames.indexOf(f)
+  assert(-1 != ord)
+  GetArrayStructFields(e, structType(ord), ord, 1, 
arrType.containsNull)
+}
+
+def getArrayItem(i : Int) : GetArrayItem = {
+  assert(e.resolved)
+  assert(e.dataType.isInstanceOf[ArrayType])
+  GetArrayItem(e, Literal(i))
+}
+
+def getMapValue(k : Expression) : Expression = {
+  assert(e.resolved)
+  assert(e.dataType.isInstanceOf[MapType])
+  val mapType = e.dataType.asInstanceOf[MapType]
+  assert(k.dataType == mapType.keyType)
+  GetMapValue(e, k)
+}
+
+def addCaseWhen( cond : Expression, v : Expression) : Expression = {
+  assert(e.resolved)
+  assert(e.isInstanceOf[CaseWhen])
+  assert(cond.dataType == BooleanType)
+  assert(v.dataType == e.dataType)
+  val CaseWhen(branches, default) = e
+  CaseWhen( branches :+ (cond, v), default)
+}
+  }
+
+  test("explicit") {
+val rel = baseOptimizedPlan.select(
+  CreateNamedStruct("att" :: idRef :: Nil).getStructField("att") as 
"outerAtt"
+   )
+
+assertResult(StructType(StructField("outerAtt", LongType, nullable = 
false) :: Nil))(rel.schema)
+
+val optimized = Optimize execute rel
+
+val expected = baseOptimizedPlan.select(idRef as "outerAtt")
+
+

[GitHub] spark pull request #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2017-02-04 Thread eyalfa
Github user eyalfa commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r99475096
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/complexTypesSuite.scala
 ---
@@ -0,0 +1,499 @@
+/*
+ * 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.scalatest.Matchers
+
+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.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.Range
+import org.apache.spark.sql.catalyst.rules.RuleExecutor
+import org.apache.spark.sql.types._
+
+/**
+* SPARK-18601 discusses simplification direct access to complex types 
creators.
+* i.e. {{{create_named_struct(square, `x` * `x`).square}}} can be 
simplified to {{{`x` * `x`}}}.
+* sam applies to create_array and create_map
+*/
+class ComplexTypesSuite extends PlanTest{
+
+  object Optimize extends RuleExecutor[LogicalPlan] {
+val batches =
+  Batch("collapse projections", FixedPoint(10),
+  CollapseProject) ::
+  Batch("Constant Folding", FixedPoint(10),
+  NullPropagation(conf),
+  ConstantFolding,
+  BooleanSimplification,
+  SimplifyConditionals,
+  SimplifyBinaryComparison,
+  SimplifyCreateStructOps,
+  SimplifyCreateArrayOps,
+  SimplifyCreateMapOps) :: Nil
+  }
+
+  val idAtt = ('id).long.notNull
+
+  lazy val baseOptimizedPlan = Range(1L, 1000L, 1, Some(2), idAtt :: Nil)
+
+  val idRef = baseOptimizedPlan.output.head
+
+  implicit class ComplexTypeDslSupport(e : Expression) {
+def getStructField(f : String): GetStructField = {
+  assert(e.resolved)
+  assert(e.dataType.isInstanceOf[StructType])
+  val structType = e.dataType.asInstanceOf[StructType]
+  val ord = structType.fieldNames.indexOf(f)
+  assert(-1 != ord)
+  GetStructField(e, ord, Some(f))
+}
+
+def getArrayStructField(f : String) : Expression = {
+  assert(e.resolved)
+  assert(e.dataType.isInstanceOf[ArrayType])
+  val arrType = e.dataType.asInstanceOf[ArrayType]
+  assert(arrType.elementType.isInstanceOf[StructType])
+  val structType = arrType.elementType.asInstanceOf[StructType]
+  val ord = structType.fieldNames.indexOf(f)
+  assert(-1 != ord)
+  GetArrayStructFields(e, structType(ord), ord, 1, 
arrType.containsNull)
+}
+
+def getArrayItem(i : Int) : GetArrayItem = {
+  assert(e.resolved)
+  assert(e.dataType.isInstanceOf[ArrayType])
+  GetArrayItem(e, Literal(i))
+}
+
+def getMapValue(k : Expression) : Expression = {
+  assert(e.resolved)
+  assert(e.dataType.isInstanceOf[MapType])
+  val mapType = e.dataType.asInstanceOf[MapType]
+  assert(k.dataType == mapType.keyType)
+  GetMapValue(e, k)
+}
+
+def addCaseWhen( cond : Expression, v : Expression) : Expression = {
+  assert(e.resolved)
+  assert(e.isInstanceOf[CaseWhen])
+  assert(cond.dataType == BooleanType)
+  assert(v.dataType == e.dataType)
+  val CaseWhen(branches, default) = e
+  CaseWhen( branches :+ (cond, v), default)
+}
+  }
+
+  test("explicit") {
+val rel = baseOptimizedPlan.select(
+  CreateNamedStruct("att" :: idRef :: Nil).getStructField("att") as 
"outerAtt"
+   )
+
+assertResult(StructType(StructField("outerAtt", LongType, nullable = 
false) :: Nil))(rel.schema)
+
+val optimized = Optimize execute rel
+
+val expected = baseOptimizedPlan.select(idRef as "outerAtt")
+
+

[GitHub] spark pull request #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2017-01-31 Thread eyalfa
Github user eyalfa commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r98660310
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala
 ---
@@ -0,0 +1,166 @@
+/*
+ * 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.expressions._
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+* push down operations into [[CreateNamedStructLike]].
+*/
+object SimplifyCreateStructOps extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp {
+  // push down field extraction
+  case GetStructField(createNamedStructLike: CreateNamedStructLike, 
ordinal, _) =>
+createNamedStructLike.valExprs(ordinal)
+}
+  }
+}
+
+/**
+* push down operations into [[CreateArray]].
+*/
+object SimplifyCreateArrayOps extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp {
+  // push down field selection (array of structs)
+  case GetArrayStructFields(CreateArray(elems), field, ordinal, 
numFields, containsNull) =>
+// instead f selecting the field on the entire array,
+// select it from each member of the array.
+// pushing down the operation this way open other optimizations 
opportunities
+// (i.e. struct(...,x,...).x)
+CreateArray(elems.map(GetStructField(_, ordinal, 
Some(field.name
+  // push down item selection.
+  case ga @ GetArrayItem(CreateArray(elems), IntegerLiteral(idx)) =>
+// instead of creating the array and then selecting one row,
+// remove array creation altgether.
+if (idx >= 0 && idx < elems.size) {
+  // valid index
+  elems(idx)
+} else {
+  // out of bounds, mimic the runtime behavior and return null
+  Cast(Literal(null), ga.dataType)
--- End diff --

yep


---
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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2017-01-31 Thread eyalfa
Github user eyalfa commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r98660085
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala
 ---
@@ -293,6 +293,12 @@ object SimplifyConditionals extends Rule[LogicalPlan] 
with PredicateHelper {
 // from that. Note that CaseWhen.branches should never be empty, 
and as a result the
 // headOption (rather than head) added above is just an extra (and 
unnecessary) safeguard.
 branches.head._2
+
+  case e @ CaseWhen(branches, _) if branches.exists(_._1 == 
Literal(true)) =>
+// a branc with a TRue condition eliminates all following branches,
+// these branches can be pruned away
+val (h, t) = branches.span(_._1 != Literal(true))
+CaseWhen( h :+ t.head, None)
--- End diff --

sorry, please explain


---
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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2017-01-31 Thread hvanhovell
Github user hvanhovell commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r98613798
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala
 ---
@@ -0,0 +1,166 @@
+/*
+ * 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.expressions._
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+* push down operations into [[CreateNamedStructLike]].
+*/
+object SimplifyCreateStructOps extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp {
+  // push down field extraction
+  case GetStructField(createNamedStructLike: CreateNamedStructLike, 
ordinal, _) =>
+createNamedStructLike.valExprs(ordinal)
+}
+  }
+}
+
+/**
+* push down operations into [[CreateArray]].
+*/
+object SimplifyCreateArrayOps extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp {
+  // push down field selection (array of structs)
+  case GetArrayStructFields(CreateArray(elems), field, ordinal, 
numFields, containsNull) =>
+// instead f selecting the field on the entire array,
+// select it from each member of the array.
+// pushing down the operation this way open other optimizations 
opportunities
+// (i.e. struct(...,x,...).x)
+CreateArray(elems.map(GetStructField(_, ordinal, 
Some(field.name
+  // push down item selection.
+  case ga @ GetArrayItem(CreateArray(elems), IntegerLiteral(idx)) =>
+// instead of creating the array and then selecting one row,
+// remove array creation altgether.
+if (idx >= 0 && idx < elems.size) {
+  // valid index
+  elems(idx)
+} else {
+  // out of bounds, mimic the runtime behavior and return null
+  Cast(Literal(null), ga.dataType)
--- End diff --

`Literal(null, ga.dataType)`?


---
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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2017-01-30 Thread hvanhovell
Github user hvanhovell commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r98613398
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala
 ---
@@ -293,6 +293,12 @@ object SimplifyConditionals extends Rule[LogicalPlan] 
with PredicateHelper {
 // from that. Note that CaseWhen.branches should never be empty, 
and as a result the
 // headOption (rather than head) added above is just an extra (and 
unnecessary) safeguard.
 branches.head._2
+
+  case e @ CaseWhen(branches, _) if branches.exists(_._1 == 
Literal(true)) =>
--- End diff --

`e @` not needed. Use `TrueLiteral` for comparisons.


---
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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2017-01-30 Thread hvanhovell
Github user hvanhovell commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r98613430
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala
 ---
@@ -293,6 +293,12 @@ object SimplifyConditionals extends Rule[LogicalPlan] 
with PredicateHelper {
 // from that. Note that CaseWhen.branches should never be empty, 
and as a result the
 // headOption (rather than head) added above is just an extra (and 
unnecessary) safeguard.
 branches.head._2
+
+  case e @ CaseWhen(branches, _) if branches.exists(_._1 == 
Literal(true)) =>
+// a branc with a TRue condition eliminates all following branches,
+// these branches can be pruned away
+val (h, t) = branches.span(_._1 != Literal(true))
--- End diff --

Use `TrueLiteral`


---
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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2017-01-30 Thread hvanhovell
Github user hvanhovell commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r98613413
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala
 ---
@@ -293,6 +293,12 @@ object SimplifyConditionals extends Rule[LogicalPlan] 
with PredicateHelper {
 // from that. Note that CaseWhen.branches should never be empty, 
and as a result the
 // headOption (rather than head) added above is just an extra (and 
unnecessary) safeguard.
 branches.head._2
+
+  case e @ CaseWhen(branches, _) if branches.exists(_._1 == 
Literal(true)) =>
+// a branc with a TRue condition eliminates all following branches,
+// these branches can be pruned away
+val (h, t) = branches.span(_._1 != Literal(true))
+CaseWhen( h :+ t.head, None)
--- End diff --

Nit space


---
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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2017-01-27 Thread hvanhovell
Github user hvanhovell commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r98240562
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala
 ---
@@ -0,0 +1,131 @@
+/*
+ * 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.expressions.{Cast, Coalesce, 
CreateArray, CreateMap, CreateNamedStructLike, Expression, GetArrayItem, 
GetArrayStructFields, GetMapValue, GetStructField, IntegerLiteral, Literal}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+* push down operations into [[CreateNamedStructLike]].
+*/
+object SimplifyCreateStructOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // push down field extraction
+  case GetStructField(createNamedStructLike : CreateNamedStructLike, 
ordinal, _) =>
+createNamedStructLike.valExprs(ordinal)
+}
+  }
+}
+
+/**
+* push down operations into [[CreateArray]].
+*/
+object SimplifyCreateArrayOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // push down field selection (array of structs)
+  case GetArrayStructFields(CreateArray(elems), field, ordinal, 
numFields, containsNull) =>
+def getStructField(elem : Expression) = {
+  GetStructField(elem, ordinal, Some(field.name))
+}
+CreateArray(elems.map(getStructField))
+  // push down item selection.
+  case ga @ GetArrayItem(CreateArray(elems), IntegerLiteral(idx)) =>
+if (idx >= 0 && idx < elems.size) {
+  elems(idx)
+} else {
+  Cast(Literal(null), ga.dataType)
+}
+}
+  }
+}
+
+/**
+* push down operations into [[CreateMap]].
+*/
+object SimplifyCreateMapOps extends Rule[LogicalPlan]{
+  object ComparisonResult extends Enumeration {
+val PositiveMatch = Value
+val NegativeMatch = Value
+val UnDetermined = Value
+  }
+
+  def compareKeys(k1 : Expression, k2 : Expression) : 
ComparisonResult.Value = {
+(k1, k2) match {
+  case (x, y) if x.semanticEquals(y) => ComparisonResult.PositiveMatch
+  // make surethis is null safe, especially when datatypes differ
+  // is this even possible?
+  case (_ : Literal, _ : Literal) => ComparisonResult.NegativeMatch
+  case _ => ComparisonResult.UnDetermined
+}
+  }
+
+  case class ClassifiedEntries(undetermined : Seq[Expression],
+   nullable : Boolean,
+   firstPositive : Option[Expression]) {
+def normalize( k : Expression ) : ClassifiedEntries = this match {
+  /**
+  * when we have undetermined matches that might bproduce a null value,
+  * we can't separate a positive match and use [[Coalesce]] to choose 
the final result.
+  * so we 'hide' the positive match as an undetermined match.
+  */
+  case ClassifiedEntries( u, true, Some(p)) if u.nonEmpty =>
+ClassifiedEntries(u ++ Seq(k, p), true, None)
+  case _ => this
+}
+  }
+
+  def classifyEntries(mapEntries : Seq[(Expression, Expression)],
+  requestedKey : Expression) : ClassifiedEntries = {
+val res1 = mapEntries.foldLeft(ClassifiedEntries(Seq.empty, nullable = 
false, None)) {
+  case (prev @ ClassifiedEntries(_, _, Some(_)), _) => prev
+  case (ClassifiedEntries(prev, nullable, None), (k, v)) =>
+compareKeys(k, requestedKey) match {
+  case ComparisonResult.UnDetermined =>
+val vIsNullable = v.nullable
+val nextNullbale = nullable || vIsNullable
+

[GitHub] spark pull request #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2017-01-27 Thread hvanhovell
Github user hvanhovell commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r98240343
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala
 ---
@@ -0,0 +1,128 @@
+/*
+ * 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.expressions._
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+* push down operations into [[CreateNamedStructLike]].
+*/
+object SimplifyCreateStructOps extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp {
+  // push down field extraction
+  case GetStructField(createNamedStructLike: CreateNamedStructLike, 
ordinal, _) =>
+createNamedStructLike.valExprs(ordinal)
+}
+  }
+}
+
+/**
+* push down operations into [[CreateArray]].
+*/
+object SimplifyCreateArrayOps extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp {
+  // push down field selection (array of structs)
+  case GetArrayStructFields(CreateArray(elems), field, ordinal, 
numFields, containsNull) =>
+CreateArray(elems.map(GetStructField(_, ordinal, 
Some(field.name
+  // push down item selection.
+  case ga @ GetArrayItem(CreateArray(elems), IntegerLiteral(idx)) =>
+if (idx >= 0 && idx < elems.size) {
+  elems(idx)
+} else {
+  Cast(Literal(null), ga.dataType)
+}
+}
+  }
+}
+
+/**
+* push down operations into [[CreateMap]].
+*/
+object SimplifyCreateMapOps extends Rule[LogicalPlan] {
+  object ComparisonResult extends Enumeration {
+val PositiveMatch = Value
+val NegativeMatch = Value
+val UnDetermined = Value
+  }
+
+  def compareKeys(k1 : Expression, k2 : Expression) : 
ComparisonResult.Value = {
+(k1, k2) match {
+  case (x, y) if x.semanticEquals(y) => ComparisonResult.PositiveMatch
+  // make surethis is null safe, especially when datatypes differ
+  // is this even possible?
+  case (_ : Literal, _ : Literal) => ComparisonResult.NegativeMatch
+  case _ => ComparisonResult.UnDetermined
+}
+  }
+
+  case class ClassifiedEntries(
+undetermined : Seq[Expression],
+nullable : Boolean,
+firstPositive : Option[Expression]) {
+def normalize(k : Expression) : ClassifiedEntries = this match {
+  /**
+  * when we have undetermined matches that might bproduce a null value,
+  * we can't separate a positive match and use [[Coalesce]] to choose 
the final result.
+  * so we 'hide' the positive match as an undetermined match.
+  */
+  case ClassifiedEntries(u, true, Some(p)) if u.nonEmpty =>
+ClassifiedEntries(u ++ Seq(k, p), true, None)
+  case _ => this
+}
+  }
+
+  def classifyEntries(mapEntries : Seq[(Expression, Expression)],
+  requestedKey : Expression) : ClassifiedEntries = {
+val res1 = mapEntries.foldLeft(ClassifiedEntries(Seq.empty, nullable = 
false, None)) {
+  case (prev @ ClassifiedEntries(_, _, Some(_)), _) => prev
+  case (ClassifiedEntries(prev, nullable, None), (k, v)) =>
+compareKeys(k, requestedKey) match {
+  case ComparisonResult.UnDetermined =>
+val vIsNullable = v.nullable
+val nextNullbale = nullable || vIsNullable
+ClassifiedEntries(prev ++ Seq(k, v), nullable = nextNullbale, 
None)
+  case ComparisonResult.NegativeMatch => ClassifiedEntries(prev, 
nullable, None)
+  case ComparisonResult.PositiveMatch => ClassifiedEntries(prev, 
nullable, Some(v))
+}
+}
+

[GitHub] spark pull request #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2017-01-27 Thread eyalfa
Github user eyalfa commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r98222460
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala
 ---
@@ -0,0 +1,128 @@
+/*
+ * 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.expressions._
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+* push down operations into [[CreateNamedStructLike]].
+*/
+object SimplifyCreateStructOps extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp {
+  // push down field extraction
+  case GetStructField(createNamedStructLike: CreateNamedStructLike, 
ordinal, _) =>
+createNamedStructLike.valExprs(ordinal)
+}
+  }
+}
+
+/**
+* push down operations into [[CreateArray]].
+*/
+object SimplifyCreateArrayOps extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp {
+  // push down field selection (array of structs)
+  case GetArrayStructFields(CreateArray(elems), field, ordinal, 
numFields, containsNull) =>
+CreateArray(elems.map(GetStructField(_, ordinal, 
Some(field.name
+  // push down item selection.
+  case ga @ GetArrayItem(CreateArray(elems), IntegerLiteral(idx)) =>
+if (idx >= 0 && idx < elems.size) {
+  elems(idx)
+} else {
+  Cast(Literal(null), ga.dataType)
+}
+}
+  }
+}
+
+/**
+* push down operations into [[CreateMap]].
+*/
+object SimplifyCreateMapOps extends Rule[LogicalPlan] {
+  object ComparisonResult extends Enumeration {
+val PositiveMatch = Value
+val NegativeMatch = Value
+val UnDetermined = Value
+  }
+
+  def compareKeys(k1 : Expression, k2 : Expression) : 
ComparisonResult.Value = {
+(k1, k2) match {
+  case (x, y) if x.semanticEquals(y) => ComparisonResult.PositiveMatch
+  // make surethis is null safe, especially when datatypes differ
+  // is this even possible?
+  case (_ : Literal, _ : Literal) => ComparisonResult.NegativeMatch
+  case _ => ComparisonResult.UnDetermined
+}
+  }
+
+  case class ClassifiedEntries(
+undetermined : Seq[Expression],
+nullable : Boolean,
+firstPositive : Option[Expression]) {
+def normalize(k : Expression) : ClassifiedEntries = this match {
+  /**
+  * when we have undetermined matches that might bproduce a null value,
+  * we can't separate a positive match and use [[Coalesce]] to choose 
the final result.
+  * so we 'hide' the positive match as an undetermined match.
+  */
+  case ClassifiedEntries(u, true, Some(p)) if u.nonEmpty =>
+ClassifiedEntries(u ++ Seq(k, p), true, None)
+  case _ => this
+}
+  }
+
+  def classifyEntries(mapEntries : Seq[(Expression, Expression)],
+  requestedKey : Expression) : ClassifiedEntries = {
+val res1 = mapEntries.foldLeft(ClassifiedEntries(Seq.empty, nullable = 
false, None)) {
+  case (prev @ ClassifiedEntries(_, _, Some(_)), _) => prev
+  case (ClassifiedEntries(prev, nullable, None), (k, v)) =>
+compareKeys(k, requestedKey) match {
+  case ComparisonResult.UnDetermined =>
+val vIsNullable = v.nullable
+val nextNullbale = nullable || vIsNullable
+ClassifiedEntries(prev ++ Seq(k, v), nullable = nextNullbale, 
None)
+  case ComparisonResult.NegativeMatch => ClassifiedEntries(prev, 
nullable, None)
+  case ComparisonResult.PositiveMatch => ClassifiedEntries(prev, 
nullable, Some(v))
+}
+}
+

[GitHub] spark pull request #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2017-01-27 Thread eyalfa
Github user eyalfa commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r98216632
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala
 ---
@@ -0,0 +1,128 @@
+/*
+ * 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.expressions._
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+* push down operations into [[CreateNamedStructLike]].
+*/
+object SimplifyCreateStructOps extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp {
+  // push down field extraction
+  case GetStructField(createNamedStructLike: CreateNamedStructLike, 
ordinal, _) =>
+createNamedStructLike.valExprs(ordinal)
+}
+  }
+}
+
+/**
+* push down operations into [[CreateArray]].
+*/
+object SimplifyCreateArrayOps extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp {
+  // push down field selection (array of structs)
+  case GetArrayStructFields(CreateArray(elems), field, ordinal, 
numFields, containsNull) =>
+CreateArray(elems.map(GetStructField(_, ordinal, 
Some(field.name
+  // push down item selection.
+  case ga @ GetArrayItem(CreateArray(elems), IntegerLiteral(idx)) =>
+if (idx >= 0 && idx < elems.size) {
+  elems(idx)
+} else {
+  Cast(Literal(null), ga.dataType)
+}
+}
+  }
+}
+
+/**
+* push down operations into [[CreateMap]].
+*/
+object SimplifyCreateMapOps extends Rule[LogicalPlan] {
+  object ComparisonResult extends Enumeration {
+val PositiveMatch = Value
+val NegativeMatch = Value
+val UnDetermined = Value
+  }
+
+  def compareKeys(k1 : Expression, k2 : Expression) : 
ComparisonResult.Value = {
+(k1, k2) match {
+  case (x, y) if x.semanticEquals(y) => ComparisonResult.PositiveMatch
+  // make surethis is null safe, especially when datatypes differ
+  // is this even possible?
+  case (_ : Literal, _ : Literal) => ComparisonResult.NegativeMatch
+  case _ => ComparisonResult.UnDetermined
+}
+  }
+
+  case class ClassifiedEntries(
+undetermined : Seq[Expression],
+nullable : Boolean,
+firstPositive : Option[Expression]) {
+def normalize(k : Expression) : ClassifiedEntries = this match {
+  /**
+  * when we have undetermined matches that might bproduce a null value,
+  * we can't separate a positive match and use [[Coalesce]] to choose 
the final result.
+  * so we 'hide' the positive match as an undetermined match.
+  */
+  case ClassifiedEntries(u, true, Some(p)) if u.nonEmpty =>
+ClassifiedEntries(u ++ Seq(k, p), true, None)
+  case _ => this
+}
+  }
+
+  def classifyEntries(mapEntries : Seq[(Expression, Expression)],
+  requestedKey : Expression) : ClassifiedEntries = {
+val res1 = mapEntries.foldLeft(ClassifiedEntries(Seq.empty, nullable = 
false, None)) {
+  case (prev @ ClassifiedEntries(_, _, Some(_)), _) => prev
+  case (ClassifiedEntries(prev, nullable, None), (k, v)) =>
+compareKeys(k, requestedKey) match {
+  case ComparisonResult.UnDetermined =>
+val vIsNullable = v.nullable
+val nextNullbale = nullable || vIsNullable
+ClassifiedEntries(prev ++ Seq(k, v), nullable = nextNullbale, 
None)
+  case ComparisonResult.NegativeMatch => ClassifiedEntries(prev, 
nullable, None)
+  case ComparisonResult.PositiveMatch => ClassifiedEntries(prev, 
nullable, Some(v))
+}
+}
+

[GitHub] spark pull request #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2017-01-27 Thread hvanhovell
Github user hvanhovell commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r98201359
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala
 ---
@@ -0,0 +1,128 @@
+/*
+ * 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.expressions._
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+* push down operations into [[CreateNamedStructLike]].
+*/
+object SimplifyCreateStructOps extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp {
+  // push down field extraction
+  case GetStructField(createNamedStructLike : CreateNamedStructLike, 
ordinal, _) =>
+createNamedStructLike.valExprs(ordinal)
+}
+  }
+}
+
+/**
+* push down operations into [[CreateArray]].
+*/
+object SimplifyCreateArrayOps extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp {
+  // push down field selection (array of structs)
+  case GetArrayStructFields(CreateArray(elems), field, ordinal, 
numFields, containsNull) =>
+CreateArray(elems.map(GetStructField(_, ordinal, 
Some(field.name
+  // push down item selection.
+  case ga @ GetArrayItem(CreateArray(elems), IntegerLiteral(idx)) =>
+if (idx >= 0 && idx < elems.size) {
+  elems(idx)
+} else {
+  Cast(Literal(null), ga.dataType)
+}
+}
+  }
+}
+
+/**
+* push down operations into [[CreateMap]].
+*/
+object SimplifyCreateMapOps extends Rule[LogicalPlan] {
+  object ComparisonResult extends Enumeration {
+val PositiveMatch = Value
+val NegativeMatch = Value
+val UnDetermined = Value
+  }
+
+  def compareKeys(k1 : Expression, k2 : Expression) : 
ComparisonResult.Value = {
+(k1, k2) match {
+  case (x, y) if x.semanticEquals(y) => ComparisonResult.PositiveMatch
+  // make surethis is null safe, especially when datatypes differ
--- End diff --

Literals are only equal when the dataTypes and the values are the same.


---
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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2017-01-27 Thread eyalfa
Github user eyalfa commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r98193617
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala
 ---
@@ -0,0 +1,128 @@
+/*
+ * 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.expressions._
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+* push down operations into [[CreateNamedStructLike]].
+*/
+object SimplifyCreateStructOps extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp {
+  // push down field extraction
+  case GetStructField(createNamedStructLike : CreateNamedStructLike, 
ordinal, _) =>
+createNamedStructLike.valExprs(ordinal)
+}
+  }
+}
+
+/**
+* push down operations into [[CreateArray]].
+*/
+object SimplifyCreateArrayOps extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp {
+  // push down field selection (array of structs)
+  case GetArrayStructFields(CreateArray(elems), field, ordinal, 
numFields, containsNull) =>
+CreateArray(elems.map(GetStructField(_, ordinal, 
Some(field.name
+  // push down item selection.
+  case ga @ GetArrayItem(CreateArray(elems), IntegerLiteral(idx)) =>
+if (idx >= 0 && idx < elems.size) {
+  elems(idx)
+} else {
+  Cast(Literal(null), ga.dataType)
+}
+}
+  }
+}
+
+/**
+* push down operations into [[CreateMap]].
+*/
+object SimplifyCreateMapOps extends Rule[LogicalPlan] {
+  object ComparisonResult extends Enumeration {
+val PositiveMatch = Value
+val NegativeMatch = Value
+val UnDetermined = Value
+  }
+
+  def compareKeys(k1 : Expression, k2 : Expression) : 
ComparisonResult.Value = {
+(k1, k2) match {
+  case (x, y) if x.semanticEquals(y) => ComparisonResult.PositiveMatch
+  // make surethis is null safe, especially when datatypes differ
--- End diff --

I this my comment relates to the case where one literal in an integer while 
the other is a long, not sue if this is possible (as stated in the next 
comment),
but if it is, does null : Int equals null : Long?


---
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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2017-01-27 Thread eyalfa
Github user eyalfa commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r98192298
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/complexTypesSuite.scala
 ---
@@ -0,0 +1,482 @@
+/*
+ * 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.scalatest.Matchers
+
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.dsl.plans._
+import org.apache.spark.sql.catalyst.expressions.{Coalesce, CreateArray, 
CreateMap, CreateNamedStruct, Expression, GetArrayItem, GetArrayStructFields, 
GetMapValue, GetStructField, Literal}
+import org.apache.spark.sql.catalyst.plans.PlanTest
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.Range
+import org.apache.spark.sql.catalyst.rules.RuleExecutor
+import org.apache.spark.sql.types._
+
+/**
+* Created by eyalf on 11/4/2016.
+* SPARK-18601 discusses simplification direct access to complex types 
creators.
+* i.e. {{{create_named_struct(square, `x` * `x`).square}}} can be 
simplified to {{{`x` * `x`}}}.
+* sam applies to create_array and create_map
+*/
+class ComplexTypesSuite extends PlanTest with Matchers{
+
+  object Optimize extends RuleExecutor[LogicalPlan] {
+val batches =
+  Batch("collapse projections", FixedPoint(10),
+  CollapseProject) ::
+  Batch("Constant Folding", FixedPoint(10),
+  NullPropagation,
+  ConstantFolding,
+  BooleanSimplification,
+  SimplifyConditionals,
+  SimplifyCreateStructOps,
+  SimplifyCreateArrayOps,
+  SimplifyCreateMapOps) :: Nil
+  }
+
+  val idAtt = ('id).long.notNull
+
+  lazy val baseOptimizedPlan = Range(1L, 1000L, 1, Some(2), idAtt :: Nil)
+
+  val idRef = baseOptimizedPlan.output.head
+
+
+//  val idRefColumn = Column("id")
+//  val struct1RefColumn = Column("struct1")
+
+  implicit class ComplexTypeDslSupport(e : Expression) {
+def getStructField(f : String): GetStructField = {
+  e should be ('resolved)
+  e.dataType should be (a[StructType])
--- End diff --

will change to assertion


---
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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2017-01-27 Thread eyalfa
Github user eyalfa commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r98192112
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala
 ---
@@ -0,0 +1,131 @@
+/*
+ * 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.expressions.{Cast, Coalesce, 
CreateArray, CreateMap, CreateNamedStructLike, Expression, GetArrayItem, 
GetArrayStructFields, GetMapValue, GetStructField, IntegerLiteral, Literal}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+* push down operations into [[CreateNamedStructLike]].
+*/
+object SimplifyCreateStructOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // push down field extraction
+  case GetStructField(createNamedStructLike : CreateNamedStructLike, 
ordinal, _) =>
+createNamedStructLike.valExprs(ordinal)
+}
+  }
+}
+
+/**
+* push down operations into [[CreateArray]].
+*/
+object SimplifyCreateArrayOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // push down field selection (array of structs)
+  case GetArrayStructFields(CreateArray(elems), field, ordinal, 
numFields, containsNull) =>
+def getStructField(elem : Expression) = {
+  GetStructField(elem, ordinal, Some(field.name))
+}
+CreateArray(elems.map(getStructField))
+  // push down item selection.
+  case ga @ GetArrayItem(CreateArray(elems), IntegerLiteral(idx)) =>
+if (idx >= 0 && idx < elems.size) {
+  elems(idx)
+} else {
+  Cast(Literal(null), ga.dataType)
+}
+}
+  }
+}
+
+/**
+* push down operations into [[CreateMap]].
+*/
+object SimplifyCreateMapOps extends Rule[LogicalPlan]{
+  object ComparisonResult extends Enumeration {
+val PositiveMatch = Value
+val NegativeMatch = Value
+val UnDetermined = Value
+  }
+
+  def compareKeys(k1 : Expression, k2 : Expression) : 
ComparisonResult.Value = {
+(k1, k2) match {
+  case (x, y) if x.semanticEquals(y) => ComparisonResult.PositiveMatch
+  // make surethis is null safe, especially when datatypes differ
+  // is this even possible?
+  case (_ : Literal, _ : Literal) => ComparisonResult.NegativeMatch
+  case _ => ComparisonResult.UnDetermined
+}
+  }
+
+  case class ClassifiedEntries(undetermined : Seq[Expression],
+   nullable : Boolean,
+   firstPositive : Option[Expression]) {
+def normalize( k : Expression ) : ClassifiedEntries = this match {
+  /**
+  * when we have undetermined matches that might bproduce a null value,
+  * we can't separate a positive match and use [[Coalesce]] to choose 
the final result.
+  * so we 'hide' the positive match as an undetermined match.
+  */
+  case ClassifiedEntries( u, true, Some(p)) if u.nonEmpty =>
+ClassifiedEntries(u ++ Seq(k, p), true, None)
+  case _ => this
+}
+  }
+
+  def classifyEntries(mapEntries : Seq[(Expression, Expression)],
+  requestedKey : Expression) : ClassifiedEntries = {
+val res1 = mapEntries.foldLeft(ClassifiedEntries(Seq.empty, nullable = 
false, None)) {
+  case (prev @ ClassifiedEntries(_, _, Some(_)), _) => prev
+  case (ClassifiedEntries(prev, nullable, None), (k, v)) =>
+compareKeys(k, requestedKey) match {
+  case ComparisonResult.UnDetermined =>
+val vIsNullable = v.nullable
+val nextNullbale = nullable || vIsNullable
+

[GitHub] spark pull request #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2017-01-27 Thread eyalfa
Github user eyalfa commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r98191785
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala
 ---
@@ -0,0 +1,128 @@
+/*
+ * 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.expressions._
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+* push down operations into [[CreateNamedStructLike]].
+*/
+object SimplifyCreateStructOps extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp {
+  // push down field extraction
+  case GetStructField(createNamedStructLike: CreateNamedStructLike, 
ordinal, _) =>
+createNamedStructLike.valExprs(ordinal)
+}
+  }
+}
+
+/**
+* push down operations into [[CreateArray]].
+*/
+object SimplifyCreateArrayOps extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp {
+  // push down field selection (array of structs)
+  case GetArrayStructFields(CreateArray(elems), field, ordinal, 
numFields, containsNull) =>
+CreateArray(elems.map(GetStructField(_, ordinal, 
Some(field.name
+  // push down item selection.
+  case ga @ GetArrayItem(CreateArray(elems), IntegerLiteral(idx)) =>
+if (idx >= 0 && idx < elems.size) {
+  elems(idx)
+} else {
+  Cast(Literal(null), ga.dataType)
+}
+}
+  }
+}
+
+/**
+* push down operations into [[CreateMap]].
+*/
+object SimplifyCreateMapOps extends Rule[LogicalPlan] {
+  object ComparisonResult extends Enumeration {
+val PositiveMatch = Value
+val NegativeMatch = Value
+val UnDetermined = Value
+  }
+
+  def compareKeys(k1 : Expression, k2 : Expression) : 
ComparisonResult.Value = {
+(k1, k2) match {
+  case (x, y) if x.semanticEquals(y) => ComparisonResult.PositiveMatch
+  // make surethis is null safe, especially when datatypes differ
+  // is this even possible?
+  case (_ : Literal, _ : Literal) => ComparisonResult.NegativeMatch
+  case _ => ComparisonResult.UnDetermined
+}
+  }
+
+  case class ClassifiedEntries(
+undetermined : Seq[Expression],
+nullable : Boolean,
+firstPositive : Option[Expression]) {
+def normalize(k : Expression) : ClassifiedEntries = this match {
+  /**
+  * when we have undetermined matches that might bproduce a null value,
+  * we can't separate a positive match and use [[Coalesce]] to choose 
the final result.
+  * so we 'hide' the positive match as an undetermined match.
+  */
+  case ClassifiedEntries(u, true, Some(p)) if u.nonEmpty =>
+ClassifiedEntries(u ++ Seq(k, p), true, None)
+  case _ => this
+}
+  }
+
+  def classifyEntries(mapEntries : Seq[(Expression, Expression)],
+  requestedKey : Expression) : ClassifiedEntries = {
+val res1 = mapEntries.foldLeft(ClassifiedEntries(Seq.empty, nullable = 
false, None)) {
+  case (prev @ ClassifiedEntries(_, _, Some(_)), _) => prev
+  case (ClassifiedEntries(prev, nullable, None), (k, v)) =>
+compareKeys(k, requestedKey) match {
+  case ComparisonResult.UnDetermined =>
+val vIsNullable = v.nullable
+val nextNullbale = nullable || vIsNullable
+ClassifiedEntries(prev ++ Seq(k, v), nullable = nextNullbale, 
None)
+  case ComparisonResult.NegativeMatch => ClassifiedEntries(prev, 
nullable, None)
+  case ComparisonResult.PositiveMatch => ClassifiedEntries(prev, 
nullable, Some(v))
+}
+}
+

[GitHub] spark pull request #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2017-01-27 Thread hvanhovell
Github user hvanhovell commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r98027355
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala
 ---
@@ -0,0 +1,131 @@
+/*
+ * 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.expressions.{Cast, Coalesce, 
CreateArray, CreateMap, CreateNamedStructLike, Expression, GetArrayItem, 
GetArrayStructFields, GetMapValue, GetStructField, IntegerLiteral, Literal}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+* push down operations into [[CreateNamedStructLike]].
+*/
+object SimplifyCreateStructOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // push down field extraction
+  case GetStructField(createNamedStructLike : CreateNamedStructLike, 
ordinal, _) =>
+createNamedStructLike.valExprs(ordinal)
+}
+  }
+}
+
+/**
+* push down operations into [[CreateArray]].
+*/
+object SimplifyCreateArrayOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // push down field selection (array of structs)
+  case GetArrayStructFields(CreateArray(elems), field, ordinal, 
numFields, containsNull) =>
+def getStructField(elem : Expression) = {
+  GetStructField(elem, ordinal, Some(field.name))
+}
+CreateArray(elems.map(getStructField))
+  // push down item selection.
+  case ga @ GetArrayItem(CreateArray(elems), IntegerLiteral(idx)) =>
+if (idx >= 0 && idx < elems.size) {
+  elems(idx)
+} else {
+  Cast(Literal(null), ga.dataType)
+}
+}
+  }
+}
+
+/**
+* push down operations into [[CreateMap]].
+*/
+object SimplifyCreateMapOps extends Rule[LogicalPlan]{
+  object ComparisonResult extends Enumeration {
+val PositiveMatch = Value
+val NegativeMatch = Value
+val UnDetermined = Value
+  }
+
+  def compareKeys(k1 : Expression, k2 : Expression) : 
ComparisonResult.Value = {
+(k1, k2) match {
+  case (x, y) if x.semanticEquals(y) => ComparisonResult.PositiveMatch
+  // make surethis is null safe, especially when datatypes differ
+  // is this even possible?
+  case (_ : Literal, _ : Literal) => ComparisonResult.NegativeMatch
+  case _ => ComparisonResult.UnDetermined
+}
+  }
+
+  case class ClassifiedEntries(undetermined : Seq[Expression],
+   nullable : Boolean,
+   firstPositive : Option[Expression]) {
+def normalize( k : Expression ) : ClassifiedEntries = this match {
+  /**
+  * when we have undetermined matches that might bproduce a null value,
+  * we can't separate a positive match and use [[Coalesce]] to choose 
the final result.
+  * so we 'hide' the positive match as an undetermined match.
+  */
+  case ClassifiedEntries( u, true, Some(p)) if u.nonEmpty =>
+ClassifiedEntries(u ++ Seq(k, p), true, None)
+  case _ => this
+}
+  }
+
+  def classifyEntries(mapEntries : Seq[(Expression, Expression)],
+  requestedKey : Expression) : ClassifiedEntries = {
+val res1 = mapEntries.foldLeft(ClassifiedEntries(Seq.empty, nullable = 
false, None)) {
+  case (prev @ ClassifiedEntries(_, _, Some(_)), _) => prev
+  case (ClassifiedEntries(prev, nullable, None), (k, v)) =>
+compareKeys(k, requestedKey) match {
+  case ComparisonResult.UnDetermined =>
+val vIsNullable = v.nullable
+val nextNullbale = nullable || vIsNullable
+

[GitHub] spark pull request #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2017-01-27 Thread hvanhovell
Github user hvanhovell commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r98026974
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala
 ---
@@ -0,0 +1,128 @@
+/*
+ * 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.expressions._
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+* push down operations into [[CreateNamedStructLike]].
+*/
+object SimplifyCreateStructOps extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp {
+  // push down field extraction
+  case GetStructField(createNamedStructLike : CreateNamedStructLike, 
ordinal, _) =>
--- End diff --

This happens a lot in your code please fix them.


---
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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2017-01-27 Thread hvanhovell
Github user hvanhovell commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r98183051
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala
 ---
@@ -0,0 +1,128 @@
+/*
+ * 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.expressions._
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+* push down operations into [[CreateNamedStructLike]].
+*/
+object SimplifyCreateStructOps extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp {
+  // push down field extraction
+  case GetStructField(createNamedStructLike: CreateNamedStructLike, 
ordinal, _) =>
+createNamedStructLike.valExprs(ordinal)
+}
+  }
+}
+
+/**
+* push down operations into [[CreateArray]].
+*/
+object SimplifyCreateArrayOps extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp {
+  // push down field selection (array of structs)
+  case GetArrayStructFields(CreateArray(elems), field, ordinal, 
numFields, containsNull) =>
+CreateArray(elems.map(GetStructField(_, ordinal, 
Some(field.name
+  // push down item selection.
+  case ga @ GetArrayItem(CreateArray(elems), IntegerLiteral(idx)) =>
+if (idx >= 0 && idx < elems.size) {
+  elems(idx)
+} else {
+  Cast(Literal(null), ga.dataType)
+}
+}
+  }
+}
+
+/**
+* push down operations into [[CreateMap]].
+*/
+object SimplifyCreateMapOps extends Rule[LogicalPlan] {
+  object ComparisonResult extends Enumeration {
+val PositiveMatch = Value
+val NegativeMatch = Value
+val UnDetermined = Value
+  }
+
+  def compareKeys(k1 : Expression, k2 : Expression) : 
ComparisonResult.Value = {
+(k1, k2) match {
+  case (x, y) if x.semanticEquals(y) => ComparisonResult.PositiveMatch
+  // make surethis is null safe, especially when datatypes differ
+  // is this even possible?
+  case (_ : Literal, _ : Literal) => ComparisonResult.NegativeMatch
+  case _ => ComparisonResult.UnDetermined
+}
+  }
+
+  case class ClassifiedEntries(
+undetermined : Seq[Expression],
+nullable : Boolean,
+firstPositive : Option[Expression]) {
+def normalize(k : Expression) : ClassifiedEntries = this match {
+  /**
+  * when we have undetermined matches that might bproduce a null value,
--- End diff --

Language `bproduce`.

Also make sure the comment is properly aligned.


---
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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2017-01-27 Thread hvanhovell
Github user hvanhovell commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r98027807
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/complexTypesSuite.scala
 ---
@@ -0,0 +1,482 @@
+/*
+ * 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.scalatest.Matchers
+
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.dsl.plans._
+import org.apache.spark.sql.catalyst.expressions.{Coalesce, CreateArray, 
CreateMap, CreateNamedStruct, Expression, GetArrayItem, GetArrayStructFields, 
GetMapValue, GetStructField, Literal}
+import org.apache.spark.sql.catalyst.plans.PlanTest
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.Range
+import org.apache.spark.sql.catalyst.rules.RuleExecutor
+import org.apache.spark.sql.types._
+
+/**
+* Created by eyalf on 11/4/2016.
+* SPARK-18601 discusses simplification direct access to complex types 
creators.
+* i.e. {{{create_named_struct(square, `x` * `x`).square}}} can be 
simplified to {{{`x` * `x`}}}.
+* sam applies to create_array and create_map
+*/
+class ComplexTypesSuite extends PlanTest with Matchers{
+
+  object Optimize extends RuleExecutor[LogicalPlan] {
+val batches =
+  Batch("collapse projections", FixedPoint(10),
+  CollapseProject) ::
+  Batch("Constant Folding", FixedPoint(10),
+  NullPropagation,
+  ConstantFolding,
+  BooleanSimplification,
+  SimplifyConditionals,
+  SimplifyCreateStructOps,
+  SimplifyCreateArrayOps,
+  SimplifyCreateMapOps) :: Nil
+  }
+
+  val idAtt = ('id).long.notNull
+
+  lazy val baseOptimizedPlan = Range(1L, 1000L, 1, Some(2), idAtt :: Nil)
+
+  val idRef = baseOptimizedPlan.output.head
+
+
+//  val idRefColumn = Column("id")
+//  val struct1RefColumn = Column("struct1")
+
+  implicit class ComplexTypeDslSupport(e : Expression) {
+def getStructField(f : String): GetStructField = {
+  e should be ('resolved)
+  e.dataType should be (a[StructType])
--- End diff --

It really is not, consistency is key here. We have a few thousand tests 
that are written in a different style. This makes it hard for people like me to 
review 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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2017-01-27 Thread hvanhovell
Github user hvanhovell commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r98027489
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/complexTypesSuite.scala
 ---
@@ -0,0 +1,482 @@
+/*
+ * 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.scalatest.Matchers
+
+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.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.Range
+import org.apache.spark.sql.catalyst.rules.RuleExecutor
+import org.apache.spark.sql.types._
+
+/**
+* Created by eyalf on 11/4/2016.
--- End diff --

Remove this line. Also fix the header indentation.


---
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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2017-01-27 Thread hvanhovell
Github user hvanhovell commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r98182938
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala
 ---
@@ -0,0 +1,128 @@
+/*
+ * 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.expressions._
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+* push down operations into [[CreateNamedStructLike]].
+*/
+object SimplifyCreateStructOps extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp {
+  // push down field extraction
+  case GetStructField(createNamedStructLike: CreateNamedStructLike, 
ordinal, _) =>
+createNamedStructLike.valExprs(ordinal)
+}
+  }
+}
+
+/**
+* push down operations into [[CreateArray]].
+*/
+object SimplifyCreateArrayOps extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp {
+  // push down field selection (array of structs)
+  case GetArrayStructFields(CreateArray(elems), field, ordinal, 
numFields, containsNull) =>
+CreateArray(elems.map(GetStructField(_, ordinal, 
Some(field.name
+  // push down item selection.
+  case ga @ GetArrayItem(CreateArray(elems), IntegerLiteral(idx)) =>
+if (idx >= 0 && idx < elems.size) {
+  elems(idx)
+} else {
+  Cast(Literal(null), ga.dataType)
+}
+}
+  }
+}
+
+/**
+* push down operations into [[CreateMap]].
+*/
+object SimplifyCreateMapOps extends Rule[LogicalPlan] {
+  object ComparisonResult extends Enumeration {
+val PositiveMatch = Value
+val NegativeMatch = Value
+val UnDetermined = Value
+  }
+
+  def compareKeys(k1 : Expression, k2 : Expression) : 
ComparisonResult.Value = {
+(k1, k2) match {
+  case (x, y) if x.semanticEquals(y) => ComparisonResult.PositiveMatch
+  // make surethis is null safe, especially when datatypes differ
+  // is this even possible?
+  case (_ : Literal, _ : Literal) => ComparisonResult.NegativeMatch
+  case _ => ComparisonResult.UnDetermined
+}
+  }
+
+  case class ClassifiedEntries(
+undetermined : Seq[Expression],
--- End diff --

Style: 4 space and not space before the colon


---
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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2017-01-27 Thread hvanhovell
Github user hvanhovell commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r98183279
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala
 ---
@@ -0,0 +1,128 @@
+/*
+ * 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.expressions._
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+* push down operations into [[CreateNamedStructLike]].
+*/
+object SimplifyCreateStructOps extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp {
+  // push down field extraction
+  case GetStructField(createNamedStructLike: CreateNamedStructLike, 
ordinal, _) =>
+createNamedStructLike.valExprs(ordinal)
+}
+  }
+}
+
+/**
+* push down operations into [[CreateArray]].
+*/
+object SimplifyCreateArrayOps extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp {
+  // push down field selection (array of structs)
+  case GetArrayStructFields(CreateArray(elems), field, ordinal, 
numFields, containsNull) =>
+CreateArray(elems.map(GetStructField(_, ordinal, 
Some(field.name
+  // push down item selection.
+  case ga @ GetArrayItem(CreateArray(elems), IntegerLiteral(idx)) =>
+if (idx >= 0 && idx < elems.size) {
+  elems(idx)
+} else {
+  Cast(Literal(null), ga.dataType)
+}
+}
+  }
+}
+
+/**
+* push down operations into [[CreateMap]].
--- End diff --

This needs a lot more documentation. Your approach is far from trivial, and 
an unsuspecting reader needs some help here.

Also align comment.


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

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



[GitHub] spark pull request #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2017-01-27 Thread hvanhovell
Github user hvanhovell commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r98045247
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala
 ---
@@ -0,0 +1,128 @@
+/*
+ * 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.expressions._
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+* push down operations into [[CreateNamedStructLike]].
+*/
+object SimplifyCreateStructOps extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp {
+  // push down field extraction
+  case GetStructField(createNamedStructLike : CreateNamedStructLike, 
ordinal, _) =>
+createNamedStructLike.valExprs(ordinal)
+}
+  }
+}
+
+/**
+* push down operations into [[CreateArray]].
+*/
+object SimplifyCreateArrayOps extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp {
+  // push down field selection (array of structs)
+  case GetArrayStructFields(CreateArray(elems), field, ordinal, 
numFields, containsNull) =>
+CreateArray(elems.map(GetStructField(_, ordinal, 
Some(field.name
+  // push down item selection.
+  case ga @ GetArrayItem(CreateArray(elems), IntegerLiteral(idx)) =>
+if (idx >= 0 && idx < elems.size) {
+  elems(idx)
+} else {
+  Cast(Literal(null), ga.dataType)
+}
+}
+  }
+}
+
+/**
+* push down operations into [[CreateMap]].
+*/
+object SimplifyCreateMapOps extends Rule[LogicalPlan] {
+  object ComparisonResult extends Enumeration {
+val PositiveMatch = Value
+val NegativeMatch = Value
+val UnDetermined = Value
+  }
+
+  def compareKeys(k1 : Expression, k2 : Expression) : 
ComparisonResult.Value = {
+(k1, k2) match {
+  case (x, y) if x.semanticEquals(y) => ComparisonResult.PositiveMatch
+  // make surethis is null safe, especially when datatypes differ
--- End diff --

Why would null be an issue?


---
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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2017-01-27 Thread hvanhovell
Github user hvanhovell commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r98186441
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala
 ---
@@ -0,0 +1,128 @@
+/*
+ * 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.expressions._
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+* push down operations into [[CreateNamedStructLike]].
+*/
+object SimplifyCreateStructOps extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp {
+  // push down field extraction
+  case GetStructField(createNamedStructLike: CreateNamedStructLike, 
ordinal, _) =>
+createNamedStructLike.valExprs(ordinal)
+}
+  }
+}
+
+/**
+* push down operations into [[CreateArray]].
+*/
+object SimplifyCreateArrayOps extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp {
+  // push down field selection (array of structs)
+  case GetArrayStructFields(CreateArray(elems), field, ordinal, 
numFields, containsNull) =>
+CreateArray(elems.map(GetStructField(_, ordinal, 
Some(field.name
+  // push down item selection.
+  case ga @ GetArrayItem(CreateArray(elems), IntegerLiteral(idx)) =>
+if (idx >= 0 && idx < elems.size) {
+  elems(idx)
+} else {
+  Cast(Literal(null), ga.dataType)
+}
+}
+  }
+}
+
+/**
+* push down operations into [[CreateMap]].
+*/
+object SimplifyCreateMapOps extends Rule[LogicalPlan] {
+  object ComparisonResult extends Enumeration {
+val PositiveMatch = Value
+val NegativeMatch = Value
+val UnDetermined = Value
+  }
+
+  def compareKeys(k1 : Expression, k2 : Expression) : 
ComparisonResult.Value = {
+(k1, k2) match {
+  case (x, y) if x.semanticEquals(y) => ComparisonResult.PositiveMatch
+  // make surethis is null safe, especially when datatypes differ
+  // is this even possible?
+  case (_ : Literal, _ : Literal) => ComparisonResult.NegativeMatch
+  case _ => ComparisonResult.UnDetermined
+}
+  }
+
+  case class ClassifiedEntries(
+undetermined : Seq[Expression],
+nullable : Boolean,
+firstPositive : Option[Expression]) {
+def normalize(k : Expression) : ClassifiedEntries = this match {
+  /**
+  * when we have undetermined matches that might bproduce a null value,
+  * we can't separate a positive match and use [[Coalesce]] to choose 
the final result.
+  * so we 'hide' the positive match as an undetermined match.
+  */
+  case ClassifiedEntries(u, true, Some(p)) if u.nonEmpty =>
+ClassifiedEntries(u ++ Seq(k, p), true, None)
+  case _ => this
+}
+  }
+
+  def classifyEntries(mapEntries : Seq[(Expression, Expression)],
+  requestedKey : Expression) : ClassifiedEntries = {
+val res1 = mapEntries.foldLeft(ClassifiedEntries(Seq.empty, nullable = 
false, None)) {
+  case (prev @ ClassifiedEntries(_, _, Some(_)), _) => prev
+  case (ClassifiedEntries(prev, nullable, None), (k, v)) =>
+compareKeys(k, requestedKey) match {
+  case ComparisonResult.UnDetermined =>
+val vIsNullable = v.nullable
+val nextNullbale = nullable || vIsNullable
+ClassifiedEntries(prev ++ Seq(k, v), nullable = nextNullbale, 
None)
+  case ComparisonResult.NegativeMatch => ClassifiedEntries(prev, 
nullable, None)
+  case ComparisonResult.PositiveMatch => ClassifiedEntries(prev, 
nullable, Some(v))
+}
+}
+

[GitHub] spark pull request #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2017-01-26 Thread hvanhovell
Github user hvanhovell commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r98026636
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala
 ---
@@ -0,0 +1,128 @@
+/*
+ * 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.expressions._
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+* push down operations into [[CreateNamedStructLike]].
+*/
+object SimplifyCreateStructOps extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp {
+  // push down field extraction
+  case GetStructField(createNamedStructLike : CreateNamedStructLike, 
ordinal, _) =>
--- End diff --

NIT: style, no space before the colon


---
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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2016-12-08 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r91625006
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/complexTypesSuite.scala
 ---
@@ -0,0 +1,482 @@
+/*
+ * 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.scalatest.Matchers
+
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.dsl.plans._
+import org.apache.spark.sql.catalyst.expressions.{Coalesce, CreateArray, 
CreateMap, CreateNamedStruct, Expression, GetArrayItem, GetArrayStructFields, 
GetMapValue, GetStructField, Literal}
+import org.apache.spark.sql.catalyst.plans.PlanTest
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.Range
+import org.apache.spark.sql.catalyst.rules.RuleExecutor
+import org.apache.spark.sql.types._
+
+/**
+* Created by eyalf on 11/4/2016.
+* SPARK-18601 discusses simplification direct access to complex types 
creators.
+* i.e. {{{create_named_struct(square, `x` * `x`).square}}} can be 
simplified to {{{`x` * `x`}}}.
+* sam applies to create_array and create_map
+*/
+class ComplexTypesSuite extends PlanTest with Matchers{
+
+  object Optimize extends RuleExecutor[LogicalPlan] {
+val batches =
+  Batch("collapse projections", FixedPoint(10),
+  CollapseProject) ::
+  Batch("Constant Folding", FixedPoint(10),
+  NullPropagation,
+  ConstantFolding,
+  BooleanSimplification,
+  SimplifyConditionals,
+  SimplifyCreateStructOps,
+  SimplifyCreateArrayOps,
+  SimplifyCreateMapOps) :: Nil
+  }
+
+  val idAtt = ('id).long.notNull
+
+  lazy val baseOptimizedPlan = Range(1L, 1000L, 1, Some(2), idAtt :: Nil)
+
+  val idRef = baseOptimizedPlan.output.head
+
+
+//  val idRefColumn = Column("id")
+//  val struct1RefColumn = Column("struct1")
+
+  implicit class ComplexTypeDslSupport(e : Expression) {
+def getStructField(f : String): GetStructField = {
+  e should be ('resolved)
+  e.dataType should be (a[StructType])
--- End diff --

Actually I am not confident of this. If we don't have a clear reason to 
force the style, I am fine as is. I think I am not supposed to decide this 
though.


---
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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2016-12-08 Thread eyalfa
Github user eyalfa commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r91596315
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/complexTypesSuite.scala
 ---
@@ -0,0 +1,482 @@
+/*
+ * 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.scalatest.Matchers
+
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.dsl.plans._
+import org.apache.spark.sql.catalyst.expressions.{Coalesce, CreateArray, 
CreateMap, CreateNamedStruct, Expression, GetArrayItem, GetArrayStructFields, 
GetMapValue, GetStructField, Literal}
+import org.apache.spark.sql.catalyst.plans.PlanTest
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.Range
+import org.apache.spark.sql.catalyst.rules.RuleExecutor
+import org.apache.spark.sql.types._
+
+/**
+* Created by eyalf on 11/4/2016.
+* SPARK-18601 discusses simplification direct access to complex types 
creators.
+* i.e. {{{create_named_struct(square, `x` * `x`).square}}} can be 
simplified to {{{`x` * `x`}}}.
+* sam applies to create_array and create_map
+*/
+class ComplexTypesSuite extends PlanTest with Matchers{
+
+  object Optimize extends RuleExecutor[LogicalPlan] {
+val batches =
+  Batch("collapse projections", FixedPoint(10),
+  CollapseProject) ::
+  Batch("Constant Folding", FixedPoint(10),
+  NullPropagation,
+  ConstantFolding,
+  BooleanSimplification,
+  SimplifyConditionals,
+  SimplifyCreateStructOps,
+  SimplifyCreateArrayOps,
+  SimplifyCreateMapOps) :: Nil
+  }
+
+  val idAtt = ('id).long.notNull
+
+  lazy val baseOptimizedPlan = Range(1L, 1000L, 1, Some(2), idAtt :: Nil)
+
+  val idRef = baseOptimizedPlan.output.head
+
+
+//  val idRefColumn = Column("id")
+//  val struct1RefColumn = Column("struct1")
+
+  implicit class ComplexTypeDslSupport(e : Expression) {
+def getStructField(f : String): GetStructField = {
+  e should be ('resolved)
+  e.dataType should be (a[StructType])
--- End diff --

@HyukjinKwon , @hvanhovell ,
is the Matchers syntax acceptable or not? I personally like it, but it's up 
to project's policies


---
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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2016-12-08 Thread eyalfa
Github user eyalfa commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r91595458
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala
 ---
@@ -0,0 +1,131 @@
+/*
+ * 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.expressions.{Cast, Coalesce, 
CreateArray, CreateMap, CreateNamedStructLike, Expression, GetArrayItem, 
GetArrayStructFields, GetMapValue, GetStructField, IntegerLiteral, Literal}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+* push down operations into [[CreateNamedStructLike]].
+*/
+object SimplifyCreateStructOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // push down field extraction
+  case GetStructField(createNamedStructLike : CreateNamedStructLike, 
ordinal, _) =>
+createNamedStructLike.valExprs(ordinal)
+}
+  }
+}
+
+/**
+* push down operations into [[CreateArray]].
+*/
+object SimplifyCreateArrayOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // push down field selection (array of structs)
+  case GetArrayStructFields(CreateArray(elems), field, ordinal, 
numFields, containsNull) =>
+def getStructField(elem : Expression) = {
+  GetStructField(elem, ordinal, Some(field.name))
+}
+CreateArray(elems.map(getStructField))
+  // push down item selection.
+  case ga @ GetArrayItem(CreateArray(elems), IntegerLiteral(idx)) =>
+if (idx >= 0 && idx < elems.size) {
+  elems(idx)
+} else {
+  Cast(Literal(null), ga.dataType)
+}
+}
+  }
+}
+
+/**
+* push down operations into [[CreateMap]].
+*/
+object SimplifyCreateMapOps extends Rule[LogicalPlan]{
+  object ComparisonResult extends Enumeration {
+val PositiveMatch = Value
+val NegativeMatch = Value
+val UnDetermined = Value
+  }
+
+  def compareKeys(k1 : Expression, k2 : Expression) : 
ComparisonResult.Value = {
+(k1, k2) match {
+  case (x, y) if x.semanticEquals(y) => ComparisonResult.PositiveMatch
+  // make surethis is null safe, especially when datatypes differ
+  // is this even possible?
+  case (_ : Literal, _ : Literal) => ComparisonResult.NegativeMatch
+  case _ => ComparisonResult.UnDetermined
+}
+  }
+
+  case class ClassifiedEntries(undetermined : Seq[Expression],
+   nullable : Boolean,
+   firstPositive : Option[Expression]) {
+def normalize( k : Expression ) : ClassifiedEntries = this match {
+  /**
+  * when we have undetermined matches that might bproduce a null value,
+  * we can't separate a positive match and use [[Coalesce]] to choose 
the final result.
+  * so we 'hide' the positive match as an undetermined match.
+  */
+  case ClassifiedEntries( u, true, Some(p)) if u.nonEmpty =>
+ClassifiedEntries(u ++ Seq(k, p), true, None)
+  case _ => this
+}
+  }
+
+  def classifyEntries(mapEntries : Seq[(Expression, Expression)],
+  requestedKey : Expression) : ClassifiedEntries = {
+val res1 = mapEntries.foldLeft(ClassifiedEntries(Seq.empty, nullable = 
false, None)) {
+  case (prev @ ClassifiedEntries(_, _, Some(_)), _) => prev
+  case (ClassifiedEntries(prev, nullable, None), (k, v)) =>
+compareKeys(k, requestedKey) match {
+  case ComparisonResult.UnDetermined =>
+val vIsNullable = v.nullable
+val nextNullbale = nullable || vIsNullable
+

[GitHub] spark pull request #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2016-12-07 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r91453918
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala
 ---
@@ -0,0 +1,131 @@
+/*
+ * 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.expressions.{Cast, Coalesce, 
CreateArray, CreateMap, CreateNamedStructLike, Expression, GetArrayItem, 
GetArrayStructFields, GetMapValue, GetStructField, IntegerLiteral, Literal}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+* push down operations into [[CreateNamedStructLike]].
+*/
+object SimplifyCreateStructOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // push down field extraction
+  case GetStructField(createNamedStructLike : CreateNamedStructLike, 
ordinal, _) =>
+createNamedStructLike.valExprs(ordinal)
+}
+  }
+}
+
+/**
+* push down operations into [[CreateArray]].
+*/
+object SimplifyCreateArrayOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // push down field selection (array of structs)
+  case GetArrayStructFields(CreateArray(elems), field, ordinal, 
numFields, containsNull) =>
+def getStructField(elem : Expression) = {
+  GetStructField(elem, ordinal, Some(field.name))
+}
+CreateArray(elems.map(getStructField))
--- End diff --

Could we do this like something as below?:

```scala
CreateArray(elems.map(elem => GetStructField(elem, ordinal, 
Some(field.name
```

It seems `getStructField(...)` is only used in this scope and I think it is 
good to remove 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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2016-12-07 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r91454208
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala
 ---
@@ -0,0 +1,131 @@
+/*
+ * 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.expressions.{Cast, Coalesce, 
CreateArray, CreateMap, CreateNamedStructLike, Expression, GetArrayItem, 
GetArrayStructFields, GetMapValue, GetStructField, IntegerLiteral, Literal}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+* push down operations into [[CreateNamedStructLike]].
+*/
+object SimplifyCreateStructOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // push down field extraction
+  case GetStructField(createNamedStructLike : CreateNamedStructLike, 
ordinal, _) =>
+createNamedStructLike.valExprs(ordinal)
+}
+  }
+}
+
+/**
+* push down operations into [[CreateArray]].
+*/
+object SimplifyCreateArrayOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // push down field selection (array of structs)
+  case GetArrayStructFields(CreateArray(elems), field, ordinal, 
numFields, containsNull) =>
+def getStructField(elem : Expression) = {
+  GetStructField(elem, ordinal, Some(field.name))
+}
+CreateArray(elems.map(getStructField))
+  // push down item selection.
+  case ga @ GetArrayItem(CreateArray(elems), IntegerLiteral(idx)) =>
+if (idx >= 0 && idx < elems.size) {
+  elems(idx)
+} else {
+  Cast(Literal(null), ga.dataType)
+}
+}
+  }
+}
+
+/**
+* push down operations into [[CreateMap]].
+*/
+object SimplifyCreateMapOps extends Rule[LogicalPlan]{
+  object ComparisonResult extends Enumeration {
+val PositiveMatch = Value
+val NegativeMatch = Value
+val UnDetermined = Value
+  }
+
+  def compareKeys(k1 : Expression, k2 : Expression) : 
ComparisonResult.Value = {
+(k1, k2) match {
+  case (x, y) if x.semanticEquals(y) => ComparisonResult.PositiveMatch
+  // make surethis is null safe, especially when datatypes differ
+  // is this even possible?
+  case (_ : Literal, _ : Literal) => ComparisonResult.NegativeMatch
+  case _ => ComparisonResult.UnDetermined
+}
+  }
+
+  case class ClassifiedEntries(undetermined : Seq[Expression],
+   nullable : Boolean,
+   firstPositive : Option[Expression]) {
--- End diff --

Oh @eyalfa, I believe we should make the indentation as below if it does 
not fit in 100 character length:

```scala
case class ClassifiedEntries(
undetermined : Seq[Expression],
nullable : Boolean,
firstPositive : Option[Expression]) {
...
```


---
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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2016-12-07 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r91455499
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala
 ---
@@ -0,0 +1,131 @@
+/*
+ * 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.expressions.{Cast, Coalesce, 
CreateArray, CreateMap, CreateNamedStructLike, Expression, GetArrayItem, 
GetArrayStructFields, GetMapValue, GetStructField, IntegerLiteral, Literal}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+* push down operations into [[CreateNamedStructLike]].
+*/
+object SimplifyCreateStructOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // push down field extraction
+  case GetStructField(createNamedStructLike : CreateNamedStructLike, 
ordinal, _) =>
+createNamedStructLike.valExprs(ordinal)
+}
+  }
+}
+
+/**
+* push down operations into [[CreateArray]].
+*/
+object SimplifyCreateArrayOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // push down field selection (array of structs)
+  case GetArrayStructFields(CreateArray(elems), field, ordinal, 
numFields, containsNull) =>
+def getStructField(elem : Expression) = {
+  GetStructField(elem, ordinal, Some(field.name))
+}
+CreateArray(elems.map(getStructField))
+  // push down item selection.
+  case ga @ GetArrayItem(CreateArray(elems), IntegerLiteral(idx)) =>
+if (idx >= 0 && idx < elems.size) {
+  elems(idx)
+} else {
+  Cast(Literal(null), ga.dataType)
+}
+}
+  }
+}
+
+/**
+* push down operations into [[CreateMap]].
+*/
+object SimplifyCreateMapOps extends Rule[LogicalPlan]{
+  object ComparisonResult extends Enumeration {
+val PositiveMatch = Value
+val NegativeMatch = Value
+val UnDetermined = Value
+  }
+
+  def compareKeys(k1 : Expression, k2 : Expression) : 
ComparisonResult.Value = {
+(k1, k2) match {
+  case (x, y) if x.semanticEquals(y) => ComparisonResult.PositiveMatch
+  // make surethis is null safe, especially when datatypes differ
+  // is this even possible?
+  case (_ : Literal, _ : Literal) => ComparisonResult.NegativeMatch
+  case _ => ComparisonResult.UnDetermined
+}
+  }
+
+  case class ClassifiedEntries(undetermined : Seq[Expression],
+   nullable : Boolean,
+   firstPositive : Option[Expression]) {
+def normalize( k : Expression ) : ClassifiedEntries = this match {
+  /**
+  * when we have undetermined matches that might bproduce a null value,
+  * we can't separate a positive match and use [[Coalesce]] to choose 
the final result.
+  * so we 'hide' the positive match as an undetermined match.
+  */
+  case ClassifiedEntries( u, true, Some(p)) if u.nonEmpty =>
+ClassifiedEntries(u ++ Seq(k, p), true, None)
+  case _ => this
+}
+  }
+
+  def classifyEntries(mapEntries : Seq[(Expression, Expression)],
+  requestedKey : Expression) : ClassifiedEntries = {
+val res1 = mapEntries.foldLeft(ClassifiedEntries(Seq.empty, nullable = 
false, None)) {
+  case (prev @ ClassifiedEntries(_, _, Some(_)), _) => prev
+  case (ClassifiedEntries(prev, nullable, None), (k, v)) =>
+compareKeys(k, requestedKey) match {
+  case ComparisonResult.UnDetermined =>
+val vIsNullable = v.nullable
+val nextNullbale = nullable || vIsNullable
+   

[GitHub] spark pull request #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2016-12-07 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r91455356
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala
 ---
@@ -0,0 +1,131 @@
+/*
+ * 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.expressions.{Cast, Coalesce, 
CreateArray, CreateMap, CreateNamedStructLike, Expression, GetArrayItem, 
GetArrayStructFields, GetMapValue, GetStructField, IntegerLiteral, Literal}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+* push down operations into [[CreateNamedStructLike]].
+*/
+object SimplifyCreateStructOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // push down field extraction
+  case GetStructField(createNamedStructLike : CreateNamedStructLike, 
ordinal, _) =>
+createNamedStructLike.valExprs(ordinal)
+}
+  }
+}
+
+/**
+* push down operations into [[CreateArray]].
+*/
+object SimplifyCreateArrayOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // push down field selection (array of structs)
+  case GetArrayStructFields(CreateArray(elems), field, ordinal, 
numFields, containsNull) =>
+def getStructField(elem : Expression) = {
+  GetStructField(elem, ordinal, Some(field.name))
+}
+CreateArray(elems.map(getStructField))
+  // push down item selection.
+  case ga @ GetArrayItem(CreateArray(elems), IntegerLiteral(idx)) =>
+if (idx >= 0 && idx < elems.size) {
+  elems(idx)
+} else {
+  Cast(Literal(null), ga.dataType)
+}
+}
+  }
+}
+
+/**
+* push down operations into [[CreateMap]].
+*/
+object SimplifyCreateMapOps extends Rule[LogicalPlan]{
+  object ComparisonResult extends Enumeration {
+val PositiveMatch = Value
+val NegativeMatch = Value
+val UnDetermined = Value
+  }
+
+  def compareKeys(k1 : Expression, k2 : Expression) : 
ComparisonResult.Value = {
+(k1, k2) match {
+  case (x, y) if x.semanticEquals(y) => ComparisonResult.PositiveMatch
+  // make surethis is null safe, especially when datatypes differ
+  // is this even possible?
+  case (_ : Literal, _ : Literal) => ComparisonResult.NegativeMatch
+  case _ => ComparisonResult.UnDetermined
+}
+  }
+
+  case class ClassifiedEntries(undetermined : Seq[Expression],
+   nullable : Boolean,
+   firstPositive : Option[Expression]) {
+def normalize( k : Expression ) : ClassifiedEntries = this match {
+  /**
+  * when we have undetermined matches that might bproduce a null value,
+  * we can't separate a positive match and use [[Coalesce]] to choose 
the final result.
+  * so we 'hide' the positive match as an undetermined match.
+  */
+  case ClassifiedEntries( u, true, Some(p)) if u.nonEmpty =>
+ClassifiedEntries(u ++ Seq(k, p), true, None)
+  case _ => this
+}
+  }
+
+  def classifyEntries(mapEntries : Seq[(Expression, Expression)],
+  requestedKey : Expression) : ClassifiedEntries = {
+val res1 = mapEntries.foldLeft(ClassifiedEntries(Seq.empty, nullable = 
false, None)) {
+  case (prev @ ClassifiedEntries(_, _, Some(_)), _) => prev
+  case (ClassifiedEntries(prev, nullable, None), (k, v)) =>
+compareKeys(k, requestedKey) match {
+  case ComparisonResult.UnDetermined =>
+val vIsNullable = v.nullable
+val nextNullbale = nullable || vIsNullable
+   

[GitHub] spark pull request #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2016-12-07 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r91457070
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/complexTypesSuite.scala
 ---
@@ -0,0 +1,482 @@
+/*
+ * 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.scalatest.Matchers
+
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.dsl.plans._
+import org.apache.spark.sql.catalyst.expressions.{Coalesce, CreateArray, 
CreateMap, CreateNamedStruct, Expression, GetArrayItem, GetArrayStructFields, 
GetMapValue, GetStructField, Literal}
+import org.apache.spark.sql.catalyst.plans.PlanTest
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.Range
+import org.apache.spark.sql.catalyst.rules.RuleExecutor
+import org.apache.spark.sql.types._
+
+/**
+* Created by eyalf on 11/4/2016.
+* SPARK-18601 discusses simplification direct access to complex types 
creators.
+* i.e. {{{create_named_struct(square, `x` * `x`).square}}} can be 
simplified to {{{`x` * `x`}}}.
+* sam applies to create_array and create_map
+*/
+class ComplexTypesSuite extends PlanTest with Matchers{
+
+  object Optimize extends RuleExecutor[LogicalPlan] {
+val batches =
+  Batch("collapse projections", FixedPoint(10),
+  CollapseProject) ::
+  Batch("Constant Folding", FixedPoint(10),
+  NullPropagation,
+  ConstantFolding,
+  BooleanSimplification,
+  SimplifyConditionals,
+  SimplifyCreateStructOps,
+  SimplifyCreateArrayOps,
+  SimplifyCreateMapOps) :: Nil
+  }
+
+  val idAtt = ('id).long.notNull
+
+  lazy val baseOptimizedPlan = Range(1L, 1000L, 1, Some(2), idAtt :: Nil)
+
+  val idRef = baseOptimizedPlan.output.head
+
+
+//  val idRefColumn = Column("id")
+//  val struct1RefColumn = Column("struct1")
+
+  implicit class ComplexTypeDslSupport(e : Expression) {
+def getStructField(f : String): GetStructField = {
+  e should be ('resolved)
+  e.dataType should be (a[StructType])
--- End diff --

I guess infix annotation is discouraged according to 
http://spark.apache.org/contributing.html. I see `assert` is being used much 
commonly. This might be acceptable but honestly I have seen `should be` not 
often although I understand there are some usages of this across codebase..


---
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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2016-12-07 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r91453489
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.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.expressions.{Cast, CreateArray, 
CreateMap, CreateNamedStructLike, Expression, GetArrayItem, 
GetArrayStructFields, GetMapValue, GetStructField, IntegerLiteral, Literal}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+* push down operations into [[CreateNamedStructLike]].
+*/
+object SimplifyCreateStructOps extends Rule[LogicalPlan]{
--- End diff --

> `]{ -> ] {`

I believe most of them have this indentation and think this is a good to 
do. It seems there are several same instances for 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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2016-12-07 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r91453426
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala
 ---
@@ -0,0 +1,131 @@
+/*
+ * 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.expressions.{Cast, Coalesce, 
CreateArray, CreateMap, CreateNamedStructLike, Expression, GetArrayItem, 
GetArrayStructFields, GetMapValue, GetStructField, IntegerLiteral, Literal}
--- End diff --

I believe it is nicer if it has a multiple-line import or a wild card one 
as it imports more than 6 ones.


---
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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2016-12-07 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r91453593
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala
 ---
@@ -0,0 +1,131 @@
+/*
+ * 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.expressions.{Cast, Coalesce, 
CreateArray, CreateMap, CreateNamedStructLike, Expression, GetArrayItem, 
GetArrayStructFields, GetMapValue, GetStructField, IntegerLiteral, Literal}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+* push down operations into [[CreateNamedStructLike]].
+*/
+object SimplifyCreateStructOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
--- End diff --

Here too, `plan.transformExpressionsUp {`. I think it is good to follow 
other code styles. It seems there are several same instances.


---
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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2016-12-07 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r91457208
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/complexTypesSuite.scala
 ---
@@ -0,0 +1,482 @@
+/*
+ * 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.scalatest.Matchers
+
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.dsl.plans._
+import org.apache.spark.sql.catalyst.expressions.{Coalesce, CreateArray, 
CreateMap, CreateNamedStruct, Expression, GetArrayItem, GetArrayStructFields, 
GetMapValue, GetStructField, Literal}
+import org.apache.spark.sql.catalyst.plans.PlanTest
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.Range
+import org.apache.spark.sql.catalyst.rules.RuleExecutor
+import org.apache.spark.sql.types._
+
+/**
+* Created by eyalf on 11/4/2016.
+* SPARK-18601 discusses simplification direct access to complex types 
creators.
+* i.e. {{{create_named_struct(square, `x` * `x`).square}}} can be 
simplified to {{{`x` * `x`}}}.
+* sam applies to create_array and create_map
+*/
+class ComplexTypesSuite extends PlanTest with Matchers{
+
+  object Optimize extends RuleExecutor[LogicalPlan] {
+val batches =
+  Batch("collapse projections", FixedPoint(10),
+  CollapseProject) ::
+  Batch("Constant Folding", FixedPoint(10),
+  NullPropagation,
+  ConstantFolding,
+  BooleanSimplification,
+  SimplifyConditionals,
+  SimplifyCreateStructOps,
+  SimplifyCreateArrayOps,
+  SimplifyCreateMapOps) :: Nil
+  }
+
+  val idAtt = ('id).long.notNull
+
+  lazy val baseOptimizedPlan = Range(1L, 1000L, 1, Some(2), idAtt :: Nil)
+
+  val idRef = baseOptimizedPlan.output.head
+
+
+//  val idRefColumn = Column("id")
+//  val struct1RefColumn = Column("struct1")
+
+  implicit class ComplexTypeDslSupport(e : Expression) {
+def getStructField(f : String): GetStructField = {
+  e should be ('resolved)
+  e.dataType should be (a[StructType])
+  val structType = e.dataType.asInstanceOf[StructType]
+  val ord = structType.fieldNames.indexOf(f)
+  ord shouldNot be (-1)
+  GetStructField(e, ord, Some(f))
+}
+def getArrayStructField(f : String) : Expression = {
--- End diff --

I believe we need a single newline between consecutive methods according to 
https://github.com/databricks/scala-style-guide#blank-lines-vertical-whitespace 
and for the same instances 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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2016-12-07 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r9148
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/complexTypesSuite.scala
 ---
@@ -0,0 +1,482 @@
+/*
+ * 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.scalatest.Matchers
+
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.dsl.plans._
+import org.apache.spark.sql.catalyst.expressions.{Coalesce, CreateArray, 
CreateMap, CreateNamedStruct, Expression, GetArrayItem, GetArrayStructFields, 
GetMapValue, GetStructField, Literal}
+import org.apache.spark.sql.catalyst.plans.PlanTest
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.Range
+import org.apache.spark.sql.catalyst.rules.RuleExecutor
+import org.apache.spark.sql.types._
+
+/**
+* Created by eyalf on 11/4/2016.
+* SPARK-18601 discusses simplification direct access to complex types 
creators.
+* i.e. {{{create_named_struct(square, `x` * `x`).square}}} can be 
simplified to {{{`x` * `x`}}}.
+* sam applies to create_array and create_map
+*/
+class ComplexTypesSuite extends PlanTest with Matchers{
+
+  object Optimize extends RuleExecutor[LogicalPlan] {
+val batches =
+  Batch("collapse projections", FixedPoint(10),
+  CollapseProject) ::
+  Batch("Constant Folding", FixedPoint(10),
+  NullPropagation,
+  ConstantFolding,
+  BooleanSimplification,
+  SimplifyConditionals,
+  SimplifyCreateStructOps,
+  SimplifyCreateArrayOps,
+  SimplifyCreateMapOps) :: Nil
+  }
+
+  val idAtt = ('id).long.notNull
+
+  lazy val baseOptimizedPlan = Range(1L, 1000L, 1, Some(2), idAtt :: Nil)
+
+  val idRef = baseOptimizedPlan.output.head
+
+
+//  val idRefColumn = Column("id")
+//  val struct1RefColumn = Column("struct1")
--- End diff --

It seems removing those was missed.


---
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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2016-12-07 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r91454252
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala
 ---
@@ -0,0 +1,131 @@
+/*
+ * 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.expressions.{Cast, Coalesce, 
CreateArray, CreateMap, CreateNamedStructLike, Expression, GetArrayItem, 
GetArrayStructFields, GetMapValue, GetStructField, IntegerLiteral, Literal}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+* push down operations into [[CreateNamedStructLike]].
+*/
+object SimplifyCreateStructOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // push down field extraction
+  case GetStructField(createNamedStructLike : CreateNamedStructLike, 
ordinal, _) =>
+createNamedStructLike.valExprs(ordinal)
+}
+  }
+}
+
+/**
+* push down operations into [[CreateArray]].
+*/
+object SimplifyCreateArrayOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // push down field selection (array of structs)
+  case GetArrayStructFields(CreateArray(elems), field, ordinal, 
numFields, containsNull) =>
+def getStructField(elem : Expression) = {
+  GetStructField(elem, ordinal, Some(field.name))
+}
+CreateArray(elems.map(getStructField))
+  // push down item selection.
+  case ga @ GetArrayItem(CreateArray(elems), IntegerLiteral(idx)) =>
+if (idx >= 0 && idx < elems.size) {
+  elems(idx)
+} else {
+  Cast(Literal(null), ga.dataType)
+}
+}
+  }
+}
+
+/**
+* push down operations into [[CreateMap]].
+*/
+object SimplifyCreateMapOps extends Rule[LogicalPlan]{
+  object ComparisonResult extends Enumeration {
+val PositiveMatch = Value
+val NegativeMatch = Value
+val UnDetermined = Value
+  }
+
+  def compareKeys(k1 : Expression, k2 : Expression) : 
ComparisonResult.Value = {
+(k1, k2) match {
+  case (x, y) if x.semanticEquals(y) => ComparisonResult.PositiveMatch
+  // make surethis is null safe, especially when datatypes differ
+  // is this even possible?
+  case (_ : Literal, _ : Literal) => ComparisonResult.NegativeMatch
+  case _ => ComparisonResult.UnDetermined
+}
+  }
+
+  case class ClassifiedEntries(undetermined : Seq[Expression],
+   nullable : Boolean,
+   firstPositive : Option[Expression]) {
+def normalize( k : Expression ) : ClassifiedEntries = this match {
--- End diff --

We could remove the extra spaces between braces as `def normalize(k : 
Expression)` and for the same instances.


---
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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2016-12-07 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r91457315
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/complexTypesSuite.scala
 ---
@@ -0,0 +1,482 @@
+/*
+ * 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.scalatest.Matchers
+
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.dsl.plans._
+import org.apache.spark.sql.catalyst.expressions.{Coalesce, CreateArray, 
CreateMap, CreateNamedStruct, Expression, GetArrayItem, GetArrayStructFields, 
GetMapValue, GetStructField, Literal}
+import org.apache.spark.sql.catalyst.plans.PlanTest
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.Range
+import org.apache.spark.sql.catalyst.rules.RuleExecutor
+import org.apache.spark.sql.types._
+
+/**
+* Created by eyalf on 11/4/2016.
+* SPARK-18601 discusses simplification direct access to complex types 
creators.
+* i.e. {{{create_named_struct(square, `x` * `x`).square}}} can be 
simplified to {{{`x` * `x`}}}.
+* sam applies to create_array and create_map
+*/
+class ComplexTypesSuite extends PlanTest with Matchers{
+
+  object Optimize extends RuleExecutor[LogicalPlan] {
+val batches =
+  Batch("collapse projections", FixedPoint(10),
+  CollapseProject) ::
+  Batch("Constant Folding", FixedPoint(10),
+  NullPropagation,
+  ConstantFolding,
+  BooleanSimplification,
+  SimplifyConditionals,
+  SimplifyCreateStructOps,
+  SimplifyCreateArrayOps,
+  SimplifyCreateMapOps) :: Nil
+  }
+
+  val idAtt = ('id).long.notNull
+
+  lazy val baseOptimizedPlan = Range(1L, 1000L, 1, Some(2), idAtt :: Nil)
+
+  val idRef = baseOptimizedPlan.output.head
+
+
+//  val idRefColumn = Column("id")
+//  val struct1RefColumn = Column("struct1")
+
+  implicit class ComplexTypeDslSupport(e : Expression) {
+def getStructField(f : String): GetStructField = {
+  e should be ('resolved)
+  e.dataType should be (a[StructType])
+  val structType = e.dataType.asInstanceOf[StructType]
+  val ord = structType.fieldNames.indexOf(f)
+  ord shouldNot be (-1)
+  GetStructField(e, ord, Some(f))
+}
+def getArrayStructField(f : String) : Expression = {
+  e should be ('resolved)
+  e.dataType should be (a[ArrayType])
+  val arrType = e.dataType.asInstanceOf[ArrayType]
+  arrType.elementType should be (a[StructType])
+  val structType = arrType.elementType.asInstanceOf[StructType]
+  val ord = structType.fieldNames.indexOf(f)
+  ord shouldNot be (-1)
+  GetArrayStructFields(e, structType(ord), ord, 1, 
arrType.containsNull)
+}
+def getArrayItem(i : Int) : GetArrayItem = {
+  e should be ('resolved)
+  e.dataType should be (a[ArrayType])
+  GetArrayItem(e, Literal(i))
+}
+def getMapValue(k : Expression) : Expression = {
+  e should be ('resolved)
+  e.dataType should be (a[MapType])
+  val mapType = e.dataType.asInstanceOf[MapType]
+  k.dataType shouldEqual mapType.keyType
+  GetMapValue(e, k)
+}
+  }
+
+  test("explicit") {
+val rel = baseOptimizedPlan.select(
+  CreateNamedStruct("att" :: idRef :: Nil).getStructField("att") as 
"outerAtt"
+   )
+
+rel.schema shouldEqual
+  StructType(StructField("outerAtt", LongType, nullable = false) :: 
Nil)
+
+val optimized = Optimize execute rel
+
+val expected = baseOptimizedPlan.select(idRef as "outerAtt")
+
+comparePlans(optimized, expected)
+  }
+
+  ignore("explicit - deduced att name") {
+val rel = 

[GitHub] spark pull request #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2016-12-03 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r90757729
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.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.expressions.{Cast, CreateArray, 
CreateMap, CreateNamedStructLike, Expression, GetArrayItem, 
GetArrayStructFields, GetMapValue, GetStructField, IntegerLiteral, Literal}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+* push down operations into [[CreateNamedStructLike]].
+*/
+object SimplifyCreateStructOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // push down field extraction
+  case GetStructField( createNamedStructLike : CreateNamedStructLike, 
ordinal, _ ) =>
+createNamedStructLike.valExprs(ordinal)
+}
+  }
+}
+
+/**
+* push down operations into [[CreateArray]].
+*/
+object SimplifyCreateArrayOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // push down field selection (array of structs)
+  case GetArrayStructFields(CreateArray(elems), field, ordinal, 
numFields, containsNull) =>
+def getStructField( elem : Expression ) = {
+  GetStructField( elem, ordinal, Some(field.name) )
+}
+CreateArray( elems.map(getStructField) )
+  // push down item selection.
+  case ga @ GetArrayItem( CreateArray(elems), IntegerLiteral( idx ) ) 
=>
+if ( idx >= 0 && idx < elems.size ) {
+  elems(idx)
+} else {
+  Cast( Literal( null), ga.dataType )
+}
+}
+  }
+}
+
+/**
+* push down operations into [[CreateMap]].
+*/
+object SimplifyCreateMapOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
--- End diff --

Oh @eyalfa, I understand it might be up to a personal preference if it is 
not documented and there are same instances with this but I believe the space 
between them is more common. Maybe you could leave `[WIP]` in the title in 
order to prevent the review if you are workinh on 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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2016-12-03 Thread eyalfa
Github user eyalfa commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r90752975
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.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.expressions.{Cast, CreateArray, 
CreateMap, CreateNamedStructLike, Expression, GetArrayItem, 
GetArrayStructFields, GetMapValue, GetStructField, IntegerLiteral, Literal}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+* push down operations into [[CreateNamedStructLike]].
+*/
+object SimplifyCreateStructOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // push down field extraction
+  case GetStructField( createNamedStructLike : CreateNamedStructLike, 
ordinal, _ ) =>
+createNamedStructLike.valExprs(ordinal)
+}
+  }
+}
+
+/**
+* push down operations into [[CreateArray]].
+*/
+object SimplifyCreateArrayOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // push down field selection (array of structs)
+  case GetArrayStructFields(CreateArray(elems), field, ordinal, 
numFields, containsNull) =>
+def getStructField( elem : Expression ) = {
+  GetStructField( elem, ordinal, Some(field.name) )
+}
+CreateArray( elems.map(getStructField) )
+  // push down item selection.
+  case ga @ GetArrayItem( CreateArray(elems), IntegerLiteral( idx ) ) 
=>
+if ( idx >= 0 && idx < elems.size ) {
+  elems(idx)
+} else {
+  Cast( Literal( null), ga.dataType )
+}
+}
+  }
+}
+
+/**
+* push down operations into [[CreateMap]].
+*/
+object SimplifyCreateMapOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
--- End diff --

@gatorsmile I've run a small regex on the spark source tree:
`git grep -En '[a-zA-Z][{]' -- *.scala`

this returns 277 places where this space is missing, am I missing anything?


---
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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2016-12-02 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r90719531
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.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.expressions.{Cast, CreateArray, 
CreateMap, CreateNamedStructLike, Expression, GetArrayItem, 
GetArrayStructFields, GetMapValue, GetStructField, IntegerLiteral, Literal}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+* push down operations into [[CreateNamedStructLike]].
+*/
+object SimplifyCreateStructOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // push down field extraction
+  case GetStructField( createNamedStructLike : CreateNamedStructLike, 
ordinal, _ ) =>
+createNamedStructLike.valExprs(ordinal)
+}
+  }
+}
+
+/**
+* push down operations into [[CreateArray]].
+*/
+object SimplifyCreateArrayOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // push down field selection (array of structs)
+  case GetArrayStructFields(CreateArray(elems), field, ordinal, 
numFields, containsNull) =>
+def getStructField( elem : Expression ) = {
+  GetStructField( elem, ordinal, Some(field.name) )
+}
+CreateArray( elems.map(getStructField) )
+  // push down item selection.
+  case ga @ GetArrayItem( CreateArray(elems), IntegerLiteral( idx ) ) 
=>
+if ( idx >= 0 && idx < elems.size ) {
+  elems(idx)
+} else {
+  Cast( Literal( null), ga.dataType )
+}
+}
+  }
+}
+
+/**
+* push down operations into [[CreateMap]].
+*/
+object SimplifyCreateMapOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // attempt to unfold 'constant' key extraction,
+  // this enables other optimizations to take place.
+  case gmv @ GetMapValue(cm @ CreateMap(elems), key @ Literal(v, t)) =>
+val idx = cm.keys.indexOf( key )
+if( -1 != idx ) {
+  cm.values(idx)
+} else {
+  Literal.create(null, gmv.dataType)
+}
--- End diff --

->
`if (-1 != idx) cm.values(idx) else Literal.create(null, gmv.dataType)`


---
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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2016-12-02 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r90719830
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.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.expressions.{Cast, CreateArray, 
CreateMap, CreateNamedStructLike, Expression, GetArrayItem, 
GetArrayStructFields, GetMapValue, GetStructField, IntegerLiteral, Literal}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+* push down operations into [[CreateNamedStructLike]].
+*/
+object SimplifyCreateStructOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // push down field extraction
+  case GetStructField( createNamedStructLike : CreateNamedStructLike, 
ordinal, _ ) =>
+createNamedStructLike.valExprs(ordinal)
+}
+  }
+}
+
+/**
+* push down operations into [[CreateArray]].
+*/
+object SimplifyCreateArrayOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // push down field selection (array of structs)
+  case GetArrayStructFields(CreateArray(elems), field, ordinal, 
numFields, containsNull) =>
+def getStructField( elem : Expression ) = {
+  GetStructField( elem, ordinal, Some(field.name) )
+}
+CreateArray( elems.map(getStructField) )
+  // push down item selection.
+  case ga @ GetArrayItem( CreateArray(elems), IntegerLiteral( idx ) ) 
=>
+if ( idx >= 0 && idx < elems.size ) {
+  elems(idx)
+} else {
+  Cast( Literal( null), ga.dataType )
+}
+}
+  }
+}
+
+/**
+* push down operations into [[CreateMap]].
+*/
+object SimplifyCreateMapOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // attempt to unfold 'constant' key extraction,
+  // this enables other optimizations to take place.
+  case gmv @ GetMapValue(cm @ CreateMap(elems), key @ Literal(v, t)) =>
--- End diff --

`case gmv @ GetMapValue(cm @ CreateMap(elems), key @ Literal(v, t))`
->
`case gmv @ GetMapValue(cm: CreateMap, key: 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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2016-12-02 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r90719622
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.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.expressions.{Cast, CreateArray, 
CreateMap, CreateNamedStructLike, Expression, GetArrayItem, 
GetArrayStructFields, GetMapValue, GetStructField, IntegerLiteral, Literal}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+* push down operations into [[CreateNamedStructLike]].
+*/
+object SimplifyCreateStructOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // push down field extraction
+  case GetStructField( createNamedStructLike : CreateNamedStructLike, 
ordinal, _ ) =>
+createNamedStructLike.valExprs(ordinal)
+}
+  }
+}
+
+/**
+* push down operations into [[CreateArray]].
+*/
+object SimplifyCreateArrayOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // push down field selection (array of structs)
+  case GetArrayStructFields(CreateArray(elems), field, ordinal, 
numFields, containsNull) =>
+def getStructField( elem : Expression ) = {
+  GetStructField( elem, ordinal, Some(field.name) )
+}
+CreateArray( elems.map(getStructField) )
+  // push down item selection.
+  case ga @ GetArrayItem( CreateArray(elems), IntegerLiteral( idx ) ) 
=>
+if ( idx >= 0 && idx < elems.size ) {
+  elems(idx)
+} else {
+  Cast( Literal( null), ga.dataType )
+}
+}
+  }
+}
+
+/**
+* push down operations into [[CreateMap]].
+*/
+object SimplifyCreateMapOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
--- End diff --

`p{` -> `p {`


---
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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2016-12-02 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r90719159
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.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.expressions.{Cast, CreateArray, 
CreateMap, CreateNamedStructLike, Expression, GetArrayItem, 
GetArrayStructFields, GetMapValue, GetStructField, IntegerLiteral, Literal}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+* push down operations into [[CreateNamedStructLike]].
+*/
+object SimplifyCreateStructOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // push down field extraction
+  case GetStructField( createNamedStructLike : CreateNamedStructLike, 
ordinal, _ ) =>
+createNamedStructLike.valExprs(ordinal)
+}
+  }
+}
+
+/**
+* push down operations into [[CreateArray]].
+*/
+object SimplifyCreateArrayOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // push down field selection (array of structs)
+  case GetArrayStructFields(CreateArray(elems), field, ordinal, 
numFields, containsNull) =>
+def getStructField( elem : Expression ) = {
+  GetStructField( elem, ordinal, Some(field.name) )
+}
+CreateArray( elems.map(getStructField) )
+  // push down item selection.
+  case ga @ GetArrayItem( CreateArray(elems), IntegerLiteral( idx ) ) 
=>
+if ( idx >= 0 && idx < elems.size ) {
--- End diff --

`( idx` -> `(idx`
`size )` -> `size)`


---
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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2016-12-02 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r90719232
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.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.expressions.{Cast, CreateArray, 
CreateMap, CreateNamedStructLike, Expression, GetArrayItem, 
GetArrayStructFields, GetMapValue, GetStructField, IntegerLiteral, Literal}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+* push down operations into [[CreateNamedStructLike]].
+*/
+object SimplifyCreateStructOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // push down field extraction
+  case GetStructField( createNamedStructLike : CreateNamedStructLike, 
ordinal, _ ) =>
+createNamedStructLike.valExprs(ordinal)
+}
+  }
+}
+
+/**
+* push down operations into [[CreateArray]].
+*/
+object SimplifyCreateArrayOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // push down field selection (array of structs)
+  case GetArrayStructFields(CreateArray(elems), field, ordinal, 
numFields, containsNull) =>
+def getStructField( elem : Expression ) = {
+  GetStructField( elem, ordinal, Some(field.name) )
+}
+CreateArray( elems.map(getStructField) )
+  // push down item selection.
+  case ga @ GetArrayItem( CreateArray(elems), IntegerLiteral( idx ) ) 
=>
+if ( idx >= 0 && idx < elems.size ) {
+  elems(idx)
+} else {
+  Cast( Literal( null), ga.dataType )
--- End diff --

`Cast( Literal( null)` -> Cast(Literal(null)


---
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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2016-12-02 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r90719049
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.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.expressions.{Cast, CreateArray, 
CreateMap, CreateNamedStructLike, Expression, GetArrayItem, 
GetArrayStructFields, GetMapValue, GetStructField, IntegerLiteral, Literal}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+* push down operations into [[CreateNamedStructLike]].
+*/
+object SimplifyCreateStructOps extends Rule[LogicalPlan]{
--- End diff --

`]{` -> `] {`


---
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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2016-12-02 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r90718839
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/complexTypesSuite.scala
 ---
@@ -0,0 +1,357 @@
+/*
+ * 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.scalatest.Matchers
+
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.dsl.plans._
+import org.apache.spark.sql.catalyst.expressions.{CreateArray, CreateMap, 
CreateNamedStruct, Expression, GetArrayItem, GetArrayStructFields, GetMapValue, 
GetStructField, Literal}
+import org.apache.spark.sql.catalyst.plans.PlanTest
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.Range
+import org.apache.spark.sql.catalyst.rules.RuleExecutor
+import org.apache.spark.sql.types._
+
+/**
+* Created by eyalf on 11/4/2016.
+* SPARK-18601 discusses simplification direct access to complex types 
creators.
+* i.e. {{{create_named_struct(square, `x` * `x`).square}}} can be 
simplified to {{{`x` * `x`}}}.
+* sam applies to create_array and create_map
+*/
+class ComplexTypesSuite extends PlanTest with Matchers{
+
+  object Optimize extends RuleExecutor[LogicalPlan] {
+val batches =
+  Batch( "collapse projections", FixedPoint(10),
+  CollapseProject) ::
+  Batch("Constant Folding", FixedPoint(10),
+  NullPropagation,
+  ConstantFolding,
+  BooleanSimplification,
+  SimplifyConditionals,
+  SimplifyCreateStructOps,
+  SimplifyCreateArrayOps,
+  SimplifyCreateMapOps) :: Nil
+  }
+
+  val idAtt = ('id).long.notNull
+
+  lazy val baseOptimizedPlan = Range( 1L, 1000L, 1, Some(2), idAtt :: Nil )
+
+  val idRef = baseOptimizedPlan.output.head
+
+
+//  val idRefColumn = Column( "id" )
+//  val struct1RefColumn = Column( "struct1" )
+
+  implicit class ComplexTypeDslSupport( e : Expression ) {
+def getStructField( f : String ): GetStructField = {
+  e should be ('resolved)
+  e.dataType should be (a[StructType])
+  val structType = e.dataType.asInstanceOf[StructType]
+  val ord = structType.fieldNames.indexOf(f)
+  ord shouldNot be (-1)
+  GetStructField( e, ord, Some(f))
+}
+def getArrayStructField( f : String ) : Expression = {
+  e should be ('resolved)
+  e.dataType should be (a[ArrayType])
+  val arrType = e.dataType.asInstanceOf[ArrayType]
+  arrType.elementType should be (a[StructType])
+  val structType = arrType.elementType.asInstanceOf[StructType]
+  val ord = structType.fieldNames.indexOf(f)
+  ord shouldNot be (-1)
+  GetArrayStructFields(e, structType(ord), ord, 1, 
arrType.containsNull )
+}
+def getArrayItem( i : Int ) : GetArrayItem = {
+  e should be ('resolved)
+  e.dataType should be (a[ArrayType])
+  GetArrayItem( e, Literal(i))
+}
+def getMapValue( k : Expression ) : Expression = {
+  e should be ('resolved)
+  e.dataType should be (a[MapType])
+  val mapType = e.dataType.asInstanceOf[MapType]
+  k.dataType shouldEqual mapType.keyType
+  GetMapValue( e, k )
--- End diff --

Style issues: `GetMapValue( e, k )` -> `GetMapValue(e, k)`


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

[GitHub] spark pull request #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2016-12-02 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r90718774
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/complexTypesSuite.scala
 ---
@@ -0,0 +1,357 @@
+/*
+ * 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.scalatest.Matchers
+
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.dsl.plans._
+import org.apache.spark.sql.catalyst.expressions.{CreateArray, CreateMap, 
CreateNamedStruct, Expression, GetArrayItem, GetArrayStructFields, GetMapValue, 
GetStructField, Literal}
+import org.apache.spark.sql.catalyst.plans.PlanTest
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.Range
+import org.apache.spark.sql.catalyst.rules.RuleExecutor
+import org.apache.spark.sql.types._
+
+/**
+* Created by eyalf on 11/4/2016.
+* SPARK-18601 discusses simplification direct access to complex types 
creators.
+* i.e. {{{create_named_struct(square, `x` * `x`).square}}} can be 
simplified to {{{`x` * `x`}}}.
+* sam applies to create_array and create_map
+*/
+class ComplexTypesSuite extends PlanTest with Matchers{
+
+  object Optimize extends RuleExecutor[LogicalPlan] {
+val batches =
+  Batch( "collapse projections", FixedPoint(10),
+  CollapseProject) ::
+  Batch("Constant Folding", FixedPoint(10),
+  NullPropagation,
+  ConstantFolding,
+  BooleanSimplification,
+  SimplifyConditionals,
+  SimplifyCreateStructOps,
+  SimplifyCreateArrayOps,
+  SimplifyCreateMapOps) :: Nil
+  }
+
+  val idAtt = ('id).long.notNull
+
+  lazy val baseOptimizedPlan = Range( 1L, 1000L, 1, Some(2), idAtt :: Nil )
+
+  val idRef = baseOptimizedPlan.output.head
+
+
+//  val idRefColumn = Column( "id" )
+//  val struct1RefColumn = Column( "struct1" )
+
+  implicit class ComplexTypeDslSupport( e : Expression ) {
--- End diff --

Style issues: `( e : Expression )` -> `(e: Expression)`


---
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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2016-11-28 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r89918782
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/Spark18601Suite.scala ---
@@ -0,0 +1,331 @@
+/*
+ * 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
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast, 
CreateArray, CreateMap, CreateNamedStruct, CreateNamedStructLike, 
CreateNamedStructUnsafe, Expression, GetArrayItem, GetArrayStructFields, 
GetMapValue, GetStructField, IntegerLiteral, Literal}
--- End diff --

Or simply import with a wild card if you are importing more than 6 entities.


---
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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2016-11-28 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r89906204
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/Spark18601Suite.scala ---
@@ -0,0 +1,331 @@
+/*
+ * 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
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast, 
CreateArray, CreateMap, CreateNamedStruct, CreateNamedStructLike, 
CreateNamedStructUnsafe, Expression, GetArrayItem, GetArrayStructFields, 
GetMapValue, GetStructField, IntegerLiteral, Literal}
+import org.apache.spark.sql.catalyst.optimizer.Optimizer
+import org.apache.spark.sql.catalyst.plans.PlanTest
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.types._
+import org.scalatest.{FunSuite, ShouldMatchers, Suite, Tag}
+
+import scala.collection.immutable.IndexedSeq
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.dsl.plans._
+
+/**
+* Created by eyalf on 11/4/2016.
+*/
+class Spark18601Suite extends PlanTest with SharedSQLContext with 
ShouldMatchers{
+  //import testImplicits._
+
+  lazy val baseRelation = sqlContext.range( 1L, 1000L)
+  lazy val baseOptimizedPlan = baseRelation.queryExecution.optimizedPlan
+
+  val idRef = ('id).long.notNull
+  val idRefColumn = Column( "id" )
+  val struct1RefColumn = Column( "struct1" )
+
+  test( "explicit" ) {
+val rel = baseRelation.select(
+  functions.struct( idRefColumn as "att" ).getField("att") as 
"outerAtt"
+)
+rel.schema shouldEqual
+  StructType( StructField( "outerAtt", LongType, nullable = false ) :: 
Nil )
+
+val optimized = rel.queryExecution.optimizedPlan
+
+val expected = baseOptimizedPlan.select(
+  idRef as "outerAtt"
+)
+
+comparePlans(optimized, expected)
+  }
+
+  test( "explicit - deduced att name") {
+val rel = baseRelation.select(
+  functions.struct( idRefColumn as "att" ).getField("att")
+)
+rel.schema shouldEqual
+  StructType(
+StructField( "named_struct(att, id AS `att`).att", LongType, 
nullable = false ) :: Nil
+  )
+
+val optimized = rel.queryExecution.optimizedPlan
+
+val expected = baseOptimizedPlan.select(
+  idRef as "named_struct(att, id AS `att`).att"
+)
--- End diff --

Try 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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2016-11-28 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r89906005
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypeConstructors.scala
 ---
@@ -0,0 +1,79 @@
+/*
+ * 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.expressions.{Cast, CreateArray, 
CreateMap, CreateNamedStructLike, Expression, GetArrayItem, 
GetArrayStructFields, GetMapValue, GetStructField, IntegerLiteral, Literal}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+* push down operations into [[CreateNamedStructLike]].
+*/
+object SimplifyCreateStructOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // push down field extraction
+  case GetStructField( createNamedStructLike : CreateNamedStructLike, 
ordinal, _ ) =>
+val value = createNamedStructLike.valExprs(ordinal)
+value
+}
+  }
+}
+
+/**
+* push down operations into [[CreateArray]].
+*/
+object SimplifyCreateArrayOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // push down field selection (array of structs)
+  case GetArrayStructFields(CreateArray(elems), field, ordinal, 
numFields, containsNull) =>
+def getStructField( elem : Expression ) = {
+  GetStructField( elem, ordinal, Some(field.name) )
+}
+CreateArray( elems.map(getStructField) )
+  // push down item selection.
+  case ga @ GetArrayItem( CreateArray(elems), IntegerLiteral( idx ) ) 
=>
+if ( idx >= 0 && idx < elems.size ) {
+  elems(idx)
+} else {
+  Cast( Literal( null), ga.dataType )
+}
+}
+  }
+}
+
+/**
+* push down operations into [[CreateMap]].
+*/
+object SimplifyCreateMapOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // attempt to unfold 'constant' key extraction,
+  // this enables other optimizations to take place.
+  case gmv @ GetMapValue(cm @ CreateMap(elems), key @ Literal(v, t)) =>
+if ( cm.keys.contains( key ) ) {
--- End diff --

This is just another comment about the space. Please follow the code base 
about the space usage.


---
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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2016-11-28 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r89905892
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypeConstructors.scala
 ---
@@ -0,0 +1,79 @@
+/*
+ * 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.expressions.{Cast, CreateArray, 
CreateMap, CreateNamedStructLike, Expression, GetArrayItem, 
GetArrayStructFields, GetMapValue, GetStructField, IntegerLiteral, Literal}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+* push down operations into [[CreateNamedStructLike]].
+*/
+object SimplifyCreateStructOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // push down field extraction
+  case GetStructField( createNamedStructLike : CreateNamedStructLike, 
ordinal, _ ) =>
+val value = createNamedStructLike.valExprs(ordinal)
+value
+}
+  }
+}
+
+/**
+* push down operations into [[CreateArray]].
+*/
+object SimplifyCreateArrayOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // push down field selection (array of structs)
+  case GetArrayStructFields(CreateArray(elems), field, ordinal, 
numFields, containsNull) =>
+def getStructField( elem : Expression ) = {
+  GetStructField( elem, ordinal, Some(field.name) )
+}
+CreateArray( elems.map(getStructField) )
+  // push down item selection.
+  case ga @ GetArrayItem( CreateArray(elems), IntegerLiteral( idx ) ) 
=>
+if ( idx >= 0 && idx < elems.size ) {
+  elems(idx)
+} else {
+  Cast( Literal( null), ga.dataType )
+}
+}
+  }
+}
+
+/**
+* push down operations into [[CreateMap]].
+*/
+object SimplifyCreateMapOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // attempt to unfold 'constant' key extraction,
+  // this enables other optimizations to take place.
+  case gmv @ GetMapValue(cm @ CreateMap(elems), key @ Literal(v, t)) =>
+if ( cm.keys.contains( key ) ) {
+  val idx = cm.keys.indexOf(key)
+  cm.values(idx)
+} else {
+  Cast( Literal( null ), gmv.dataType)
--- End diff --

My comment is just the style issue. Remove the useless space. 


---
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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2016-11-28 Thread eyalfa
Github user eyalfa commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r89896286
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/Spark18601Suite.scala ---
@@ -0,0 +1,331 @@
+/*
--- End diff --

I've tried, now I can't import SharedSQLContext, any idea?


---
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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2016-11-28 Thread eyalfa
Github user eyalfa commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r89889982
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypeConstructors.scala
 ---
@@ -0,0 +1,79 @@
+/*
+ * 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.expressions.{Cast, CreateArray, 
CreateMap, CreateNamedStructLike, Expression, GetArrayItem, 
GetArrayStructFields, GetMapValue, GetStructField, IntegerLiteral, Literal}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+* push down operations into [[CreateNamedStructLike]].
+*/
+object SimplifyCreateStructOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // push down field extraction
+  case GetStructField( createNamedStructLike : CreateNamedStructLike, 
ordinal, _ ) =>
+val value = createNamedStructLike.valExprs(ordinal)
+value
+}
+  }
+}
+
+/**
+* push down operations into [[CreateArray]].
+*/
+object SimplifyCreateArrayOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // push down field selection (array of structs)
+  case GetArrayStructFields(CreateArray(elems), field, ordinal, 
numFields, containsNull) =>
+def getStructField( elem : Expression ) = {
+  GetStructField( elem, ordinal, Some(field.name) )
+}
+CreateArray( elems.map(getStructField) )
+  // push down item selection.
+  case ga @ GetArrayItem( CreateArray(elems), IntegerLiteral( idx ) ) 
=>
+if ( idx >= 0 && idx < elems.size ) {
+  elems(idx)
+} else {
+  Cast( Literal( null), ga.dataType )
+}
+}
+  }
+}
+
+/**
+* push down operations into [[CreateMap]].
+*/
+object SimplifyCreateMapOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // attempt to unfold 'constant' key extraction,
+  // this enables other optimizations to take place.
+  case gmv @ GetMapValue(cm @ CreateMap(elems), key @ Literal(v, t)) =>
+if ( cm.keys.contains( key ) ) {
--- End diff --

already modified according to @hvanhovell  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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2016-11-28 Thread eyalfa
Github user eyalfa commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r89889823
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypeConstructors.scala
 ---
@@ -0,0 +1,79 @@
+/*
+ * 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.expressions.{Cast, CreateArray, 
CreateMap, CreateNamedStructLike, Expression, GetArrayItem, 
GetArrayStructFields, GetMapValue, GetStructField, IntegerLiteral, Literal}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+* push down operations into [[CreateNamedStructLike]].
+*/
+object SimplifyCreateStructOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // push down field extraction
+  case GetStructField( createNamedStructLike : CreateNamedStructLike, 
ordinal, _ ) =>
+val value = createNamedStructLike.valExprs(ordinal)
+value
+}
+  }
+}
+
+/**
+* push down operations into [[CreateArray]].
+*/
+object SimplifyCreateArrayOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // push down field selection (array of structs)
+  case GetArrayStructFields(CreateArray(elems), field, ordinal, 
numFields, containsNull) =>
+def getStructField( elem : Expression ) = {
+  GetStructField( elem, ordinal, Some(field.name) )
+}
+CreateArray( elems.map(getStructField) )
+  // push down item selection.
+  case ga @ GetArrayItem( CreateArray(elems), IntegerLiteral( idx ) ) 
=>
+if ( idx >= 0 && idx < elems.size ) {
+  elems(idx)
+} else {
+  Cast( Literal( null), ga.dataType )
+}
+}
+  }
+}
+
+/**
+* push down operations into [[CreateMap]].
+*/
+object SimplifyCreateMapOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // attempt to unfold 'constant' key extraction,
+  // this enables other optimizations to take place.
+  case gmv @ GetMapValue(cm @ CreateMap(elems), key @ Literal(v, t)) =>
+if ( cm.keys.contains( key ) ) {
+  val idx = cm.keys.indexOf(key)
+  cm.values(idx)
+} else {
+  Cast( Literal( null ), gmv.dataType)
--- End diff --

actually modified to  Literal.create(null, gmv.dataType)
constants folding would have done thins anyway



---
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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2016-11-28 Thread eyalfa
Github user eyalfa commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r89889287
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/Spark18601Suite.scala ---
@@ -0,0 +1,331 @@
+/*
+ * 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
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast, 
CreateArray, CreateMap, CreateNamedStruct, CreateNamedStructLike, 
CreateNamedStructUnsafe, Expression, GetArrayItem, GetArrayStructFields, 
GetMapValue, GetStructField, IntegerLiteral, Literal}
+import org.apache.spark.sql.catalyst.optimizer.Optimizer
+import org.apache.spark.sql.catalyst.plans.PlanTest
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.types._
+import org.scalatest.{FunSuite, ShouldMatchers, Suite, Tag}
+
+import scala.collection.immutable.IndexedSeq
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.dsl.plans._
+
+/**
+* Created by eyalf on 11/4/2016.
+*/
+class Spark18601Suite extends PlanTest with SharedSQLContext with 
ShouldMatchers{
+  //import testImplicits._
+
+  lazy val baseRelation = sqlContext.range( 1L, 1000L)
+  lazy val baseOptimizedPlan = baseRelation.queryExecution.optimizedPlan
+
+  val idRef = ('id).long.notNull
+  val idRefColumn = Column( "id" )
+  val struct1RefColumn = Column( "struct1" )
+
+  test( "explicit" ) {
+val rel = baseRelation.select(
+  functions.struct( idRefColumn as "att" ).getField("att") as 
"outerAtt"
+)
+rel.schema shouldEqual
+  StructType( StructField( "outerAtt", LongType, nullable = false ) :: 
Nil )
+
+val optimized = rel.queryExecution.optimizedPlan
+
+val expected = baseOptimizedPlan.select(
+  idRef as "outerAtt"
+)
+
+comparePlans(optimized, expected)
+  }
+
+  test( "explicit - deduced att name") {
+val rel = baseRelation.select(
+  functions.struct( idRefColumn as "att" ).getField("att")
+)
+rel.schema shouldEqual
+  StructType(
+StructField( "named_struct(att, id AS `att`).att", LongType, 
nullable = false ) :: Nil
+  )
+
+val optimized = rel.queryExecution.optimizedPlan
+
+val expected = baseOptimizedPlan.select(
+  idRef as "named_struct(att, id AS `att`).att"
+)
--- End diff --

I think it exceeds max line length


---
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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2016-11-28 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r89887357
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/Spark18601Suite.scala ---
@@ -0,0 +1,331 @@
+/*
+ * 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
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast, 
CreateArray, CreateMap, CreateNamedStruct, CreateNamedStructLike, 
CreateNamedStructUnsafe, Expression, GetArrayItem, GetArrayStructFields, 
GetMapValue, GetStructField, IntegerLiteral, Literal}
--- End diff --

You can split it to multiple lines


---
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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2016-11-28 Thread hvanhovell
Github user hvanhovell commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r89887374
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/Spark18601Suite.scala ---
@@ -0,0 +1,331 @@
+/*
+ * 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
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast, 
CreateArray, CreateMap, CreateNamedStruct, CreateNamedStructLike, 
CreateNamedStructUnsafe, Expression, GetArrayItem, GetArrayStructFields, 
GetMapValue, GetStructField, IntegerLiteral, Literal}
+import org.apache.spark.sql.catalyst.optimizer.Optimizer
+import org.apache.spark.sql.catalyst.plans.PlanTest
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.types._
+import org.scalatest.{FunSuite, ShouldMatchers, Suite, Tag}
+
+import scala.collection.immutable.IndexedSeq
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.dsl.plans._
+
+/**
+* Created by eyalf on 11/4/2016.
+*/
+class Spark18601Suite extends PlanTest with SharedSQLContext with 
ShouldMatchers{
+  //import testImplicits._
+
+  lazy val baseRelation = sqlContext.range( 1L, 1000L)
+  lazy val baseOptimizedPlan = baseRelation.queryExecution.optimizedPlan
+
+  val idRef = ('id).long.notNull
+  val idRefColumn = Column( "id" )
+  val struct1RefColumn = Column( "struct1" )
+
+  test( "explicit" ) {
+val rel = baseRelation.select(
+  functions.struct( idRefColumn as "att" ).getField("att") as 
"outerAtt"
+)
+rel.schema shouldEqual
+  StructType( StructField( "outerAtt", LongType, nullable = false ) :: 
Nil )
+
+val optimized = rel.queryExecution.optimizedPlan
+
+val expected = baseOptimizedPlan.select(
+  idRef as "outerAtt"
+)
+
+comparePlans(optimized, expected)
+  }
+
+  test( "explicit - deduced att name") {
+val rel = baseRelation.select(
+  functions.struct( idRefColumn as "att" ).getField("att")
+)
+rel.schema shouldEqual
+  StructType(
+StructField( "named_struct(att, id AS `att`).att", LongType, 
nullable = false ) :: Nil
+  )
+
+val optimized = rel.queryExecution.optimizedPlan
+
+val expected = baseOptimizedPlan.select(
+  idRef as "named_struct(att, id AS `att`).att"
+)
+
+comparePlans(optimized, expected)
+  }
+
+  test( "collapsed" ) {
+val rel = baseRelation.select(
+  functions.struct( idRefColumn as "att" ) as "struct1"
+)
+rel.schema shouldEqual
+  StructType(
+StructField(
+  "struct1",
+  StructType(
+StructField(
+  "att",
+  LongType,
+  false
+) :: Nil
+  ),
+  false
+) :: Nil
+  )
--- End diff --

The dsl is also nice to use.


---
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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2016-11-28 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r89886992
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypeConstructors.scala
 ---
@@ -0,0 +1,79 @@
+/*
+ * 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.expressions.{Cast, CreateArray, 
CreateMap, CreateNamedStructLike, Expression, GetArrayItem, 
GetArrayStructFields, GetMapValue, GetStructField, IntegerLiteral, Literal}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+* push down operations into [[CreateNamedStructLike]].
+*/
+object SimplifyCreateStructOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // push down field extraction
+  case GetStructField( createNamedStructLike : CreateNamedStructLike, 
ordinal, _ ) =>
+val value = createNamedStructLike.valExprs(ordinal)
+value
+}
+  }
+}
+
+/**
+* push down operations into [[CreateArray]].
+*/
+object SimplifyCreateArrayOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // push down field selection (array of structs)
+  case GetArrayStructFields(CreateArray(elems), field, ordinal, 
numFields, containsNull) =>
+def getStructField( elem : Expression ) = {
+  GetStructField( elem, ordinal, Some(field.name) )
+}
+CreateArray( elems.map(getStructField) )
+  // push down item selection.
+  case ga @ GetArrayItem( CreateArray(elems), IntegerLiteral( idx ) ) 
=>
+if ( idx >= 0 && idx < elems.size ) {
+  elems(idx)
+} else {
+  Cast( Literal( null), ga.dataType )
+}
+}
+  }
+}
+
+/**
+* push down operations into [[CreateMap]].
+*/
+object SimplifyCreateMapOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // attempt to unfold 'constant' key extraction,
+  // this enables other optimizations to take place.
+  case gmv @ GetMapValue(cm @ CreateMap(elems), key @ Literal(v, t)) =>
+if ( cm.keys.contains( key ) ) {
--- End diff --

`if ( cm.keys.contains( key ) ) {` -> if (cm.keys.contains(key)) {


---
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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2016-11-28 Thread eyalfa
Github user eyalfa commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r89886974
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/Spark18601Suite.scala ---
@@ -0,0 +1,331 @@
+/*
+ * 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
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast, 
CreateArray, CreateMap, CreateNamedStruct, CreateNamedStructLike, 
CreateNamedStructUnsafe, Expression, GetArrayItem, GetArrayStructFields, 
GetMapValue, GetStructField, IntegerLiteral, Literal}
+import org.apache.spark.sql.catalyst.optimizer.Optimizer
+import org.apache.spark.sql.catalyst.plans.PlanTest
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.types._
+import org.scalatest.{FunSuite, ShouldMatchers, Suite, Tag}
+
+import scala.collection.immutable.IndexedSeq
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.dsl.plans._
+
+/**
+* Created by eyalf on 11/4/2016.
+*/
+class Spark18601Suite extends PlanTest with SharedSQLContext with 
ShouldMatchers{
+  //import testImplicits._
+
+  lazy val baseRelation = sqlContext.range( 1L, 1000L)
+  lazy val baseOptimizedPlan = baseRelation.queryExecution.optimizedPlan
+
+  val idRef = ('id).long.notNull
+  val idRefColumn = Column( "id" )
+  val struct1RefColumn = Column( "struct1" )
+
+  test( "explicit" ) {
+val rel = baseRelation.select(
+  functions.struct( idRefColumn as "att" ).getField("att") as 
"outerAtt"
+)
+rel.schema shouldEqual
+  StructType( StructField( "outerAtt", LongType, nullable = false ) :: 
Nil )
+
+val optimized = rel.queryExecution.optimizedPlan
+
+val expected = baseOptimizedPlan.select(
+  idRef as "outerAtt"
+)
+
+comparePlans(optimized, expected)
+  }
+
+  test( "explicit - deduced att name") {
+val rel = baseRelation.select(
+  functions.struct( idRefColumn as "att" ).getField("att")
+)
+rel.schema shouldEqual
+  StructType(
+StructField( "named_struct(att, id AS `att`).att", LongType, 
nullable = false ) :: Nil
+  )
+
+val optimized = rel.queryExecution.optimizedPlan
+
+val expected = baseOptimizedPlan.select(
+  idRef as "named_struct(att, id AS `att`).att"
+)
+
+comparePlans(optimized, expected)
+  }
+
+  test( "collapsed" ) {
+val rel = baseRelation.select(
+  functions.struct( idRefColumn as "att" ) as "struct1"
+)
+rel.schema shouldEqual
+  StructType(
+StructField(
+  "struct1",
+  StructType(
+StructField(
+  "att",
+  LongType,
+  false
+) :: Nil
+  ),
+  false
+) :: Nil
+  )
--- End diff --

I'll have a try at 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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2016-11-28 Thread hvanhovell
Github user hvanhovell commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r89886872
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypeConstructors.scala
 ---
@@ -0,0 +1,79 @@
+/*
+ * 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.expressions.{Cast, CreateArray, 
CreateMap, CreateNamedStructLike, Expression, GetArrayItem, 
GetArrayStructFields, GetMapValue, GetStructField, IntegerLiteral, Literal}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+* push down operations into [[CreateNamedStructLike]].
+*/
+object SimplifyCreateStructOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // push down field extraction
+  case GetStructField( createNamedStructLike : CreateNamedStructLike, 
ordinal, _ ) =>
+val value = createNamedStructLike.valExprs(ordinal)
+value
+}
+  }
+}
+
+/**
+* push down operations into [[CreateArray]].
+*/
+object SimplifyCreateArrayOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // push down field selection (array of structs)
+  case GetArrayStructFields(CreateArray(elems), field, ordinal, 
numFields, containsNull) =>
+def getStructField( elem : Expression ) = {
+  GetStructField( elem, ordinal, Some(field.name) )
+}
+CreateArray( elems.map(getStructField) )
+  // push down item selection.
+  case ga @ GetArrayItem( CreateArray(elems), IntegerLiteral( idx ) ) 
=>
+if ( idx >= 0 && idx < elems.size ) {
+  elems(idx)
+} else {
+  Cast( Literal( null), ga.dataType )
+}
+}
+  }
+}
+
+/**
+* push down operations into [[CreateMap]].
+*/
+object SimplifyCreateMapOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // attempt to unfold 'constant' key extraction,
+  // this enables other optimizations to take place.
+  case gmv @ GetMapValue(cm @ CreateMap(elems), key @ Literal(v, t)) =>
--- End diff --

Yeah, that is fair to assume. The I mean you don't need to desugar the 
entire literal. So you could also write:
```scala
case gmv @ GetMapValue(cm @ CreateMap(elems), key: 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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2016-11-28 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r89886696
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypeConstructors.scala
 ---
@@ -0,0 +1,79 @@
+/*
+ * 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.expressions.{Cast, CreateArray, 
CreateMap, CreateNamedStructLike, Expression, GetArrayItem, 
GetArrayStructFields, GetMapValue, GetStructField, IntegerLiteral, Literal}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+* push down operations into [[CreateNamedStructLike]].
+*/
+object SimplifyCreateStructOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // push down field extraction
+  case GetStructField( createNamedStructLike : CreateNamedStructLike, 
ordinal, _ ) =>
+val value = createNamedStructLike.valExprs(ordinal)
+value
+}
+  }
+}
+
+/**
+* push down operations into [[CreateArray]].
+*/
+object SimplifyCreateArrayOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // push down field selection (array of structs)
+  case GetArrayStructFields(CreateArray(elems), field, ordinal, 
numFields, containsNull) =>
+def getStructField( elem : Expression ) = {
+  GetStructField( elem, ordinal, Some(field.name) )
+}
+CreateArray( elems.map(getStructField) )
+  // push down item selection.
+  case ga @ GetArrayItem( CreateArray(elems), IntegerLiteral( idx ) ) 
=>
+if ( idx >= 0 && idx < elems.size ) {
+  elems(idx)
+} else {
+  Cast( Literal( null), ga.dataType )
+}
+}
+  }
+}
+
+/**
+* push down operations into [[CreateMap]].
+*/
+object SimplifyCreateMapOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // attempt to unfold 'constant' key extraction,
+  // this enables other optimizations to take place.
+  case gmv @ GetMapValue(cm @ CreateMap(elems), key @ Literal(v, t)) =>
--- End diff --

`cm: CreateMap(elems)`?


---
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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2016-11-28 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r89886559
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypeConstructors.scala
 ---
@@ -0,0 +1,79 @@
+/*
+ * 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.expressions.{Cast, CreateArray, 
CreateMap, CreateNamedStructLike, Expression, GetArrayItem, 
GetArrayStructFields, GetMapValue, GetStructField, IntegerLiteral, Literal}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+* push down operations into [[CreateNamedStructLike]].
+*/
+object SimplifyCreateStructOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // push down field extraction
+  case GetStructField( createNamedStructLike : CreateNamedStructLike, 
ordinal, _ ) =>
+val value = createNamedStructLike.valExprs(ordinal)
+value
+}
+  }
+}
+
+/**
+* push down operations into [[CreateArray]].
+*/
+object SimplifyCreateArrayOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // push down field selection (array of structs)
+  case GetArrayStructFields(CreateArray(elems), field, ordinal, 
numFields, containsNull) =>
+def getStructField( elem : Expression ) = {
+  GetStructField( elem, ordinal, Some(field.name) )
+}
+CreateArray( elems.map(getStructField) )
+  // push down item selection.
+  case ga @ GetArrayItem( CreateArray(elems), IntegerLiteral( idx ) ) 
=>
+if ( idx >= 0 && idx < elems.size ) {
+  elems(idx)
+} else {
+  Cast( Literal( null), ga.dataType )
+}
+}
+  }
+}
+
+/**
+* push down operations into [[CreateMap]].
+*/
+object SimplifyCreateMapOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // attempt to unfold 'constant' key extraction,
+  // this enables other optimizations to take place.
+  case gmv @ GetMapValue(cm @ CreateMap(elems), key @ Literal(v, t)) =>
+if ( cm.keys.contains( key ) ) {
+  val idx = cm.keys.indexOf(key)
+  cm.values(idx)
+} else {
+  Cast( Literal( null ), gmv.dataType)
--- End diff --

Style issue: `Cast( Literal( null ), gmv.dataType)` -> `Cast(Literal(null), 
gmv.dataType)`


---
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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2016-11-28 Thread eyalfa
Github user eyalfa commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r89886382
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypeConstructors.scala
 ---
@@ -0,0 +1,79 @@
+/*
+ * 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.expressions.{Cast, CreateArray, 
CreateMap, CreateNamedStructLike, Expression, GetArrayItem, 
GetArrayStructFields, GetMapValue, GetStructField, IntegerLiteral, Literal}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+* push down operations into [[CreateNamedStructLike]].
+*/
+object SimplifyCreateStructOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // push down field extraction
+  case GetStructField( createNamedStructLike : CreateNamedStructLike, 
ordinal, _ ) =>
+val value = createNamedStructLike.valExprs(ordinal)
+value
+}
+  }
+}
+
+/**
+* push down operations into [[CreateArray]].
+*/
+object SimplifyCreateArrayOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // push down field selection (array of structs)
+  case GetArrayStructFields(CreateArray(elems), field, ordinal, 
numFields, containsNull) =>
+def getStructField( elem : Expression ) = {
+  GetStructField( elem, ordinal, Some(field.name) )
+}
+CreateArray( elems.map(getStructField) )
+  // push down item selection.
+  case ga @ GetArrayItem( CreateArray(elems), IntegerLiteral( idx ) ) 
=>
+if ( idx >= 0 && idx < elems.size ) {
+  elems(idx)
+} else {
+  Cast( Literal( null), ga.dataType )
+}
+}
+  }
+}
+
+/**
+* push down operations into [[CreateMap]].
+*/
+object SimplifyCreateMapOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // attempt to unfold 'constant' key extraction,
+  // this enables other optimizations to take place.
+  case gmv @ GetMapValue(cm @ CreateMap(elems), key @ Literal(v, t)) =>
--- End diff --

assume constant folding already did its thing, otherwise there's not much 
we can do anyway


---
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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2016-11-28 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r89886358
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypeConstructors.scala
 ---
@@ -0,0 +1,79 @@
+/*
+ * 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.expressions.{Cast, CreateArray, 
CreateMap, CreateNamedStructLike, Expression, GetArrayItem, 
GetArrayStructFields, GetMapValue, GetStructField, IntegerLiteral, Literal}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+* push down operations into [[CreateNamedStructLike]].
+*/
+object SimplifyCreateStructOps extends Rule[LogicalPlan]{
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+plan.transformExpressionsUp{
+  // push down field extraction
+  case GetStructField( createNamedStructLike : CreateNamedStructLike, 
ordinal, _ ) =>
+val value = createNamedStructLike.valExprs(ordinal)
+value
--- End diff --

Nit: directly call `createNamedStructLike.valExprs(ordinal)`


---
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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2016-11-28 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r89885601
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/Spark18601Suite.scala ---
@@ -0,0 +1,331 @@
+/*
+ * 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
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast, 
CreateArray, CreateMap, CreateNamedStruct, CreateNamedStructLike, 
CreateNamedStructUnsafe, Expression, GetArrayItem, GetArrayStructFields, 
GetMapValue, GetStructField, IntegerLiteral, Literal}
+import org.apache.spark.sql.catalyst.optimizer.Optimizer
+import org.apache.spark.sql.catalyst.plans.PlanTest
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.types._
+import org.scalatest.{FunSuite, ShouldMatchers, Suite, Tag}
+
+import scala.collection.immutable.IndexedSeq
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.dsl.plans._
+
+/**
+* Created by eyalf on 11/4/2016.
+*/
+class Spark18601Suite extends PlanTest with SharedSQLContext with 
ShouldMatchers{
+  //import testImplicits._
+
+  lazy val baseRelation = sqlContext.range( 1L, 1000L)
+  lazy val baseOptimizedPlan = baseRelation.queryExecution.optimizedPlan
+
+  val idRef = ('id).long.notNull
+  val idRefColumn = Column( "id" )
+  val struct1RefColumn = Column( "struct1" )
+
+  test( "explicit" ) {
+val rel = baseRelation.select(
+  functions.struct( idRefColumn as "att" ).getField("att") as 
"outerAtt"
+)
+rel.schema shouldEqual
+  StructType( StructField( "outerAtt", LongType, nullable = false ) :: 
Nil )
+
+val optimized = rel.queryExecution.optimizedPlan
+
+val expected = baseOptimizedPlan.select(
+  idRef as "outerAtt"
+)
+
+comparePlans(optimized, expected)
+  }
+
+  test( "explicit - deduced att name") {
+val rel = baseRelation.select(
+  functions.struct( idRefColumn as "att" ).getField("att")
+)
+rel.schema shouldEqual
+  StructType(
+StructField( "named_struct(att, id AS `att`).att", LongType, 
nullable = false ) :: Nil
+  )
+
+val optimized = rel.queryExecution.optimizedPlan
+
+val expected = baseOptimizedPlan.select(
+  idRef as "named_struct(att, id AS `att`).att"
+)
+
+comparePlans(optimized, expected)
+  }
+
+  test( "collapsed" ) {
--- End diff --

Nit: `test( "collapsed" )` -> `test("collapsed")`

Please correct all the style issues in all the above test cases.


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

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



[GitHub] spark pull request #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2016-11-28 Thread eyalfa
Github user eyalfa commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r89885103
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/Spark18601Suite.scala ---
@@ -0,0 +1,331 @@
+/*
+ * 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
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast, 
CreateArray, CreateMap, CreateNamedStruct, CreateNamedStructLike, 
CreateNamedStructUnsafe, Expression, GetArrayItem, GetArrayStructFields, 
GetMapValue, GetStructField, IntegerLiteral, Literal}
+import org.apache.spark.sql.catalyst.optimizer.Optimizer
+import org.apache.spark.sql.catalyst.plans.PlanTest
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.types._
+import org.scalatest.{FunSuite, ShouldMatchers, Suite, Tag}
+
+import scala.collection.immutable.IndexedSeq
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.dsl.plans._
+
+/**
+* Created by eyalf on 11/4/2016.
+*/
+class Spark18601Suite extends PlanTest with SharedSQLContext with 
ShouldMatchers{
+  //import testImplicits._
--- End diff --

already did, will push shortly


---
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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2016-11-28 Thread hvanhovell
Github user hvanhovell commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r89883879
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/Spark18601Suite.scala ---
@@ -0,0 +1,331 @@
+/*
--- End diff --

See the tests in that package for a example of how to create your own 
optimizer and stuff.


---
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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2016-11-28 Thread hvanhovell
Github user hvanhovell commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r89883383
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypeConstructors.scala
 ---
@@ -0,0 +1,79 @@
+/*
--- End diff --

name the file `complexTypes.scala`


---
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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2016-11-28 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r89884183
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/Spark18601Suite.scala ---
@@ -0,0 +1,331 @@
+/*
+ * 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
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast, 
CreateArray, CreateMap, CreateNamedStruct, CreateNamedStructLike, 
CreateNamedStructUnsafe, Expression, GetArrayItem, GetArrayStructFields, 
GetMapValue, GetStructField, IntegerLiteral, Literal}
+import org.apache.spark.sql.catalyst.optimizer.Optimizer
+import org.apache.spark.sql.catalyst.plans.PlanTest
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.types._
+import org.scalatest.{FunSuite, ShouldMatchers, Suite, Tag}
+
+import scala.collection.immutable.IndexedSeq
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.dsl.plans._
+
+/**
+* Created by eyalf on 11/4/2016.
+*/
+class Spark18601Suite extends PlanTest with SharedSQLContext with 
ShouldMatchers{
+  //import testImplicits._
--- End diff --

Nit: clean 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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2016-11-28 Thread hvanhovell
Github user hvanhovell commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r89883711
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/Spark18601Suite.scala ---
@@ -0,0 +1,331 @@
+/*
--- End diff --

Move this into catalyst, and place this in the 
`org.apache.spark.sql.catalyst.optimizer`. I would call it the 
`complexTypesSuite.scala`


---
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 #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2016-11-28 Thread hvanhovell
Github user hvanhovell commented on a diff in the pull request:

https://github.com/apache/spark/pull/16043#discussion_r89883785
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/Spark18601Suite.scala ---
@@ -0,0 +1,331 @@
+/*
+ * 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
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast, 
CreateArray, CreateMap, CreateNamedStruct, CreateNamedStructLike, 
CreateNamedStructUnsafe, Expression, GetArrayItem, GetArrayStructFields, 
GetMapValue, GetStructField, IntegerLiteral, Literal}
+import org.apache.spark.sql.catalyst.optimizer.Optimizer
+import org.apache.spark.sql.catalyst.plans.PlanTest
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.types._
+import org.scalatest.{FunSuite, ShouldMatchers, Suite, Tag}
+
+import scala.collection.immutable.IndexedSeq
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.dsl.plans._
+
+/**
+* Created by eyalf on 11/4/2016.
--- End diff --

Short description of the tests?


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

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



[GitHub] spark pull request #16043: [SPARK-18601][SQL] Simplify Create/Get complex ex...

2016-11-28 Thread eyalfa
GitHub user eyalfa opened a pull request:

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

[SPARK-18601][SQL] Simplify Create/Get complex expression pairs in optimizer

## What changes were proposed in this pull request?
It often happens that a complex object (struct/map/array) is created only 
to get elements from it in an subsequent expression. We can add an optimizer 
rule for this.

## How was this patch tested?
unit-tests

Please review http://spark.apache.org/contributing.html before opening a 
pull request.


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

$ git pull https://github.com/eyalfa/spark SPARK-18601

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

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


commit b49a644dd4fb77251d1349366e38c51f3e1bd237
Author: eyal farago 
Date:   2016-11-22T15:59:20Z

getOnCreateStruct: added tests to show the problem, and an initial 
implementation.

commit 27afcaf1fffe3cd2927cacaa1c9a6168d26ab4fe
Author: eyal farago 
Date:   2016-11-22T19:09:18Z

getOnCreateStruct: move the new ruleinto an existing batch.

commit 7dd168f41c10ebf79b5ed56e53b697e9250eb633
Author: eyal farago 
Date:   2016-11-26T16:21:37Z

getOnCreateStruct: added opt rules for createArray and createMap, found a 
bug in arr.getField.

commit 6d42111d4f094d35634883dea2fac2f3b4eb50fa
Author: eyal farago 
Date:   2016-11-28T20:18:26Z

SPARK-18601: removed the CreateNamedStructLike extractor.

commit 0f6823907ec862e479f4cacdcd1869ffa8ce7858
Author: eyal farago 
Date:   2016-11-28T20:19:51Z

SPARK-18601: test suite for spark-18601

commit 07a35ea11a635a87c7e32f5fda8f60c0815d9479
Author: eyal farago 
Date:   2016-11-28T20:21:00Z

SPARK-18601: optimizer rules for simplifying operations on complex type 
constructors.




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