[GitHub] spark pull request #14676: [SPARK-16947][SQL] Support type coercion and fold...

2016-08-18 Thread asfgit
Github user asfgit closed the pull request at:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes 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 #14676: [SPARK-16947][SQL] Support type coercion and fold...

2016-08-18 Thread hvanhovell
Github user hvanhovell commented on a diff in the pull request:

https://github.com/apache/spark/pull/14676#discussion_r75277151
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTables.scala
 ---
@@ -0,0 +1,105 @@
+/*
+ * 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.analysis
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Cast, 
InterpretedProjection, Unevaluable}
+import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, 
LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.types.{StructField, StructType}
+
+/**
+ * An analyzer rule that replaces [[UnresolvedInlineTable]] with 
[[LocalRelation]].
+ */
+object ResolveInlineTables extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
+case table: UnresolvedInlineTable if table.expressionsResolved =>
+  validateInputDimension(table)
+  validateInputEvaluable(table)
+  convert(table)
+  }
+
+  /**
+   * Validates that all inline table data are foldable expressions.
+   *
+   * This is package visible for unit testing.
+   */
+  private[analysis] def validateInputEvaluable(table: 
UnresolvedInlineTable): Unit = {
+table.rows.foreach { row =>
+  row.foreach { e =>
+if (!e.resolved || e.isInstanceOf[Unevaluable]) {
--- End diff --

So I did some more thinking, and I cannot really think of use case for 
`rand()` or any other non-deterministic expression in a fixed table. Lets not 
support this, and required that expressions must be foldable.

I am sorry for changing my mind on this one.


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

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



[GitHub] spark pull request #14676: [SPARK-16947][SQL] Support type coercion and fold...

2016-08-18 Thread hvanhovell
Github user hvanhovell commented on a diff in the pull request:

https://github.com/apache/spark/pull/14676#discussion_r75276635
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTables.scala
 ---
@@ -0,0 +1,109 @@
+/*
+ * 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.analysis
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Cast
+import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, 
LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.types.{StructField, StructType}
+
+/**
+ * An analyzer rule that replaces [[UnresolvedInlineTable]] with 
[[LocalRelation]].
+ */
+object ResolveInlineTables extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
+case table: UnresolvedInlineTable if table.expressionsResolved =>
+  validateInputDimension(table)
+  validateInputFoldable(table)
+  convert(table)
+  }
+
+  /**
+   * Validates that all inline table data are foldable expressions.
+   *
+   * This is publicly visible for unit testing.
+   */
+  def validateInputFoldable(table: UnresolvedInlineTable): Unit = {
+table.rows.foreach { row =>
+  row.foreach { e =>
+if (!e.resolved || !e.foldable) {
+  e.failAnalysis(s"cannot evaluate expression ${e.sql} in inline 
table definition")
+}
+  }
+}
+  }
+
+  /**
+   * Validates the input data dimension:
+   * 1. All rows have the same cardinality.
+   * 2. The number of column aliases defined is consistent with the number 
of columns in data.
+   *
+   * This is publicly visible for unit testing.
+   */
+  def validateInputDimension(table: UnresolvedInlineTable): Unit = {
+if (table.rows.nonEmpty) {
+  val numCols = table.rows.head.size
+  table.rows.zipWithIndex.foreach { case (row, ri) =>
+if (row.size != numCols) {
+  table.failAnalysis(s"expected $numCols columns but found 
${row.size} columns in row $ri")
+}
+  }
+
+  if (table.names.size != numCols) {
+table.failAnalysis(s"expected ${table.names.size} columns but 
found $numCols in first row")
+  }
+}
+  }
+
+  /**
+   * Convert a valid (with right shape and foldable inputs) 
[[UnresolvedInlineTable]]
+   * into a [[LocalRelation]].
+   *
+   * This function attempts to coerce inputs into consistent types.
+   *
+   * This is publicly visible for unit testing.
+   */
+  def convert(table: UnresolvedInlineTable): LocalRelation = {
+val numCols = table.rows.head.size
+
+// For each column, traverse all the values and find a common data 
type.
+val targetTypes = Seq.tabulate(numCols) { ci =>
--- End diff --

Yeah, this might be ok for most testing scenario's. However we are 
currently uncovering all sorts of tricky nullable bugs in the optimizer; it is 
useful in such a case to be able to control nullability. The other thing is 
that these features tend to be used in production, and then these things start 
to matter. Finally it is literally a one-liner `column.exists(_.nullable)` (if 
you combine this with creating the schema). 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes 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 #14676: [SPARK-16947][SQL] Support type coercion and fold...

2016-08-18 Thread hvanhovell
Github user hvanhovell commented on a diff in the pull request:

https://github.com/apache/spark/pull/14676#discussion_r75274360
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTables.scala
 ---
@@ -0,0 +1,105 @@
+/*
+ * 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.analysis
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Cast, 
InterpretedProjection, Unevaluable}
+import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, 
LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.types.{StructField, StructType}
+
+/**
+ * An analyzer rule that replaces [[UnresolvedInlineTable]] with 
[[LocalRelation]].
+ */
+object ResolveInlineTables extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
+case table: UnresolvedInlineTable if table.expressionsResolved =>
+  validateInputDimension(table)
+  validateInputEvaluable(table)
+  convert(table)
+  }
+
+  /**
+   * Validates that all inline table data are foldable expressions.
+   *
+   * This is package visible for unit testing.
+   */
+  private[analysis] def validateInputEvaluable(table: 
UnresolvedInlineTable): Unit = {
+table.rows.foreach { row =>
+  row.foreach { e =>
+if (!e.resolved || e.isInstanceOf[Unevaluable]) {
+  e.failAnalysis(s"cannot evaluate expression ${e.sql} in inline 
table definition")
+}
+  }
+}
+  }
+
+  /**
+   * Validates the input data dimension:
+   * 1. All rows have the same cardinality.
+   * 2. The number of column aliases defined is consistent with the number 
of columns in data.
+   *
+   * This is package visible for unit testing.
+   */
+  private[analysis] def validateInputDimension(table: 
UnresolvedInlineTable): Unit = {
+if (table.rows.nonEmpty) {
+  val numCols = table.rows.head.size
+  table.rows.zipWithIndex.foreach { case (row, ri) =>
+if (row.size != numCols) {
+  table.failAnalysis(s"expected $numCols columns but found 
${row.size} columns in row $ri")
+}
+  }
+
+  if (table.names.size != numCols) {
+table.failAnalysis(s"expected ${table.names.size} columns but 
found $numCols in first row")
+  }
+}
+  }
+
+  /**
+   * Convert a valid (with right shape and foldable inputs) 
[[UnresolvedInlineTable]]
+   * into a [[LocalRelation]].
+   *
+   * This function attempts to coerce inputs into consistent types.
+   *
+   * This is package visible for unit testing.
+   */
+  private[analysis] def convert(table: UnresolvedInlineTable): 
LocalRelation = {
+val numCols = table.rows.head.size
+
+// For each column, traverse all the values and find a common data 
type.
+val targetTypes = table.rows.transpose.zip(table.names).map { case 
(column, name) =>
+  val inputTypes = column.map(_.dataType)
+  
TypeCoercion.findWiderTypeWithoutStringPromotion(inputTypes).getOrElse {
--- End diff --

Lets go with the Postgres semantics.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes 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 #14676: [SPARK-16947][SQL] Support type coercion and fold...

2016-08-17 Thread petermaxlee
Github user petermaxlee commented on a diff in the pull request:

https://github.com/apache/spark/pull/14676#discussion_r75251858
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTables.scala
 ---
@@ -0,0 +1,105 @@
+/*
+ * 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.analysis
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Cast, 
InterpretedProjection, Unevaluable}
+import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, 
LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.types.{StructField, StructType}
+
+/**
+ * An analyzer rule that replaces [[UnresolvedInlineTable]] with 
[[LocalRelation]].
+ */
+object ResolveInlineTables extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
+case table: UnresolvedInlineTable if table.expressionsResolved =>
+  validateInputDimension(table)
+  validateInputEvaluable(table)
+  convert(table)
+  }
+
+  /**
+   * Validates that all inline table data are foldable expressions.
+   *
+   * This is package visible for unit testing.
+   */
+  private[analysis] def validateInputEvaluable(table: 
UnresolvedInlineTable): Unit = {
+table.rows.foreach { row =>
+  row.foreach { e =>
+if (!e.resolved || e.isInstanceOf[Unevaluable]) {
--- End diff --

I added some 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 #14676: [SPARK-16947][SQL] Support type coercion and fold...

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

https://github.com/apache/spark/pull/14676#discussion_r75249808
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTables.scala
 ---
@@ -0,0 +1,105 @@
+/*
+ * 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.analysis
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Cast, 
InterpretedProjection, Unevaluable}
+import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, 
LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.types.{StructField, StructType}
+
+/**
+ * An analyzer rule that replaces [[UnresolvedInlineTable]] with 
[[LocalRelation]].
+ */
+object ResolveInlineTables extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
+case table: UnresolvedInlineTable if table.expressionsResolved =>
+  validateInputDimension(table)
+  validateInputEvaluable(table)
+  convert(table)
+  }
+
+  /**
+   * Validates that all inline table data are foldable expressions.
+   *
+   * This is package visible for unit testing.
+   */
+  private[analysis] def validateInputEvaluable(table: 
UnresolvedInlineTable): Unit = {
+table.rows.foreach { row =>
+  row.foreach { e =>
+if (!e.resolved || e.isInstanceOf[Unevaluable]) {
--- End diff --

This looks tricky to me, as ideally `foldable` matches the semantic better.

Actually we are making assumptions here, i.e. in the case of inline table, 
evaluable always mean foldable, because `UnresolvedInlineTable` can't resolve 
`UnresolvedAttribute` to `AttributeReference` as it's a leaf node.

We should either document this, or not support rand, cc @hvanhovell 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes 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 #14676: [SPARK-16947][SQL] Support type coercion and fold...

2016-08-17 Thread petermaxlee
Github user petermaxlee commented on a diff in the pull request:

https://github.com/apache/spark/pull/14676#discussion_r75249521
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTablesSuite.scala
 ---
@@ -0,0 +1,89 @@
+/*
+ * 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.analysis
+
+import org.scalatest.BeforeAndAfter
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Literal, Rand}
+import org.apache.spark.sql.catalyst.expressions.aggregate.Count
+import org.apache.spark.sql.catalyst.plans.PlanTest
+import org.apache.spark.sql.types.LongType
+
+/**
+ * Unit tests for [[ResolveInlineTables]]. Note that there are also test 
cases defined in
+ * end-to-end tests (in sql/core module) for verifying the correct error 
messages are shown
+ * in negative cases.
+ */
+class ResolveInlineTablesSuite extends PlanTest with BeforeAndAfter {
+
+  private def lit(v: Any): Literal = Literal(v)
+
+  test("validate inputs are foldable") {
+ResolveInlineTables.validateInputEvaluable(
+  UnresolvedInlineTable(Seq("c1", "c2"), Seq(Seq(lit(1)
+
+// nondeterministic (rand) should be fine
+ResolveInlineTables.validateInputEvaluable(
+  UnresolvedInlineTable(Seq("c1", "c2"), Seq(Seq(Rand(1)
+
+// aggregate should not work
+intercept[AnalysisException] {
+  ResolveInlineTables.validateInputEvaluable(
+UnresolvedInlineTable(Seq("c1", "c2"), Seq(Seq(Count(lit(1))
+}
+
+// unresolved attribute should not work
+intercept[AnalysisException] {
+  ResolveInlineTables.validateInputEvaluable(
+UnresolvedInlineTable(Seq("c1", "c2"), 
Seq(Seq(UnresolvedAttribute("A")
--- End diff --

But how would a user construct an AttributeReference?



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes 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 #14676: [SPARK-16947][SQL] Support type coercion and fold...

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

https://github.com/apache/spark/pull/14676#discussion_r75249457
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTables.scala
 ---
@@ -0,0 +1,105 @@
+/*
+ * 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.analysis
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Cast, 
InterpretedProjection, Unevaluable}
+import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, 
LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.types.{StructField, StructType}
+
+/**
+ * An analyzer rule that replaces [[UnresolvedInlineTable]] with 
[[LocalRelation]].
+ */
+object ResolveInlineTables extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
+case table: UnresolvedInlineTable if table.expressionsResolved =>
+  validateInputDimension(table)
+  validateInputEvaluable(table)
+  convert(table)
+  }
+
+  /**
+   * Validates that all inline table data are foldable expressions.
+   *
+   * This is package visible for unit testing.
+   */
+  private[analysis] def validateInputEvaluable(table: 
UnresolvedInlineTable): Unit = {
+table.rows.foreach { row =>
+  row.foreach { e =>
+if (!e.resolved || e.isInstanceOf[Unevaluable]) {
+  e.failAnalysis(s"cannot evaluate expression ${e.sql} in inline 
table definition")
+}
+  }
+}
+  }
+
+  /**
+   * Validates the input data dimension:
+   * 1. All rows have the same cardinality.
+   * 2. The number of column aliases defined is consistent with the number 
of columns in data.
+   *
+   * This is package visible for unit testing.
+   */
+  private[analysis] def validateInputDimension(table: 
UnresolvedInlineTable): Unit = {
+if (table.rows.nonEmpty) {
+  val numCols = table.rows.head.size
+  table.rows.zipWithIndex.foreach { case (row, ri) =>
+if (row.size != numCols) {
+  table.failAnalysis(s"expected $numCols columns but found 
${row.size} columns in row $ri")
+}
+  }
+
+  if (table.names.size != numCols) {
+table.failAnalysis(s"expected ${table.names.size} columns but 
found $numCols in first row")
+  }
+}
+  }
+
+  /**
+   * Convert a valid (with right shape and foldable inputs) 
[[UnresolvedInlineTable]]
+   * into a [[LocalRelation]].
+   *
+   * This function attempts to coerce inputs into consistent types.
+   *
+   * This is package visible for unit testing.
+   */
+  private[analysis] def convert(table: UnresolvedInlineTable): 
LocalRelation = {
+val numCols = table.rows.head.size
+
+// For each column, traverse all the values and find a common data 
type.
+val targetTypes = table.rows.transpose.zip(table.names).map { case 
(column, name) =>
+  val inputTypes = column.map(_.dataType)
+  
TypeCoercion.findWiderTypeWithoutStringPromotion(inputTypes).getOrElse {
--- End diff --

I don't have a strong preference, cc @hvanhovell 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes 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 #14676: [SPARK-16947][SQL] Support type coercion and fold...

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

https://github.com/apache/spark/pull/14676#discussion_r75249365
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTablesSuite.scala
 ---
@@ -0,0 +1,89 @@
+/*
+ * 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.analysis
+
+import org.scalatest.BeforeAndAfter
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Literal, Rand}
+import org.apache.spark.sql.catalyst.expressions.aggregate.Count
+import org.apache.spark.sql.catalyst.plans.PlanTest
+import org.apache.spark.sql.types.LongType
+
+/**
+ * Unit tests for [[ResolveInlineTables]]. Note that there are also test 
cases defined in
+ * end-to-end tests (in sql/core module) for verifying the correct error 
messages are shown
+ * in negative cases.
+ */
+class ResolveInlineTablesSuite extends PlanTest with BeforeAndAfter {
+
+  private def lit(v: Any): Literal = Literal(v)
+
+  test("validate inputs are foldable") {
+ResolveInlineTables.validateInputEvaluable(
+  UnresolvedInlineTable(Seq("c1", "c2"), Seq(Seq(lit(1)
+
+// nondeterministic (rand) should be fine
+ResolveInlineTables.validateInputEvaluable(
+  UnresolvedInlineTable(Seq("c1", "c2"), Seq(Seq(Rand(1)
+
+// aggregate should not work
+intercept[AnalysisException] {
+  ResolveInlineTables.validateInputEvaluable(
+UnresolvedInlineTable(Seq("c1", "c2"), Seq(Seq(Count(lit(1))
+}
+
+// unresolved attribute should not work
+intercept[AnalysisException] {
+  ResolveInlineTables.validateInputEvaluable(
+UnresolvedInlineTable(Seq("c1", "c2"), 
Seq(Seq(UnresolvedAttribute("A")
--- End diff --

the `Add` will be resolved and evaluable, but not foldable.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes 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 #14676: [SPARK-16947][SQL] Support type coercion and fold...

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

https://github.com/apache/spark/pull/14676#discussion_r75249341
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTablesSuite.scala
 ---
@@ -0,0 +1,89 @@
+/*
+ * 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.analysis
+
+import org.scalatest.BeforeAndAfter
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Literal, Rand}
+import org.apache.spark.sql.catalyst.expressions.aggregate.Count
+import org.apache.spark.sql.catalyst.plans.PlanTest
+import org.apache.spark.sql.types.LongType
+
+/**
+ * Unit tests for [[ResolveInlineTables]]. Note that there are also test 
cases defined in
+ * end-to-end tests (in sql/core module) for verifying the correct error 
messages are shown
+ * in negative cases.
+ */
+class ResolveInlineTablesSuite extends PlanTest with BeforeAndAfter {
+
+  private def lit(v: Any): Literal = Literal(v)
+
+  test("validate inputs are foldable") {
+ResolveInlineTables.validateInputEvaluable(
+  UnresolvedInlineTable(Seq("c1", "c2"), Seq(Seq(lit(1)
+
+// nondeterministic (rand) should be fine
+ResolveInlineTables.validateInputEvaluable(
+  UnresolvedInlineTable(Seq("c1", "c2"), Seq(Seq(Rand(1)
+
+// aggregate should not work
+intercept[AnalysisException] {
+  ResolveInlineTables.validateInputEvaluable(
+UnresolvedInlineTable(Seq("c1", "c2"), Seq(Seq(Count(lit(1))
+}
+
+// unresolved attribute should not work
+intercept[AnalysisException] {
+  ResolveInlineTables.validateInputEvaluable(
+UnresolvedInlineTable(Seq("c1", "c2"), 
Seq(Seq(UnresolvedAttribute("A")
--- End diff --

how about `UnresolvedInlineTable(Seq("c1", "c2"), 
Seq(Seq(AttributeReference("A") + 1)))`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes 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 #14676: [SPARK-16947][SQL] Support type coercion and fold...

2016-08-17 Thread petermaxlee
Github user petermaxlee commented on a diff in the pull request:

https://github.com/apache/spark/pull/14676#discussion_r75248833
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTables.scala
 ---
@@ -0,0 +1,105 @@
+/*
+ * 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.analysis
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Cast, 
InterpretedProjection, Unevaluable}
+import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, 
LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.types.{StructField, StructType}
+
+/**
+ * An analyzer rule that replaces [[UnresolvedInlineTable]] with 
[[LocalRelation]].
+ */
+object ResolveInlineTables extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
+case table: UnresolvedInlineTable if table.expressionsResolved =>
+  validateInputDimension(table)
+  validateInputEvaluable(table)
+  convert(table)
+  }
+
+  /**
+   * Validates that all inline table data are foldable expressions.
+   *
+   * This is package visible for unit testing.
+   */
+  private[analysis] def validateInputEvaluable(table: 
UnresolvedInlineTable): Unit = {
+table.rows.foreach { row =>
+  row.foreach { e =>
+if (!e.resolved || e.isInstanceOf[Unevaluable]) {
+  e.failAnalysis(s"cannot evaluate expression ${e.sql} in inline 
table definition")
+}
+  }
+}
+  }
+
+  /**
+   * Validates the input data dimension:
+   * 1. All rows have the same cardinality.
+   * 2. The number of column aliases defined is consistent with the number 
of columns in data.
+   *
+   * This is package visible for unit testing.
+   */
+  private[analysis] def validateInputDimension(table: 
UnresolvedInlineTable): Unit = {
+if (table.rows.nonEmpty) {
+  val numCols = table.rows.head.size
+  table.rows.zipWithIndex.foreach { case (row, ri) =>
+if (row.size != numCols) {
+  table.failAnalysis(s"expected $numCols columns but found 
${row.size} columns in row $ri")
+}
+  }
+
+  if (table.names.size != numCols) {
+table.failAnalysis(s"expected ${table.names.size} columns but 
found $numCols in first row")
+  }
+}
+  }
+
+  /**
+   * Convert a valid (with right shape and foldable inputs) 
[[UnresolvedInlineTable]]
+   * into a [[LocalRelation]].
+   *
+   * This function attempts to coerce inputs into consistent types.
+   *
+   * This is package visible for unit testing.
+   */
+  private[analysis] def convert(table: UnresolvedInlineTable): 
LocalRelation = {
+val numCols = table.rows.head.size
+
+// For each column, traverse all the values and find a common data 
type.
+val targetTypes = table.rows.transpose.zip(table.names).map { case 
(column, name) =>
+  val inputTypes = column.map(_.dataType)
+  
TypeCoercion.findWiderTypeWithoutStringPromotion(inputTypes).getOrElse {
--- End diff --

Postgres doesn't allow it. We can choose to be consistent with union 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 #14676: [SPARK-16947][SQL] Support type coercion and fold...

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

https://github.com/apache/spark/pull/14676#discussion_r75248623
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTables.scala
 ---
@@ -0,0 +1,105 @@
+/*
+ * 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.analysis
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Cast, 
InterpretedProjection, Unevaluable}
+import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, 
LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.types.{StructField, StructType}
+
+/**
+ * An analyzer rule that replaces [[UnresolvedInlineTable]] with 
[[LocalRelation]].
+ */
+object ResolveInlineTables extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
+case table: UnresolvedInlineTable if table.expressionsResolved =>
+  validateInputDimension(table)
+  validateInputEvaluable(table)
+  convert(table)
+  }
+
+  /**
+   * Validates that all inline table data are foldable expressions.
+   *
+   * This is package visible for unit testing.
+   */
+  private[analysis] def validateInputEvaluable(table: 
UnresolvedInlineTable): Unit = {
+table.rows.foreach { row =>
+  row.foreach { e =>
+if (!e.resolved || e.isInstanceOf[Unevaluable]) {
+  e.failAnalysis(s"cannot evaluate expression ${e.sql} in inline 
table definition")
+}
+  }
+}
+  }
+
+  /**
+   * Validates the input data dimension:
+   * 1. All rows have the same cardinality.
+   * 2. The number of column aliases defined is consistent with the number 
of columns in data.
+   *
+   * This is package visible for unit testing.
+   */
+  private[analysis] def validateInputDimension(table: 
UnresolvedInlineTable): Unit = {
+if (table.rows.nonEmpty) {
+  val numCols = table.rows.head.size
+  table.rows.zipWithIndex.foreach { case (row, ri) =>
+if (row.size != numCols) {
+  table.failAnalysis(s"expected $numCols columns but found 
${row.size} columns in row $ri")
+}
+  }
+
+  if (table.names.size != numCols) {
+table.failAnalysis(s"expected ${table.names.size} columns but 
found $numCols in first row")
+  }
+}
+  }
+
+  /**
+   * Convert a valid (with right shape and foldable inputs) 
[[UnresolvedInlineTable]]
+   * into a [[LocalRelation]].
+   *
+   * This function attempts to coerce inputs into consistent types.
+   *
+   * This is package visible for unit testing.
+   */
+  private[analysis] def convert(table: UnresolvedInlineTable): 
LocalRelation = {
+val numCols = table.rows.head.size
+
+// For each column, traverse all the values and find a common data 
type.
+val targetTypes = table.rows.transpose.zip(table.names).map { case 
(column, name) =>
+  val inputTypes = column.map(_.dataType)
+  
TypeCoercion.findWiderTypeWithoutStringPromotion(inputTypes).getOrElse {
--- End diff --

Can you check with other databases? Should we do string promotion for 
inline table? FYI expressions in `Union` can promote to string.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes 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 #14676: [SPARK-16947][SQL] Support type coercion and fold...

2016-08-17 Thread petermaxlee
Github user petermaxlee commented on a diff in the pull request:

https://github.com/apache/spark/pull/14676#discussion_r75248513
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTables.scala
 ---
@@ -0,0 +1,105 @@
+/*
+ * 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.analysis
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Cast, 
InterpretedProjection, Unevaluable}
+import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, 
LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.types.{StructField, StructType}
+
+/**
+ * An analyzer rule that replaces [[UnresolvedInlineTable]] with 
[[LocalRelation]].
+ */
+object ResolveInlineTables extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
+case table: UnresolvedInlineTable if table.expressionsResolved =>
+  validateInputDimension(table)
+  validateInputEvaluable(table)
+  convert(table)
+  }
+
+  /**
+   * Validates that all inline table data are foldable expressions.
+   *
+   * This is package visible for unit testing.
+   */
+  private[analysis] def validateInputEvaluable(table: 
UnresolvedInlineTable): Unit = {
+table.rows.foreach { row =>
+  row.foreach { e =>
+if (!e.resolved || e.isInstanceOf[Unevaluable]) {
+  e.failAnalysis(s"cannot evaluate expression ${e.sql} in inline 
table definition")
+}
+  }
+}
+  }
+
+  /**
+   * Validates the input data dimension:
+   * 1. All rows have the same cardinality.
+   * 2. The number of column aliases defined is consistent with the number 
of columns in data.
+   *
+   * This is package visible for unit testing.
+   */
+  private[analysis] def validateInputDimension(table: 
UnresolvedInlineTable): Unit = {
+if (table.rows.nonEmpty) {
+  val numCols = table.rows.head.size
+  table.rows.zipWithIndex.foreach { case (row, ri) =>
--- End diff --

That's a good idea. Let me do that.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes 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 #14676: [SPARK-16947][SQL] Support type coercion and fold...

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

https://github.com/apache/spark/pull/14676#discussion_r75248490
  
--- Diff: sql/core/src/test/resources/sql-tests/inputs/inline-table.sql ---
@@ -0,0 +1,48 @@
+
+-- single row, without table and column alias
+select * from values ("one", 1);
+
+-- single row, without column alias
+select * from values ("one", 1) as data;
+
+-- single row
+select * from values ("one", 1) as data(a, b);
+
+-- single column multiple rows
+select * from values 1, 2, 3 as data(a);
+
+-- two rows
--- End diff --

nit: 3 rows


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes 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 #14676: [SPARK-16947][SQL] Support type coercion and fold...

2016-08-17 Thread petermaxlee
Github user petermaxlee commented on a diff in the pull request:

https://github.com/apache/spark/pull/14676#discussion_r75248427
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTables.scala
 ---
@@ -0,0 +1,105 @@
+/*
+ * 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.analysis
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Cast, 
InterpretedProjection, Unevaluable}
+import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, 
LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.types.{StructField, StructType}
+
+/**
+ * An analyzer rule that replaces [[UnresolvedInlineTable]] with 
[[LocalRelation]].
+ */
+object ResolveInlineTables extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
+case table: UnresolvedInlineTable if table.expressionsResolved =>
+  validateInputDimension(table)
+  validateInputEvaluable(table)
+  convert(table)
+  }
+
+  /**
+   * Validates that all inline table data are foldable expressions.
+   *
+   * This is package visible for unit testing.
+   */
+  private[analysis] def validateInputEvaluable(table: 
UnresolvedInlineTable): Unit = {
+table.rows.foreach { row =>
+  row.foreach { e =>
+if (!e.resolved || e.isInstanceOf[Unevaluable]) {
+  e.failAnalysis(s"cannot evaluate expression ${e.sql} in inline 
table definition")
+}
+  }
+}
+  }
+
+  /**
+   * Validates the input data dimension:
+   * 1. All rows have the same cardinality.
+   * 2. The number of column aliases defined is consistent with the number 
of columns in data.
+   *
+   * This is package visible for unit testing.
+   */
+  private[analysis] def validateInputDimension(table: 
UnresolvedInlineTable): Unit = {
+if (table.rows.nonEmpty) {
+  val numCols = table.rows.head.size
+  table.rows.zipWithIndex.foreach { case (row, ri) =>
+if (row.size != numCols) {
+  table.failAnalysis(s"expected $numCols columns but found 
${row.size} columns in row $ri")
+}
+  }
+
+  if (table.names.size != numCols) {
+table.failAnalysis(s"expected ${table.names.size} columns but 
found $numCols in first row")
+  }
+}
+  }
+
+  /**
+   * Convert a valid (with right shape and foldable inputs) 
[[UnresolvedInlineTable]]
+   * into a [[LocalRelation]].
+   *
+   * This function attempts to coerce inputs into consistent types.
+   *
+   * This is package visible for unit testing.
+   */
+  private[analysis] def convert(table: UnresolvedInlineTable): 
LocalRelation = {
+val numCols = table.rows.head.size
+
+// For each column, traverse all the values and find a common data 
type.
+val targetTypes = table.rows.transpose.zip(table.names).map { case 
(column, name) =>
+  val inputTypes = column.map(_.dataType)
+  
TypeCoercion.findWiderTypeWithoutStringPromotion(inputTypes).getOrElse {
+table.failAnalysis(s"incompatible types found in column $name for 
inline table")
+  }
+}
+assert(targetTypes.size == table.names.size)
--- End diff --

asserts are not meant to be user facing. They are meant to be defensive 
against programming errors (i.e. bugs in Spark).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes 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, 

[GitHub] spark pull request #14676: [SPARK-16947][SQL] Support type coercion and fold...

2016-08-17 Thread petermaxlee
Github user petermaxlee commented on a diff in the pull request:

https://github.com/apache/spark/pull/14676#discussion_r75248346
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTables.scala
 ---
@@ -0,0 +1,105 @@
+/*
+ * 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.analysis
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Cast, 
InterpretedProjection, Unevaluable}
+import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, 
LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.types.{StructField, StructType}
+
+/**
+ * An analyzer rule that replaces [[UnresolvedInlineTable]] with 
[[LocalRelation]].
+ */
+object ResolveInlineTables extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
+case table: UnresolvedInlineTable if table.expressionsResolved =>
+  validateInputDimension(table)
+  validateInputEvaluable(table)
+  convert(table)
+  }
+
+  /**
+   * Validates that all inline table data are foldable expressions.
+   *
+   * This is package visible for unit testing.
+   */
+  private[analysis] def validateInputEvaluable(table: 
UnresolvedInlineTable): Unit = {
--- End diff --

This was suggested by @hvanhovell.

I think private functions are still meant to be private. This is only 
package visible for the purpose of testing. That is to say, I don't expect 
developers to be calling this function either.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes 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 #14676: [SPARK-16947][SQL] Support type coercion and fold...

2016-08-17 Thread petermaxlee
Github user petermaxlee commented on a diff in the pull request:

https://github.com/apache/spark/pull/14676#discussion_r75248351
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTables.scala
 ---
@@ -0,0 +1,105 @@
+/*
+ * 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.analysis
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Cast, 
InterpretedProjection, Unevaluable}
+import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, 
LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.types.{StructField, StructType}
+
+/**
+ * An analyzer rule that replaces [[UnresolvedInlineTable]] with 
[[LocalRelation]].
+ */
+object ResolveInlineTables extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
+case table: UnresolvedInlineTable if table.expressionsResolved =>
+  validateInputDimension(table)
+  validateInputEvaluable(table)
+  convert(table)
+  }
+
+  /**
+   * Validates that all inline table data are foldable expressions.
+   *
+   * This is package visible for unit testing.
+   */
+  private[analysis] def validateInputEvaluable(table: 
UnresolvedInlineTable): Unit = {
+table.rows.foreach { row =>
+  row.foreach { e =>
+if (!e.resolved || e.isInstanceOf[Unevaluable]) {
--- End diff --

If we check foldable rand() wouldn't work.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes 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 #14676: [SPARK-16947][SQL] Support type coercion and fold...

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

https://github.com/apache/spark/pull/14676#discussion_r75248289
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTables.scala
 ---
@@ -0,0 +1,105 @@
+/*
+ * 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.analysis
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Cast, 
InterpretedProjection, Unevaluable}
+import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, 
LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.types.{StructField, StructType}
+
+/**
+ * An analyzer rule that replaces [[UnresolvedInlineTable]] with 
[[LocalRelation]].
+ */
+object ResolveInlineTables extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
+case table: UnresolvedInlineTable if table.expressionsResolved =>
+  validateInputDimension(table)
+  validateInputEvaluable(table)
+  convert(table)
+  }
+
+  /**
+   * Validates that all inline table data are foldable expressions.
+   *
+   * This is package visible for unit testing.
+   */
+  private[analysis] def validateInputEvaluable(table: 
UnresolvedInlineTable): Unit = {
+table.rows.foreach { row =>
+  row.foreach { e =>
+if (!e.resolved || e.isInstanceOf[Unevaluable]) {
+  e.failAnalysis(s"cannot evaluate expression ${e.sql} in inline 
table definition")
+}
+  }
+}
+  }
+
+  /**
+   * Validates the input data dimension:
+   * 1. All rows have the same cardinality.
+   * 2. The number of column aliases defined is consistent with the number 
of columns in data.
+   *
+   * This is package visible for unit testing.
+   */
+  private[analysis] def validateInputDimension(table: 
UnresolvedInlineTable): Unit = {
+if (table.rows.nonEmpty) {
+  val numCols = table.rows.head.size
+  table.rows.zipWithIndex.foreach { case (row, ri) =>
+if (row.size != numCols) {
+  table.failAnalysis(s"expected $numCols columns but found 
${row.size} columns in row $ri")
+}
+  }
+
+  if (table.names.size != numCols) {
+table.failAnalysis(s"expected ${table.names.size} columns but 
found $numCols in first row")
+  }
+}
+  }
+
+  /**
+   * Convert a valid (with right shape and foldable inputs) 
[[UnresolvedInlineTable]]
+   * into a [[LocalRelation]].
+   *
+   * This function attempts to coerce inputs into consistent types.
+   *
+   * This is package visible for unit testing.
+   */
+  private[analysis] def convert(table: UnresolvedInlineTable): 
LocalRelation = {
+val numCols = table.rows.head.size
+
+// For each column, traverse all the values and find a common data 
type.
+val targetTypes = table.rows.transpose.zip(table.names).map { case 
(column, name) =>
+  val inputTypes = column.map(_.dataType)
+  
TypeCoercion.findWiderTypeWithoutStringPromotion(inputTypes).getOrElse {
+table.failAnalysis(s"incompatible types found in column $name for 
inline table")
+  }
+}
+assert(targetTypes.size == table.names.size)
--- End diff --

it's duplicated, `validateInputDimension` already guarantees 
`table.names.size` is equal to number of column


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes 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 #14676: [SPARK-16947][SQL] Support type coercion and fold...

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

https://github.com/apache/spark/pull/14676#discussion_r75248129
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTables.scala
 ---
@@ -0,0 +1,105 @@
+/*
+ * 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.analysis
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Cast, 
InterpretedProjection, Unevaluable}
+import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, 
LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.types.{StructField, StructType}
+
+/**
+ * An analyzer rule that replaces [[UnresolvedInlineTable]] with 
[[LocalRelation]].
+ */
+object ResolveInlineTables extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
+case table: UnresolvedInlineTable if table.expressionsResolved =>
+  validateInputDimension(table)
+  validateInputEvaluable(table)
+  convert(table)
+  }
+
+  /**
+   * Validates that all inline table data are foldable expressions.
+   *
+   * This is package visible for unit testing.
+   */
+  private[analysis] def validateInputEvaluable(table: 
UnresolvedInlineTable): Unit = {
+table.rows.foreach { row =>
+  row.foreach { e =>
+if (!e.resolved || e.isInstanceOf[Unevaluable]) {
+  e.failAnalysis(s"cannot evaluate expression ${e.sql} in inline 
table definition")
+}
+  }
+}
+  }
+
+  /**
+   * Validates the input data dimension:
+   * 1. All rows have the same cardinality.
+   * 2. The number of column aliases defined is consistent with the number 
of columns in data.
+   *
+   * This is package visible for unit testing.
+   */
+  private[analysis] def validateInputDimension(table: 
UnresolvedInlineTable): Unit = {
+if (table.rows.nonEmpty) {
+  val numCols = table.rows.head.size
+  table.rows.zipWithIndex.foreach { case (row, ri) =>
--- End diff --

shall we just get the `table.names.size` first and iterate the rows?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes 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 #14676: [SPARK-16947][SQL] Support type coercion and fold...

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

https://github.com/apache/spark/pull/14676#discussion_r75248030
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTables.scala
 ---
@@ -0,0 +1,105 @@
+/*
+ * 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.analysis
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Cast, 
InterpretedProjection, Unevaluable}
+import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, 
LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.types.{StructField, StructType}
+
+/**
+ * An analyzer rule that replaces [[UnresolvedInlineTable]] with 
[[LocalRelation]].
+ */
+object ResolveInlineTables extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
+case table: UnresolvedInlineTable if table.expressionsResolved =>
+  validateInputDimension(table)
+  validateInputEvaluable(table)
+  convert(table)
+  }
+
+  /**
+   * Validates that all inline table data are foldable expressions.
+   *
+   * This is package visible for unit testing.
+   */
+  private[analysis] def validateInputEvaluable(table: 
UnresolvedInlineTable): Unit = {
+table.rows.foreach { row =>
+  row.foreach { e =>
+if (!e.resolved || e.isInstanceOf[Unevaluable]) {
--- End diff --

why do we check `Unevaluable` instead of `foldable` 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 #14676: [SPARK-16947][SQL] Support type coercion and fold...

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

https://github.com/apache/spark/pull/14676#discussion_r75247958
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTables.scala
 ---
@@ -0,0 +1,105 @@
+/*
+ * 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.analysis
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Cast, 
InterpretedProjection, Unevaluable}
+import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, 
LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.types.{StructField, StructType}
+
+/**
+ * An analyzer rule that replaces [[UnresolvedInlineTable]] with 
[[LocalRelation]].
+ */
+object ResolveInlineTables extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
+case table: UnresolvedInlineTable if table.expressionsResolved =>
+  validateInputDimension(table)
+  validateInputEvaluable(table)
+  convert(table)
+  }
+
+  /**
+   * Validates that all inline table data are foldable expressions.
+   *
+   * This is package visible for unit testing.
+   */
+  private[analysis] def validateInputEvaluable(table: 
UnresolvedInlineTable): Unit = {
--- End diff --

it doesn't need to be package private, see 
https://github.com/apache/spark/commit/064d91ff7342002414d3274694a8e2e37f154986


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes 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 #14676: [SPARK-16947][SQL] Support type coercion and fold...

2016-08-17 Thread petermaxlee
Github user petermaxlee commented on a diff in the pull request:

https://github.com/apache/spark/pull/14676#discussion_r75244731
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTables.scala
 ---
@@ -0,0 +1,109 @@
+/*
+ * 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.analysis
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Cast
+import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, 
LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.types.{StructField, StructType}
+
+/**
+ * An analyzer rule that replaces [[UnresolvedInlineTable]] with 
[[LocalRelation]].
+ */
+object ResolveInlineTables extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
+case table: UnresolvedInlineTable if table.expressionsResolved =>
+  validateInputDimension(table)
+  validateInputFoldable(table)
+  convert(table)
+  }
+
+  /**
+   * Validates that all inline table data are foldable expressions.
+   *
+   * This is publicly visible for unit testing.
+   */
+  def validateInputFoldable(table: UnresolvedInlineTable): Unit = {
+table.rows.foreach { row =>
+  row.foreach { e =>
+if (!e.resolved || !e.foldable) {
+  e.failAnalysis(s"cannot evaluate expression ${e.sql} in inline 
table definition")
+}
+  }
+}
+  }
+
+  /**
+   * Validates the input data dimension:
+   * 1. All rows have the same cardinality.
+   * 2. The number of column aliases defined is consistent with the number 
of columns in data.
+   *
+   * This is publicly visible for unit testing.
+   */
+  def validateInputDimension(table: UnresolvedInlineTable): Unit = {
+if (table.rows.nonEmpty) {
+  val numCols = table.rows.head.size
+  table.rows.zipWithIndex.foreach { case (row, ri) =>
+if (row.size != numCols) {
+  table.failAnalysis(s"expected $numCols columns but found 
${row.size} columns in row $ri")
+}
+  }
+
+  if (table.names.size != numCols) {
+table.failAnalysis(s"expected ${table.names.size} columns but 
found $numCols in first row")
+  }
+}
+  }
+
+  /**
+   * Convert a valid (with right shape and foldable inputs) 
[[UnresolvedInlineTable]]
+   * into a [[LocalRelation]].
+   *
+   * This function attempts to coerce inputs into consistent types.
+   *
+   * This is publicly visible for unit testing.
+   */
+  def convert(table: UnresolvedInlineTable): LocalRelation = {
+val numCols = table.rows.head.size
+
+// For each column, traverse all the values and find a common data 
type.
+val targetTypes = Seq.tabulate(numCols) { ci =>
--- End diff --

I didn't check for nullability, since I don't think it actually matters 
much for the purpose we are using this feature. What would be useful is to be 
able to define the data type explicitly, and then we can do controlled tests 
for nullability.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes 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 #14676: [SPARK-16947][SQL] Support type coercion and fold...

2016-08-17 Thread hvanhovell
Github user hvanhovell commented on a diff in the pull request:

https://github.com/apache/spark/pull/14676#discussion_r75228521
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTables.scala
 ---
@@ -0,0 +1,105 @@
+/*
+ * 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.analysis
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Cast, 
InterpretedProjection}
+import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, 
LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.types.{StructField, StructType}
+
+/**
+ * An analyzer rule that replaces [[UnresolvedInlineTable]] with 
[[LocalRelation]].
+ */
+object ResolveInlineTables extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
+case table: UnresolvedInlineTable if table.expressionsResolved =>
+  validateInputDimension(table)
+  // validateInputFoldable(table)
--- End diff --

Check if the expression does not implement `Unevaluable`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes 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 #14676: [SPARK-16947][SQL] Support type coercion and fold...

2016-08-17 Thread petermaxlee
Github user petermaxlee commented on a diff in the pull request:

https://github.com/apache/spark/pull/14676#discussion_r75228342
  
--- Diff: 
sql/core/src/test/resources/sql-tests/results/inline-table.sql.out ---
@@ -0,0 +1,135 @@
+-- Automatically generated by SQLQueryTestSuite
+-- Number of queries: 15
+
+
+-- !query 0
+select * from values ("one", 1)
+-- !query 0 schema
+struct
+-- !query 0 output
+one1
+
+
+-- !query 1
+select * from values ("one", 1) as data
+-- !query 1 schema
+struct
+-- !query 1 output
+one1
+
+
+-- !query 2
+select * from values ("one", 1) as data(a, b)
+-- !query 2 schema
+struct
+-- !query 2 output
+one1
+
+
+-- !query 3
+select * from values ("one", 1), ("two", 2), ("three", null) as data(a, b)
+-- !query 3 schema
+struct
+-- !query 3 output
+one1
+three  NULL
+two2
+
+
+-- !query 4
+select * from values ("one", null), ("two", null) as data(a, b)
+-- !query 4 schema
+struct
+-- !query 4 output
+oneNULL
+twoNULL
+
+
+-- !query 5
+select * from values ("one", 1), ("two", 2L) as data(a, b)
+-- !query 5 schema
+struct
+-- !query 5 output
+one1
+two2
+
+
+-- !query 6
+select * from values ("one", 1 + 0), ("two", 1 + 3L) as data(a, b)
+-- !query 6 schema
+struct
+-- !query 6 output
+one1
+two4
+
+
+-- !query 7
+select * from values ("one", array(0, 1)), ("two", array(2, 3)) as data(a, 
b)
+-- !query 7 schema
+struct
+-- !query 7 output
+one[0,1]
+two[2,3]
+
+
+-- !query 8
+select * from values ("one", 2.0), ("two", 3.0D) as data(a, b)
+-- !query 8 schema
+struct
+-- !query 8 output
+one2.0
+two3.0
+
+
+-- !query 9
+select * from values ("one", rand(5)), ("two", 3.0D) as data(a, b)
+-- !query 9 schema
+struct
+-- !query 9 output
+one0.087440518337355
+two3.0
+
+
+-- !query 10
+select * from values ("one", 2.0), ("two") as data(a, b)
+-- !query 10 schema
+struct<>
+-- !query 10 output
+org.apache.spark.sql.AnalysisException
+expected 2 columns but found 1 columns in row 1; line 1 pos 14
+
+
+-- !query 11
+select * from values ("one", array(0, 1)), ("two", struct(1, 2)) as 
data(a, b)
+-- !query 11 schema
+struct<>
+-- !query 11 output
+org.apache.spark.sql.AnalysisException
+incompatible types found in column b for inline table; line 1 pos 14
+
+
+-- !query 12
+select * from values ("one"), ("two") as data(a, b)
+-- !query 12 schema
+struct<>
+-- !query 12 output
+org.apache.spark.sql.AnalysisException
+expected 2 columns but found 1 in first row; line 1 pos 14
+
+
+-- !query 13
+select * from values ("one", random_not_exist_func(1)), ("two", 2) as 
data(a, b)
+-- !query 13 schema
+struct<>
+-- !query 13 output
+org.apache.spark.sql.AnalysisException
+Undefined function: 'random_not_exist_func'. This function is neither a 
registered temporary function nor a permanent function registered in the 
database 'default'.; line 1 pos 29
+
+
+-- !query 14
+select * from values ("one", count(1)), ("two", 2) as data(a, b)
+-- !query 14 schema
+struct<>
+-- !query 14 output
+java.lang.UnsupportedOperationException
+Cannot evaluate expression: count(1)
--- End diff --

this is not a great exception because it is thrown by the projection - not 
during analysis.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes 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 #14676: [SPARK-16947][SQL] Support type coercion and fold...

2016-08-17 Thread hvanhovell
Github user hvanhovell commented on a diff in the pull request:

https://github.com/apache/spark/pull/14676#discussion_r75228317
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTables.scala
 ---
@@ -0,0 +1,109 @@
+/*
+ * 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.analysis
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Cast
+import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, 
LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.types.{StructField, StructType}
+
+/**
+ * An analyzer rule that replaces [[UnresolvedInlineTable]] with 
[[LocalRelation]].
+ */
+object ResolveInlineTables extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
+case table: UnresolvedInlineTable if table.expressionsResolved =>
+  validateInputDimension(table)
+  validateInputFoldable(table)
+  convert(table)
+  }
+
+  /**
+   * Validates that all inline table data are foldable expressions.
+   *
+   * This is publicly visible for unit testing.
+   */
+  def validateInputFoldable(table: UnresolvedInlineTable): Unit = {
+table.rows.foreach { row =>
+  row.foreach { e =>
+if (!e.resolved || !e.foldable) {
+  e.failAnalysis(s"cannot evaluate expression ${e.sql} in inline 
table definition")
+}
+  }
+}
+  }
+
+  /**
+   * Validates the input data dimension:
+   * 1. All rows have the same cardinality.
+   * 2. The number of column aliases defined is consistent with the number 
of columns in data.
+   *
+   * This is publicly visible for unit testing.
+   */
+  def validateInputDimension(table: UnresolvedInlineTable): Unit = {
+if (table.rows.nonEmpty) {
+  val numCols = table.rows.head.size
+  table.rows.zipWithIndex.foreach { case (row, ri) =>
+if (row.size != numCols) {
+  table.failAnalysis(s"expected $numCols columns but found 
${row.size} columns in row $ri")
+}
+  }
+
+  if (table.names.size != numCols) {
+table.failAnalysis(s"expected ${table.names.size} columns but 
found $numCols in first row")
+  }
+}
+  }
+
+  /**
+   * Convert a valid (with right shape and foldable inputs) 
[[UnresolvedInlineTable]]
+   * into a [[LocalRelation]].
+   *
+   * This function attempts to coerce inputs into consistent types.
+   *
+   * This is publicly visible for unit testing.
+   */
+  def convert(table: UnresolvedInlineTable): LocalRelation = {
+val numCols = table.rows.head.size
+
+// For each column, traverse all the values and find a common data 
type.
+val targetTypes = Seq.tabulate(numCols) { ci =>
+  val inputTypes = table.rows.map(_(ci).dataType)
+  
TypeCoercion.findWiderTypeWithoutStringPromotion(inputTypes).getOrElse {
+table.failAnalysis(s"incompatible types found in column $ci for 
inline table")
+  }
+}
+assert(targetTypes.size == table.names.size)
+
+val newRows: Seq[InternalRow] = table.rows.map { row =>
+  InternalRow.fromSeq(row.zipWithIndex.map { case (e, ci) =>
+val targetType = targetTypes(ci)
+if (e.dataType.sameType(targetType)) {
+  e.eval()
--- End diff --

You should be fine if you rule out `Unevaluable` expressions. Count is one 
of those.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes 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 

[GitHub] spark pull request #14676: [SPARK-16947][SQL] Support type coercion and fold...

2016-08-17 Thread petermaxlee
Github user petermaxlee commented on a diff in the pull request:

https://github.com/apache/spark/pull/14676#discussion_r75228185
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTables.scala
 ---
@@ -0,0 +1,105 @@
+/*
+ * 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.analysis
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Cast, 
InterpretedProjection}
+import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, 
LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.types.{StructField, StructType}
+
+/**
+ * An analyzer rule that replaces [[UnresolvedInlineTable]] with 
[[LocalRelation]].
+ */
+object ResolveInlineTables extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
+case table: UnresolvedInlineTable if table.expressionsResolved =>
+  validateInputDimension(table)
+  // validateInputFoldable(table)
--- End diff --

@hvanhovell what's a good check to do here? I don't want to rule out rand 
but do want to rule out count(1).



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes 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 #14676: [SPARK-16947][SQL] Support type coercion and fold...

2016-08-17 Thread petermaxlee
Github user petermaxlee commented on a diff in the pull request:

https://github.com/apache/spark/pull/14676#discussion_r75227645
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTables.scala
 ---
@@ -0,0 +1,109 @@
+/*
+ * 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.analysis
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Cast
+import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, 
LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.types.{StructField, StructType}
+
+/**
+ * An analyzer rule that replaces [[UnresolvedInlineTable]] with 
[[LocalRelation]].
+ */
+object ResolveInlineTables extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
+case table: UnresolvedInlineTable if table.expressionsResolved =>
+  validateInputDimension(table)
+  validateInputFoldable(table)
+  convert(table)
+  }
+
+  /**
+   * Validates that all inline table data are foldable expressions.
+   *
+   * This is publicly visible for unit testing.
+   */
+  def validateInputFoldable(table: UnresolvedInlineTable): Unit = {
+table.rows.foreach { row =>
+  row.foreach { e =>
+if (!e.resolved || !e.foldable) {
+  e.failAnalysis(s"cannot evaluate expression ${e.sql} in inline 
table definition")
+}
+  }
+}
+  }
+
+  /**
+   * Validates the input data dimension:
+   * 1. All rows have the same cardinality.
+   * 2. The number of column aliases defined is consistent with the number 
of columns in data.
+   *
+   * This is publicly visible for unit testing.
+   */
+  def validateInputDimension(table: UnresolvedInlineTable): Unit = {
+if (table.rows.nonEmpty) {
+  val numCols = table.rows.head.size
+  table.rows.zipWithIndex.foreach { case (row, ri) =>
+if (row.size != numCols) {
+  table.failAnalysis(s"expected $numCols columns but found 
${row.size} columns in row $ri")
+}
+  }
+
+  if (table.names.size != numCols) {
+table.failAnalysis(s"expected ${table.names.size} columns but 
found $numCols in first row")
+  }
+}
+  }
+
+  /**
+   * Convert a valid (with right shape and foldable inputs) 
[[UnresolvedInlineTable]]
+   * into a [[LocalRelation]].
+   *
+   * This function attempts to coerce inputs into consistent types.
+   *
+   * This is publicly visible for unit testing.
+   */
+  def convert(table: UnresolvedInlineTable): LocalRelation = {
+val numCols = table.rows.head.size
+
+// For each column, traverse all the values and find a common data 
type.
+val targetTypes = Seq.tabulate(numCols) { ci =>
+  val inputTypes = table.rows.map(_(ci).dataType)
+  
TypeCoercion.findWiderTypeWithoutStringPromotion(inputTypes).getOrElse {
+table.failAnalysis(s"incompatible types found in column $ci for 
inline table")
+  }
+}
+assert(targetTypes.size == table.names.size)
+
+val newRows: Seq[InternalRow] = table.rows.map { row =>
+  InternalRow.fromSeq(row.zipWithIndex.map { case (e, ci) =>
+val targetType = targetTypes(ci)
+if (e.dataType.sameType(targetType)) {
+  e.eval()
--- End diff --

How do we determine if something is valid here? If we don't do a foldable 
check and want to support nondeterministic functions, how do we rule out 
something like count(1)?



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

[GitHub] spark pull request #14676: [SPARK-16947][SQL] Support type coercion and fold...

2016-08-17 Thread petermaxlee
Github user petermaxlee commented on a diff in the pull request:

https://github.com/apache/spark/pull/14676#discussion_r75173027
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala
 ---
@@ -50,6 +50,23 @@ case class UnresolvedRelation(
 }
 
 /**
+ * An inline table that has not been resolved yet. Once resolved, it is 
turned by the analyzer into
+ * a [[org.apache.spark.sql.catalyst.plans.logical.LocalRelation]].
+ *
+ * @param names list of column names
+ * @param rows expressions for the data
+ */
+case class UnresolvedInlineTable(
+names: Seq[String],
+rows: Seq[Seq[Expression]])
+  extends LeafNode {
+
+  lazy val expressionsResolved: Boolean = rows.forall(_.forall(_.resolved))
--- End diff --

I do want this memoized, so a lazy val is better 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 #14676: [SPARK-16947][SQL] Support type coercion and fold...

2016-08-17 Thread hvanhovell
Github user hvanhovell commented on a diff in the pull request:

https://github.com/apache/spark/pull/14676#discussion_r75110862
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTables.scala
 ---
@@ -0,0 +1,109 @@
+/*
+ * 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.analysis
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Cast
+import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, 
LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.types.{StructField, StructType}
+
+/**
+ * An analyzer rule that replaces [[UnresolvedInlineTable]] with 
[[LocalRelation]].
+ */
+object ResolveInlineTables extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
+case table: UnresolvedInlineTable if table.expressionsResolved =>
+  validateInputDimension(table)
+  validateInputFoldable(table)
+  convert(table)
+  }
+
+  /**
+   * Validates that all inline table data are foldable expressions.
+   *
+   * This is publicly visible for unit testing.
+   */
+  def validateInputFoldable(table: UnresolvedInlineTable): Unit = {
+table.rows.foreach { row =>
+  row.foreach { e =>
+if (!e.resolved || !e.foldable) {
+  e.failAnalysis(s"cannot evaluate expression ${e.sql} in inline 
table definition")
--- End diff --

You could also mention the row and the column for better UX.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes 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 #14676: [SPARK-16947][SQL] Support type coercion and fold...

2016-08-17 Thread hvanhovell
Github user hvanhovell commented on a diff in the pull request:

https://github.com/apache/spark/pull/14676#discussion_r75110610
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTables.scala
 ---
@@ -0,0 +1,109 @@
+/*
+ * 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.analysis
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Cast
+import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, 
LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.types.{StructField, StructType}
+
+/**
+ * An analyzer rule that replaces [[UnresolvedInlineTable]] with 
[[LocalRelation]].
+ */
+object ResolveInlineTables extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
+case table: UnresolvedInlineTable if table.expressionsResolved =>
+  validateInputDimension(table)
+  validateInputFoldable(table)
+  convert(table)
+  }
+
+  /**
+   * Validates that all inline table data are foldable expressions.
+   *
+   * This is publicly visible for unit testing.
--- End diff --

The test is located in the same package, so you could reduce visibility to 
protected/package.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes 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 #14676: [SPARK-16947][SQL] Support type coercion and fold...

2016-08-17 Thread hvanhovell
Github user hvanhovell commented on a diff in the pull request:

https://github.com/apache/spark/pull/14676#discussion_r7507
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTables.scala
 ---
@@ -0,0 +1,109 @@
+/*
+ * 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.analysis
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Cast
+import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, 
LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.types.{StructField, StructType}
+
+/**
+ * An analyzer rule that replaces [[UnresolvedInlineTable]] with 
[[LocalRelation]].
+ */
+object ResolveInlineTables extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
+case table: UnresolvedInlineTable if table.expressionsResolved =>
+  validateInputDimension(table)
+  validateInputFoldable(table)
+  convert(table)
+  }
+
+  /**
+   * Validates that all inline table data are foldable expressions.
+   *
+   * This is publicly visible for unit testing.
+   */
+  def validateInputFoldable(table: UnresolvedInlineTable): Unit = {
+table.rows.foreach { row =>
+  row.foreach { e =>
+if (!e.resolved || !e.foldable) {
+  e.failAnalysis(s"cannot evaluate expression ${e.sql} in inline 
table definition")
+}
+  }
+}
+  }
+
+  /**
+   * Validates the input data dimension:
+   * 1. All rows have the same cardinality.
+   * 2. The number of column aliases defined is consistent with the number 
of columns in data.
+   *
+   * This is publicly visible for unit testing.
+   */
+  def validateInputDimension(table: UnresolvedInlineTable): Unit = {
+if (table.rows.nonEmpty) {
+  val numCols = table.rows.head.size
+  table.rows.zipWithIndex.foreach { case (row, ri) =>
+if (row.size != numCols) {
+  table.failAnalysis(s"expected $numCols columns but found 
${row.size} columns in row $ri")
+}
+  }
+
+  if (table.names.size != numCols) {
+table.failAnalysis(s"expected ${table.names.size} columns but 
found $numCols in first row")
+  }
+}
+  }
+
+  /**
+   * Convert a valid (with right shape and foldable inputs) 
[[UnresolvedInlineTable]]
+   * into a [[LocalRelation]].
+   *
+   * This function attempts to coerce inputs into consistent types.
+   *
+   * This is publicly visible for unit testing.
+   */
+  def convert(table: UnresolvedInlineTable): LocalRelation = {
+val numCols = table.rows.head.size
+
+// For each column, traverse all the values and find a common data 
type.
+val targetTypes = Seq.tabulate(numCols) { ci =>
+  val inputTypes = table.rows.map(_(ci).dataType)
+  
TypeCoercion.findWiderTypeWithoutStringPromotion(inputTypes).getOrElse {
+table.failAnalysis(s"incompatible types found in column $ci for 
inline table")
+  }
+}
+assert(targetTypes.size == table.names.size)
+
+val newRows: Seq[InternalRow] = table.rows.map { row =>
+  InternalRow.fromSeq(row.zipWithIndex.map { case (e, ci) =>
+val targetType = targetTypes(ci)
+if (e.dataType.sameType(targetType)) {
+  e.eval()
--- End diff --

Try this with a `rand()` or any other nondeterministic expression and it 
will fail. In order to support these you have to create an 
InterpretedProjection per row and use this once, see the following code for a 
similar situation: 
https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala#L1575-L1576


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. 

[GitHub] spark pull request #14676: [SPARK-16947][SQL] Support type coercion and fold...

2016-08-17 Thread hvanhovell
Github user hvanhovell commented on a diff in the pull request:

https://github.com/apache/spark/pull/14676#discussion_r75078500
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTables.scala
 ---
@@ -0,0 +1,109 @@
+/*
+ * 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.analysis
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Cast
+import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, 
LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.types.{StructField, StructType}
+
+/**
+ * An analyzer rule that replaces [[UnresolvedInlineTable]] with 
[[LocalRelation]].
+ */
+object ResolveInlineTables extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
+case table: UnresolvedInlineTable if table.expressionsResolved =>
+  validateInputDimension(table)
+  validateInputFoldable(table)
+  convert(table)
+  }
+
+  /**
+   * Validates that all inline table data are foldable expressions.
+   *
+   * This is publicly visible for unit testing.
+   */
+  def validateInputFoldable(table: UnresolvedInlineTable): Unit = {
+table.rows.foreach { row =>
+  row.foreach { e =>
+if (!e.resolved || !e.foldable) {
+  e.failAnalysis(s"cannot evaluate expression ${e.sql} in inline 
table definition")
+}
+  }
+}
+  }
+
+  /**
+   * Validates the input data dimension:
+   * 1. All rows have the same cardinality.
+   * 2. The number of column aliases defined is consistent with the number 
of columns in data.
+   *
+   * This is publicly visible for unit testing.
+   */
+  def validateInputDimension(table: UnresolvedInlineTable): Unit = {
+if (table.rows.nonEmpty) {
+  val numCols = table.rows.head.size
+  table.rows.zipWithIndex.foreach { case (row, ri) =>
+if (row.size != numCols) {
+  table.failAnalysis(s"expected $numCols columns but found 
${row.size} columns in row $ri")
+}
+  }
+
+  if (table.names.size != numCols) {
+table.failAnalysis(s"expected ${table.names.size} columns but 
found $numCols in first row")
+  }
+}
+  }
+
+  /**
+   * Convert a valid (with right shape and foldable inputs) 
[[UnresolvedInlineTable]]
+   * into a [[LocalRelation]].
+   *
+   * This function attempts to coerce inputs into consistent types.
+   *
+   * This is publicly visible for unit testing.
+   */
+  def convert(table: UnresolvedInlineTable): LocalRelation = {
+val numCols = table.rows.head.size
+
+// For each column, traverse all the values and find a common data 
type.
+val targetTypes = Seq.tabulate(numCols) { ci =>
--- End diff --

It would be nice if we would create the entire schema here (including 
nullability).

You could also transpose the `Seq[Seq[Expression]]`. Something like: 
`table.rows.transpose.zip(table.names).map { ... }`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes 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 #14676: [SPARK-16947][SQL] Support type coercion and fold...

2016-08-17 Thread hvanhovell
Github user hvanhovell commented on a diff in the pull request:

https://github.com/apache/spark/pull/14676#discussion_r75078278
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala
 ---
@@ -50,6 +50,23 @@ case class UnresolvedRelation(
 }
 
 /**
+ * An inline table that has not been resolved yet. Once resolved, it is 
turned by the analyzer into
+ * a [[org.apache.spark.sql.catalyst.plans.logical.LocalRelation]].
+ *
+ * @param names list of column names
+ * @param rows expressions for the data
+ */
+case class UnresolvedInlineTable(
+names: Seq[String],
+rows: Seq[Seq[Expression]])
+  extends LeafNode {
+
+  lazy val expressionsResolved: Boolean = rows.forall(_.forall(_.resolved))
--- End diff --

This is used only once. Lets move this code into that location.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes 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 #14676: [SPARK-16947][SQL] Support type coercion and fold...

2016-08-17 Thread hvanhovell
Github user hvanhovell commented on a diff in the pull request:

https://github.com/apache/spark/pull/14676#discussion_r75075723
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTables.scala
 ---
@@ -0,0 +1,109 @@
+/*
+ * 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.analysis
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Cast
+import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, 
LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.types.{StructField, StructType}
+
+/**
+ * An analyzer rule that replaces [[UnresolvedInlineTable]] with 
[[LocalRelation]].
+ */
+object ResolveInlineTables extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
+case table: UnresolvedInlineTable if table.expressionsResolved =>
+  validateInputDimension(table)
+  validateInputFoldable(table)
+  convert(table)
+  }
+
+  /**
+   * Validates that all inline table data are foldable expressions.
+   *
+   * This is publicly visible for unit testing.
+   */
+  def validateInputFoldable(table: UnresolvedInlineTable): Unit = {
+table.rows.foreach { row =>
+  row.foreach { e =>
+if (!e.resolved || !e.foldable) {
+  e.failAnalysis(s"cannot evaluate expression ${e.sql} in inline 
table definition")
+}
+  }
+}
+  }
+
+  /**
+   * Validates the input data dimension:
+   * 1. All rows have the same cardinality.
+   * 2. The number of column aliases defined is consistent with the number 
of columns in data.
+   *
+   * This is publicly visible for unit testing.
+   */
+  def validateInputDimension(table: UnresolvedInlineTable): Unit = {
+if (table.rows.nonEmpty) {
+  val numCols = table.rows.head.size
+  table.rows.zipWithIndex.foreach { case (row, ri) =>
+if (row.size != numCols) {
+  table.failAnalysis(s"expected $numCols columns but found 
${row.size} columns in row $ri")
+}
+  }
+
+  if (table.names.size != numCols) {
+table.failAnalysis(s"expected ${table.names.size} columns but 
found $numCols in first row")
+  }
+}
+  }
+
+  /**
+   * Convert a valid (with right shape and foldable inputs) 
[[UnresolvedInlineTable]]
+   * into a [[LocalRelation]].
+   *
+   * This function attempts to coerce inputs into consistent types.
+   *
+   * This is publicly visible for unit testing.
+   */
+  def convert(table: UnresolvedInlineTable): LocalRelation = {
+val numCols = table.rows.head.size
+
+// For each column, traverse all the values and find a common data 
type.
+val targetTypes = Seq.tabulate(numCols) { ci =>
+  val inputTypes = table.rows.map(_(ci).dataType)
+  
TypeCoercion.findWiderTypeWithoutStringPromotion(inputTypes).getOrElse {
+table.failAnalysis(s"incompatible types found in column $ci for 
inline table")
+  }
+}
+assert(targetTypes.size == table.names.size)
+
+val newRows: Seq[InternalRow] = table.rows.map { row =>
+  InternalRow.fromSeq(row.zipWithIndex.map { case (e, ci) =>
+val targetType = targetTypes(ci)
+if (e.dataType.sameType(targetType)) {
+  e.eval()
+} else {
+  Cast(e, targetType).eval()
+}
+  })
+}
+
+val attributes = StructType(targetTypes.zip(table.names)
+  .map { case (typ, name) => StructField(name, typ) }).toAttributes
--- End diff --

Infer nullability?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but 

[GitHub] spark pull request #14676: [SPARK-16947][SQL] Support type coercion and fold...

2016-08-16 Thread petermaxlee
Github user petermaxlee commented on a diff in the pull request:

https://github.com/apache/spark/pull/14676#discussion_r75065140
  
--- Diff: sql/core/src/test/resources/sql-tests/inputs/inline-table.sql ---
@@ -0,0 +1,39 @@
+
+-- single row, without table and column alias
+select * from values ("one", 1);
+
+-- single row, without column alias
+select * from values ("one", 1) as data;
+
+-- single row
+select * from values ("one", 1) as data(a, b);
--- End diff --

added


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes 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 #14676: [SPARK-16947][SQL] Support type coercion and fold...

2016-08-16 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/14676#discussion_r75062469
  
--- Diff: sql/core/src/test/resources/sql-tests/inputs/inline-table.sql ---
@@ -0,0 +1,39 @@
+
+-- single row, without table and column alias
+select * from values ("one", 1);
+
+-- single row, without column alias
+select * from values ("one", 1) as data;
+
+-- single row
+select * from values ("one", 1) as data(a, b);
--- End diff --

Could you add a case for `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 #14676: [SPARK-16947][SQL] Support type coercion and fold...

2016-08-16 Thread petermaxlee
GitHub user petermaxlee opened a pull request:

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

[SPARK-16947][SQL] Support type coercion and foldable expression for inline 
tables

## What changes were proposed in this pull request?

(Please fill in changes proposed in this fix)


## How was this patch tested?
Added a new unit test suite ResolveInlineTablesSuite and a new file-based 
end-to-end test inline-table.sql.


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

$ git pull https://github.com/petermaxlee/spark SPARK-16947

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

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


commit d7acae55034d4ff5da3e7579cf44acb7b704b4a1
Author: petermaxlee 
Date:   2016-08-17T00:40:07Z

[SPARK-16947][SQL] Support type coercion and foldable expression for inline 
tables




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes 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