[GitHub] spark pull request #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-12 Thread jiangxb1987
Github user jiangxb1987 commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95929443
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala 
---
@@ -0,0 +1,80 @@
+/*
+ * 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.AnalysisException
+import org.apache.spark.sql.catalyst.CatalystConf
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, 
View}
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+ * This file defines analysis rules related to views.
+ */
+
+/**
+ * Make sure that a view's child plan produces the view's output 
attributes. We wrap the child
+ * with a Project and add an alias for each output attribute. The 
attributes are resolved by
+ * name. This should be only done after the batch of Resolution, because 
the view attributes are
+ * not completely resolved during the batch of Resolution.
+ */
+case class AliasViewChild(conf: CatalystConf) extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan 
resolveOperators {
+case v @ View(_, output, child) if child.resolved =>
+  val resolver = conf.resolver
+  val newOutput = output.map { attr =>
+val originAttr = findAttributeByName(attr.name, child.output, 
resolver)
+// The dataType of the output attributes may be not the same with 
that of the view output,
+// so we should cast the attribute to the dataType of the view 
output attribute. If the
+// cast can't perform, will throw an AnalysisException.
+Alias(Cast(originAttr, attr.dataType), attr.name)(exprId = 
attr.exprId,
+  qualifier = attr.qualifier, explicitMetadata = 
Some(attr.metadata))
+  }
+  v.copy(child = Project(newOutput, child))
+  }
--- End diff --

I happen to have discussed the same issue with @cloud-fan , the approach we 
think about is to rewrite the query as "SELECT ${child.outputAttributes} FROM 
(originalViewText) AS t", and save that to CatalogTable.viewText on write path. 
This make it pretty easy to handle schema reorder/extra columns.

Note that we have to check for duplicate attribute name for the child 
output, and throw an AnalysisException if there exists duplicate attributes, 
this follows the Hive's behavior.


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-12 Thread yhuai
Github user yhuai commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95896683
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala 
---
@@ -0,0 +1,80 @@
+/*
+ * 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.AnalysisException
+import org.apache.spark.sql.catalyst.CatalystConf
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, 
View}
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+ * This file defines analysis rules related to views.
+ */
+
+/**
+ * Make sure that a view's child plan produces the view's output 
attributes. We wrap the child
+ * with a Project and add an alias for each output attribute. The 
attributes are resolved by
+ * name. This should be only done after the batch of Resolution, because 
the view attributes are
+ * not completely resolved during the batch of Resolution.
+ */
+case class AliasViewChild(conf: CatalystConf) extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan 
resolveOperators {
+case v @ View(_, output, child) if child.resolved =>
+  val resolver = conf.resolver
+  val newOutput = output.map { attr =>
+val originAttr = findAttributeByName(attr.name, child.output, 
resolver)
+// The dataType of the output attributes may be not the same with 
that of the view output,
+// so we should cast the attribute to the dataType of the view 
output attribute. If the
+// cast can't perform, will throw an AnalysisException.
+Alias(Cast(originAttr, attr.dataType), attr.name)(exprId = 
attr.exprId,
+  qualifier = attr.qualifier, explicitMetadata = 
Some(attr.metadata))
+  }
+  v.copy(child = Project(newOutput, child))
+  }
--- End diff --

btw, I talked with @hvanhovell, the tricky case is when the view definition 
query is `select *`. So, we need first result the query and put a mapping from 
the view column name to the query output. Then, when we read the view back, 
even the column ordering of `select *` query is changed, we can still get the 
correct view column ordering.


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

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

https://github.com/apache/spark/pull/16233#discussion_r95738554
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala 
---
@@ -0,0 +1,80 @@
+/*
+ * 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.AnalysisException
+import org.apache.spark.sql.catalyst.CatalystConf
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, 
View}
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+ * This file defines analysis rules related to views.
+ */
+
+/**
+ * Make sure that a view's child plan produces the view's output 
attributes. We wrap the child
+ * with a Project and add an alias for each output attribute. The 
attributes are resolved by
+ * name. This should be only done after the batch of Resolution, because 
the view attributes are
+ * not completely resolved during the batch of Resolution.
+ */
+case class AliasViewChild(conf: CatalystConf) extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan 
resolveOperators {
+case v @ View(_, output, child) if child.resolved =>
+  val resolver = conf.resolver
+  val newOutput = output.map { attr =>
+val originAttr = findAttributeByName(attr.name, child.output, 
resolver)
+// The dataType of the output attributes may be not the same with 
that of the view output,
+// so we should cast the attribute to the dataType of the view 
output attribute. If the
+// cast can't perform, will throw an AnalysisException.
+Alias(Cast(originAttr, attr.dataType), attr.name)(exprId = 
attr.exprId,
+  qualifier = attr.qualifier, explicitMetadata = 
Some(attr.metadata))
+  }
+  v.copy(child = Project(newOutput, child))
+  }
--- End diff --

good catch! I think we should match by index and add an assert to guarantee 
that `View.output` has same number of columns with `View.child`


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

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

https://github.com/apache/spark/pull/16233#discussion_r95735905
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala 
---
@@ -0,0 +1,80 @@
+/*
+ * 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.AnalysisException
+import org.apache.spark.sql.catalyst.CatalystConf
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, 
View}
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+ * This file defines analysis rules related to views.
+ */
+
+/**
+ * Make sure that a view's child plan produces the view's output 
attributes. We wrap the child
+ * with a Project and add an alias for each output attribute. The 
attributes are resolved by
+ * name. This should be only done after the batch of Resolution, because 
the view attributes are
+ * not completely resolved during the batch of Resolution.
+ */
+case class AliasViewChild(conf: CatalystConf) extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan 
resolveOperators {
+case v @ View(_, output, child) if child.resolved =>
+  val resolver = conf.resolver
+  val newOutput = output.map { attr =>
+val originAttr = findAttributeByName(attr.name, child.output, 
resolver)
+// The dataType of the output attributes may be not the same with 
that of the view output,
+// so we should cast the attribute to the dataType of the view 
output attribute. If the
+// cast can't perform, will throw an AnalysisException.
+Alias(Cast(originAttr, attr.dataType), attr.name)(exprId = 
attr.exprId,
+  qualifier = attr.qualifier, explicitMetadata = 
Some(attr.metadata))
+  }
+  v.copy(child = Project(newOutput, child))
+  }
--- End diff --

an example from Hive:
```
CREATE VIEW view1(b, a) AS SELECT 1 a, 2 b FROM (SELECT 1) t;
SELECT * FROM view1;
+-+-+
|b|a|
+-+-+
|1|2|
+-+-+

CREATE VIEW view2 AS SELECT a, b FROM view1;
SELECT * FROM view2;
+-+-+
|a|b|
+-+-+
|2|1|
+-+-+

ALTER VIEW view1 AS SELECT 1 b, 2 a FROM (SELECT 1) t;
SELECT * FROM view1;
+-+-+
|b|a|
+-+-+
|1|2|
+-+-+

SELECT * FROM view2;
+-+-+
|a|b|
+-+-+
|2|1|
+-+-+
```
You can see that, after we alter `view1`, the query of `view1` and `view2` 
has no change at all(including schema). This is because `view1` specifies 
custom column names, and we match the custom columns names with the actual view 
definition column names by index, that why `ALTER VIEW view1 AS SELECT 1 b, 2 a 
FROM (SELECT 1) t` has no effect at all.


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-11 Thread jiangxb1987
Github user jiangxb1987 commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95731778
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala 
---
@@ -0,0 +1,80 @@
+/*
+ * 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.AnalysisException
+import org.apache.spark.sql.catalyst.CatalystConf
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, 
View}
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+ * This file defines analysis rules related to views.
+ */
+
+/**
+ * Make sure that a view's child plan produces the view's output 
attributes. We wrap the child
+ * with a Project and add an alias for each output attribute. The 
attributes are resolved by
+ * name. This should be only done after the batch of Resolution, because 
the view attributes are
+ * not completely resolved during the batch of Resolution.
+ */
+case class AliasViewChild(conf: CatalystConf) extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan 
resolveOperators {
+case v @ View(_, output, child) if child.resolved =>
+  val resolver = conf.resolver
+  val newOutput = output.map { attr =>
+val originAttr = findAttributeByName(attr.name, child.output, 
resolver)
+// The dataType of the output attributes may be not the same with 
that of the view output,
+// so we should cast the attribute to the dataType of the view 
output attribute. If the
+// cast can't perform, will throw an AnalysisException.
+Alias(Cast(originAttr, attr.dataType), attr.name)(exprId = 
attr.exprId,
+  qualifier = attr.qualifier, explicitMetadata = 
Some(attr.metadata))
--- End diff --

Okay we just compare the both dataTypes and throw an AnalysisException if 
they don't match up.


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-11 Thread jiangxb1987
Github user jiangxb1987 commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95723547
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala 
---
@@ -0,0 +1,80 @@
+/*
+ * 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.AnalysisException
+import org.apache.spark.sql.catalyst.CatalystConf
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, 
View}
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+ * This file defines analysis rules related to views.
+ */
+
+/**
+ * Make sure that a view's child plan produces the view's output 
attributes. We wrap the child
+ * with a Project and add an alias for each output attribute. The 
attributes are resolved by
+ * name. This should be only done after the batch of Resolution, because 
the view attributes are
+ * not completely resolved during the batch of Resolution.
+ */
+case class AliasViewChild(conf: CatalystConf) extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan 
resolveOperators {
+case v @ View(_, output, child) if child.resolved =>
+  val resolver = conf.resolver
+  val newOutput = output.map { attr =>
+val originAttr = findAttributeByName(attr.name, child.output, 
resolver)
+// The dataType of the output attributes may be not the same with 
that of the view output,
+// so we should cast the attribute to the dataType of the view 
output attribute. If the
+// cast can't perform, will throw an AnalysisException.
+Alias(Cast(originAttr, attr.dataType), attr.name)(exprId = 
attr.exprId,
+  qualifier = attr.qualifier, explicitMetadata = 
Some(attr.metadata))
+  }
+  v.copy(child = Project(newOutput, child))
+  }
--- End diff --

I think @cloud-fan get a point that the column names in the schema of the 
view is not necessarily the same with that of the child. Hive also resolves the 
view columns by name, for example:
```
hive> create table testtable as select 1 a, 2 b, 3 c;
hive> create view testview(a, b, c) as select c, b, a from testtable;
hive> select * from testview;
3   2   1
Time taken: 0.111 seconds, Fetched: 1 row(s)
hive> alter table testtable change column `b` `b` int AFTER `c`;
hive> select * from testview;
2   3   1
Time taken: 0.109 seconds, Fetched: 1 row(s)
```


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

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

https://github.com/apache/spark/pull/16233#discussion_r95715663
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala 
---
@@ -0,0 +1,80 @@
+/*
+ * 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.AnalysisException
+import org.apache.spark.sql.catalyst.CatalystConf
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, 
View}
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+ * This file defines analysis rules related to views.
+ */
+
+/**
+ * Make sure that a view's child plan produces the view's output 
attributes. We wrap the child
+ * with a Project and add an alias for each output attribute. The 
attributes are resolved by
+ * name. This should be only done after the batch of Resolution, because 
the view attributes are
+ * not completely resolved during the batch of Resolution.
+ */
+case class AliasViewChild(conf: CatalystConf) extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan 
resolveOperators {
+case v @ View(_, output, child) if child.resolved =>
+  val resolver = conf.resolver
+  val newOutput = output.map { attr =>
+val originAttr = findAttributeByName(attr.name, child.output, 
resolver)
+// The dataType of the output attributes may be not the same with 
that of the view output,
+// so we should cast the attribute to the dataType of the view 
output attribute. If the
+// cast can't perform, will throw an AnalysisException.
+Alias(Cast(originAttr, attr.dataType), attr.name)(exprId = 
attr.exprId,
+  qualifier = attr.qualifier, explicitMetadata = 
Some(attr.metadata))
+  }
+  v.copy(child = Project(newOutput, child))
+  }
--- End diff --

We will map the view columns `(b, c, a)` to the real columns `(a, b, c)` by 
index. When we resolve a view, we are trying to map the real columns(`a, b, c`) 
to the real plan schema(the schema of table `testbase`) by name, which is 
nothing to do with the view columns `(b, c, a)`, it can be anything else like 
`(x, y, z)`.

We need to take care of the first matching when we implement persisting 
view.


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

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

https://github.com/apache/spark/pull/16233#discussion_r95714934
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala 
---
@@ -0,0 +1,80 @@
+/*
+ * 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.AnalysisException
+import org.apache.spark.sql.catalyst.CatalystConf
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, 
View}
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+ * This file defines analysis rules related to views.
+ */
+
+/**
+ * Make sure that a view's child plan produces the view's output 
attributes. We wrap the child
+ * with a Project and add an alias for each output attribute. The 
attributes are resolved by
+ * name. This should be only done after the batch of Resolution, because 
the view attributes are
+ * not completely resolved during the batch of Resolution.
+ */
+case class AliasViewChild(conf: CatalystConf) extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan 
resolveOperators {
+case v @ View(_, output, child) if child.resolved =>
+  val resolver = conf.resolver
+  val newOutput = output.map { attr =>
+val originAttr = findAttributeByName(attr.name, child.output, 
resolver)
+// The dataType of the output attributes may be not the same with 
that of the view output,
+// so we should cast the attribute to the dataType of the view 
output attribute. If the
+// cast can't perform, will throw an AnalysisException.
+Alias(Cast(originAttr, attr.dataType), attr.name)(exprId = 
attr.exprId,
+  qualifier = attr.qualifier, explicitMetadata = 
Some(attr.metadata))
--- End diff --

how about we use `UpCast`?


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-11 Thread jiangxb1987
Github user jiangxb1987 commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95711504
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -50,6 +50,36 @@ object SimpleAnalyzer extends Analyzer(
 new SimpleCatalystConf(caseSensitiveAnalysis = true))
 
 /**
+ * Provides a way to keep state during the analysis, this enables us to 
decouple the concerns
+ * of analysis environment from the catalog.
+ *
+ * Note this is thread local.
+ *
+ * @param defaultDatabase The default database used in the view 
resolution, this overrules the
+ *current catalog database.
+ * @param nestedViewLevel The nested level in the view resolution, this 
enables us to limit the
+ *depth of nested views.
--- End diff --

I've added a TODO under this.


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

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



[GitHub] spark pull request #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-11 Thread asfgit
Github user asfgit closed the pull request at:

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


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-11 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95678535
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala 
---
@@ -0,0 +1,80 @@
+/*
+ * 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.AnalysisException
+import org.apache.spark.sql.catalyst.CatalystConf
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, 
View}
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+ * This file defines analysis rules related to views.
+ */
+
+/**
+ * Make sure that a view's child plan produces the view's output 
attributes. We wrap the child
+ * with a Project and add an alias for each output attribute. The 
attributes are resolved by
+ * name. This should be only done after the batch of Resolution, because 
the view attributes are
+ * not completely resolved during the batch of Resolution.
+ */
+case class AliasViewChild(conf: CatalystConf) extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan 
resolveOperators {
+case v @ View(_, output, child) if child.resolved =>
+  val resolver = conf.resolver
+  val newOutput = output.map { attr =>
+val originAttr = findAttributeByName(attr.name, child.output, 
resolver)
+// The dataType of the output attributes may be not the same with 
that of the view output,
+// so we should cast the attribute to the dataType of the view 
output attribute. If the
+// cast can't perform, will throw an AnalysisException.
+Alias(Cast(originAttr, attr.dataType), attr.name)(exprId = 
attr.exprId,
+  qualifier = attr.qualifier, explicitMetadata = 
Some(attr.metadata))
--- End diff --

Agree.


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-11 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95677700
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala 
---
@@ -0,0 +1,80 @@
+/*
+ * 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.AnalysisException
+import org.apache.spark.sql.catalyst.CatalystConf
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, 
View}
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+ * This file defines analysis rules related to views.
+ */
+
+/**
+ * Make sure that a view's child plan produces the view's output 
attributes. We wrap the child
+ * with a Project and add an alias for each output attribute. The 
attributes are resolved by
+ * name. This should be only done after the batch of Resolution, because 
the view attributes are
+ * not completely resolved during the batch of Resolution.
+ */
+case class AliasViewChild(conf: CatalystConf) extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan 
resolveOperators {
+case v @ View(_, output, child) if child.resolved =>
+  val resolver = conf.resolver
+  val newOutput = output.map { attr =>
+val originAttr = findAttributeByName(attr.name, child.output, 
resolver)
+// The dataType of the output attributes may be not the same with 
that of the view output,
+// so we should cast the attribute to the dataType of the view 
output attribute. If the
+// cast can't perform, will throw an AnalysisException.
--- End diff --

This test case is the example. 
https://github.com/jiangxb1987/spark/blob/3a6dd3e0185423a88541dd83e313690afdcd2543/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala#L683-L699


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-11 Thread yhuai
Github user yhuai commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95669988
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala 
---
@@ -0,0 +1,80 @@
+/*
+ * 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.AnalysisException
+import org.apache.spark.sql.catalyst.CatalystConf
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, 
View}
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+ * This file defines analysis rules related to views.
+ */
+
+/**
+ * Make sure that a view's child plan produces the view's output 
attributes. We wrap the child
+ * with a Project and add an alias for each output attribute. The 
attributes are resolved by
+ * name. This should be only done after the batch of Resolution, because 
the view attributes are
+ * not completely resolved during the batch of Resolution.
+ */
+case class AliasViewChild(conf: CatalystConf) extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan 
resolveOperators {
+case v @ View(_, output, child) if child.resolved =>
+  val resolver = conf.resolver
+  val newOutput = output.map { attr =>
+val originAttr = findAttributeByName(attr.name, child.output, 
resolver)
+// The dataType of the output attributes may be not the same with 
that of the view output,
+// so we should cast the attribute to the dataType of the view 
output attribute. If the
+// cast can't perform, will throw an AnalysisException.
+Alias(Cast(originAttr, attr.dataType), attr.name)(exprId = 
attr.exprId,
+  qualifier = attr.qualifier, explicitMetadata = 
Some(attr.metadata))
--- End diff --

If the data type does not match, I feel we should throw an exception 
instead of just adding the cast at here. 

When the data type is changed, the meaning of the column may be changed. 
So, the user who defines the view may need to make some actions like modifying 
the applications using the view. I feel throwing an exception is better because 
it lets users know that the column data type is changed and some actions are 
required. 


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-11 Thread nsyca
Github user nsyca commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95665834
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala 
---
@@ -0,0 +1,80 @@
+/*
+ * 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.AnalysisException
+import org.apache.spark.sql.catalyst.CatalystConf
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, 
View}
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+ * This file defines analysis rules related to views.
+ */
+
+/**
+ * Make sure that a view's child plan produces the view's output 
attributes. We wrap the child
+ * with a Project and add an alias for each output attribute. The 
attributes are resolved by
+ * name. This should be only done after the batch of Resolution, because 
the view attributes are
+ * not completely resolved during the batch of Resolution.
+ */
+case class AliasViewChild(conf: CatalystConf) extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan 
resolveOperators {
+case v @ View(_, output, child) if child.resolved =>
+  val resolver = conf.resolver
+  val newOutput = output.map { attr =>
+val originAttr = findAttributeByName(attr.name, child.output, 
resolver)
+// The dataType of the output attributes may be not the same with 
that of the view output,
+// so we should cast the attribute to the dataType of the view 
output attribute. If the
+// cast can't perform, will throw an AnalysisException.
--- End diff --

I am curious in which scenario that the data types of the child will not 
match the view. Do you have an example? Or this is just a safety net. If it is, 
can it be done as an assertion instead?


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

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



[GitHub] spark pull request #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-11 Thread yhuai
Github user yhuai commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95662299
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala 
---
@@ -0,0 +1,80 @@
+/*
+ * 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.AnalysisException
+import org.apache.spark.sql.catalyst.CatalystConf
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, 
View}
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+ * This file defines analysis rules related to views.
+ */
+
+/**
+ * Make sure that a view's child plan produces the view's output 
attributes. We wrap the child
+ * with a Project and add an alias for each output attribute. The 
attributes are resolved by
+ * name. This should be only done after the batch of Resolution, because 
the view attributes are
+ * not completely resolved during the batch of Resolution.
+ */
+case class AliasViewChild(conf: CatalystConf) extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan 
resolveOperators {
+case v @ View(_, output, child) if child.resolved =>
+  val resolver = conf.resolver
+  val newOutput = output.map { attr =>
+val originAttr = findAttributeByName(attr.name, child.output, 
resolver)
+// The dataType of the output attributes may be not the same with 
that of the view output,
+// so we should cast the attribute to the dataType of the view 
output attribute. If the
+// cast can't perform, will throw an AnalysisException.
+Alias(Cast(originAttr, attr.dataType), attr.name)(exprId = 
attr.exprId,
+  qualifier = attr.qualifier, explicitMetadata = 
Some(attr.metadata))
+  }
+  v.copy(child = Project(newOutput, child))
+  }
--- End diff --

Sorry, I may miss something obvious. But, I am still not sure it is the 
right thing to do.

I tried the following in the postgres

```
yhuai=# create table testbase (a int, b int, c int);
CREATE TABLE
yhuai=# insert into testbase values (1, 2, 3)  ;
INSERT 0 1
yhuai=# insert into testbase values (4, 5, 6);  
   
INSERT 0 1
yhuai=# create view testview (b, c, a) as select a, b, c from testbase;
CREATE VIEW
yhuai=# select * from testview;
 b | c | a 
---+---+---
 1 | 2 | 3
 4 | 5 | 6
(2 rows)

yhuai=# create view testview1 (b, c, a) as select * from testbase;
CREATE VIEW
yhuai=# select * from testview1;
 b | c | a 
---+---+---
 1 | 2 | 3
 4 | 5 | 6
(2 rows)

```

I am not sure why we are resolving those columns by name.


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

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



[GitHub] spark pull request #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-11 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95655805
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala
 ---
@@ -378,6 +379,35 @@ case class InsertIntoTable(
 }
 
 /**
+ * A container for holding the view description(CatalogTable), and the 
output of the view. The
+ * child should be a logical plan parsed from the `CatalogTable.viewText`, 
should throw an error
+ * if the `viewText` is not defined.
+ * This operator will be removed at the end of analysis stage.
+ *
+ * @param desc A view description(CatalogTable) that provides necessary 
information to resolve the
+ * view.
+ * @param output The output of a view operator, this is generated during 
planning the view, so that
+ *   we are able to decouple the output from the underlying 
structure.
+ * @param child The logical plan of a view operator, it should be a 
logical plan parsed from the
+ *  `CatalogTable.viewText`, should throw an error if the 
`viewText` is not defined.
+ */
+case class View(
+desc: CatalogTable,
+output: Seq[Attribute],
+child: LogicalPlan) extends LogicalPlan with MultiInstanceRelation {
--- End diff --

Agree. Let us keep it unchanged, although we are unable to create a test 
case to trigger it. 


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

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



[GitHub] spark pull request #16233: [SPARK-18801][SQL] Support resolve a nested view

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

https://github.com/apache/spark/pull/16233#discussion_r95597513
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala
 ---
@@ -378,6 +379,35 @@ case class InsertIntoTable(
 }
 
 /**
+ * A container for holding the view description(CatalogTable), and the 
output of the view. The
+ * child should be a logical plan parsed from the `CatalogTable.viewText`, 
should throw an error
+ * if the `viewText` is not defined.
+ * This operator will be removed at the end of analysis stage.
+ *
+ * @param desc A view description(CatalogTable) that provides necessary 
information to resolve the
+ * view.
+ * @param output The output of a view operator, this is generated during 
planning the view, so that
+ *   we are able to decouple the output from the underlying 
structure.
+ * @param child The logical plan of a view operator, it should be a 
logical plan parsed from the
+ *  `CatalogTable.viewText`, should throw an error if the 
`viewText` is not defined.
+ */
+case class View(
+desc: CatalogTable,
+output: Seq[Attribute],
+child: LogicalPlan) extends LogicalPlan with MultiInstanceRelation {
--- End diff --

makes sense to me


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

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

https://github.com/apache/spark/pull/16233#discussion_r95598084
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala 
---
@@ -0,0 +1,80 @@
+/*
+ * 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.AnalysisException
+import org.apache.spark.sql.catalyst.CatalystConf
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, 
View}
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+ * This file defines analysis rules related to views.
+ */
+
+/**
+ * Make sure that a view's child plan produces the view's output 
attributes. We wrap the child
+ * with a Project and add an alias for each output attribute. The 
attributes are resolved by
+ * name. This should be only done after the batch of Resolution, because 
the view attributes are
+ * not completely resolved during the batch of Resolution.
+ */
+case class AliasViewChild(conf: CatalystConf) extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan 
resolveOperators {
+case v @ View(_, output, child) if child.resolved =>
+  val resolver = conf.resolver
+  val newOutput = output.map { attr =>
+val originAttr = findAttributeByName(attr.name, child.output, 
resolver)
+// The dataType of the output attributes may be not the same with 
that of the view output,
+// so we should cast the attribute to the dataType of the view 
output attribute. If the
+// cast cann't perform, will throw an AnalysisException.
+Alias(Cast(originAttr, attr.dataType), attr.name)(exprId = 
attr.exprId,
+  qualifier = attr.qualifier, explicitMetadata = 
Some(attr.metadata))
+  }
+  v.copy(child = Project(newOutput, child))
+  }
+
+  /**
+   * Find the attribute that has the expected attribute name from an 
attribute list, the names
+   * are compared using conf.resolver.
+   * If the expected attribute is not found, throw an AnalysisException.
+   */
+  private def findAttributeByName(
+  name: String,
+  attrs: Seq[Attribute],
+  resolver: Resolver): Attribute = {
+attrs.collectFirst {
--- End diff --

`find` is more accurate here, as we won't change the type. While 
`collectFirst` can, e.g. `List[A].collectFirst[B] {...}`


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

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

https://github.com/apache/spark/pull/16233#discussion_r95593666
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala
 ---
@@ -378,6 +379,35 @@ case class InsertIntoTable(
 }
 
 /**
+ * A container for holding the view description(CatalogTable), and the 
output of the view. The
+ * child should be a logical plan parsed from the `CatalogTable.viewText`, 
should throw an error
+ * if the `viewText` is not defined.
+ * This operator will be removed at the end of analysis stage.
+ *
+ * @param desc A view description(CatalogTable) that provides necessary 
information to resolve the
+ * view.
+ * @param output The output of a view operator, this is generated during 
planning the view, so that
+ *   we are able to decouple the output from the underlying 
structure.
+ * @param child The logical plan of a view operator, it should be a 
logical plan parsed from the
+ *  `CatalogTable.viewText`, should throw an error if the 
`viewText` is not defined.
+ */
+case class View(
+desc: CatalogTable,
+output: Seq[Attribute],
+child: LogicalPlan) extends LogicalPlan with MultiInstanceRelation {
--- End diff --

@gatorsmile MultiInstanceRelation can be mixed in with anything. The chance 
of it generating duplicate expression IDs is infinitesmally small (all other 
`MultiInstanceRelation`s implement exactly the same logic). The whole idea of 
using a `MultiInstanceRelations` is to reduce the amount of tree rewriting 
required when there are duplicate attribute (ids).


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-11 Thread jiangxb1987
Github user jiangxb1987 commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95534400
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala 
---
@@ -0,0 +1,80 @@
+/*
+ * 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.AnalysisException
+import org.apache.spark.sql.catalyst.CatalystConf
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, 
View}
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+ * This file defines analysis rules related to views.
+ */
+
+/**
+ * Make sure that a view's child plan produces the view's output 
attributes. We wrap the child
+ * with a Project and add an alias for each output attribute. The 
attributes are resolved by
+ * name. This should be only done after the batch of Resolution, because 
the view attributes are
+ * not completely resolved during the batch of Resolution.
+ */
+case class AliasViewChild(conf: CatalystConf) extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan 
resolveOperators {
+case v @ View(_, output, child) if child.resolved =>
+  val resolver = conf.resolver
+  val newOutput = output.map { attr =>
+val originAttr = findAttributeByName(attr.name, child.output, 
resolver)
+// The dataType of the output attributes may be not the same with 
that of the view output,
+// so we should cast the attribute to the dataType of the view 
output attribute. If the
+// cast cann't perform, will throw an AnalysisException.
+Alias(Cast(originAttr, attr.dataType), attr.name)(exprId = 
attr.exprId,
+  qualifier = attr.qualifier, explicitMetadata = 
Some(attr.metadata))
+  }
+  v.copy(child = Project(newOutput, child))
+  }
+
+  /**
+   * Find the attribute that has the expected attribute name from an 
attribute list, the names
+   * are compared using conf.resolver.
+   * If the expected attribute is not found, throw an AnalysisException.
+   */
+  private def findAttributeByName(
+  name: String,
+  attrs: Seq[Attribute],
+  resolver: Resolver): Attribute = {
+attrs.collectFirst {
--- End diff --

May I ask why `find` is better than `collectFirst` here? Thanks!


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-11 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95528408
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala
 ---
@@ -378,6 +379,35 @@ case class InsertIntoTable(
 }
 
 /**
+ * A container for holding the view description(CatalogTable), and the 
output of the view. The
+ * child should be a logical plan parsed from the `CatalogTable.viewText`, 
should throw an error
+ * if the `viewText` is not defined.
+ * This operator will be removed at the end of analysis stage.
+ *
+ * @param desc A view description(CatalogTable) that provides necessary 
information to resolve the
+ * view.
+ * @param output The output of a view operator, this is generated during 
planning the view, so that
+ *   we are able to decouple the output from the underlying 
structure.
+ * @param child The logical plan of a view operator, it should be a 
logical plan parsed from the
+ *  `CatalogTable.viewText`, should throw an error if the 
`viewText` is not defined.
+ */
+case class View(
+desc: CatalogTable,
+output: Seq[Attribute],
+child: LogicalPlan) extends LogicalPlan with MultiInstanceRelation {
--- End diff --

It generates the output attributes. In theory, it could generate the 
duplicate expression ID, but it will not be a top child node under `Join`, 
`Intersect` and `Except`. Thus, I am fine to add it or not. Let us leave it 
unchanged for safety. 


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-11 Thread jiangxb1987
Github user jiangxb1987 commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95527384
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala 
---
@@ -543,4 +545,157 @@ class SQLViewSuite extends QueryTest with 
SQLTestUtils with TestHiveSingleton {
   }
 }
   }
+
+  test("correctly resolve a nested view") {
+withTempDatabase { db =>
+  withView(s"$db.view1", s"$db.view2") {
+val view1 = CatalogTable(
+  identifier = TableIdentifier("view1", Some(db)),
+  tableType = CatalogTableType.VIEW,
+  storage = CatalogStorageFormat.empty,
+  schema = new StructType().add("id", "int").add("id1", "int"),
+  viewOriginalText = Some("SELECT * FROM jt"),
+  viewText = Some("SELECT * FROM jt"),
+  properties = Map[String, String] 
{CatalogTable.VIEW_DEFAULT_DATABASE -> "default"})
+val view2 = CatalogTable(
+  identifier = TableIdentifier("view2", Some(db)),
+  tableType = CatalogTableType.VIEW,
+  storage = CatalogStorageFormat.empty,
+  schema = new StructType().add("id", "int").add("id1", "int"),
+  viewOriginalText = Some("SELECT * FROM view1"),
+  viewText = Some("SELECT * FROM view1"),
+  properties = Map[String, String] 
{CatalogTable.VIEW_DEFAULT_DATABASE -> db})
+activateDatabase(db) {
+  hiveContext.sessionState.catalog.createTable(view1, 
ignoreIfExists = false)
+  hiveContext.sessionState.catalog.createTable(view2, 
ignoreIfExists = false)
+  checkAnswer(sql("SELECT * FROM view2 ORDER BY id"), (1 to 
9).map(i => Row(i, i)))
+}
+  }
+}
+  }
+
+  test("correctly resolve a view with CTE") {
+withView("cte_view") {
+  val cte_view = CatalogTable(
+identifier = TableIdentifier("cte_view"),
+tableType = CatalogTableType.VIEW,
+storage = CatalogStorageFormat.empty,
+schema = new StructType().add("n", "int"),
+viewOriginalText = Some("WITH w AS (SELECT 1 AS n) SELECT n FROM 
w"),
+viewText = Some("WITH w AS (SELECT 1 AS n) SELECT n FROM w"),
+properties = Map[String, String] 
{CatalogTable.VIEW_DEFAULT_DATABASE -> "default"})
+  hiveContext.sessionState.catalog.createTable(cte_view, 
ignoreIfExists = false)
+  checkAnswer(sql("SELECT * FROM cte_view"), Row(1))
+}
+  }
+
+  test("correctly resolve a view in a self join") {
+withView("join_view") {
+  val join_view = CatalogTable(
+identifier = TableIdentifier("join_view"),
+tableType = CatalogTableType.VIEW,
+storage = CatalogStorageFormat.empty,
+schema = new StructType().add("id", "int").add("id1", "int"),
+viewOriginalText = Some("SELECT * FROM jt"),
+viewText = Some("SELECT * FROM jt"),
+properties = Map[String, String] 
{CatalogTable.VIEW_DEFAULT_DATABASE -> "default"})
+  hiveContext.sessionState.catalog.createTable(join_view, 
ignoreIfExists = false)
+  checkAnswer(
+sql("SELECT * FROM join_view t1 JOIN join_view t2 ON t1.id = t2.id 
ORDER BY t1.id"),
+(1 to 9).map(i => Row(i, i, i, i)))
+}
+  }
+
+  private def assertInvalidReference(query: String): Unit = {
+val e = intercept[AnalysisException] {
+  sql(query)
+}.getMessage
+assert(e.contains("Table or view not found"))
+  }
+
+  test("error handling: fail if the referenced table or view is invalid") {
+withView("view1", "view2", "view3") {
+  // Fail if the referenced table is defined in a invalid database.
+  val view1 = CatalogTable(
+identifier = TableIdentifier("view1"),
+tableType = CatalogTableType.VIEW,
+storage = CatalogStorageFormat.empty,
+schema = new StructType().add("id", "int").add("id1", "int"),
+viewOriginalText = Some("SELECT * FROM invalid_db.jt"),
+viewText = Some("SELECT * FROM invalid_db.jt"),
+properties = Map[String, String] 
{CatalogTable.VIEW_DEFAULT_DATABASE -> "default"})
+  hiveContext.sessionState.catalog.createTable(view1, ignoreIfExists = 
false)
+  assertInvalidReference("SELECT * FROM view1")
+
+  // Fail if the referenced table is invalid.
+  val view2 = CatalogTable(
+identifier = TableIdentifier("view2"),
+tableType = CatalogTableType.VIEW,
+storage = CatalogStorageFormat.empty,
+schema = new StructType().add("id", "int").add("id1", "int"),
+viewOriginalText = Some("SELECT * FROM invalid_table"),
+viewText = Some("SELE

[GitHub] spark pull request #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-10 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95526238
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala
 ---
@@ -378,6 +379,35 @@ case class InsertIntoTable(
 }
 
 /**
+ * A container for holding the view description(CatalogTable), and the 
output of the view. The
+ * child should be a logical plan parsed from the `CatalogTable.viewText`, 
should throw an error
+ * if the `viewText` is not defined.
+ * This operator will be removed at the end of analysis stage.
+ *
+ * @param desc A view description(CatalogTable) that provides necessary 
information to resolve the
+ * view.
+ * @param output The output of a view operator, this is generated during 
planning the view, so that
+ *   we are able to decouple the output from the underlying 
structure.
+ * @param child The logical plan of a view operator, it should be a 
logical plan parsed from the
+ *  `CatalogTable.viewText`, should throw an error if the 
`viewText` is not defined.
+ */
+case class View(
+desc: CatalogTable,
+output: Seq[Attribute],
+child: LogicalPlan) extends LogicalPlan with MultiInstanceRelation {
--- End diff --

We only extend `MultiInstanceRelation` for the leaf node. Any reason why it 
is needed. 


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-10 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95525588
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala 
---
@@ -543,4 +545,157 @@ class SQLViewSuite extends QueryTest with 
SQLTestUtils with TestHiveSingleton {
   }
 }
   }
+
+  test("correctly resolve a nested view") {
+withTempDatabase { db =>
+  withView(s"$db.view1", s"$db.view2") {
+val view1 = CatalogTable(
+  identifier = TableIdentifier("view1", Some(db)),
+  tableType = CatalogTableType.VIEW,
+  storage = CatalogStorageFormat.empty,
+  schema = new StructType().add("id", "int").add("id1", "int"),
+  viewOriginalText = Some("SELECT * FROM jt"),
+  viewText = Some("SELECT * FROM jt"),
+  properties = Map[String, String] 
{CatalogTable.VIEW_DEFAULT_DATABASE -> "default"})
+val view2 = CatalogTable(
+  identifier = TableIdentifier("view2", Some(db)),
+  tableType = CatalogTableType.VIEW,
+  storage = CatalogStorageFormat.empty,
+  schema = new StructType().add("id", "int").add("id1", "int"),
+  viewOriginalText = Some("SELECT * FROM view1"),
+  viewText = Some("SELECT * FROM view1"),
+  properties = Map[String, String] 
{CatalogTable.VIEW_DEFAULT_DATABASE -> db})
+activateDatabase(db) {
+  hiveContext.sessionState.catalog.createTable(view1, 
ignoreIfExists = false)
+  hiveContext.sessionState.catalog.createTable(view2, 
ignoreIfExists = false)
+  checkAnswer(sql("SELECT * FROM view2 ORDER BY id"), (1 to 
9).map(i => Row(i, i)))
+}
+  }
+}
+  }
+
+  test("correctly resolve a view with CTE") {
+withView("cte_view") {
+  val cte_view = CatalogTable(
+identifier = TableIdentifier("cte_view"),
+tableType = CatalogTableType.VIEW,
+storage = CatalogStorageFormat.empty,
+schema = new StructType().add("n", "int"),
+viewOriginalText = Some("WITH w AS (SELECT 1 AS n) SELECT n FROM 
w"),
+viewText = Some("WITH w AS (SELECT 1 AS n) SELECT n FROM w"),
+properties = Map[String, String] 
{CatalogTable.VIEW_DEFAULT_DATABASE -> "default"})
+  hiveContext.sessionState.catalog.createTable(cte_view, 
ignoreIfExists = false)
+  checkAnswer(sql("SELECT * FROM cte_view"), Row(1))
+}
+  }
+
+  test("correctly resolve a view in a self join") {
--- End diff --

Without `View` extending `MultiInstanceRelation `, it still works. 


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-10 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95525456
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala
 ---
@@ -378,6 +379,35 @@ case class InsertIntoTable(
 }
 
 /**
+ * A container for holding the view description(CatalogTable), and the 
output of the view. The
+ * child should be a logical plan parsed from the `CatalogTable.viewText`, 
should throw an error
+ * if the `viewText` is not defined.
+ * This operator will be removed at the end of analysis stage.
+ *
+ * @param desc A view description(CatalogTable) that provides necessary 
information to resolve the
+ * view.
+ * @param output The output of a view operator, this is generated during 
planning the view, so that
+ *   we are able to decouple the output from the underlying 
structure.
+ * @param child The logical plan of a view operator, it should be a 
logical plan parsed from the
+ *  `CatalogTable.viewText`, should throw an error if the 
`viewText` is not defined.
+ */
+case class View(
+desc: CatalogTable,
+output: Seq[Attribute],
+child: LogicalPlan) extends LogicalPlan with MultiInstanceRelation {
--- End diff --

I still cannot get the point why we need to extend `MultiInstanceRelation` 
here. We only do it for the leaf node, right?


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

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



[GitHub] spark pull request #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-10 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95522422
  
--- Diff: sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala 
---
@@ -2476,4 +2476,14 @@ class SQLQuerySuite extends QueryTest with 
SharedSQLContext {
   assert(sql("SELECT * FROM array_tbl where arr = ARRAY(1L)").count == 
1)
 }
   }
+
+  test("should be able to resolve a persistent view") {
--- End diff --

I see. That means, this PR enables the view support without enabling Hive 
support. This test case is just covering a very basic case. We need to check 
more scenarios, like ALTER VIEW. Please remember this in the follow-up PRs. 

Also, update the PR description and mention this in a separate bullet. 


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

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

https://github.com/apache/spark/pull/16233#discussion_r95520270
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala 
---
@@ -0,0 +1,80 @@
+/*
+ * 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.AnalysisException
+import org.apache.spark.sql.catalyst.CatalystConf
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, 
View}
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+ * This file defines analysis rules related to views.
+ */
+
+/**
+ * Make sure that a view's child plan produces the view's output 
attributes. We wrap the child
+ * with a Project and add an alias for each output attribute. The 
attributes are resolved by
+ * name. This should be only done after the batch of Resolution, because 
the view attributes are
+ * not completely resolved during the batch of Resolution.
+ */
+case class AliasViewChild(conf: CatalystConf) extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan 
resolveOperators {
+case v @ View(_, output, child) if child.resolved =>
+  val resolver = conf.resolver
+  val newOutput = output.map { attr =>
+val originAttr = findAttributeByName(attr.name, child.output, 
resolver)
+// The dataType of the output attributes may be not the same with 
that of the view output,
+// so we should cast the attribute to the dataType of the view 
output attribute. If the
+// cast cann't perform, will throw an AnalysisException.
+Alias(Cast(originAttr, attr.dataType), attr.name)(exprId = 
attr.exprId,
--- End diff --

Note that all tests can pass without this `Cast`, but it does fix a weird 
behavior: the result of a view query may have different schema if the view 
definition has been changed. shall we pull it out into a follow-up PR or do it 
here? cc @hvanhovell @yhuai 


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-10 Thread jiangxb1987
Github user jiangxb1987 commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95519850
  
--- Diff: sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala 
---
@@ -2476,4 +2476,14 @@ class SQLQuerySuite extends QueryTest with 
SharedSQLContext {
   assert(sql("SELECT * FROM array_tbl where arr = ARRAY(1L)").count == 
1)
 }
   }
+
+  test("should be able to resolve a persistent view") {
--- End diff --

We don't define the behavior to resolve a view using a SQLContext in 
current master, this test case is to define that behavior.


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-10 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95519288
  
--- Diff: sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala 
---
@@ -2476,4 +2476,14 @@ class SQLQuerySuite extends QueryTest with 
SharedSQLContext {
   assert(sql("SELECT * FROM array_tbl where arr = ARRAY(1L)").count == 
1)
 }
   }
+
+  test("should be able to resolve a persistent view") {
--- End diff --

What is your goal for this test case? Any reason?

BTW, we should move this test case to `SQLQueryTestSuite`. Now, we are 
trying to migrate such test cases to there.


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-10 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95517753
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala 
---
@@ -543,4 +545,157 @@ class SQLViewSuite extends QueryTest with 
SQLTestUtils with TestHiveSingleton {
   }
 }
   }
+
+  test("correctly resolve a nested view") {
+withTempDatabase { db =>
+  withView(s"$db.view1", s"$db.view2") {
+val view1 = CatalogTable(
+  identifier = TableIdentifier("view1", Some(db)),
+  tableType = CatalogTableType.VIEW,
+  storage = CatalogStorageFormat.empty,
+  schema = new StructType().add("id", "int").add("id1", "int"),
+  viewOriginalText = Some("SELECT * FROM jt"),
+  viewText = Some("SELECT * FROM jt"),
+  properties = Map[String, String] 
{CatalogTable.VIEW_DEFAULT_DATABASE -> "default"})
+val view2 = CatalogTable(
+  identifier = TableIdentifier("view2", Some(db)),
+  tableType = CatalogTableType.VIEW,
+  storage = CatalogStorageFormat.empty,
+  schema = new StructType().add("id", "int").add("id1", "int"),
+  viewOriginalText = Some("SELECT * FROM view1"),
+  viewText = Some("SELECT * FROM view1"),
+  properties = Map[String, String] 
{CatalogTable.VIEW_DEFAULT_DATABASE -> db})
+activateDatabase(db) {
+  hiveContext.sessionState.catalog.createTable(view1, 
ignoreIfExists = false)
+  hiveContext.sessionState.catalog.createTable(view2, 
ignoreIfExists = false)
+  checkAnswer(sql("SELECT * FROM view2 ORDER BY id"), (1 to 
9).map(i => Row(i, i)))
+}
+  }
+}
+  }
+
+  test("correctly resolve a view with CTE") {
+withView("cte_view") {
+  val cte_view = CatalogTable(
+identifier = TableIdentifier("cte_view"),
+tableType = CatalogTableType.VIEW,
+storage = CatalogStorageFormat.empty,
+schema = new StructType().add("n", "int"),
+viewOriginalText = Some("WITH w AS (SELECT 1 AS n) SELECT n FROM 
w"),
+viewText = Some("WITH w AS (SELECT 1 AS n) SELECT n FROM w"),
+properties = Map[String, String] 
{CatalogTable.VIEW_DEFAULT_DATABASE -> "default"})
+  hiveContext.sessionState.catalog.createTable(cte_view, 
ignoreIfExists = false)
+  checkAnswer(sql("SELECT * FROM cte_view"), Row(1))
+}
+  }
+
+  test("correctly resolve a view in a self join") {
+withView("join_view") {
+  val join_view = CatalogTable(
+identifier = TableIdentifier("join_view"),
+tableType = CatalogTableType.VIEW,
+storage = CatalogStorageFormat.empty,
+schema = new StructType().add("id", "int").add("id1", "int"),
+viewOriginalText = Some("SELECT * FROM jt"),
+viewText = Some("SELECT * FROM jt"),
+properties = Map[String, String] 
{CatalogTable.VIEW_DEFAULT_DATABASE -> "default"})
+  hiveContext.sessionState.catalog.createTable(join_view, 
ignoreIfExists = false)
+  checkAnswer(
+sql("SELECT * FROM join_view t1 JOIN join_view t2 ON t1.id = t2.id 
ORDER BY t1.id"),
+(1 to 9).map(i => Row(i, i, i, i)))
+}
+  }
+
+  private def assertInvalidReference(query: String): Unit = {
+val e = intercept[AnalysisException] {
+  sql(query)
+}.getMessage
+assert(e.contains("Table or view not found"))
+  }
+
+  test("error handling: fail if the referenced table or view is invalid") {
+withView("view1", "view2", "view3") {
+  // Fail if the referenced table is defined in a invalid database.
+  val view1 = CatalogTable(
+identifier = TableIdentifier("view1"),
+tableType = CatalogTableType.VIEW,
+storage = CatalogStorageFormat.empty,
+schema = new StructType().add("id", "int").add("id1", "int"),
+viewOriginalText = Some("SELECT * FROM invalid_db.jt"),
+viewText = Some("SELECT * FROM invalid_db.jt"),
+properties = Map[String, String] 
{CatalogTable.VIEW_DEFAULT_DATABASE -> "default"})
+  hiveContext.sessionState.catalog.createTable(view1, ignoreIfExists = 
false)
+  assertInvalidReference("SELECT * FROM view1")
+
+  // Fail if the referenced table is invalid.
+  val view2 = CatalogTable(
+identifier = TableIdentifier("view2"),
+tableType = CatalogTableType.VIEW,
+storage = CatalogStorageFormat.empty,
+schema = new StructType().add("id", "int").add("id1", "int"),
+viewOriginalText = Some("SELECT * FROM invalid_table"),
+viewText = Some("SELEC

[GitHub] spark pull request #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-10 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95515867
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLViewSuite.scala 
---
@@ -543,4 +545,157 @@ class SQLViewSuite extends QueryTest with 
SQLTestUtils with TestHiveSingleton {
   }
 }
   }
+
+  test("correctly resolve a nested view") {
+withTempDatabase { db =>
+  withView(s"$db.view1", s"$db.view2") {
+val view1 = CatalogTable(
+  identifier = TableIdentifier("view1", Some(db)),
+  tableType = CatalogTableType.VIEW,
+  storage = CatalogStorageFormat.empty,
+  schema = new StructType().add("id", "int").add("id1", "int"),
+  viewOriginalText = Some("SELECT * FROM jt"),
+  viewText = Some("SELECT * FROM jt"),
+  properties = Map[String, String] 
{CatalogTable.VIEW_DEFAULT_DATABASE -> "default"})
+val view2 = CatalogTable(
+  identifier = TableIdentifier("view2", Some(db)),
+  tableType = CatalogTableType.VIEW,
+  storage = CatalogStorageFormat.empty,
+  schema = new StructType().add("id", "int").add("id1", "int"),
+  viewOriginalText = Some("SELECT * FROM view1"),
+  viewText = Some("SELECT * FROM view1"),
+  properties = Map[String, String] 
{CatalogTable.VIEW_DEFAULT_DATABASE -> db})
+activateDatabase(db) {
+  hiveContext.sessionState.catalog.createTable(view1, 
ignoreIfExists = false)
+  hiveContext.sessionState.catalog.createTable(view2, 
ignoreIfExists = false)
+  checkAnswer(sql("SELECT * FROM view2 ORDER BY id"), (1 to 
9).map(i => Row(i, i)))
+}
+  }
+}
+  }
+
+  test("correctly resolve a view with CTE") {
+withView("cte_view") {
+  val cte_view = CatalogTable(
+identifier = TableIdentifier("cte_view"),
+tableType = CatalogTableType.VIEW,
+storage = CatalogStorageFormat.empty,
+schema = new StructType().add("n", "int"),
+viewOriginalText = Some("WITH w AS (SELECT 1 AS n) SELECT n FROM 
w"),
+viewText = Some("WITH w AS (SELECT 1 AS n) SELECT n FROM w"),
+properties = Map[String, String] 
{CatalogTable.VIEW_DEFAULT_DATABASE -> "default"})
+  hiveContext.sessionState.catalog.createTable(cte_view, 
ignoreIfExists = false)
+  checkAnswer(sql("SELECT * FROM cte_view"), Row(1))
+}
+  }
+
+  test("correctly resolve a view in a self join") {
+withView("join_view") {
+  val join_view = CatalogTable(
+identifier = TableIdentifier("join_view"),
+tableType = CatalogTableType.VIEW,
+storage = CatalogStorageFormat.empty,
+schema = new StructType().add("id", "int").add("id1", "int"),
+viewOriginalText = Some("SELECT * FROM jt"),
+viewText = Some("SELECT * FROM jt"),
+properties = Map[String, String] 
{CatalogTable.VIEW_DEFAULT_DATABASE -> "default"})
+  hiveContext.sessionState.catalog.createTable(join_view, 
ignoreIfExists = false)
+  checkAnswer(
+sql("SELECT * FROM join_view t1 JOIN join_view t2 ON t1.id = t2.id 
ORDER BY t1.id"),
+(1 to 9).map(i => Row(i, i, i, i)))
+}
+  }
+
+  private def assertInvalidReference(query: String): Unit = {
+val e = intercept[AnalysisException] {
+  sql(query)
+}.getMessage
+assert(e.contains("Table or view not found"))
+  }
+
+  test("error handling: fail if the referenced table or view is invalid") {
+withView("view1", "view2", "view3") {
+  // Fail if the referenced table is defined in a invalid database.
+  val view1 = CatalogTable(
+identifier = TableIdentifier("view1"),
+tableType = CatalogTableType.VIEW,
+storage = CatalogStorageFormat.empty,
+schema = new StructType().add("id", "int").add("id1", "int"),
+viewOriginalText = Some("SELECT * FROM invalid_db.jt"),
+viewText = Some("SELECT * FROM invalid_db.jt"),
+properties = Map[String, String] 
{CatalogTable.VIEW_DEFAULT_DATABASE -> "default"})
+  hiveContext.sessionState.catalog.createTable(view1, ignoreIfExists = 
false)
+  assertInvalidReference("SELECT * FROM view1")
+
+  // Fail if the referenced table is invalid.
+  val view2 = CatalogTable(
+identifier = TableIdentifier("view2"),
+tableType = CatalogTableType.VIEW,
+storage = CatalogStorageFormat.empty,
+schema = new StructType().add("id", "int").add("id1", "int"),
+viewOriginalText = Some("SELECT * FROM invalid_table"),
+viewText = Some("SELEC

[GitHub] spark pull request #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-10 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95515728
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala 
---
@@ -0,0 +1,80 @@
+/*
+ * 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.AnalysisException
+import org.apache.spark.sql.catalyst.CatalystConf
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, 
View}
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+ * This file defines analysis rules related to views.
+ */
+
+/**
+ * Make sure that a view's child plan produces the view's output 
attributes. We wrap the child
+ * with a Project and add an alias for each output attribute. The 
attributes are resolved by
+ * name. This should be only done after the batch of Resolution, because 
the view attributes are
+ * not completely resolved during the batch of Resolution.
+ */
+case class AliasViewChild(conf: CatalystConf) extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan 
resolveOperators {
+case v @ View(_, output, child) if child.resolved =>
+  val resolver = conf.resolver
+  val newOutput = output.map { attr =>
+val originAttr = findAttributeByName(attr.name, child.output, 
resolver)
+// The dataType of the output attributes may be not the same with 
that of the view output,
+// so we should cast the attribute to the dataType of the view 
output attribute. If the
+// cast cann't perform, will throw an AnalysisException.
--- End diff --

Nit: `cann't ` -> `can't `


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-10 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95515250
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -510,32 +545,87 @@ class Analyzer(
* Replaces [[UnresolvedRelation]]s with concrete relations from the 
catalog.
*/
   object ResolveRelations extends Rule[LogicalPlan] {
-private def lookupTableFromCatalog(u: UnresolvedRelation): LogicalPlan 
= {
+
+// If the unresolved relation is running directly on files, we just 
return the original
+// UnresolvedRelation, the plan will get resolved later. Else we look 
up the table from catalog
+// and change the default database name(in AnalysisContext) if it is a 
view.
+// We usually look up a table from the default database if the table 
identifier has an empty
+// database part, for a view the default database should be the 
currentDb when the view was
+// created. When the case comes to resolving a nested view, the view 
may have different default
+// database with that the referenced view has, so we need to use
+// `AnalysisContext.defaultDatabase` to track the current default 
database.
+// When the relation we resolve is a view, we fetch the 
view.desc(which is a CatalogTable), and
+// then set the value of `CatalogTable.viewDefaultDatabase` to
+// `AnalysisContext.defaultDatabase`, we look up the relations that 
the view references using
+// the default database.
+// For example:
+// |- view1 (defaultDatabase = db1)
+//   |- operator
+// |- table2 (defaultDatabase = db1)
+// |- view2 (defaultDatabase = db2)
+//|- view3 (defaultDatabase = db3)
+//   |- view4 (defaultDatabase = db4)
+// In this case, the view `view1` is a nested view, it directly 
references `table2`、`view2`
+// and `view4`, the view `view2` references `view3`. On resolving the 
table, we look up the
+// relations `table2`、`view2`、`view4` using the default database 
`db1`, and look up the
+// relation `view3` using the default database `db2`.
+//
+// Note this is compatible with the views defined by older versions of 
Spark(before 2.2), which
+// have empty defaultDatabase and all the relations in viewText have 
database part defined.
+def resolveRelation(plan: LogicalPlan): LogicalPlan = plan match {
+  case u: UnresolvedRelation if 
!isRunningDirectlyOnFiles(u.tableIdentifier) =>
+val defaultDatabase = AnalysisContext.get.defaultDatabase
+val relation = lookupTableFromCatalog(u, defaultDatabase)
+resolveRelation(relation)
+  // The view's child should be a logical plan parsed from the 
`desc.viewText`, the variable
+  // `viewText` should be defined, or else we throw an error on the 
generation of the View
+  // operator.
+  case view @ View(desc, _, child) if !child.resolved =>
+// Resolve all the UnresolvedRelations and Views in the child.
+val newChild = 
AnalysisContext.withAnalysisContext(desc.viewDefaultDatabase) {
+  execute(child)
+}
+view.copy(child = newChild)
+  case p @ SubqueryAlias(_, view: View, _) =>
+val newChild = resolveRelation(view)
+p.copy(child = newChild)
+  case _ => plan
+}
+
+def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+  case i @ InsertIntoTable(u: UnresolvedRelation, parts, child, _, _) 
if child.resolved =>
+i.copy(table = EliminateSubqueryAliases(lookupTableFromCatalog(u)))
+  case u: UnresolvedRelation => resolveRelation(u)
+}
+
+// Look up the table with the given name from catalog. The database we 
used is decided by the
+// precedence:
+// 1. Use the database part of the table identifier, if it is defined;
+// 2. Use defaultDatabase, if it is defined(In this case, no temporary 
objects can be used,
+//and the default database is only used to look up a view);
+// 3. Use the currentDb of the SessionCatalog.
+private def lookupTableFromCatalog(
+u: UnresolvedRelation,
+defaultDatabase: Option[String] = None): LogicalPlan = {
   try {
-catalog.lookupRelation(u.tableIdentifier, u.alias)
+val tableIdentWithDb = u.tableIdentifier.copy(
+  database = u.tableIdentifier.database.orElse(defaultDatabase))
+catalog.lookupRelation(tableIdentWithDb, u.alias)
   } catch {
 case _: NoSuchTableException =>
   u.failAnalysis(s"Table or view not found: ${u.tableName}")
   }
 }
 
-def apply(plan: LogicalPlan): LogicalPla

[GitHub] spark pull request #16233: [SPARK-18801][SQL] Support resolve a nested view

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

https://github.com/apache/spark/pull/16233#discussion_r95514996
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala
 ---
@@ -860,6 +864,24 @@ abstract class CatalogTestUtils {
   bucketSpec = Some(BucketSpec(4, Seq("col1"), Nil)))
   }
 
+  def newView(
+  name: String,
+  database: Option[String] = None): CatalogTable = {
+val viewDefaultDatabase = database.getOrElse("default")
+CatalogTable(
+  identifier = TableIdentifier(name, database),
+  tableType = CatalogTableType.VIEW,
+  storage = CatalogStorageFormat.empty,
+  schema = new StructType()
+.add("col1", "int")
+.add("col2", "string")
+.add("a", "int")
+.add("b", "string"),
+  viewOriginalText = Some("SELECT * FROM tbl1"),
+  viewText = Some("SELECT * FROM tbl1"),
+  properties = Map[String, String] {CatalogTable.VIEW_DEFAULT_DATABASE 
-> viewDefaultDatabase})
--- End diff --

nit: `Map(CatalogTable.VIEW_DEFAULT_DATABASE -> viewDefaultDatabase)`, 
scala comiler will infer the type for us


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

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

https://github.com/apache/spark/pull/16233#discussion_r95514898
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala
 ---
@@ -378,6 +379,35 @@ case class InsertIntoTable(
 }
 
 /**
+ * A container for holding the view description(CatalogTable), and the 
output of the view. The
+ * child should be a logical plan parsed from the `CatalogTable.viewText`, 
should throw an error
+ * if the `viewText` is not defined.
+ * This operator will be removed at the end of analysis stage.
+ *
+ * @param desc A view description(CatalogTable) that provides necessary 
information to resolve the
+ * view.
+ * @param output The output of a view operator, this is generated during 
planning the view, so that
+ *   we are able to decouple the output from the underlying 
structure.
+ * @param child The logical plan of a view operator, it should be a 
logical plan parsed from the
+ *  `CatalogTable.viewText`, should throw an error if the 
`viewText` is not defined.
+ */
+case class View(
+desc: CatalogTable,
+output: Seq[Attribute],
+child: LogicalPlan) extends LogicalPlan with MultiInstanceRelation {
--- End diff --

nit: extends `UnaryNode`?


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

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

https://github.com/apache/spark/pull/16233#discussion_r95514867
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala 
---
@@ -0,0 +1,80 @@
+/*
+ * 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.AnalysisException
+import org.apache.spark.sql.catalyst.CatalystConf
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, 
View}
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+ * This file defines analysis rules related to views.
+ */
+
+/**
+ * Make sure that a view's child plan produces the view's output 
attributes. We wrap the child
+ * with a Project and add an alias for each output attribute. The 
attributes are resolved by
+ * name. This should be only done after the batch of Resolution, because 
the view attributes are
+ * not completely resolved during the batch of Resolution.
+ */
+case class AliasViewChild(conf: CatalystConf) extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan 
resolveOperators {
+case v @ View(_, output, child) if child.resolved =>
+  val resolver = conf.resolver
+  val newOutput = output.map { attr =>
+val originAttr = findAttributeByName(attr.name, child.output, 
resolver)
+// The dataType of the output attributes may be not the same with 
that of the view output,
+// so we should cast the attribute to the dataType of the view 
output attribute. If the
+// cast cann't perform, will throw an AnalysisException.
+Alias(Cast(originAttr, attr.dataType), attr.name)(exprId = 
attr.exprId,
+  qualifier = attr.qualifier, explicitMetadata = 
Some(attr.metadata))
+  }
+  v.copy(child = Project(newOutput, child))
+  }
+
+  /**
+   * Find the attribute that has the expected attribute name from an 
attribute list, the names
+   * are compared using conf.resolver.
+   * If the expected attribute is not found, throw an AnalysisException.
+   */
+  private def findAttributeByName(
+  name: String,
+  attrs: Seq[Attribute],
+  resolver: Resolver): Attribute = {
+attrs.collectFirst {
--- End diff --

nit: use `find`?


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-10 Thread jiangxb1987
Github user jiangxb1987 commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95512712
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
@@ -125,11 +132,16 @@ private[hive] class 
HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
   // Otherwise, wrap the table with a Subquery using the table name.
   alias.map(a => SubqueryAlias(a, qualifiedTable, 
None)).getOrElse(qualifiedTable)
 } else if (table.tableType == CatalogTableType.VIEW) {
+  val tableIdentifier = table.identifier
   val viewText = table.viewText.getOrElse(sys.error("Invalid view 
without text."))
-  SubqueryAlias(
-alias.getOrElse(table.identifier.table),
-sparkSession.sessionState.sqlParser.parsePlan(viewText),
-Option(table.identifier))
+  // The relation is a view, so we wrap the relation by:
+  // 1. Add a [[View]] operator over the relation to keep track of the 
view desc;
+  // 2. Wrap the logical plan in a [[SubqueryAlias]] which tracks the 
name of the view.
+  val child = View(
+desc = table,
+output = table.schema.toAttributes,
+child = sparkSession.sessionState.sqlParser.parsePlan(viewText))
--- End diff --

yup


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

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

https://github.com/apache/spark/pull/16233#discussion_r95512479
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
@@ -125,11 +132,16 @@ private[hive] class 
HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
   // Otherwise, wrap the table with a Subquery using the table name.
   alias.map(a => SubqueryAlias(a, qualifiedTable, 
None)).getOrElse(qualifiedTable)
 } else if (table.tableType == CatalogTableType.VIEW) {
+  val tableIdentifier = table.identifier
   val viewText = table.viewText.getOrElse(sys.error("Invalid view 
without text."))
-  SubqueryAlias(
-alias.getOrElse(table.identifier.table),
-sparkSession.sessionState.sqlParser.parsePlan(viewText),
-Option(table.identifier))
+  // The relation is a view, so we wrap the relation by:
+  // 1. Add a [[View]] operator over the relation to keep track of the 
view desc;
+  // 2. Wrap the logical plan in a [[SubqueryAlias]] which tracks the 
name of the view.
+  val child = View(
+desc = table,
+output = table.schema.toAttributes,
+child = sparkSession.sessionState.sqlParser.parsePlan(viewText))
--- End diff --

ah i see, so when we reference a cached view, we wanna a different set of 
outputs, to avoid conflicts.


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

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

https://github.com/apache/spark/pull/16233#discussion_r95511706
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -510,32 +545,88 @@ class Analyzer(
* Replaces [[UnresolvedRelation]]s with concrete relations from the 
catalog.
*/
   object ResolveRelations extends Rule[LogicalPlan] {
-private def lookupTableFromCatalog(u: UnresolvedRelation): LogicalPlan 
= {
+
+// If the unresolved relation is running directly on files, we just 
return the original
+// UnresolvedRelation, the plan will get resolved later. Else we look 
up the table from catalog
+// and change the default database name(in AnalysisContext) if it is a 
view.
+// We usually look up a table from the default database if the table 
identifier has an empty
+// database part, for a view the default database should be the 
currentDb when the view was
+// created. When the case comes to resolving a nested view, the view 
may have different default
+// database with that the referenced view has, so we need to use
+// `AnalysisContext.defaultDatabase` to track the current default 
database.
+// When the relation we resolve is a view, we fetch the 
view.desc(which is a CatalogTable), and
+// then set the value of `CatalogTable.viewDefaultDatabase` to
+// `AnalysisContext.defaultDatabase`, we look up the relations that 
the view references using
+// the default database.
+// For example:
+// |- view1 (defaultDatabase = db1)
+//   |- operator
+// |- table2 (defaultDatabase = db1)
+// |- view2 (defaultDatabase = db2)
+//|- view3 (defaultDatabase = db3)
+//   |- view4 (defaultDatabase = db4)
+// In this case, the view `view1` is a nested view, it directly 
references `table2`、`view2`
+// and `view4`, the view `view2` references `view3`. On resolving the 
table, we look up the
+// relations `table2`、`view2`、`view4` using the default database 
`db1`, and look up the
+// relation `view3` using the default database `db2`.
+//
+// Note this is compatible with the views defined by older versions of 
Spark(before 2.2), which
+// have empty defaultDatabase and all the relations in viewText have 
database part defined.
+def resolveRelation(plan: LogicalPlan): LogicalPlan = plan match {
+  case u: UnresolvedRelation if 
!isRunningDirectlyOnFiles(u.tableIdentifier) =>
+val defaultDatabase = AnalysisContext.get.defaultDatabase
+val relation = lookupTableFromCatalog(u, defaultDatabase)
+resolveRelation(relation)
+  // The view's child should be a logical plan parsed from the 
`desc.viewText`, the variable
+  // `viewText` should be defined, or else we throw an error on the 
generation of the View
+  // operator.
+  case view @ View(desc, _, child) if !child.resolved =>
+// Resolve all the UnresolvedRelations and Views in the child.
+val newChild = 
AnalysisContext.withAnalysisContext(desc.viewDefaultDatabase) {
+  execute(child)
+}
+view.copy(child = newChild)
+  case p @ SubqueryAlias(_, view: View, _) =>
+val newChild = resolveRelation(view)
+p.copy(child = newChild)
+  case _ => plan
+}
+
+def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+  case i @ InsertIntoTable(u: UnresolvedRelation, parts, child, _, _) 
if child.resolved =>
+i.copy(table = EliminateSubqueryAliases(lookupTableFromCatalog(u)))
+  case u: UnresolvedRelation => resolveRelation(u)
+}
+
+// Look up the table with the given name from catalog. The database we 
used is decided by the
+// precedence:
+// 1. Use the database part of the table identifier, if it is defined;
+// 2. Use defaultDatabase, if it is defined(In this case, no temporary 
objects can be used,
+//and the default database is only used to look up a view);
+// 3. Use the currentDb of the SessionCatalog.
+private def lookupTableFromCatalog(
+u: UnresolvedRelation,
+defaultDatabase: Option[String] = None): LogicalPlan = {
   try {
-catalog.lookupRelation(u.tableIdentifier, u.alias)
+// If `u.tableIdentifier` has a empty database part, copy the 
identifier with
+// `defaultDatabase` as the database part.
+val tableIdentWithDb = 
u.tableIdentifier.withDatabase(defaultDatabase)
--- End diff --

Sure lets remove this then.


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

[GitHub] spark pull request #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-10 Thread jiangxb1987
Github user jiangxb1987 commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95510561
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -510,32 +545,88 @@ class Analyzer(
* Replaces [[UnresolvedRelation]]s with concrete relations from the 
catalog.
*/
   object ResolveRelations extends Rule[LogicalPlan] {
-private def lookupTableFromCatalog(u: UnresolvedRelation): LogicalPlan 
= {
+
+// If the unresolved relation is running directly on files, we just 
return the original
+// UnresolvedRelation, the plan will get resolved later. Else we look 
up the table from catalog
+// and change the default database name(in AnalysisContext) if it is a 
view.
+// We usually look up a table from the default database if the table 
identifier has an empty
+// database part, for a view the default database should be the 
currentDb when the view was
+// created. When the case comes to resolving a nested view, the view 
may have different default
+// database with that the referenced view has, so we need to use
+// `AnalysisContext.defaultDatabase` to track the current default 
database.
+// When the relation we resolve is a view, we fetch the 
view.desc(which is a CatalogTable), and
+// then set the value of `CatalogTable.viewDefaultDatabase` to
+// `AnalysisContext.defaultDatabase`, we look up the relations that 
the view references using
+// the default database.
+// For example:
+// |- view1 (defaultDatabase = db1)
+//   |- operator
+// |- table2 (defaultDatabase = db1)
+// |- view2 (defaultDatabase = db2)
+//|- view3 (defaultDatabase = db3)
+//   |- view4 (defaultDatabase = db4)
+// In this case, the view `view1` is a nested view, it directly 
references `table2`、`view2`
+// and `view4`, the view `view2` references `view3`. On resolving the 
table, we look up the
+// relations `table2`、`view2`、`view4` using the default database 
`db1`, and look up the
+// relation `view3` using the default database `db2`.
+//
+// Note this is compatible with the views defined by older versions of 
Spark(before 2.2), which
+// have empty defaultDatabase and all the relations in viewText have 
database part defined.
+def resolveRelation(plan: LogicalPlan): LogicalPlan = plan match {
+  case u: UnresolvedRelation if 
!isRunningDirectlyOnFiles(u.tableIdentifier) =>
+val defaultDatabase = AnalysisContext.get.defaultDatabase
+val relation = lookupTableFromCatalog(u, defaultDatabase)
+resolveRelation(relation)
+  // The view's child should be a logical plan parsed from the 
`desc.viewText`, the variable
+  // `viewText` should be defined, or else we throw an error on the 
generation of the View
+  // operator.
+  case view @ View(desc, _, child) if !child.resolved =>
+// Resolve all the UnresolvedRelations and Views in the child.
+val newChild = 
AnalysisContext.withAnalysisContext(desc.viewDefaultDatabase) {
+  execute(child)
+}
+view.copy(child = newChild)
+  case p @ SubqueryAlias(_, view: View, _) =>
+val newChild = resolveRelation(view)
+p.copy(child = newChild)
+  case _ => plan
+}
+
+def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+  case i @ InsertIntoTable(u: UnresolvedRelation, parts, child, _, _) 
if child.resolved =>
+i.copy(table = EliminateSubqueryAliases(lookupTableFromCatalog(u)))
+  case u: UnresolvedRelation => resolveRelation(u)
+}
+
+// Look up the table with the given name from catalog. The database we 
used is decided by the
+// precedence:
+// 1. Use the database part of the table identifier, if it is defined;
+// 2. Use defaultDatabase, if it is defined(In this case, no temporary 
objects can be used,
+//and the default database is only used to look up a view);
+// 3. Use the currentDb of the SessionCatalog.
+private def lookupTableFromCatalog(
+u: UnresolvedRelation,
+defaultDatabase: Option[String] = None): LogicalPlan = {
   try {
-catalog.lookupRelation(u.tableIdentifier, u.alias)
+// If `u.tableIdentifier` has a empty database part, copy the 
identifier with
+// `defaultDatabase` as the database part.
+val tableIdentWithDb = 
u.tableIdentifier.withDatabase(defaultDatabase)
--- End diff --

I'm neutural with this issue, cc @hvanhovell for more advise.


---
If your project is set up for it, you can reply to this email and have your
reply appear 

[GitHub] spark pull request #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-10 Thread jiangxb1987
Github user jiangxb1987 commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95510361
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
@@ -125,11 +132,16 @@ private[hive] class 
HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
   // Otherwise, wrap the table with a Subquery using the table name.
   alias.map(a => SubqueryAlias(a, qualifiedTable, 
None)).getOrElse(qualifiedTable)
 } else if (table.tableType == CatalogTableType.VIEW) {
+  val tableIdentifier = table.identifier
   val viewText = table.viewText.getOrElse(sys.error("Invalid view 
without text."))
-  SubqueryAlias(
-alias.getOrElse(table.identifier.table),
-sparkSession.sessionState.sqlParser.parsePlan(viewText),
-Option(table.identifier))
+  // The relation is a view, so we wrap the relation by:
+  // 1. Add a [[View]] operator over the relation to keep track of the 
view desc;
+  // 2. Wrap the logical plan in a [[SubqueryAlias]] which tracks the 
name of the view.
+  val child = View(
+desc = table,
+output = table.schema.toAttributes,
+child = sparkSession.sessionState.sqlParser.parsePlan(viewText))
--- End diff --

The resolution of a View operator takes place by two stages:
For the first stage, we look up the relation and generate a View node by:
1. Parse the `viewText` to generate a child logical plan, which is 
unresolved;
2. Generate the output attribute from the `CatalogTable.schema`, which is 
resolved;
This stage focus on planning of the view.

For the second stage, we resolve the child logical plan of the View node, 
the logic is in `ResolveRelations.resolveRelation()`.
In this stage, we resolve the child logical plan, which was parsed from the 
`viewText`(the sql query string that creates the view).

At last, in `AliasViewChild` rule, we alias the resolved child plan to the 
view node. In this way, we successfully resolve a view.

In the future, we could cache the `viewText` and its corresponding logical 
plan. In case a view is referenced for multiple times, we only have to parse 
and resolve the query for once, and for later references we can read directly 
from the cache.


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

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

https://github.com/apache/spark/pull/16233#discussion_r95498331
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
@@ -125,11 +132,16 @@ private[hive] class 
HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
   // Otherwise, wrap the table with a Subquery using the table name.
   alias.map(a => SubqueryAlias(a, qualifiedTable, 
None)).getOrElse(qualifiedTable)
 } else if (table.tableType == CatalogTableType.VIEW) {
+  val tableIdentifier = table.identifier
   val viewText = table.viewText.getOrElse(sys.error("Invalid view 
without text."))
-  SubqueryAlias(
-alias.getOrElse(table.identifier.table),
-sparkSession.sessionState.sqlParser.parsePlan(viewText),
-Option(table.identifier))
+  // The relation is a view, so we wrap the relation by:
+  // 1. Add a [[View]] operator over the relation to keep track of the 
view desc;
+  // 2. Wrap the logical plan in a [[SubqueryAlias]] which tracks the 
name of the view.
+  val child = View(
+desc = table,
+output = table.schema.toAttributes,
+child = sparkSession.sessionState.sqlParser.parsePlan(viewText))
--- End diff --

can you explain it more? How does an individual `output` field help us 
decouple planning of query from the planning of the view?


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

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

https://github.com/apache/spark/pull/16233#discussion_r95494148
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -510,32 +545,88 @@ class Analyzer(
* Replaces [[UnresolvedRelation]]s with concrete relations from the 
catalog.
*/
   object ResolveRelations extends Rule[LogicalPlan] {
-private def lookupTableFromCatalog(u: UnresolvedRelation): LogicalPlan 
= {
+
+// If the unresolved relation is running directly on files, we just 
return the original
+// UnresolvedRelation, the plan will get resolved later. Else we look 
up the table from catalog
+// and change the default database name(in AnalysisContext) if it is a 
view.
+// We usually look up a table from the default database if the table 
identifier has an empty
+// database part, for a view the default database should be the 
currentDb when the view was
+// created. When the case comes to resolving a nested view, the view 
may have different default
+// database with that the referenced view has, so we need to use
+// `AnalysisContext.defaultDatabase` to track the current default 
database.
+// When the relation we resolve is a view, we fetch the 
view.desc(which is a CatalogTable), and
+// then set the value of `CatalogTable.viewDefaultDatabase` to
+// `AnalysisContext.defaultDatabase`, we look up the relations that 
the view references using
+// the default database.
+// For example:
+// |- view1 (defaultDatabase = db1)
+//   |- operator
+// |- table2 (defaultDatabase = db1)
+// |- view2 (defaultDatabase = db2)
+//|- view3 (defaultDatabase = db3)
+//   |- view4 (defaultDatabase = db4)
+// In this case, the view `view1` is a nested view, it directly 
references `table2`、`view2`
+// and `view4`, the view `view2` references `view3`. On resolving the 
table, we look up the
+// relations `table2`、`view2`、`view4` using the default database 
`db1`, and look up the
+// relation `view3` using the default database `db2`.
+//
+// Note this is compatible with the views defined by older versions of 
Spark(before 2.2), which
+// have empty defaultDatabase and all the relations in viewText have 
database part defined.
+def resolveRelation(plan: LogicalPlan): LogicalPlan = plan match {
+  case u: UnresolvedRelation if 
!isRunningDirectlyOnFiles(u.tableIdentifier) =>
+val defaultDatabase = AnalysisContext.get.defaultDatabase
+val relation = lookupTableFromCatalog(u, defaultDatabase)
+resolveRelation(relation)
+  // The view's child should be a logical plan parsed from the 
`desc.viewText`, the variable
+  // `viewText` should be defined, or else we throw an error on the 
generation of the View
+  // operator.
+  case view @ View(desc, _, child) if !child.resolved =>
+// Resolve all the UnresolvedRelations and Views in the child.
+val newChild = 
AnalysisContext.withAnalysisContext(desc.viewDefaultDatabase) {
+  execute(child)
+}
+view.copy(child = newChild)
+  case p @ SubqueryAlias(_, view: View, _) =>
+val newChild = resolveRelation(view)
+p.copy(child = newChild)
+  case _ => plan
+}
+
+def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+  case i @ InsertIntoTable(u: UnresolvedRelation, parts, child, _, _) 
if child.resolved =>
+i.copy(table = EliminateSubqueryAliases(lookupTableFromCatalog(u)))
+  case u: UnresolvedRelation => resolveRelation(u)
+}
+
+// Look up the table with the given name from catalog. The database we 
used is decided by the
+// precedence:
+// 1. Use the database part of the table identifier, if it is defined;
+// 2. Use defaultDatabase, if it is defined(In this case, no temporary 
objects can be used,
+//and the default database is only used to look up a view);
+// 3. Use the currentDb of the SessionCatalog.
+private def lookupTableFromCatalog(
+u: UnresolvedRelation,
+defaultDatabase: Option[String] = None): LogicalPlan = {
   try {
-catalog.lookupRelation(u.tableIdentifier, u.alias)
+// If `u.tableIdentifier` has a empty database part, copy the 
identifier with
+// `defaultDatabase` as the database part.
+val tableIdentWithDb = 
u.tableIdentifier.withDatabase(defaultDatabase)
--- End diff --

I don't think `withDatabase` is general purpose. A general purpose 
`withDatabase` should always respect the passed in database no matter the 
original databas

[GitHub] spark pull request #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-10 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95455394
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -510,32 +542,90 @@ class Analyzer(
* Replaces [[UnresolvedRelation]]s with concrete relations from the 
catalog.
*/
   object ResolveRelations extends Rule[LogicalPlan] {
-private def lookupTableFromCatalog(u: UnresolvedRelation): LogicalPlan 
= {
+
+// If the unresolved relation is running directly on files, we just 
return the original
+// UnresolvedRelation, the plan will get resolved later. Else we look 
up the table from catalog
+// and change the default database name if it is a view.
+// We usually look up a table from the default database if the table 
identifier has an empty
+// database part, for a view the default database should be the 
currentDb when the view was
+// created. When the case comes to resolving a nested view, the view 
may have different default
+// database with that the referenced view has, so we need to use the 
variable `defaultDatabase`
+// to track the current default database.
+// When the relation we resolve is a view, we fetch the 
view.desc(which is a CatalogTable), and
+// then set the value of `CatalogTable.viewDefaultDatabase` to the 
variable `defaultDatabase`,
+// we look up the relations that the view references using the default 
database.
+// For example:
+// |- view1 (defaultDatabase = db1)
+//   |- operator
+// |- table2 (defaultDatabase = db1)
+// |- view2 (defaultDatabase = db2)
+//|- view3 (defaultDatabase = db3)
+//   |- view4 (defaultDatabase = db4)
+// In this case, the view `view1` is a nested view, it directly 
references `table2`、`view2`
+// and `view4`, the view `view2` references `view3`. On resolving the 
table, we look up the
+// relations `table2`、`view2`、`view4` using the default database 
`db1`, and look up the
+// relation `view3` using the default database `db2`.
+//
+// Note this is compatible with the views defined by older versions of 
Spark(before 2.2), which
+// have empty defaultDatabase and all the relations in viewText have 
database part defined.
+def resolveRelation(
+plan: LogicalPlan,
+defaultDatabase: Option[String] = None): LogicalPlan = plan match {
+  case u: UnresolvedRelation if 
!isRunningDirectlyOnFiles(u.tableIdentifier) =>
+val defaultDatabase = AnalysisContext.get.defaultDatabase
+val relation = lookupTableFromCatalog(u, defaultDatabase)
+resolveRelation(relation, defaultDatabase)
+  // The view's child should be a logical plan parsed from the 
`desc.viewText`, the variable
+  // `viewText` should be defined, or else we throw an error on the 
generation of the View
+  // operator.
+  case view @ View(desc, _, child) if !child.resolved =>
+val nestedViewLevel = AnalysisContext.get.nestedViewLevel + 1
+val context = AnalysisContext(defaultDatabase = 
desc.viewDefaultDatabase,
+  nestedViewLevel = nestedViewLevel)
+// Resolve all the UnresolvedRelations and Views in the child.
+val newChild = AnalysisContext.withAnalysisContext(context) {
+  execute(child)
+}
--- End diff --

We can then improve the interface when it happens. So far, it is not needed.


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-10 Thread jiangxb1987
Github user jiangxb1987 commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95431784
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -510,32 +542,90 @@ class Analyzer(
* Replaces [[UnresolvedRelation]]s with concrete relations from the 
catalog.
*/
   object ResolveRelations extends Rule[LogicalPlan] {
-private def lookupTableFromCatalog(u: UnresolvedRelation): LogicalPlan 
= {
+
+// If the unresolved relation is running directly on files, we just 
return the original
+// UnresolvedRelation, the plan will get resolved later. Else we look 
up the table from catalog
+// and change the default database name if it is a view.
+// We usually look up a table from the default database if the table 
identifier has an empty
+// database part, for a view the default database should be the 
currentDb when the view was
+// created. When the case comes to resolving a nested view, the view 
may have different default
+// database with that the referenced view has, so we need to use the 
variable `defaultDatabase`
+// to track the current default database.
+// When the relation we resolve is a view, we fetch the 
view.desc(which is a CatalogTable), and
+// then set the value of `CatalogTable.viewDefaultDatabase` to the 
variable `defaultDatabase`,
+// we look up the relations that the view references using the default 
database.
+// For example:
+// |- view1 (defaultDatabase = db1)
+//   |- operator
+// |- table2 (defaultDatabase = db1)
+// |- view2 (defaultDatabase = db2)
+//|- view3 (defaultDatabase = db3)
+//   |- view4 (defaultDatabase = db4)
+// In this case, the view `view1` is a nested view, it directly 
references `table2`、`view2`
+// and `view4`, the view `view2` references `view3`. On resolving the 
table, we look up the
+// relations `table2`、`view2`、`view4` using the default database 
`db1`, and look up the
+// relation `view3` using the default database `db2`.
+//
+// Note this is compatible with the views defined by older versions of 
Spark(before 2.2), which
+// have empty defaultDatabase and all the relations in viewText have 
database part defined.
+def resolveRelation(
+plan: LogicalPlan,
+defaultDatabase: Option[String] = None): LogicalPlan = plan match {
+  case u: UnresolvedRelation if 
!isRunningDirectlyOnFiles(u.tableIdentifier) =>
+val defaultDatabase = AnalysisContext.get.defaultDatabase
+val relation = lookupTableFromCatalog(u, defaultDatabase)
+resolveRelation(relation, defaultDatabase)
+  // The view's child should be a logical plan parsed from the 
`desc.viewText`, the variable
+  // `viewText` should be defined, or else we throw an error on the 
generation of the View
+  // operator.
+  case view @ View(desc, _, child) if !child.resolved =>
+val nestedViewLevel = AnalysisContext.get.nestedViewLevel + 1
+val context = AnalysisContext(defaultDatabase = 
desc.viewDefaultDatabase,
+  nestedViewLevel = nestedViewLevel)
+// Resolve all the UnresolvedRelations and Views in the child.
+val newChild = AnalysisContext.withAnalysisContext(context) {
+  execute(child)
+}
--- End diff --

What if the `AnalysisContext` is used by other operator in the future? That 
may cause wrong counting of `nestedViewLevel`.


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-10 Thread jiangxb1987
Github user jiangxb1987 commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95430942
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -510,32 +542,90 @@ class Analyzer(
* Replaces [[UnresolvedRelation]]s with concrete relations from the 
catalog.
*/
   object ResolveRelations extends Rule[LogicalPlan] {
-private def lookupTableFromCatalog(u: UnresolvedRelation): LogicalPlan 
= {
+
+// If the unresolved relation is running directly on files, we just 
return the original
+// UnresolvedRelation, the plan will get resolved later. Else we look 
up the table from catalog
+// and change the default database name if it is a view.
+// We usually look up a table from the default database if the table 
identifier has an empty
+// database part, for a view the default database should be the 
currentDb when the view was
+// created. When the case comes to resolving a nested view, the view 
may have different default
+// database with that the referenced view has, so we need to use the 
variable `defaultDatabase`
+// to track the current default database.
+// When the relation we resolve is a view, we fetch the 
view.desc(which is a CatalogTable), and
+// then set the value of `CatalogTable.viewDefaultDatabase` to the 
variable `defaultDatabase`,
+// we look up the relations that the view references using the default 
database.
+// For example:
+// |- view1 (defaultDatabase = db1)
+//   |- operator
+// |- table2 (defaultDatabase = db1)
+// |- view2 (defaultDatabase = db2)
+//|- view3 (defaultDatabase = db3)
+//   |- view4 (defaultDatabase = db4)
+// In this case, the view `view1` is a nested view, it directly 
references `table2`、`view2`
+// and `view4`, the view `view2` references `view3`. On resolving the 
table, we look up the
+// relations `table2`、`view2`、`view4` using the default database 
`db1`, and look up the
+// relation `view3` using the default database `db2`.
+//
+// Note this is compatible with the views defined by older versions of 
Spark(before 2.2), which
+// have empty defaultDatabase and all the relations in viewText have 
database part defined.
+def resolveRelation(
+plan: LogicalPlan,
+defaultDatabase: Option[String] = None): LogicalPlan = plan match {
+  case u: UnresolvedRelation if 
!isRunningDirectlyOnFiles(u.tableIdentifier) =>
+val defaultDatabase = AnalysisContext.get.defaultDatabase
+val relation = lookupTableFromCatalog(u, defaultDatabase)
+resolveRelation(relation, defaultDatabase)
+  // The view's child should be a logical plan parsed from the 
`desc.viewText`, the variable
+  // `viewText` should be defined, or else we throw an error on the 
generation of the View
+  // operator.
+  case view @ View(desc, _, child) if !child.resolved =>
+val nestedViewLevel = AnalysisContext.get.nestedViewLevel + 1
+val context = AnalysisContext(defaultDatabase = 
desc.viewDefaultDatabase,
+  nestedViewLevel = nestedViewLevel)
+// Resolve all the UnresolvedRelations and Views in the child.
+val newChild = AnalysisContext.withAnalysisContext(context) {
+  execute(child)
+}
+view.copy(child = newChild)
+  case p @ SubqueryAlias(_, view: View, _) =>
+val newChild = resolveRelation(view, defaultDatabase)
+p.copy(child = newChild)
+  case _ => plan
+}
+
+def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+  case i @ InsertIntoTable(u: UnresolvedRelation, parts, child, _, _) 
if child.resolved =>
+i.copy(table = EliminateSubqueryAliases(lookupTableFromCatalog(u)))
+  case u: UnresolvedRelation => resolveRelation(u)
+}
+
+// Look up the table with the given name from catalog. The database we 
look up the table from
+// is decided follow the steps:
+// 1. If the database part is defined in the table identifier, use 
that database name;
+// 2. Else If the defaultDatabase is defined, use the default database 
name(In this case, no
+//temporary objects can be used, and the default database is only 
used to look up a view);
+// 3. Else use the currentDb of the SessionCatalog.
+private def lookupTableFromCatalog(
+u: UnresolvedRelation,
+defaultDatabase: Option[String] = None): LogicalPlan = {
   try {
-catalog.lookupRelation(u.tableIdentifier, u.alias)
+val tableIdentWithDb = 
u.tableIdentifier.withDatabase(defaultDatabase)

[GitHub] spark pull request #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-10 Thread jiangxb1987
Github user jiangxb1987 commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95429866
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -510,32 +542,90 @@ class Analyzer(
* Replaces [[UnresolvedRelation]]s with concrete relations from the 
catalog.
*/
   object ResolveRelations extends Rule[LogicalPlan] {
-private def lookupTableFromCatalog(u: UnresolvedRelation): LogicalPlan 
= {
+
+// If the unresolved relation is running directly on files, we just 
return the original
+// UnresolvedRelation, the plan will get resolved later. Else we look 
up the table from catalog
+// and change the default database name if it is a view.
+// We usually look up a table from the default database if the table 
identifier has an empty
+// database part, for a view the default database should be the 
currentDb when the view was
+// created. When the case comes to resolving a nested view, the view 
may have different default
+// database with that the referenced view has, so we need to use the 
variable `defaultDatabase`
+// to track the current default database.
+// When the relation we resolve is a view, we fetch the 
view.desc(which is a CatalogTable), and
+// then set the value of `CatalogTable.viewDefaultDatabase` to the 
variable `defaultDatabase`,
+// we look up the relations that the view references using the default 
database.
+// For example:
+// |- view1 (defaultDatabase = db1)
+//   |- operator
+// |- table2 (defaultDatabase = db1)
+// |- view2 (defaultDatabase = db2)
+//|- view3 (defaultDatabase = db3)
+//   |- view4 (defaultDatabase = db4)
+// In this case, the view `view1` is a nested view, it directly 
references `table2`、`view2`
+// and `view4`, the view `view2` references `view3`. On resolving the 
table, we look up the
+// relations `table2`、`view2`、`view4` using the default database 
`db1`, and look up the
+// relation `view3` using the default database `db2`.
+//
+// Note this is compatible with the views defined by older versions of 
Spark(before 2.2), which
+// have empty defaultDatabase and all the relations in viewText have 
database part defined.
+def resolveRelation(
+plan: LogicalPlan,
+defaultDatabase: Option[String] = None): LogicalPlan = plan match {
+  case u: UnresolvedRelation if 
!isRunningDirectlyOnFiles(u.tableIdentifier) =>
+val defaultDatabase = AnalysisContext.get.defaultDatabase
+val relation = lookupTableFromCatalog(u, defaultDatabase)
+resolveRelation(relation, defaultDatabase)
+  // The view's child should be a logical plan parsed from the 
`desc.viewText`, the variable
+  // `viewText` should be defined, or else we throw an error on the 
generation of the View
+  // operator.
+  case view @ View(desc, _, child) if !child.resolved =>
+val nestedViewLevel = AnalysisContext.get.nestedViewLevel + 1
+val context = AnalysisContext(defaultDatabase = 
desc.viewDefaultDatabase,
+  nestedViewLevel = nestedViewLevel)
+// Resolve all the UnresolvedRelations and Views in the child.
+val newChild = AnalysisContext.withAnalysisContext(context) {
+  execute(child)
+}
+view.copy(child = newChild)
+  case p @ SubqueryAlias(_, view: View, _) =>
+val newChild = resolveRelation(view, defaultDatabase)
+p.copy(child = newChild)
+  case _ => plan
+}
+
+def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+  case i @ InsertIntoTable(u: UnresolvedRelation, parts, child, _, _) 
if child.resolved =>
+i.copy(table = EliminateSubqueryAliases(lookupTableFromCatalog(u)))
+  case u: UnresolvedRelation => resolveRelation(u)
+}
+
+// Look up the table with the given name from catalog. The database we 
look up the table from
+// is decided follow the steps:
+// 1. If the database part is defined in the table identifier, use 
that database name;
+// 2. Else If the defaultDatabase is defined, use the default database 
name(In this case, no
+//temporary objects can be used, and the default database is only 
used to look up a view);
+// 3. Else use the currentDb of the SessionCatalog.
+private def lookupTableFromCatalog(
+u: UnresolvedRelation,
+defaultDatabase: Option[String] = None): LogicalPlan = {
   try {
-catalog.lookupRelation(u.tableIdentifier, u.alias)
+val tableIdentWithDb = 
u.tableIdentifier.withDatabase(defaultDatabase)

[GitHub] spark pull request #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-10 Thread jiangxb1987
Github user jiangxb1987 commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95429665
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -510,32 +542,90 @@ class Analyzer(
* Replaces [[UnresolvedRelation]]s with concrete relations from the 
catalog.
*/
   object ResolveRelations extends Rule[LogicalPlan] {
-private def lookupTableFromCatalog(u: UnresolvedRelation): LogicalPlan 
= {
+
+// If the unresolved relation is running directly on files, we just 
return the original
+// UnresolvedRelation, the plan will get resolved later. Else we look 
up the table from catalog
+// and change the default database name if it is a view.
+// We usually look up a table from the default database if the table 
identifier has an empty
+// database part, for a view the default database should be the 
currentDb when the view was
+// created. When the case comes to resolving a nested view, the view 
may have different default
+// database with that the referenced view has, so we need to use the 
variable `defaultDatabase`
+// to track the current default database.
+// When the relation we resolve is a view, we fetch the 
view.desc(which is a CatalogTable), and
+// then set the value of `CatalogTable.viewDefaultDatabase` to the 
variable `defaultDatabase`,
+// we look up the relations that the view references using the default 
database.
+// For example:
+// |- view1 (defaultDatabase = db1)
+//   |- operator
+// |- table2 (defaultDatabase = db1)
+// |- view2 (defaultDatabase = db2)
+//|- view3 (defaultDatabase = db3)
+//   |- view4 (defaultDatabase = db4)
+// In this case, the view `view1` is a nested view, it directly 
references `table2`、`view2`
+// and `view4`, the view `view2` references `view3`. On resolving the 
table, we look up the
+// relations `table2`、`view2`、`view4` using the default database 
`db1`, and look up the
+// relation `view3` using the default database `db2`.
+//
+// Note this is compatible with the views defined by older versions of 
Spark(before 2.2), which
+// have empty defaultDatabase and all the relations in viewText have 
database part defined.
+def resolveRelation(
+plan: LogicalPlan,
+defaultDatabase: Option[String] = None): LogicalPlan = plan match {
+  case u: UnresolvedRelation if 
!isRunningDirectlyOnFiles(u.tableIdentifier) =>
+val defaultDatabase = AnalysisContext.get.defaultDatabase
+val relation = lookupTableFromCatalog(u, defaultDatabase)
+resolveRelation(relation, defaultDatabase)
+  // The view's child should be a logical plan parsed from the 
`desc.viewText`, the variable
+  // `viewText` should be defined, or else we throw an error on the 
generation of the View
+  // operator.
+  case view @ View(desc, _, child) if !child.resolved =>
+val nestedViewLevel = AnalysisContext.get.nestedViewLevel + 1
+val context = AnalysisContext(defaultDatabase = 
desc.viewDefaultDatabase,
+  nestedViewLevel = nestedViewLevel)
+// Resolve all the UnresolvedRelations and Views in the child.
+val newChild = AnalysisContext.withAnalysisContext(context) {
+  execute(child)
+}
+view.copy(child = newChild)
+  case p @ SubqueryAlias(_, view: View, _) =>
+val newChild = resolveRelation(view, defaultDatabase)
+p.copy(child = newChild)
+  case _ => plan
+}
+
+def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+  case i @ InsertIntoTable(u: UnresolvedRelation, parts, child, _, _) 
if child.resolved =>
+i.copy(table = EliminateSubqueryAliases(lookupTableFromCatalog(u)))
+  case u: UnresolvedRelation => resolveRelation(u)
+}
+
+// Look up the table with the given name from catalog. The database we 
look up the table from
+// is decided follow the steps:
+// 1. If the database part is defined in the table identifier, use 
that database name;
+// 2. Else If the defaultDatabase is defined, use the default database 
name(In this case, no
+//temporary objects can be used, and the default database is only 
used to look up a view);
+// 3. Else use the currentDb of the SessionCatalog.
+private def lookupTableFromCatalog(
+u: UnresolvedRelation,
+defaultDatabase: Option[String] = None): LogicalPlan = {
   try {
-catalog.lookupRelation(u.tableIdentifier, u.alias)
+val tableIdentWithDb = 
u.tableIdentifier.withDatabase(defaultDatabase)

[GitHub] spark pull request #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-10 Thread jiangxb1987
Github user jiangxb1987 commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95428960
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -510,32 +542,90 @@ class Analyzer(
* Replaces [[UnresolvedRelation]]s with concrete relations from the 
catalog.
*/
   object ResolveRelations extends Rule[LogicalPlan] {
-private def lookupTableFromCatalog(u: UnresolvedRelation): LogicalPlan 
= {
+
+// If the unresolved relation is running directly on files, we just 
return the original
+// UnresolvedRelation, the plan will get resolved later. Else we look 
up the table from catalog
+// and change the default database name if it is a view.
+// We usually look up a table from the default database if the table 
identifier has an empty
+// database part, for a view the default database should be the 
currentDb when the view was
+// created. When the case comes to resolving a nested view, the view 
may have different default
+// database with that the referenced view has, so we need to use the 
variable `defaultDatabase`
+// to track the current default database.
+// When the relation we resolve is a view, we fetch the 
view.desc(which is a CatalogTable), and
+// then set the value of `CatalogTable.viewDefaultDatabase` to the 
variable `defaultDatabase`,
+// we look up the relations that the view references using the default 
database.
+// For example:
+// |- view1 (defaultDatabase = db1)
+//   |- operator
+// |- table2 (defaultDatabase = db1)
+// |- view2 (defaultDatabase = db2)
+//|- view3 (defaultDatabase = db3)
+//   |- view4 (defaultDatabase = db4)
+// In this case, the view `view1` is a nested view, it directly 
references `table2`、`view2`
+// and `view4`, the view `view2` references `view3`. On resolving the 
table, we look up the
+// relations `table2`、`view2`、`view4` using the default database 
`db1`, and look up the
+// relation `view3` using the default database `db2`.
+//
+// Note this is compatible with the views defined by older versions of 
Spark(before 2.2), which
+// have empty defaultDatabase and all the relations in viewText have 
database part defined.
+def resolveRelation(
+plan: LogicalPlan,
+defaultDatabase: Option[String] = None): LogicalPlan = plan match {
+  case u: UnresolvedRelation if 
!isRunningDirectlyOnFiles(u.tableIdentifier) =>
+val defaultDatabase = AnalysisContext.get.defaultDatabase
+val relation = lookupTableFromCatalog(u, defaultDatabase)
+resolveRelation(relation, defaultDatabase)
--- End diff --

We have merged `resolveView` to `resolveRelation`.


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-10 Thread jiangxb1987
Github user jiangxb1987 commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95428827
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
@@ -619,7 +642,11 @@ private[spark] class HiveExternalCatalog(conf: 
SparkConf, hadoopConf: Configurat
 
 var table = inputTable
 
-if (table.tableType != VIEW) {
+if (table.tableType == VIEW) {
+  // Read view default database from table properties.
+  val viewDefaultDatabase = table.properties.get(VIEW_DEFAULT_DATABASE)
--- End diff --

Yup, in that case, `viewDefaultDatabase` is empty.


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-10 Thread jiangxb1987
Github user jiangxb1987 commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95428444
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala 
---
@@ -0,0 +1,90 @@
+/*
+ * 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.AnalysisException
+import org.apache.spark.sql.catalyst.CatalystConf
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, 
View}
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+ * This file defines analysis rules related to views.
+ */
+
+/**
+ * Make sure that a view's child plan produces the view's output 
attributes. We wrap the child
+ * with a Project and add an alias for each output attribute. The 
attributes are resolved by
+ * name. This should be only done after the resolution batch, because the 
view attributes are
+ * not stable during resolution.
+ */
+case class AliasViewChild(conf: CatalystConf) extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan 
resolveOperators {
+case v @ View(_, output, child) if child.resolved =>
+  val resolver = conf.resolver
+  val newOutput = child.output.map { attr =>
+val newAttr = findAttributeByName(attr.name, output, resolver)
+// Check the dataType of the output attributes, throw an 
AnalysisException if they don't
+// match up.
+checkDataType(attr, newAttr)
--- End diff --

When the schema of the referenced table changes, for example:
```
  test("correctly handle type casting between view output and child 
output") {
withTable("testTable") {
  withView("testView") {
spark.range(1, 
10).toDF("id1").write.format("json").saveAsTable("testTable")
sql("CREATE VIEW testView AS SELECT * FROM testTable")

// Allow casting from IntegerType to LongType
val df = (1 until 10).map(i => i).toDF("id1")

df.write.format("json").mode(SaveMode.Overwrite).saveAsTable("testTable")
checkAnswer(sql("SELECT * FROM testView ORDER BY id1"), (1 to 
9).map(i => Row(i)))

// Cann't cast from ArrayType to LongType, throw an 
AnalysisException.
val df2 = (1 until 10).map(i => Seq(i)).toDF("id1")

df2.write.format("json").mode(SaveMode.Overwrite).saveAsTable("testTable")
intercept[AnalysisException](sql("SELECT * FROM testView ORDER BY 
id1"))
  }
}
  }
```


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-10 Thread jiangxb1987
Github user jiangxb1987 commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95428074
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala 
---
@@ -0,0 +1,90 @@
+/*
+ * 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.AnalysisException
+import org.apache.spark.sql.catalyst.CatalystConf
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, 
View}
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+ * This file defines analysis rules related to views.
+ */
+
+/**
+ * Make sure that a view's child plan produces the view's output 
attributes. We wrap the child
+ * with a Project and add an alias for each output attribute. The 
attributes are resolved by
+ * name. This should be only done after the resolution batch, because the 
view attributes are
+ * not stable during resolution.
+ */
+case class AliasViewChild(conf: CatalystConf) extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan 
resolveOperators {
+case v @ View(_, output, child) if child.resolved =>
--- End diff --

We match columns by name because the order of the child's output may be not 
the same with that of view output.
Will update the code to handle the case when the output have extra columns.


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-10 Thread yhuai
Github user yhuai commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95427571
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -510,32 +542,90 @@ class Analyzer(
* Replaces [[UnresolvedRelation]]s with concrete relations from the 
catalog.
*/
   object ResolveRelations extends Rule[LogicalPlan] {
-private def lookupTableFromCatalog(u: UnresolvedRelation): LogicalPlan 
= {
+
+// If the unresolved relation is running directly on files, we just 
return the original
+// UnresolvedRelation, the plan will get resolved later. Else we look 
up the table from catalog
+// and change the default database name if it is a view.
+// We usually look up a table from the default database if the table 
identifier has an empty
+// database part, for a view the default database should be the 
currentDb when the view was
+// created. When the case comes to resolving a nested view, the view 
may have different default
+// database with that the referenced view has, so we need to use the 
variable `defaultDatabase`
+// to track the current default database.
+// When the relation we resolve is a view, we fetch the 
view.desc(which is a CatalogTable), and
+// then set the value of `CatalogTable.viewDefaultDatabase` to the 
variable `defaultDatabase`,
+// we look up the relations that the view references using the default 
database.
+// For example:
+// |- view1 (defaultDatabase = db1)
+//   |- operator
+// |- table2 (defaultDatabase = db1)
+// |- view2 (defaultDatabase = db2)
+//|- view3 (defaultDatabase = db3)
+//   |- view4 (defaultDatabase = db4)
+// In this case, the view `view1` is a nested view, it directly 
references `table2`、`view2`
+// and `view4`, the view `view2` references `view3`. On resolving the 
table, we look up the
+// relations `table2`、`view2`、`view4` using the default database 
`db1`, and look up the
+// relation `view3` using the default database `db2`.
+//
+// Note this is compatible with the views defined by older versions of 
Spark(before 2.2), which
+// have empty defaultDatabase and all the relations in viewText have 
database part defined.
+def resolveRelation(
+plan: LogicalPlan,
+defaultDatabase: Option[String] = None): LogicalPlan = plan match {
+  case u: UnresolvedRelation if 
!isRunningDirectlyOnFiles(u.tableIdentifier) =>
+val defaultDatabase = AnalysisContext.get.defaultDatabase
+val relation = lookupTableFromCatalog(u, defaultDatabase)
+resolveRelation(relation, defaultDatabase)
+  // The view's child should be a logical plan parsed from the 
`desc.viewText`, the variable
+  // `viewText` should be defined, or else we throw an error on the 
generation of the View
+  // operator.
+  case view @ View(desc, _, child) if !child.resolved =>
+val nestedViewLevel = AnalysisContext.get.nestedViewLevel + 1
+val context = AnalysisContext(defaultDatabase = 
desc.viewDefaultDatabase,
+  nestedViewLevel = nestedViewLevel)
+// Resolve all the UnresolvedRelations and Views in the child.
+val newChild = AnalysisContext.withAnalysisContext(context) {
+  execute(child)
+}
+view.copy(child = newChild)
+  case p @ SubqueryAlias(_, view: View, _) =>
+val newChild = resolveRelation(view, defaultDatabase)
+p.copy(child = newChild)
+  case _ => plan
+}
+
+def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+  case i @ InsertIntoTable(u: UnresolvedRelation, parts, child, _, _) 
if child.resolved =>
+i.copy(table = EliminateSubqueryAliases(lookupTableFromCatalog(u)))
+  case u: UnresolvedRelation => resolveRelation(u)
+}
+
+// Look up the table with the given name from catalog. The database we 
look up the table from
+// is decided follow the steps:
+// 1. If the database part is defined in the table identifier, use 
that database name;
+// 2. Else If the defaultDatabase is defined, use the default database 
name(In this case, no
+//temporary objects can be used, and the default database is only 
used to look up a view);
+// 3. Else use the currentDb of the SessionCatalog.
+private def lookupTableFromCatalog(
+u: UnresolvedRelation,
+defaultDatabase: Option[String] = None): LogicalPlan = {
   try {
-catalog.lookupRelation(u.tableIdentifier, u.alias)
+val tableIdentWithDb = 
u.tableIdentifier.withDatabase(defaultDatabase)
--- En

[GitHub] spark pull request #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-10 Thread jiangxb1987
Github user jiangxb1987 commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95427497
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -767,19 +857,19 @@ class Analyzer(
 }
   }
 
- /**
-  * In many dialects of SQL it is valid to use ordinal positions in 
order/sort by and group by
-  * clauses. This rule is to convert ordinal positions to the 
corresponding expressions in the
-  * select list. This support is introduced in Spark 2.0.
-  *
-  * - When the sort references or group by expressions are not integer but 
foldable expressions,
-  * just ignore them.
-  * - When spark.sql.orderByOrdinal/spark.sql.groupByOrdinal is set to 
false, ignore the position
-  * numbers too.
-  *
-  * Before the release of Spark 2.0, the literals in order/sort by and 
group by clauses
-  * have no effect on the results.
-  */
+  /**
+   * In many dialects of SQL it is valid to use ordinal positions in 
order/sort by and group by
+   * clauses. This rule is to convert ordinal positions to the 
corresponding expressions in the
+   * select list. This support is introduced in Spark 2.0.
+   *
+   * - When the sort references or group by expressions are not integer 
but foldable expressions,
+   * just ignore them.
+   * - When spark.sql.orderByOrdinal/spark.sql.groupByOrdinal is set to 
false, ignore the position
+   * numbers too.
+   *
+   * Before the release of Spark 2.0, the literals in order/sort by and 
group by clauses
+   * have no effect on the results.
+   */
--- End diff --

The code format is wrong, should add an extra space before each line.


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-10 Thread yhuai
Github user yhuai commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95427132
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -510,32 +542,90 @@ class Analyzer(
* Replaces [[UnresolvedRelation]]s with concrete relations from the 
catalog.
*/
   object ResolveRelations extends Rule[LogicalPlan] {
-private def lookupTableFromCatalog(u: UnresolvedRelation): LogicalPlan 
= {
+
+// If the unresolved relation is running directly on files, we just 
return the original
+// UnresolvedRelation, the plan will get resolved later. Else we look 
up the table from catalog
+// and change the default database name if it is a view.
+// We usually look up a table from the default database if the table 
identifier has an empty
+// database part, for a view the default database should be the 
currentDb when the view was
+// created. When the case comes to resolving a nested view, the view 
may have different default
+// database with that the referenced view has, so we need to use the 
variable `defaultDatabase`
+// to track the current default database.
+// When the relation we resolve is a view, we fetch the 
view.desc(which is a CatalogTable), and
+// then set the value of `CatalogTable.viewDefaultDatabase` to the 
variable `defaultDatabase`,
+// we look up the relations that the view references using the default 
database.
+// For example:
+// |- view1 (defaultDatabase = db1)
+//   |- operator
+// |- table2 (defaultDatabase = db1)
+// |- view2 (defaultDatabase = db2)
+//|- view3 (defaultDatabase = db3)
+//   |- view4 (defaultDatabase = db4)
+// In this case, the view `view1` is a nested view, it directly 
references `table2`、`view2`
+// and `view4`, the view `view2` references `view3`. On resolving the 
table, we look up the
+// relations `table2`、`view2`、`view4` using the default database 
`db1`, and look up the
+// relation `view3` using the default database `db2`.
+//
+// Note this is compatible with the views defined by older versions of 
Spark(before 2.2), which
+// have empty defaultDatabase and all the relations in viewText have 
database part defined.
+def resolveRelation(
+plan: LogicalPlan,
+defaultDatabase: Option[String] = None): LogicalPlan = plan match {
+  case u: UnresolvedRelation if 
!isRunningDirectlyOnFiles(u.tableIdentifier) =>
+val defaultDatabase = AnalysisContext.get.defaultDatabase
+val relation = lookupTableFromCatalog(u, defaultDatabase)
+resolveRelation(relation, defaultDatabase)
+  // The view's child should be a logical plan parsed from the 
`desc.viewText`, the variable
+  // `viewText` should be defined, or else we throw an error on the 
generation of the View
+  // operator.
+  case view @ View(desc, _, child) if !child.resolved =>
+val nestedViewLevel = AnalysisContext.get.nestedViewLevel + 1
+val context = AnalysisContext(defaultDatabase = 
desc.viewDefaultDatabase,
+  nestedViewLevel = nestedViewLevel)
+// Resolve all the UnresolvedRelations and Views in the child.
+val newChild = AnalysisContext.withAnalysisContext(context) {
+  execute(child)
+}
+view.copy(child = newChild)
+  case p @ SubqueryAlias(_, view: View, _) =>
+val newChild = resolveRelation(view, defaultDatabase)
+p.copy(child = newChild)
+  case _ => plan
+}
+
+def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+  case i @ InsertIntoTable(u: UnresolvedRelation, parts, child, _, _) 
if child.resolved =>
+i.copy(table = EliminateSubqueryAliases(lookupTableFromCatalog(u)))
+  case u: UnresolvedRelation => resolveRelation(u)
+}
+
+// Look up the table with the given name from catalog. The database we 
look up the table from
+// is decided follow the steps:
+// 1. If the database part is defined in the table identifier, use 
that database name;
+// 2. Else If the defaultDatabase is defined, use the default database 
name(In this case, no
+//temporary objects can be used, and the default database is only 
used to look up a view);
+// 3. Else use the currentDb of the SessionCatalog.
+private def lookupTableFromCatalog(
+u: UnresolvedRelation,
+defaultDatabase: Option[String] = None): LogicalPlan = {
   try {
-catalog.lookupRelation(u.tableIdentifier, u.alias)
+val tableIdentWithDb = 
u.tableIdentifier.withDatabase(defaultDatabase)
--- En

[GitHub] spark pull request #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-10 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95426816
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -510,32 +542,90 @@ class Analyzer(
* Replaces [[UnresolvedRelation]]s with concrete relations from the 
catalog.
*/
   object ResolveRelations extends Rule[LogicalPlan] {
-private def lookupTableFromCatalog(u: UnresolvedRelation): LogicalPlan 
= {
+
+// If the unresolved relation is running directly on files, we just 
return the original
+// UnresolvedRelation, the plan will get resolved later. Else we look 
up the table from catalog
+// and change the default database name if it is a view.
+// We usually look up a table from the default database if the table 
identifier has an empty
+// database part, for a view the default database should be the 
currentDb when the view was
+// created. When the case comes to resolving a nested view, the view 
may have different default
+// database with that the referenced view has, so we need to use the 
variable `defaultDatabase`
+// to track the current default database.
+// When the relation we resolve is a view, we fetch the 
view.desc(which is a CatalogTable), and
+// then set the value of `CatalogTable.viewDefaultDatabase` to the 
variable `defaultDatabase`,
+// we look up the relations that the view references using the default 
database.
+// For example:
+// |- view1 (defaultDatabase = db1)
+//   |- operator
+// |- table2 (defaultDatabase = db1)
+// |- view2 (defaultDatabase = db2)
+//|- view3 (defaultDatabase = db3)
+//   |- view4 (defaultDatabase = db4)
+// In this case, the view `view1` is a nested view, it directly 
references `table2`、`view2`
+// and `view4`, the view `view2` references `view3`. On resolving the 
table, we look up the
+// relations `table2`、`view2`、`view4` using the default database 
`db1`, and look up the
+// relation `view3` using the default database `db2`.
+//
+// Note this is compatible with the views defined by older versions of 
Spark(before 2.2), which
+// have empty defaultDatabase and all the relations in viewText have 
database part defined.
+def resolveRelation(
+plan: LogicalPlan,
+defaultDatabase: Option[String] = None): LogicalPlan = plan match {
+  case u: UnresolvedRelation if 
!isRunningDirectlyOnFiles(u.tableIdentifier) =>
+val defaultDatabase = AnalysisContext.get.defaultDatabase
+val relation = lookupTableFromCatalog(u, defaultDatabase)
+resolveRelation(relation, defaultDatabase)
+  // The view's child should be a logical plan parsed from the 
`desc.viewText`, the variable
+  // `viewText` should be defined, or else we throw an error on the 
generation of the View
+  // operator.
+  case view @ View(desc, _, child) if !child.resolved =>
+val nestedViewLevel = AnalysisContext.get.nestedViewLevel + 1
+val context = AnalysisContext(defaultDatabase = 
desc.viewDefaultDatabase,
+  nestedViewLevel = nestedViewLevel)
+// Resolve all the UnresolvedRelations and Views in the child.
+val newChild = AnalysisContext.withAnalysisContext(context) {
+  execute(child)
+}
--- End diff --

Conceptually, `nestedViewLevel` should not be determined by the caller of 
`withAnalysisContext`. This change can improve the code readability. It reduces 
the code sizes. 




---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-10 Thread yhuai
Github user yhuai commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95426835
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -510,32 +542,90 @@ class Analyzer(
* Replaces [[UnresolvedRelation]]s with concrete relations from the 
catalog.
*/
   object ResolveRelations extends Rule[LogicalPlan] {
-private def lookupTableFromCatalog(u: UnresolvedRelation): LogicalPlan 
= {
+
+// If the unresolved relation is running directly on files, we just 
return the original
+// UnresolvedRelation, the plan will get resolved later. Else we look 
up the table from catalog
+// and change the default database name if it is a view.
+// We usually look up a table from the default database if the table 
identifier has an empty
+// database part, for a view the default database should be the 
currentDb when the view was
+// created. When the case comes to resolving a nested view, the view 
may have different default
+// database with that the referenced view has, so we need to use the 
variable `defaultDatabase`
+// to track the current default database.
+// When the relation we resolve is a view, we fetch the 
view.desc(which is a CatalogTable), and
+// then set the value of `CatalogTable.viewDefaultDatabase` to the 
variable `defaultDatabase`,
+// we look up the relations that the view references using the default 
database.
+// For example:
+// |- view1 (defaultDatabase = db1)
+//   |- operator
+// |- table2 (defaultDatabase = db1)
+// |- view2 (defaultDatabase = db2)
+//|- view3 (defaultDatabase = db3)
+//   |- view4 (defaultDatabase = db4)
+// In this case, the view `view1` is a nested view, it directly 
references `table2`、`view2`
+// and `view4`, the view `view2` references `view3`. On resolving the 
table, we look up the
+// relations `table2`、`view2`、`view4` using the default database 
`db1`, and look up the
+// relation `view3` using the default database `db2`.
+//
+// Note this is compatible with the views defined by older versions of 
Spark(before 2.2), which
+// have empty defaultDatabase and all the relations in viewText have 
database part defined.
+def resolveRelation(
+plan: LogicalPlan,
+defaultDatabase: Option[String] = None): LogicalPlan = plan match {
+  case u: UnresolvedRelation if 
!isRunningDirectlyOnFiles(u.tableIdentifier) =>
+val defaultDatabase = AnalysisContext.get.defaultDatabase
+val relation = lookupTableFromCatalog(u, defaultDatabase)
+resolveRelation(relation, defaultDatabase)
--- End diff --

We are calling resolveRelation again because `relation` may be a view?


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-10 Thread yhuai
Github user yhuai commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95421315
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -767,19 +857,19 @@ class Analyzer(
 }
   }
 
- /**
-  * In many dialects of SQL it is valid to use ordinal positions in 
order/sort by and group by
-  * clauses. This rule is to convert ordinal positions to the 
corresponding expressions in the
-  * select list. This support is introduced in Spark 2.0.
-  *
-  * - When the sort references or group by expressions are not integer but 
foldable expressions,
-  * just ignore them.
-  * - When spark.sql.orderByOrdinal/spark.sql.groupByOrdinal is set to 
false, ignore the position
-  * numbers too.
-  *
-  * Before the release of Spark 2.0, the literals in order/sort by and 
group by clauses
-  * have no effect on the results.
-  */
+  /**
+   * In many dialects of SQL it is valid to use ordinal positions in 
order/sort by and group by
+   * clauses. This rule is to convert ordinal positions to the 
corresponding expressions in the
+   * select list. This support is introduced in Spark 2.0.
+   *
+   * - When the sort references or group by expressions are not integer 
but foldable expressions,
+   * just ignore them.
+   * - When spark.sql.orderByOrdinal/spark.sql.groupByOrdinal is set to 
false, ignore the position
+   * numbers too.
+   *
+   * Before the release of Spark 2.0, the literals in order/sort by and 
group by clauses
+   * have no effect on the results.
+   */
--- End diff --

what's going on with these lines?


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

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



[GitHub] spark pull request #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-10 Thread yhuai
Github user yhuai commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95425724
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
@@ -619,7 +642,11 @@ private[spark] class HiveExternalCatalog(conf: 
SparkConf, hadoopConf: Configurat
 
 var table = inputTable
 
-if (table.tableType != VIEW) {
+if (table.tableType == VIEW) {
+  // Read view default database from table properties.
+  val viewDefaultDatabase = table.properties.get(VIEW_DEFAULT_DATABASE)
--- End diff --

We may not have VIEW_DEFAULT_DATABASE in the properties for views that are 
defined in older versions of spark, right?


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

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



[GitHub] spark pull request #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-10 Thread yhuai
Github user yhuai commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95422202
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala 
---
@@ -0,0 +1,90 @@
+/*
+ * 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.AnalysisException
+import org.apache.spark.sql.catalyst.CatalystConf
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, 
View}
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+ * This file defines analysis rules related to views.
+ */
+
+/**
+ * Make sure that a view's child plan produces the view's output 
attributes. We wrap the child
+ * with a Project and add an alias for each output attribute. The 
attributes are resolved by
+ * name. This should be only done after the resolution batch, because the 
view attributes are
+ * not stable during resolution.
+ */
+case class AliasViewChild(conf: CatalystConf) extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan 
resolveOperators {
+case v @ View(_, output, child) if child.resolved =>
+  val resolver = conf.resolver
+  val newOutput = child.output.map { attr =>
+val newAttr = findAttributeByName(attr.name, output, resolver)
+// Check the dataType of the output attributes, throw an 
AnalysisException if they don't
+// match up.
+checkDataType(attr, newAttr)
--- End diff --

When will newAttr and attr have different data types?


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-10 Thread yhuai
Github user yhuai commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95424456
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala 
---
@@ -0,0 +1,90 @@
+/*
+ * 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.AnalysisException
+import org.apache.spark.sql.catalyst.CatalystConf
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, 
View}
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+ * This file defines analysis rules related to views.
+ */
+
+/**
+ * Make sure that a view's child plan produces the view's output 
attributes. We wrap the child
+ * with a Project and add an alias for each output attribute. The 
attributes are resolved by
+ * name. This should be only done after the resolution batch, because the 
view attributes are
+ * not stable during resolution.
+ */
+case class AliasViewChild(conf: CatalystConf) extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan 
resolveOperators {
+case v @ View(_, output, child) if child.resolved =>
+  val resolver = conf.resolver
+  val newOutput = child.output.map { attr =>
+val newAttr = findAttributeByName(attr.name, output, resolver)
+// Check the dataType of the output attributes, throw an 
AnalysisException if they don't
+// match up.
+checkDataType(attr, newAttr)
+Alias(attr, attr.name)(exprId = newAttr.exprId, qualifier = 
newAttr.qualifier,
+  explicitMetadata = Some(newAttr.metadata))
+  }
+  v.copy(child = Project(newOutput, child))
+  }
+
+  /**
+   * Find the attribute that has the expected attribute name from an 
attribute list, the names
+   * are compared using conf.resolver.
+   * If the expected attribute is not found, throw an AnalysisException.
+   */
+  private def findAttributeByName(
+  name: String,
+  attrs: Seq[Attribute],
+  resolver: Resolver): Attribute = {
+attrs.collectFirst {
+  case attr if resolver(attr.name, name) => attr
+}.getOrElse(throw new AnalysisException(
+  s"Attribute with name '$name' is not found in " +
+s"'${attrs.map(_.name).mkString("(", ",", ")")}'"))
+  }
+
+  /**
+   * Check whether the dataType of `attr` could be casted to that of 
`other`, throw an
+   * AnalysisException if the both attributes don't match up.
+   */
+  private def checkDataType(attr: Attribute, other: Attribute): Unit = {
--- End diff --

Since this function is used once, I'd inline it.


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

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



[GitHub] spark pull request #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-10 Thread yhuai
Github user yhuai commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95424381
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala 
---
@@ -0,0 +1,90 @@
+/*
+ * 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.AnalysisException
+import org.apache.spark.sql.catalyst.CatalystConf
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, 
View}
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+ * This file defines analysis rules related to views.
+ */
+
+/**
+ * Make sure that a view's child plan produces the view's output 
attributes. We wrap the child
+ * with a Project and add an alias for each output attribute. The 
attributes are resolved by
+ * name. This should be only done after the resolution batch, because the 
view attributes are
+ * not stable during resolution.
+ */
+case class AliasViewChild(conf: CatalystConf) extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan 
resolveOperators {
+case v @ View(_, output, child) if child.resolved =>
--- End diff --

Is there any chance that output and child have different sizes? Also, looks 
like we are trying to match columns by name, can you explain the reason? Why we 
are not matching columns by the position? 


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-10 Thread yhuai
Github user yhuai commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95424897
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.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.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.CatalystConf
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, 
View}
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+ * This file defines analysis rules related to views.
+ */
+
+/**
+ * Make sure that a view's child plan produces the view's output 
attributes. We wrap the child
+ * with a Project and add an alias for each output attribute. The 
attributes are resolved by
+ * name. This should be only done after the resolution batch, because the 
view attributes are
+ * not stable during resolution.
+ */
+case class AliasViewChild(conf: CatalystConf) extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan 
resolveOperators {
+case v @ View(_, output, child) if child.resolved =>
+  val resolver = conf.resolver
+  val newOutput = child.output.map { attr =>
+val newAttr = findAttributeByName(attr.name, output, resolver)
+// Check the dataType of the output attributes, throw an 
AnalysisException if they don't
+// match up.
+checkDataType(attr, newAttr)
+Alias(attr, attr.name)(exprId = newAttr.exprId, qualifier = 
newAttr.qualifier,
+  explicitMetadata = Some(newAttr.metadata))
+  }
+  v.copy(child = Project(newOutput, child))
+  }
+
+  /**
+   * Find the attribute that has the expected attribute name from an 
attribute list, the names
+   * are compared using conf.resolver.
+   * If the expected attribute is not found, throw an AnalysisException.
+   */
+  private def findAttributeByName(
+  name: String,
+  attrs: Seq[Attribute],
+  resolver: Resolver): Attribute = {
+attrs.collectFirst {
+  case attr if resolver(attr.name, name) => attr
+}.getOrElse(throw new AnalysisException(
+  s"Attribute with name '$name' is not found in " +
+s"'${attrs.map(_.name).mkString("(", ",", ")")}'"))
+  }
+
+  /**
+   * Check whether the dataType of `attr` could be casted to that of 
`other`, throw an
+   * AnalysisException if the both attributes don't match up.
+   */
+  private def checkDataType(attr: Attribute, other: Attribute): Unit = {
+if (!Cast.canCast(attr.dataType, other.dataType)) {
--- End diff --

btw, I do not say where we actually do the cast.


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

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



[GitHub] spark pull request #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-10 Thread jiangxb1987
Github user jiangxb1987 commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95422458
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
@@ -125,11 +132,16 @@ private[hive] class 
HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
   // Otherwise, wrap the table with a Subquery using the table name.
   alias.map(a => SubqueryAlias(a, qualifiedTable, 
None)).getOrElse(qualifiedTable)
 } else if (table.tableType == CatalogTableType.VIEW) {
+  val tableIdentifier = table.identifier
   val viewText = table.viewText.getOrElse(sys.error("Invalid view 
without text."))
-  SubqueryAlias(
-alias.getOrElse(table.identifier.table),
-sparkSession.sessionState.sqlParser.parsePlan(viewText),
-Option(table.identifier))
+  // The relation is a view, so we wrap the relation by:
+  // 1. Add a [[View]] operator over the relation to keep track of the 
view desc;
+  // 2. Wrap the logical plan in a [[SubqueryAlias]] which tracks the 
name of the view.
+  val child = View(
+desc = table,
+output = table.schema.toAttributes,
+child = sparkSession.sessionState.sqlParser.parsePlan(viewText))
--- End diff --

It may looks a little over-engineering for now, but that enables us to 
decouple planning of query from the planning of the view, which allows us to 
cache resolved views in the future, and decoupling also allows you to deal with 
some forms of schema evolution (moving column order or columns being added to 
underlying data).


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-10 Thread jiangxb1987
Github user jiangxb1987 commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95421031
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -510,32 +542,90 @@ class Analyzer(
* Replaces [[UnresolvedRelation]]s with concrete relations from the 
catalog.
*/
   object ResolveRelations extends Rule[LogicalPlan] {
-private def lookupTableFromCatalog(u: UnresolvedRelation): LogicalPlan 
= {
+
+// If the unresolved relation is running directly on files, we just 
return the original
+// UnresolvedRelation, the plan will get resolved later. Else we look 
up the table from catalog
+// and change the default database name if it is a view.
+// We usually look up a table from the default database if the table 
identifier has an empty
+// database part, for a view the default database should be the 
currentDb when the view was
+// created. When the case comes to resolving a nested view, the view 
may have different default
+// database with that the referenced view has, so we need to use the 
variable `defaultDatabase`
+// to track the current default database.
+// When the relation we resolve is a view, we fetch the 
view.desc(which is a CatalogTable), and
+// then set the value of `CatalogTable.viewDefaultDatabase` to the 
variable `defaultDatabase`,
+// we look up the relations that the view references using the default 
database.
+// For example:
+// |- view1 (defaultDatabase = db1)
+//   |- operator
+// |- table2 (defaultDatabase = db1)
+// |- view2 (defaultDatabase = db2)
+//|- view3 (defaultDatabase = db3)
+//   |- view4 (defaultDatabase = db4)
+// In this case, the view `view1` is a nested view, it directly 
references `table2`、`view2`
+// and `view4`, the view `view2` references `view3`. On resolving the 
table, we look up the
+// relations `table2`、`view2`、`view4` using the default database 
`db1`, and look up the
+// relation `view3` using the default database `db2`.
+//
+// Note this is compatible with the views defined by older versions of 
Spark(before 2.2), which
+// have empty defaultDatabase and all the relations in viewText have 
database part defined.
+def resolveRelation(
+plan: LogicalPlan,
+defaultDatabase: Option[String] = None): LogicalPlan = plan match {
+  case u: UnresolvedRelation if 
!isRunningDirectlyOnFiles(u.tableIdentifier) =>
+val defaultDatabase = AnalysisContext.get.defaultDatabase
+val relation = lookupTableFromCatalog(u, defaultDatabase)
+resolveRelation(relation, defaultDatabase)
+  // The view's child should be a logical plan parsed from the 
`desc.viewText`, the variable
+  // `viewText` should be defined, or else we throw an error on the 
generation of the View
+  // operator.
+  case view @ View(desc, _, child) if !child.resolved =>
+val nestedViewLevel = AnalysisContext.get.nestedViewLevel + 1
+val context = AnalysisContext(defaultDatabase = 
desc.viewDefaultDatabase,
+  nestedViewLevel = nestedViewLevel)
+// Resolve all the UnresolvedRelations and Views in the child.
+val newChild = AnalysisContext.withAnalysisContext(context) {
+  execute(child)
+}
+view.copy(child = newChild)
+  case p @ SubqueryAlias(_, view: View, _) =>
+val newChild = resolveRelation(view, defaultDatabase)
+p.copy(child = newChild)
+  case _ => plan
+}
+
+def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+  case i @ InsertIntoTable(u: UnresolvedRelation, parts, child, _, _) 
if child.resolved =>
+i.copy(table = EliminateSubqueryAliases(lookupTableFromCatalog(u)))
+  case u: UnresolvedRelation => resolveRelation(u)
+}
+
+// Look up the table with the given name from catalog. The database we 
look up the table from
+// is decided follow the steps:
+// 1. If the database part is defined in the table identifier, use 
that database name;
+// 2. Else If the defaultDatabase is defined, use the default database 
name(In this case, no
+//temporary objects can be used, and the default database is only 
used to look up a view);
+// 3. Else use the currentDb of the SessionCatalog.
+private def lookupTableFromCatalog(
+u: UnresolvedRelation,
+defaultDatabase: Option[String] = None): LogicalPlan = {
   try {
-catalog.lookupRelation(u.tableIdentifier, u.alias)
+val tableIdentWithDb = 
u.tableIdentifier.withDatabase(defaultDatabase)

[GitHub] spark pull request #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-10 Thread jiangxb1987
Github user jiangxb1987 commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95420669
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -510,32 +542,90 @@ class Analyzer(
* Replaces [[UnresolvedRelation]]s with concrete relations from the 
catalog.
*/
   object ResolveRelations extends Rule[LogicalPlan] {
-private def lookupTableFromCatalog(u: UnresolvedRelation): LogicalPlan 
= {
+
+// If the unresolved relation is running directly on files, we just 
return the original
+// UnresolvedRelation, the plan will get resolved later. Else we look 
up the table from catalog
+// and change the default database name if it is a view.
+// We usually look up a table from the default database if the table 
identifier has an empty
+// database part, for a view the default database should be the 
currentDb when the view was
+// created. When the case comes to resolving a nested view, the view 
may have different default
+// database with that the referenced view has, so we need to use the 
variable `defaultDatabase`
+// to track the current default database.
+// When the relation we resolve is a view, we fetch the 
view.desc(which is a CatalogTable), and
+// then set the value of `CatalogTable.viewDefaultDatabase` to the 
variable `defaultDatabase`,
+// we look up the relations that the view references using the default 
database.
+// For example:
+// |- view1 (defaultDatabase = db1)
+//   |- operator
+// |- table2 (defaultDatabase = db1)
+// |- view2 (defaultDatabase = db2)
+//|- view3 (defaultDatabase = db3)
+//   |- view4 (defaultDatabase = db4)
+// In this case, the view `view1` is a nested view, it directly 
references `table2`、`view2`
+// and `view4`, the view `view2` references `view3`. On resolving the 
table, we look up the
+// relations `table2`、`view2`、`view4` using the default database 
`db1`, and look up the
+// relation `view3` using the default database `db2`.
+//
+// Note this is compatible with the views defined by older versions of 
Spark(before 2.2), which
+// have empty defaultDatabase and all the relations in viewText have 
database part defined.
+def resolveRelation(
+plan: LogicalPlan,
+defaultDatabase: Option[String] = None): LogicalPlan = plan match {
+  case u: UnresolvedRelation if 
!isRunningDirectlyOnFiles(u.tableIdentifier) =>
+val defaultDatabase = AnalysisContext.get.defaultDatabase
+val relation = lookupTableFromCatalog(u, defaultDatabase)
+resolveRelation(relation, defaultDatabase)
+  // The view's child should be a logical plan parsed from the 
`desc.viewText`, the variable
+  // `viewText` should be defined, or else we throw an error on the 
generation of the View
+  // operator.
+  case view @ View(desc, _, child) if !child.resolved =>
+val nestedViewLevel = AnalysisContext.get.nestedViewLevel + 1
+val context = AnalysisContext(defaultDatabase = 
desc.viewDefaultDatabase,
+  nestedViewLevel = nestedViewLevel)
+// Resolve all the UnresolvedRelations and Views in the child.
+val newChild = AnalysisContext.withAnalysisContext(context) {
+  execute(child)
+}
+view.copy(child = newChild)
+  case p @ SubqueryAlias(_, view: View, _) =>
--- End diff --

That's what we get from `lookupTableFromCatalog`, we have to match the case 
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-10 Thread jiangxb1987
Github user jiangxb1987 commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95420369
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.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.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.CatalystConf
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, 
View}
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+ * This file defines analysis rules related to views.
+ */
+
+/**
+ * Make sure that a view's child plan produces the view's output 
attributes. We wrap the child
+ * with a Project and add an alias for each output attribute. The 
attributes are resolved by
+ * name. This should be only done after the resolution batch, because the 
view attributes are
+ * not stable during resolution.
+ */
+case class AliasViewChild(conf: CatalystConf) extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan 
resolveOperators {
+case v @ View(_, output, child) if child.resolved =>
+  val resolver = conf.resolver
+  val newOutput = child.output.map { attr =>
+val newAttr = findAttributeByName(attr.name, output, resolver)
+// Check the dataType of the output attributes, throw an 
AnalysisException if they don't
+// match up.
+checkDataType(attr, newAttr)
+Alias(attr, attr.name)(exprId = newAttr.exprId, qualifier = 
newAttr.qualifier,
+  explicitMetadata = Some(newAttr.metadata))
+  }
+  v.copy(child = Project(newOutput, child))
+  }
+
+  /**
+   * Find the attribute that has the expected attribute name from an 
attribute list, the names
+   * are compared using conf.resolver.
+   * If the expected attribute is not found, throw an AnalysisException.
+   */
+  private def findAttributeByName(
+  name: String,
+  attrs: Seq[Attribute],
+  resolver: Resolver): Attribute = {
+attrs.collectFirst {
+  case attr if resolver(attr.name, name) => attr
+}.getOrElse(throw new AnalysisException(
+  s"Attribute with name '$name' is not found in " +
+s"'${attrs.map(_.name).mkString("(", ",", ")")}'"))
+  }
+
+  /**
+   * Check whether the dataType of `attr` could be casted to that of 
`other`, throw an
+   * AnalysisException if the both attributes don't match up.
+   */
+  private def checkDataType(attr: Attribute, other: Attribute): Unit = {
+if (!Cast.canCast(attr.dataType, other.dataType)) {
--- End diff --

We should add `Cast` to the child attribute, because it may have different 
dataType with the view output attribute. Will also add test case for this.


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

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



[GitHub] spark pull request #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-10 Thread jiangxb1987
Github user jiangxb1987 commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95420106
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -510,32 +542,90 @@ class Analyzer(
* Replaces [[UnresolvedRelation]]s with concrete relations from the 
catalog.
*/
   object ResolveRelations extends Rule[LogicalPlan] {
-private def lookupTableFromCatalog(u: UnresolvedRelation): LogicalPlan 
= {
+
+// If the unresolved relation is running directly on files, we just 
return the original
+// UnresolvedRelation, the plan will get resolved later. Else we look 
up the table from catalog
+// and change the default database name if it is a view.
+// We usually look up a table from the default database if the table 
identifier has an empty
+// database part, for a view the default database should be the 
currentDb when the view was
+// created. When the case comes to resolving a nested view, the view 
may have different default
+// database with that the referenced view has, so we need to use the 
variable `defaultDatabase`
+// to track the current default database.
+// When the relation we resolve is a view, we fetch the 
view.desc(which is a CatalogTable), and
+// then set the value of `CatalogTable.viewDefaultDatabase` to the 
variable `defaultDatabase`,
+// we look up the relations that the view references using the default 
database.
+// For example:
+// |- view1 (defaultDatabase = db1)
+//   |- operator
+// |- table2 (defaultDatabase = db1)
+// |- view2 (defaultDatabase = db2)
+//|- view3 (defaultDatabase = db3)
+//   |- view4 (defaultDatabase = db4)
+// In this case, the view `view1` is a nested view, it directly 
references `table2`、`view2`
+// and `view4`, the view `view2` references `view3`. On resolving the 
table, we look up the
+// relations `table2`、`view2`、`view4` using the default database 
`db1`, and look up the
+// relation `view3` using the default database `db2`.
+//
+// Note this is compatible with the views defined by older versions of 
Spark(before 2.2), which
+// have empty defaultDatabase and all the relations in viewText have 
database part defined.
+def resolveRelation(
+plan: LogicalPlan,
+defaultDatabase: Option[String] = None): LogicalPlan = plan match {
+  case u: UnresolvedRelation if 
!isRunningDirectlyOnFiles(u.tableIdentifier) =>
+val defaultDatabase = AnalysisContext.get.defaultDatabase
+val relation = lookupTableFromCatalog(u, defaultDatabase)
+resolveRelation(relation, defaultDatabase)
+  // The view's child should be a logical plan parsed from the 
`desc.viewText`, the variable
+  // `viewText` should be defined, or else we throw an error on the 
generation of the View
+  // operator.
+  case view @ View(desc, _, child) if !child.resolved =>
+val nestedViewLevel = AnalysisContext.get.nestedViewLevel + 1
+val context = AnalysisContext(defaultDatabase = 
desc.viewDefaultDatabase,
+  nestedViewLevel = nestedViewLevel)
+// Resolve all the UnresolvedRelations and Views in the child.
+val newChild = AnalysisContext.withAnalysisContext(context) {
+  execute(child)
+}
--- End diff --

Could you expand on what we gain from the change? Seems we're just moving 
the logic from `ResolveRelations` to `AnalysisContext`.


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-10 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95314984
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
@@ -111,6 +111,13 @@ private[hive] class HiveMetastoreCatalog(sparkSession: 
SparkSession) extends Log
 new Path(new Path(dbLocation), tblName).toString
   }
 
+  /**
+   * Return a [[LogicalPlan]] that represents the given table or view from 
Hive metastore.
--- End diff --

Nit: `Return` -> `Returns`


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-10 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95314839
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
@@ -606,11 +625,15 @@ private[spark] class HiveExternalCatalog(conf: 
SparkConf, hadoopConf: Configurat
   }
 
   /**
-   * Restores table metadata from the table properties if it's a datasouce 
table. This method is
-   * kind of a opposite version of [[createTable]].
+   * Restores table metadata from the table properties if it's a 
datasource table or a view. This
+   * method is kind of a opposite version of [[createTable]].
+   *
+   * For a datasource table, it reads table schema, provider, partition 
column names and bucket
--- End diff --

This is not limited to the data source table.


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-10 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95314477
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
@@ -358,6 +362,17 @@ private[spark] class HiveExternalCatalog(conf: 
SparkConf, hadoopConf: Configurat
   }
 
   /**
+   * This method puts view default database into a map, which can be used 
as table properties
+   * later. The view default database is used in view resolution, and that 
is not supported by
+   * Hive metastore.
+   */
+  private def viewParamToTableProps(table: CatalogTable): 
mutable.Map[String, String] = {
--- End diff --

Why using `mutable.Map[String, String]`? At most one map, right?


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

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



[GitHub] spark pull request #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-10 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95314243
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
@@ -358,6 +362,17 @@ private[spark] class HiveExternalCatalog(conf: 
SparkConf, hadoopConf: Configurat
   }
 
   /**
+   * This method puts view default database into a map, which can be used 
as table properties
+   * later. The view default database is used in view resolution, and that 
is not supported by
+   * Hive metastore.
+   */
+  private def viewParamToTableProps(table: CatalogTable): 
mutable.Map[String, String] = {
+val properties = new mutable.HashMap[String, String]
+table.viewDefaultDatabase.foreach {t => 
properties.put(VIEW_DEFAULT_DATABASE, t)}
--- End diff --

Nit: Style issue. `foreach(properties.put(VIEW_DEFAULT_DATABASE, _))`


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-09 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95313666
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/catalyst/SQLBuilder.scala ---
@@ -219,6 +219,9 @@ class SQLBuilder private (
 case OneRowRelation =>
   ""
 
+case p: View =>
--- End diff --

Maybe add a TODO comment above this line. 


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-09 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95310611
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala 
---
@@ -0,0 +1,90 @@
+/*
+ * 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.AnalysisException
+import org.apache.spark.sql.catalyst.CatalystConf
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, 
View}
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+ * This file defines analysis rules related to views.
+ */
+
+/**
+ * Make sure that a view's child plan produces the view's output 
attributes. We wrap the child
+ * with a Project and add an alias for each output attribute. The 
attributes are resolved by
+ * name. This should be only done after the resolution batch, because the 
view attributes are
+ * not stable during resolution.
+ */
+case class AliasViewChild(conf: CatalystConf) extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan 
resolveOperators {
+case v @ View(_, output, child) if child.resolved =>
+  val resolver = conf.resolver
+  val newOutput = child.output.map { attr =>
+val newAttr = findAttributeByName(attr.name, output, resolver)
+// Check the dataType of the output attributes, throw an 
AnalysisException if they don't
+// match up.
+checkDataType(attr, newAttr)
+Alias(attr, attr.name)(exprId = newAttr.exprId, qualifier = 
newAttr.qualifier,
+  explicitMetadata = Some(newAttr.metadata))
+  }
+  v.copy(child = Project(newOutput, child))
+  }
+
+  /**
+   * Find the attribute that has the expected attribute name from an 
attribute list, the names
+   * are compared using conf.resolver.
+   * If the expected attribute is not found, throw an AnalysisException.
+   */
+  private def findAttributeByName(
+  name: String,
+  attrs: Seq[Attribute],
+  resolver: Resolver): Attribute = {
+attrs.collectFirst {
+  case attr if resolver(attr.name, name) => attr
+}.getOrElse(throw new AnalysisException(
+  s"Attribute with name '$name' is not found in " +
+s"'${attrs.map(_.name).mkString("(", ",", ")")}'"))
+  }
+
+  /**
+   * Check whether the dataType of `attr` could be casted to that of 
`other`, throw an
+   * AnalysisException if the both attributes don't match up.
+   */
+  private def checkDataType(attr: Attribute, other: Attribute): Unit = {
+if (!Cast.canCast(attr.dataType, other.dataType)) {
+  throw new AnalysisException(
+s"The dataType of attribute '$other' is '${other.dataType}', which 
can't be casted to " +
+  s"that of '$attr', expected '${attr.dataType}'.")
+}
+  }
+}
+
+/**
+ * Removes [[View]] operators from the plan. The operator is respected 
till the end of analysis
+ * stage because we want to see which part of a analyzed logical plan is 
generated from a view.
+ */
+object EliminateView extends Rule[LogicalPlan] {
+  def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
+// The child should have the same output attributes with the View 
operator, so we simply
--- End diff --

: ) I did not see your 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: revie

[GitHub] spark pull request #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-09 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95310499
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala 
---
@@ -0,0 +1,90 @@
+/*
+ * 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.AnalysisException
+import org.apache.spark.sql.catalyst.CatalystConf
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, 
View}
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+ * This file defines analysis rules related to views.
+ */
+
+/**
+ * Make sure that a view's child plan produces the view's output 
attributes. We wrap the child
+ * with a Project and add an alias for each output attribute. The 
attributes are resolved by
+ * name. This should be only done after the resolution batch, because the 
view attributes are
+ * not stable during resolution.
+ */
+case class AliasViewChild(conf: CatalystConf) extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan 
resolveOperators {
+case v @ View(_, output, child) if child.resolved =>
+  val resolver = conf.resolver
+  val newOutput = child.output.map { attr =>
+val newAttr = findAttributeByName(attr.name, output, resolver)
+// Check the dataType of the output attributes, throw an 
AnalysisException if they don't
+// match up.
+checkDataType(attr, newAttr)
+Alias(attr, attr.name)(exprId = newAttr.exprId, qualifier = 
newAttr.qualifier,
+  explicitMetadata = Some(newAttr.metadata))
+  }
+  v.copy(child = Project(newOutput, child))
+  }
+
+  /**
+   * Find the attribute that has the expected attribute name from an 
attribute list, the names
+   * are compared using conf.resolver.
+   * If the expected attribute is not found, throw an AnalysisException.
+   */
+  private def findAttributeByName(
+  name: String,
+  attrs: Seq[Attribute],
+  resolver: Resolver): Attribute = {
+attrs.collectFirst {
+  case attr if resolver(attr.name, name) => attr
+}.getOrElse(throw new AnalysisException(
+  s"Attribute with name '$name' is not found in " +
+s"'${attrs.map(_.name).mkString("(", ",", ")")}'"))
+  }
+
+  /**
+   * Check whether the dataType of `attr` could be casted to that of 
`other`, throw an
+   * AnalysisException if the both attributes don't match up.
+   */
+  private def checkDataType(attr: Attribute, other: Attribute): Unit = {
+if (!Cast.canCast(attr.dataType, other.dataType)) {
+  throw new AnalysisException(
+s"The dataType of attribute '$other' is '${other.dataType}', which 
can't be casted to " +
+  s"that of '$attr', expected '${attr.dataType}'.")
+}
+  }
+}
+
+/**
+ * Removes [[View]] operators from the plan. The operator is respected 
till the end of analysis
+ * stage because we want to see which part of a analyzed logical plan is 
generated from a view.
+ */
+object EliminateView extends Rule[LogicalPlan] {
+  def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
+// The child should have the same output attributes with the View 
operator, so we simply
--- End diff --

Can we add an assert below to ensure the outputs are the same?


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

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For add

[GitHub] spark pull request #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-09 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95309414
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.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.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.CatalystConf
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, 
View}
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+ * This file defines analysis rules related to views.
+ */
+
+/**
+ * Make sure that a view's child plan produces the view's output 
attributes. We wrap the child
+ * with a Project and add an alias for each output attribute. The 
attributes are resolved by
+ * name. This should be only done after the resolution batch, because the 
view attributes are
+ * not stable during resolution.
+ */
+case class AliasViewChild(conf: CatalystConf) extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan 
resolveOperators {
+case v @ View(_, output, child) if child.resolved =>
+  val resolver = conf.resolver
+  val newOutput = child.output.map { attr =>
+val newAttr = findAttributeByName(attr.name, output, resolver)
+// Check the dataType of the output attributes, throw an 
AnalysisException if they don't
+// match up.
+checkDataType(attr, newAttr)
+Alias(attr, attr.name)(exprId = newAttr.exprId, qualifier = 
newAttr.qualifier,
+  explicitMetadata = Some(newAttr.metadata))
+  }
+  v.copy(child = Project(newOutput, child))
+  }
+
+  /**
+   * Find the attribute that has the expected attribute name from an 
attribute list, the names
+   * are compared using conf.resolver.
+   * If the expected attribute is not found, throw an AnalysisException.
+   */
+  private def findAttributeByName(
+  name: String,
+  attrs: Seq[Attribute],
+  resolver: Resolver): Attribute = {
+attrs.collectFirst {
+  case attr if resolver(attr.name, name) => attr
+}.getOrElse(throw new AnalysisException(
+  s"Attribute with name '$name' is not found in " +
+s"'${attrs.map(_.name).mkString("(", ",", ")")}'"))
+  }
+
+  /**
+   * Check whether the dataType of `attr` could be casted to that of 
`other`, throw an
+   * AnalysisException if the both attributes don't match up.
+   */
+  private def checkDataType(attr: Attribute, other: Attribute): Unit = {
+if (!Cast.canCast(attr.dataType, other.dataType)) {
--- End diff --

The rule `ImplicitTypeCasts` is done in the batch of `Resolution`, right?


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

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



[GitHub] spark pull request #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-09 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95309984
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala 
---
@@ -0,0 +1,90 @@
+/*
+ * 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.AnalysisException
+import org.apache.spark.sql.catalyst.CatalystConf
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, 
View}
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+ * This file defines analysis rules related to views.
+ */
+
+/**
+ * Make sure that a view's child plan produces the view's output 
attributes. We wrap the child
+ * with a Project and add an alias for each output attribute. The 
attributes are resolved by
+ * name. This should be only done after the resolution batch, because the 
view attributes are
+ * not stable during resolution.
+ */
+case class AliasViewChild(conf: CatalystConf) extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan 
resolveOperators {
+case v @ View(_, output, child) if child.resolved =>
+  val resolver = conf.resolver
+  val newOutput = child.output.map { attr =>
+val newAttr = findAttributeByName(attr.name, output, resolver)
+// Check the dataType of the output attributes, throw an 
AnalysisException if they don't
+// match up.
+checkDataType(attr, newAttr)
+Alias(attr, attr.name)(exprId = newAttr.exprId, qualifier = 
newAttr.qualifier,
+  explicitMetadata = Some(newAttr.metadata))
+  }
+  v.copy(child = Project(newOutput, child))
+  }
+
+  /**
+   * Find the attribute that has the expected attribute name from an 
attribute list, the names
+   * are compared using conf.resolver.
+   * If the expected attribute is not found, throw an AnalysisException.
+   */
+  private def findAttributeByName(
+  name: String,
+  attrs: Seq[Attribute],
+  resolver: Resolver): Attribute = {
+attrs.collectFirst {
+  case attr if resolver(attr.name, name) => attr
+}.getOrElse(throw new AnalysisException(
+  s"Attribute with name '$name' is not found in " +
+s"'${attrs.map(_.name).mkString("(", ",", ")")}'"))
+  }
+
+  /**
+   * Check whether the dataType of `attr` could be casted to that of 
`other`, throw an
+   * AnalysisException if the both attributes don't match up.
+   */
+  private def checkDataType(attr: Attribute, other: Attribute): Unit = {
+if (!Cast.canCast(attr.dataType, other.dataType)) {
+  throw new AnalysisException(
+s"The dataType of attribute '$other' is '${other.dataType}', which 
can't be casted to " +
+  s"that of '$attr', expected '${attr.dataType}'.")
+}
+  }
+}
+
+/**
+ * Removes [[View]] operators from the plan. The operator is respected 
till the end of analysis
+ * stage because we want to see which part of a analyzed logical plan is 
generated from a view.
--- End diff --

Nit: `a analyzed` ->`an analyzed`


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-09 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95309902
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala 
---
@@ -0,0 +1,90 @@
+/*
+ * 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.AnalysisException
+import org.apache.spark.sql.catalyst.CatalystConf
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, 
View}
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+ * This file defines analysis rules related to views.
+ */
+
+/**
+ * Make sure that a view's child plan produces the view's output 
attributes. We wrap the child
+ * with a Project and add an alias for each output attribute. The 
attributes are resolved by
+ * name. This should be only done after the resolution batch, because the 
view attributes are
+ * not stable during resolution.
+ */
+case class AliasViewChild(conf: CatalystConf) extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan 
resolveOperators {
+case v @ View(_, output, child) if child.resolved =>
+  val resolver = conf.resolver
+  val newOutput = child.output.map { attr =>
+val newAttr = findAttributeByName(attr.name, output, resolver)
+// Check the dataType of the output attributes, throw an 
AnalysisException if they don't
+// match up.
+checkDataType(attr, newAttr)
+Alias(attr, attr.name)(exprId = newAttr.exprId, qualifier = 
newAttr.qualifier,
+  explicitMetadata = Some(newAttr.metadata))
+  }
+  v.copy(child = Project(newOutput, child))
+  }
+
+  /**
+   * Find the attribute that has the expected attribute name from an 
attribute list, the names
+   * are compared using conf.resolver.
+   * If the expected attribute is not found, throw an AnalysisException.
+   */
+  private def findAttributeByName(
+  name: String,
+  attrs: Seq[Attribute],
+  resolver: Resolver): Attribute = {
+attrs.collectFirst {
+  case attr if resolver(attr.name, name) => attr
+}.getOrElse(throw new AnalysisException(
+  s"Attribute with name '$name' is not found in " +
+s"'${attrs.map(_.name).mkString("(", ",", ")")}'"))
+  }
+
+  /**
+   * Check whether the dataType of `attr` could be casted to that of 
`other`, throw an
+   * AnalysisException if the both attributes don't match up.
+   */
+  private def checkDataType(attr: Attribute, other: Attribute): Unit = {
+if (!Cast.canCast(attr.dataType, other.dataType)) {
+  throw new AnalysisException(
+s"The dataType of attribute '$other' is '${other.dataType}', which 
can't be casted to " +
+  s"that of '$attr', expected '${attr.dataType}'.")
+}
+  }
+}
+
+/**
+ * Removes [[View]] operators from the plan. The operator is respected 
till the end of analysis
+ * stage because we want to see which part of a analyzed logical plan is 
generated from a view.
+ */
+object EliminateView extends Rule[LogicalPlan] {
+  def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
--- End diff --

Nit: `transformUp` -> `transform`. When the order does not matter, we use 
`transform`


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-09 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95309603
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala 
---
@@ -0,0 +1,90 @@
+/*
+ * 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.AnalysisException
+import org.apache.spark.sql.catalyst.CatalystConf
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, 
View}
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+ * This file defines analysis rules related to views.
+ */
+
+/**
+ * Make sure that a view's child plan produces the view's output 
attributes. We wrap the child
+ * with a Project and add an alias for each output attribute. The 
attributes are resolved by
+ * name. This should be only done after the resolution batch, because the 
view attributes are
+ * not stable during resolution.
+ */
+case class AliasViewChild(conf: CatalystConf) extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan 
resolveOperators {
+case v @ View(_, output, child) if child.resolved =>
+  val resolver = conf.resolver
+  val newOutput = child.output.map { attr =>
+val newAttr = findAttributeByName(attr.name, output, resolver)
+// Check the dataType of the output attributes, throw an 
AnalysisException if they don't
+// match up.
+checkDataType(attr, newAttr)
+Alias(attr, attr.name)(exprId = newAttr.exprId, qualifier = 
newAttr.qualifier,
+  explicitMetadata = Some(newAttr.metadata))
+  }
+  v.copy(child = Project(newOutput, child))
+  }
+
+  /**
+   * Find the attribute that has the expected attribute name from an 
attribute list, the names
+   * are compared using conf.resolver.
+   * If the expected attribute is not found, throw an AnalysisException.
+   */
+  private def findAttributeByName(
+  name: String,
+  attrs: Seq[Attribute],
+  resolver: Resolver): Attribute = {
+attrs.collectFirst {
+  case attr if resolver(attr.name, name) => attr
+}.getOrElse(throw new AnalysisException(
+  s"Attribute with name '$name' is not found in " +
+s"'${attrs.map(_.name).mkString("(", ",", ")")}'"))
+  }
+
+  /**
+   * Check whether the dataType of `attr` could be casted to that of 
`other`, throw an
+   * AnalysisException if the both attributes don't match up.
+   */
+  private def checkDataType(attr: Attribute, other: Attribute): Unit = {
+if (!Cast.canCast(attr.dataType, other.dataType)) {
+  throw new AnalysisException(
+s"The dataType of attribute '$other' is '${other.dataType}', which 
can't be casted to " +
+  s"that of '$attr', expected '${attr.dataType}'.")
+}
+  }
+}
+
+/**
+ * Removes [[View]] operators from the plan. The operator is respected 
till the end of analysis
+ * stage because we want to see which part of a analyzed logical plan is 
generated from a view.
+ */
+object EliminateView extends Rule[LogicalPlan] {
+  def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
+// The child should have the same output attributes with the View 
operator, so we simply
+// remove the View operator.
+case View(_, output, child) => child
--- End diff --

Nit: `case View(_, output, child) => child` -> `case View(_, _, child) => 
child`


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

---

[GitHub] spark pull request #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-09 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95309481
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.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.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.CatalystConf
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, 
View}
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+ * This file defines analysis rules related to views.
+ */
+
+/**
+ * Make sure that a view's child plan produces the view's output 
attributes. We wrap the child
+ * with a Project and add an alias for each output attribute. The 
attributes are resolved by
+ * name. This should be only done after the resolution batch, because the 
view attributes are
+ * not stable during resolution.
+ */
+case class AliasViewChild(conf: CatalystConf) extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan 
resolveOperators {
+case v @ View(_, output, child) if child.resolved =>
+  val resolver = conf.resolver
+  val newOutput = child.output.map { attr =>
+val newAttr = findAttributeByName(attr.name, output, resolver)
+// Check the dataType of the output attributes, throw an 
AnalysisException if they don't
+// match up.
+checkDataType(attr, newAttr)
+Alias(attr, attr.name)(exprId = newAttr.exprId, qualifier = 
newAttr.qualifier,
+  explicitMetadata = Some(newAttr.metadata))
+  }
+  v.copy(child = Project(newOutput, child))
+  }
+
+  /**
+   * Find the attribute that has the expected attribute name from an 
attribute list, the names
+   * are compared using conf.resolver.
+   * If the expected attribute is not found, throw an AnalysisException.
+   */
+  private def findAttributeByName(
+  name: String,
+  attrs: Seq[Attribute],
+  resolver: Resolver): Attribute = {
+attrs.collectFirst {
+  case attr if resolver(attr.name, name) => attr
+}.getOrElse(throw new AnalysisException(
+  s"Attribute with name '$name' is not found in " +
+s"'${attrs.map(_.name).mkString("(", ",", ")")}'"))
+  }
+
+  /**
+   * Check whether the dataType of `attr` could be casted to that of 
`other`, throw an
+   * AnalysisException if the both attributes don't match up.
+   */
+  private def checkDataType(attr: Attribute, other: Attribute): Unit = {
+if (!Cast.canCast(attr.dataType, other.dataType)) {
--- End diff --

Have you added any test case for type casting?


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-09 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95308910
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala 
---
@@ -0,0 +1,90 @@
+/*
+ * 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.AnalysisException
+import org.apache.spark.sql.catalyst.CatalystConf
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, 
View}
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+ * This file defines analysis rules related to views.
+ */
+
+/**
+ * Make sure that a view's child plan produces the view's output 
attributes. We wrap the child
+ * with a Project and add an alias for each output attribute. The 
attributes are resolved by
+ * name. This should be only done after the resolution batch, because the 
view attributes are
+ * not stable during resolution.
--- End diff --

`stable` -> `completely resolved`


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-09 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95308787
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala 
---
@@ -0,0 +1,90 @@
+/*
+ * 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.AnalysisException
+import org.apache.spark.sql.catalyst.CatalystConf
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, 
View}
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+ * This file defines analysis rules related to views.
+ */
+
+/**
+ * Make sure that a view's child plan produces the view's output 
attributes. We wrap the child
+ * with a Project and add an alias for each output attribute. The 
attributes are resolved by
+ * name. This should be only done after the resolution batch, because the 
view attributes are
--- End diff --

Nit: `after the resolution batch` -> `after the batch of Resolution`


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-09 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95307494
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -510,32 +542,90 @@ class Analyzer(
* Replaces [[UnresolvedRelation]]s with concrete relations from the 
catalog.
*/
   object ResolveRelations extends Rule[LogicalPlan] {
-private def lookupTableFromCatalog(u: UnresolvedRelation): LogicalPlan 
= {
+
+// If the unresolved relation is running directly on files, we just 
return the original
+// UnresolvedRelation, the plan will get resolved later. Else we look 
up the table from catalog
+// and change the default database name if it is a view.
+// We usually look up a table from the default database if the table 
identifier has an empty
+// database part, for a view the default database should be the 
currentDb when the view was
+// created. When the case comes to resolving a nested view, the view 
may have different default
+// database with that the referenced view has, so we need to use the 
variable `defaultDatabase`
+// to track the current default database.
+// When the relation we resolve is a view, we fetch the 
view.desc(which is a CatalogTable), and
+// then set the value of `CatalogTable.viewDefaultDatabase` to the 
variable `defaultDatabase`,
+// we look up the relations that the view references using the default 
database.
+// For example:
+// |- view1 (defaultDatabase = db1)
+//   |- operator
+// |- table2 (defaultDatabase = db1)
+// |- view2 (defaultDatabase = db2)
+//|- view3 (defaultDatabase = db3)
+//   |- view4 (defaultDatabase = db4)
+// In this case, the view `view1` is a nested view, it directly 
references `table2`、`view2`
+// and `view4`, the view `view2` references `view3`. On resolving the 
table, we look up the
+// relations `table2`、`view2`、`view4` using the default database 
`db1`, and look up the
+// relation `view3` using the default database `db2`.
+//
+// Note this is compatible with the views defined by older versions of 
Spark(before 2.2), which
+// have empty defaultDatabase and all the relations in viewText have 
database part defined.
+def resolveRelation(
+plan: LogicalPlan,
+defaultDatabase: Option[String] = None): LogicalPlan = plan match {
+  case u: UnresolvedRelation if 
!isRunningDirectlyOnFiles(u.tableIdentifier) =>
+val defaultDatabase = AnalysisContext.get.defaultDatabase
+val relation = lookupTableFromCatalog(u, defaultDatabase)
+resolveRelation(relation, defaultDatabase)
+  // The view's child should be a logical plan parsed from the 
`desc.viewText`, the variable
+  // `viewText` should be defined, or else we throw an error on the 
generation of the View
+  // operator.
+  case view @ View(desc, _, child) if !child.resolved =>
+val nestedViewLevel = AnalysisContext.get.nestedViewLevel + 1
+val context = AnalysisContext(defaultDatabase = 
desc.viewDefaultDatabase,
+  nestedViewLevel = nestedViewLevel)
+// Resolve all the UnresolvedRelations and Views in the child.
+val newChild = AnalysisContext.withAnalysisContext(context) {
+  execute(child)
+}
+view.copy(child = newChild)
+  case p @ SubqueryAlias(_, view: View, _) =>
+val newChild = resolveRelation(view, defaultDatabase)
+p.copy(child = newChild)
+  case _ => plan
+}
+
+def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+  case i @ InsertIntoTable(u: UnresolvedRelation, parts, child, _, _) 
if child.resolved =>
+i.copy(table = EliminateSubqueryAliases(lookupTableFromCatalog(u)))
+  case u: UnresolvedRelation => resolveRelation(u)
+}
+
+// Look up the table with the given name from catalog. The database we 
look up the table from
+// is decided follow the steps:
+// 1. If the database part is defined in the table identifier, use 
that database name;
+// 2. Else If the defaultDatabase is defined, use the default database 
name(In this case, no
+//temporary objects can be used, and the default database is only 
used to look up a view);
+// 3. Else use the currentDb of the SessionCatalog.
--- End diff --

```
1. u.tableIdentifier.database, if defined
2. defaultDatabase, if defined
3. currentDb of SessionCatalog, otherwise.  
```


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

[GitHub] spark pull request #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-09 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95306192
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
@@ -125,11 +132,16 @@ private[hive] class 
HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
   // Otherwise, wrap the table with a Subquery using the table name.
   alias.map(a => SubqueryAlias(a, qualifiedTable, 
None)).getOrElse(qualifiedTable)
 } else if (table.tableType == CatalogTableType.VIEW) {
+  val tableIdentifier = table.identifier
   val viewText = table.viewText.getOrElse(sys.error("Invalid view 
without text."))
-  SubqueryAlias(
-alias.getOrElse(table.identifier.table),
-sparkSession.sessionState.sqlParser.parsePlan(viewText),
-Option(table.identifier))
+  // The relation is a view, so we wrap the relation by:
+  // 1. Add a [[View]] operator over the relation to keep track of the 
view desc;
+  // 2. Wrap the logical plan in a [[SubqueryAlias]] which tracks the 
name of the view.
+  val child = View(
+desc = table,
+output = table.schema.toAttributes,
+child = sparkSession.sessionState.sqlParser.parsePlan(viewText))
--- End diff --

We can even add `Cast` in project list so that analyzer can report data 
type mismatch for us automatically.


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-09 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95306084
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
@@ -125,11 +132,16 @@ private[hive] class 
HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
   // Otherwise, wrap the table with a Subquery using the table name.
   alias.map(a => SubqueryAlias(a, qualifiedTable, 
None)).getOrElse(qualifiedTable)
 } else if (table.tableType == CatalogTableType.VIEW) {
+  val tableIdentifier = table.identifier
   val viewText = table.viewText.getOrElse(sys.error("Invalid view 
without text."))
-  SubqueryAlias(
-alias.getOrElse(table.identifier.table),
-sparkSession.sessionState.sqlParser.parsePlan(viewText),
-Option(table.identifier))
+  // The relation is a view, so we wrap the relation by:
+  // 1. Add a [[View]] operator over the relation to keep track of the 
view desc;
+  // 2. Wrap the logical plan in a [[SubqueryAlias]] which tracks the 
name of the view.
+  val child = View(
+desc = table,
+output = table.schema.toAttributes,
+child = sparkSession.sessionState.sqlParser.parsePlan(viewText))
--- End diff --

sorry I may have asked this question before, but I can't recall your 
answer. Why do we need the `output` field of `View`? can we do something like
```
val child = sparkSession.sessionState.sqlParser.parsePlan(viewText)
val projectList = schema.map { field =>
  Alias(UnresolvedAttribute(Seq(field.name)), field.name)(explicitMetadata 
= Some(field.metadata))
}
View(
  desc = table,
  child = Project(projectList, child))

case class View(...) {
  def output = child.output
}
```


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-09 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95305635
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -510,32 +542,90 @@ class Analyzer(
* Replaces [[UnresolvedRelation]]s with concrete relations from the 
catalog.
*/
   object ResolveRelations extends Rule[LogicalPlan] {
-private def lookupTableFromCatalog(u: UnresolvedRelation): LogicalPlan 
= {
+
+// If the unresolved relation is running directly on files, we just 
return the original
+// UnresolvedRelation, the plan will get resolved later. Else we look 
up the table from catalog
+// and change the default database name if it is a view.
+// We usually look up a table from the default database if the table 
identifier has an empty
+// database part, for a view the default database should be the 
currentDb when the view was
+// created. When the case comes to resolving a nested view, the view 
may have different default
+// database with that the referenced view has, so we need to use the 
variable `defaultDatabase`
+// to track the current default database.
+// When the relation we resolve is a view, we fetch the 
view.desc(which is a CatalogTable), and
+// then set the value of `CatalogTable.viewDefaultDatabase` to the 
variable `defaultDatabase`,
+// we look up the relations that the view references using the default 
database.
+// For example:
+// |- view1 (defaultDatabase = db1)
+//   |- operator
+// |- table2 (defaultDatabase = db1)
+// |- view2 (defaultDatabase = db2)
+//|- view3 (defaultDatabase = db3)
+//   |- view4 (defaultDatabase = db4)
+// In this case, the view `view1` is a nested view, it directly 
references `table2`、`view2`
+// and `view4`, the view `view2` references `view3`. On resolving the 
table, we look up the
+// relations `table2`、`view2`、`view4` using the default database 
`db1`, and look up the
+// relation `view3` using the default database `db2`.
+//
+// Note this is compatible with the views defined by older versions of 
Spark(before 2.2), which
+// have empty defaultDatabase and all the relations in viewText have 
database part defined.
+def resolveRelation(
+plan: LogicalPlan,
+defaultDatabase: Option[String] = None): LogicalPlan = plan match {
+  case u: UnresolvedRelation if 
!isRunningDirectlyOnFiles(u.tableIdentifier) =>
+val defaultDatabase = AnalysisContext.get.defaultDatabase
+val relation = lookupTableFromCatalog(u, defaultDatabase)
+resolveRelation(relation, defaultDatabase)
+  // The view's child should be a logical plan parsed from the 
`desc.viewText`, the variable
+  // `viewText` should be defined, or else we throw an error on the 
generation of the View
+  // operator.
+  case view @ View(desc, _, child) if !child.resolved =>
+val nestedViewLevel = AnalysisContext.get.nestedViewLevel + 1
+val context = AnalysisContext(defaultDatabase = 
desc.viewDefaultDatabase,
+  nestedViewLevel = nestedViewLevel)
+// Resolve all the UnresolvedRelations and Views in the child.
+val newChild = AnalysisContext.withAnalysisContext(context) {
+  execute(child)
+}
+view.copy(child = newChild)
+  case p @ SubqueryAlias(_, view: View, _) =>
+val newChild = resolveRelation(view, defaultDatabase)
+p.copy(child = newChild)
+  case _ => plan
+}
+
+def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+  case i @ InsertIntoTable(u: UnresolvedRelation, parts, child, _, _) 
if child.resolved =>
+i.copy(table = EliminateSubqueryAliases(lookupTableFromCatalog(u)))
+  case u: UnresolvedRelation => resolveRelation(u)
+}
+
+// Look up the table with the given name from catalog. The database we 
look up the table from
+// is decided follow the steps:
--- End diff --

Nit: `follow` -> `following`


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-09 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95305370
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala
 ---
@@ -174,6 +176,7 @@ case class CatalogTable(
 stats: Option[CatalogStatistics] = None,
 viewOriginalText: Option[String] = None,
 viewText: Option[String] = None,
+viewDefaultDatabase: Option[String] = None,
--- End diff --

can we make this a table property instead of a field? something like
```
case class CatalogTable(...) {
  def viewDefaultDatabase: Opion[String] = 
properties.get(VIEW_DEFAULT_DATABASE)
}

object CatalogTable {
  val VIEW_DEFAULT_DATABASE = "view.default.database"
}
```

This is only used in a few places and I don't think it worth to add so many 
hacks in `HiveExternalCatalog`


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-09 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95305390
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -510,32 +542,90 @@ class Analyzer(
* Replaces [[UnresolvedRelation]]s with concrete relations from the 
catalog.
*/
   object ResolveRelations extends Rule[LogicalPlan] {
-private def lookupTableFromCatalog(u: UnresolvedRelation): LogicalPlan 
= {
+
+// If the unresolved relation is running directly on files, we just 
return the original
+// UnresolvedRelation, the plan will get resolved later. Else we look 
up the table from catalog
+// and change the default database name if it is a view.
+// We usually look up a table from the default database if the table 
identifier has an empty
+// database part, for a view the default database should be the 
currentDb when the view was
+// created. When the case comes to resolving a nested view, the view 
may have different default
+// database with that the referenced view has, so we need to use the 
variable `defaultDatabase`
+// to track the current default database.
+// When the relation we resolve is a view, we fetch the 
view.desc(which is a CatalogTable), and
+// then set the value of `CatalogTable.viewDefaultDatabase` to the 
variable `defaultDatabase`,
+// we look up the relations that the view references using the default 
database.
+// For example:
+// |- view1 (defaultDatabase = db1)
+//   |- operator
+// |- table2 (defaultDatabase = db1)
+// |- view2 (defaultDatabase = db2)
+//|- view3 (defaultDatabase = db3)
+//   |- view4 (defaultDatabase = db4)
+// In this case, the view `view1` is a nested view, it directly 
references `table2`、`view2`
+// and `view4`, the view `view2` references `view3`. On resolving the 
table, we look up the
+// relations `table2`、`view2`、`view4` using the default database 
`db1`, and look up the
+// relation `view3` using the default database `db2`.
+//
+// Note this is compatible with the views defined by older versions of 
Spark(before 2.2), which
+// have empty defaultDatabase and all the relations in viewText have 
database part defined.
+def resolveRelation(
+plan: LogicalPlan,
+defaultDatabase: Option[String] = None): LogicalPlan = plan match {
+  case u: UnresolvedRelation if 
!isRunningDirectlyOnFiles(u.tableIdentifier) =>
+val defaultDatabase = AnalysisContext.get.defaultDatabase
+val relation = lookupTableFromCatalog(u, defaultDatabase)
+resolveRelation(relation, defaultDatabase)
+  // The view's child should be a logical plan parsed from the 
`desc.viewText`, the variable
+  // `viewText` should be defined, or else we throw an error on the 
generation of the View
+  // operator.
+  case view @ View(desc, _, child) if !child.resolved =>
+val nestedViewLevel = AnalysisContext.get.nestedViewLevel + 1
+val context = AnalysisContext(defaultDatabase = 
desc.viewDefaultDatabase,
+  nestedViewLevel = nestedViewLevel)
+// Resolve all the UnresolvedRelations and Views in the child.
+val newChild = AnalysisContext.withAnalysisContext(context) {
+  execute(child)
+}
--- End diff --

Like 
[withWatrmark](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala#L572),
 we can simplify the above code by changing the interface of 
`withAnalysisContext`:
```Scala
val newChild = 
AnalysisContext.withAnalysisContext(desc.viewDefaultDatabase) {
  execute(child)
}
```

The `nestedViewLevel` can be incremented by one in `withAnalysisContext`


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-09 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95304793
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.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.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.CatalystConf
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, 
View}
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+ * This file defines analysis rules related to views.
+ */
+
+/**
+ * Make sure that a view's child plan produces the view's output 
attributes. We wrap the child
+ * with a Project and add an alias for each output attribute. The 
attributes are resolved by
+ * name. This should be only done after the resolution batch, because the 
view attributes are
+ * not stable during resolution.
+ */
+case class AliasViewChild(conf: CatalystConf) extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan 
resolveOperators {
+case v @ View(_, output, child) if child.resolved =>
+  val resolver = conf.resolver
+  val newOutput = child.output.map { attr =>
+val newAttr = findAttributeByName(attr.name, output, resolver)
+// Check the dataType of the output attributes, throw an 
AnalysisException if they don't
+// match up.
+checkDataType(attr, newAttr)
+Alias(attr, attr.name)(exprId = newAttr.exprId, qualifier = 
newAttr.qualifier,
+  explicitMetadata = Some(newAttr.metadata))
+  }
+  v.copy(child = Project(newOutput, child))
+  }
+
+  /**
+   * Find the attribute that has the expected attribute name from an 
attribute list, the names
+   * are compared using conf.resolver.
+   * If the expected attribute is not found, throw an AnalysisException.
+   */
+  private def findAttributeByName(
+  name: String,
+  attrs: Seq[Attribute],
+  resolver: Resolver): Attribute = {
+attrs.collectFirst {
+  case attr if resolver(attr.name, name) => attr
+}.getOrElse(throw new AnalysisException(
+  s"Attribute with name '$name' is not found in " +
+s"'${attrs.map(_.name).mkString("(", ",", ")")}'"))
+  }
+
+  /**
+   * Check whether the dataType of `attr` could be casted to that of 
`other`, throw an
+   * AnalysisException if the both attributes don't match up.
+   */
+  private def checkDataType(attr: Attribute, other: Attribute): Unit = {
+if (!Cast.canCast(attr.dataType, other.dataType)) {
--- End diff --

how did we handle view schema mismatch before?


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-09 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95304745
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -510,32 +542,90 @@ class Analyzer(
* Replaces [[UnresolvedRelation]]s with concrete relations from the 
catalog.
*/
   object ResolveRelations extends Rule[LogicalPlan] {
-private def lookupTableFromCatalog(u: UnresolvedRelation): LogicalPlan 
= {
+
+// If the unresolved relation is running directly on files, we just 
return the original
+// UnresolvedRelation, the plan will get resolved later. Else we look 
up the table from catalog
+// and change the default database name if it is a view.
+// We usually look up a table from the default database if the table 
identifier has an empty
+// database part, for a view the default database should be the 
currentDb when the view was
+// created. When the case comes to resolving a nested view, the view 
may have different default
+// database with that the referenced view has, so we need to use the 
variable `defaultDatabase`
+// to track the current default database.
+// When the relation we resolve is a view, we fetch the 
view.desc(which is a CatalogTable), and
+// then set the value of `CatalogTable.viewDefaultDatabase` to the 
variable `defaultDatabase`,
+// we look up the relations that the view references using the default 
database.
+// For example:
+// |- view1 (defaultDatabase = db1)
+//   |- operator
+// |- table2 (defaultDatabase = db1)
+// |- view2 (defaultDatabase = db2)
+//|- view3 (defaultDatabase = db3)
+//   |- view4 (defaultDatabase = db4)
+// In this case, the view `view1` is a nested view, it directly 
references `table2`、`view2`
+// and `view4`, the view `view2` references `view3`. On resolving the 
table, we look up the
+// relations `table2`、`view2`、`view4` using the default database 
`db1`, and look up the
+// relation `view3` using the default database `db2`.
+//
+// Note this is compatible with the views defined by older versions of 
Spark(before 2.2), which
+// have empty defaultDatabase and all the relations in viewText have 
database part defined.
+def resolveRelation(
+plan: LogicalPlan,
+defaultDatabase: Option[String] = None): LogicalPlan = plan match {
+  case u: UnresolvedRelation if 
!isRunningDirectlyOnFiles(u.tableIdentifier) =>
+val defaultDatabase = AnalysisContext.get.defaultDatabase
+val relation = lookupTableFromCatalog(u, defaultDatabase)
+resolveRelation(relation, defaultDatabase)
+  // The view's child should be a logical plan parsed from the 
`desc.viewText`, the variable
+  // `viewText` should be defined, or else we throw an error on the 
generation of the View
+  // operator.
+  case view @ View(desc, _, child) if !child.resolved =>
+val nestedViewLevel = AnalysisContext.get.nestedViewLevel + 1
+val context = AnalysisContext(defaultDatabase = 
desc.viewDefaultDatabase,
+  nestedViewLevel = nestedViewLevel)
+// Resolve all the UnresolvedRelations and Views in the child.
+val newChild = AnalysisContext.withAnalysisContext(context) {
+  execute(child)
+}
+view.copy(child = newChild)
+  case p @ SubqueryAlias(_, view: View, _) =>
+val newChild = resolveRelation(view, defaultDatabase)
+p.copy(child = newChild)
+  case _ => plan
+}
+
+def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+  case i @ InsertIntoTable(u: UnresolvedRelation, parts, child, _, _) 
if child.resolved =>
+i.copy(table = EliminateSubqueryAliases(lookupTableFromCatalog(u)))
+  case u: UnresolvedRelation => resolveRelation(u)
+}
+
+// Look up the table with the given name from catalog. The database we 
look up the table from
+// is decided follow the steps:
+// 1. If the database part is defined in the table identifier, use 
that database name;
+// 2. Else If the defaultDatabase is defined, use the default database 
name(In this case, no
+//temporary objects can be used, and the default database is only 
used to look up a view);
+// 3. Else use the currentDb of the SessionCatalog.
+private def lookupTableFromCatalog(
+u: UnresolvedRelation,
+defaultDatabase: Option[String] = None): LogicalPlan = {
   try {
-catalog.lookupRelation(u.tableIdentifier, u.alias)
+val tableIdentWithDb = 
u.tableIdentifier.withDatabase(defaultDatabase)
--

[GitHub] spark pull request #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-09 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95304509
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala 
---
@@ -0,0 +1,90 @@
+/*
+ * 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.AnalysisException
+import org.apache.spark.sql.catalyst.CatalystConf
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, 
View}
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+ * This file defines analysis rules related to views.
+ */
+
+/**
+ * Make sure that a view's child plan produces the view's output 
attributes. We wrap the child
+ * with a Project and add an alias for each output attribute. The 
attributes are resolved by
+ * name. This should be only done after the resolution batch, because the 
view attributes are
+ * not stable during resolution.
+ */
+case class AliasViewChild(conf: CatalystConf) extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan 
resolveOperators {
+case v @ View(_, output, child) if child.resolved =>
+  val resolver = conf.resolver
+  val newOutput = child.output.map { attr =>
+val newAttr = findAttributeByName(attr.name, output, resolver)
+// Check the dataType of the output attributes, throw an 
AnalysisException if they don't
+// match up.
+checkDataType(attr, newAttr)
+Alias(attr, attr.name)(exprId = newAttr.exprId, qualifier = 
newAttr.qualifier,
+  explicitMetadata = Some(newAttr.metadata))
+  }
+  v.copy(child = Project(newOutput, child))
+  }
+
+  /**
+   * Find the attribute that has the expected attribute name from an 
attribute list, the names
+   * are compared using conf.resolver.
+   * If the expected attribute is not found, throw an AnalysisException.
+   */
+  private def findAttributeByName(
+  name: String,
+  attrs: Seq[Attribute],
+  resolver: Resolver): Attribute = {
+attrs.collectFirst {
+  case attr if resolver(attr.name, name) => attr
+}.getOrElse(throw new AnalysisException(
+  s"Attribute with name '$name' is not found in " +
+s"'${attrs.map(_.name).mkString("(", ",", ")")}'"))
+  }
+
+  /**
+   * Check whether the dataType of `attr` could be casted to that of 
`other`, throw an
+   * AnalysisException if the both attributes don't match up.
+   */
+  private def checkDataType(attr: Attribute, other: Attribute): Unit = {
+if (!Cast.canCast(attr.dataType, other.dataType)) {
+  throw new AnalysisException(
+s"The dataType of attribute '$other' is '${other.dataType}', which 
can't be casted to " +
+  s"that of '$attr', expected '${attr.dataType}'.")
+}
+  }
+}
+
+/**
+ * Removes [[View]] operators from the plan. The operator is respected 
till the end of analysis
+ * stage because we want to see which part of a analyzed logical plan is 
generated from a view.
+ */
+object EliminateView extends Rule[LogicalPlan] {
+  def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
+// The child should have the same output attributes with the View 
operator, so we simply
--- End diff --

> The child should have the same output attributes with the View operator

shall we add an assert?


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

[GitHub] spark pull request #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-09 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95304037
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -510,32 +542,90 @@ class Analyzer(
* Replaces [[UnresolvedRelation]]s with concrete relations from the 
catalog.
*/
   object ResolveRelations extends Rule[LogicalPlan] {
-private def lookupTableFromCatalog(u: UnresolvedRelation): LogicalPlan 
= {
+
+// If the unresolved relation is running directly on files, we just 
return the original
+// UnresolvedRelation, the plan will get resolved later. Else we look 
up the table from catalog
+// and change the default database name if it is a view.
+// We usually look up a table from the default database if the table 
identifier has an empty
+// database part, for a view the default database should be the 
currentDb when the view was
+// created. When the case comes to resolving a nested view, the view 
may have different default
+// database with that the referenced view has, so we need to use the 
variable `defaultDatabase`
+// to track the current default database.
+// When the relation we resolve is a view, we fetch the 
view.desc(which is a CatalogTable), and
+// then set the value of `CatalogTable.viewDefaultDatabase` to the 
variable `defaultDatabase`,
+// we look up the relations that the view references using the default 
database.
+// For example:
+// |- view1 (defaultDatabase = db1)
+//   |- operator
+// |- table2 (defaultDatabase = db1)
+// |- view2 (defaultDatabase = db2)
+//|- view3 (defaultDatabase = db3)
+//   |- view4 (defaultDatabase = db4)
+// In this case, the view `view1` is a nested view, it directly 
references `table2`、`view2`
+// and `view4`, the view `view2` references `view3`. On resolving the 
table, we look up the
+// relations `table2`、`view2`、`view4` using the default database 
`db1`, and look up the
+// relation `view3` using the default database `db2`.
+//
+// Note this is compatible with the views defined by older versions of 
Spark(before 2.2), which
+// have empty defaultDatabase and all the relations in viewText have 
database part defined.
+def resolveRelation(
+plan: LogicalPlan,
+defaultDatabase: Option[String] = None): LogicalPlan = plan match {
+  case u: UnresolvedRelation if 
!isRunningDirectlyOnFiles(u.tableIdentifier) =>
+val defaultDatabase = AnalysisContext.get.defaultDatabase
+val relation = lookupTableFromCatalog(u, defaultDatabase)
+resolveRelation(relation, defaultDatabase)
+  // The view's child should be a logical plan parsed from the 
`desc.viewText`, the variable
+  // `viewText` should be defined, or else we throw an error on the 
generation of the View
+  // operator.
+  case view @ View(desc, _, child) if !child.resolved =>
+val nestedViewLevel = AnalysisContext.get.nestedViewLevel + 1
+val context = AnalysisContext(defaultDatabase = 
desc.viewDefaultDatabase,
+  nestedViewLevel = nestedViewLevel)
+// Resolve all the UnresolvedRelations and Views in the child.
+val newChild = AnalysisContext.withAnalysisContext(context) {
+  execute(child)
+}
+view.copy(child = newChild)
+  case p @ SubqueryAlias(_, view: View, _) =>
--- End diff --

why we have this case?


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-09 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95303945
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -510,32 +542,90 @@ class Analyzer(
* Replaces [[UnresolvedRelation]]s with concrete relations from the 
catalog.
*/
   object ResolveRelations extends Rule[LogicalPlan] {
-private def lookupTableFromCatalog(u: UnresolvedRelation): LogicalPlan 
= {
+
+// If the unresolved relation is running directly on files, we just 
return the original
+// UnresolvedRelation, the plan will get resolved later. Else we look 
up the table from catalog
+// and change the default database name if it is a view.
+// We usually look up a table from the default database if the table 
identifier has an empty
+// database part, for a view the default database should be the 
currentDb when the view was
+// created. When the case comes to resolving a nested view, the view 
may have different default
+// database with that the referenced view has, so we need to use the 
variable `defaultDatabase`
+// to track the current default database.
+// When the relation we resolve is a view, we fetch the 
view.desc(which is a CatalogTable), and
+// then set the value of `CatalogTable.viewDefaultDatabase` to the 
variable `defaultDatabase`,
+// we look up the relations that the view references using the default 
database.
+// For example:
+// |- view1 (defaultDatabase = db1)
+//   |- operator
+// |- table2 (defaultDatabase = db1)
+// |- view2 (defaultDatabase = db2)
+//|- view3 (defaultDatabase = db3)
+//   |- view4 (defaultDatabase = db4)
+// In this case, the view `view1` is a nested view, it directly 
references `table2`、`view2`
+// and `view4`, the view `view2` references `view3`. On resolving the 
table, we look up the
+// relations `table2`、`view2`、`view4` using the default database 
`db1`, and look up the
+// relation `view3` using the default database `db2`.
+//
+// Note this is compatible with the views defined by older versions of 
Spark(before 2.2), which
+// have empty defaultDatabase and all the relations in viewText have 
database part defined.
+def resolveRelation(
+plan: LogicalPlan,
+defaultDatabase: Option[String] = None): LogicalPlan = plan match {
+  case u: UnresolvedRelation if 
!isRunningDirectlyOnFiles(u.tableIdentifier) =>
+val defaultDatabase = AnalysisContext.get.defaultDatabase
+val relation = lookupTableFromCatalog(u, defaultDatabase)
+resolveRelation(relation, defaultDatabase)
+  // The view's child should be a logical plan parsed from the 
`desc.viewText`, the variable
+  // `viewText` should be defined, or else we throw an error on the 
generation of the View
+  // operator.
+  case view @ View(desc, _, child) if !child.resolved =>
+val nestedViewLevel = AnalysisContext.get.nestedViewLevel + 1
+val context = AnalysisContext(defaultDatabase = 
desc.viewDefaultDatabase,
+  nestedViewLevel = nestedViewLevel)
+// Resolve all the UnresolvedRelations and Views in the child.
+val newChild = AnalysisContext.withAnalysisContext(context) {
+  execute(child)
+}
+view.copy(child = newChild)
+  case p @ SubqueryAlias(_, view: View, _) =>
+val newChild = resolveRelation(view, defaultDatabase)
+p.copy(child = newChild)
+  case _ => plan
+}
+
+def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+  case i @ InsertIntoTable(u: UnresolvedRelation, parts, child, _, _) 
if child.resolved =>
+i.copy(table = EliminateSubqueryAliases(lookupTableFromCatalog(u)))
+  case u: UnresolvedRelation => resolveRelation(u)
+}
+
+// Look up the table with the given name from catalog. The database we 
look up the table from
+// is decided follow the steps:
+// 1. If the database part is defined in the table identifier, use 
that database name;
+// 2. Else If the defaultDatabase is defined, use the default database 
name(In this case, no
+//temporary objects can be used, and the default database is only 
used to look up a view);
+// 3. Else use the currentDb of the SessionCatalog.
--- End diff --

Looks like the current code doesn't match these comments well, shall we 
update the comments?


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

[GitHub] spark pull request #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-09 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95302345
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -50,6 +50,36 @@ object SimpleAnalyzer extends Analyzer(
 new SimpleCatalystConf(caseSensitiveAnalysis = true))
 
 /**
+ * Provides a way to keep state during the analysis, this enables us to 
decouple the concerns
+ * of analysis environment from the catalog.
+ *
+ * Note this is thread local.
+ *
+ * @param defaultDatabase The default database used in the view 
resolution, this overrules the
+ *current catalog database.
+ * @param nestedViewLevel The nested level in the view resolution, this 
enables us to limit the
+ *depth of nested views.
+ */
+case class AnalysisContext(
+defaultDatabase: Option[String] = None,
+nestedViewLevel: Int = 0)
+
+object AnalysisContext {
+  private val value = new ThreadLocal[AnalysisContext]() {
+override def initialValue: AnalysisContext = AnalysisContext()
+  }
+
+  def get: AnalysisContext = value.get()
+  def set(context: AnalysisContext): Unit = value.set(context)
--- End diff --

This function is private, right?


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

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



[GitHub] spark pull request #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-09 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95302084
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -510,32 +542,90 @@ class Analyzer(
* Replaces [[UnresolvedRelation]]s with concrete relations from the 
catalog.
*/
   object ResolveRelations extends Rule[LogicalPlan] {
-private def lookupTableFromCatalog(u: UnresolvedRelation): LogicalPlan 
= {
+
+// If the unresolved relation is running directly on files, we just 
return the original
+// UnresolvedRelation, the plan will get resolved later. Else we look 
up the table from catalog
+// and change the default database name if it is a view.
+// We usually look up a table from the default database if the table 
identifier has an empty
+// database part, for a view the default database should be the 
currentDb when the view was
+// created. When the case comes to resolving a nested view, the view 
may have different default
+// database with that the referenced view has, so we need to use the 
variable `defaultDatabase`
+// to track the current default database.
+// When the relation we resolve is a view, we fetch the 
view.desc(which is a CatalogTable), and
+// then set the value of `CatalogTable.viewDefaultDatabase` to the 
variable `defaultDatabase`,
+// we look up the relations that the view references using the default 
database.
+// For example:
+// |- view1 (defaultDatabase = db1)
+//   |- operator
+// |- table2 (defaultDatabase = db1)
+// |- view2 (defaultDatabase = db2)
+//|- view3 (defaultDatabase = db3)
+//   |- view4 (defaultDatabase = db4)
+// In this case, the view `view1` is a nested view, it directly 
references `table2`、`view2`
+// and `view4`, the view `view2` references `view3`. On resolving the 
table, we look up the
+// relations `table2`、`view2`、`view4` using the default database 
`db1`, and look up the
+// relation `view3` using the default database `db2`.
+//
+// Note this is compatible with the views defined by older versions of 
Spark(before 2.2), which
+// have empty defaultDatabase and all the relations in viewText have 
database part defined.
+def resolveRelation(
+plan: LogicalPlan,
+defaultDatabase: Option[String] = None): LogicalPlan = plan match {
--- End diff --

We already introduce `AnalysisContext`. It does not need this parm, right?
 
```Scala
def resolveRelation(plan: LogicalPlan): LogicalPlan = plan match {
  case u: UnresolvedRelation if 
!isRunningDirectlyOnFiles(u.tableIdentifier) =>
val defaultDatabase = AnalysisContext.get.defaultDatabase
val relation = lookupTableFromCatalog(u, defaultDatabase)
resolveRelation(relation)
  // The view's child should be a logical plan parsed from the 
`desc.viewText`, the variable
  // `viewText` should be defined, or else we throw an error on the 
generation of the View
  // operator.
  case view @ View(desc, _, child) if !child.resolved =>
val nestedViewLevel = AnalysisContext.get.nestedViewLevel + 1
val context = AnalysisContext(defaultDatabase = 
desc.viewDefaultDatabase,
  nestedViewLevel = nestedViewLevel)
// Resolve all the UnresolvedRelations and Views in the child.
val newChild = AnalysisContext.withAnalysisContext(context) {
  execute(child)
}
view.copy(child = newChild)
  case p @ SubqueryAlias(_, view: View, _) =>
val newChild = resolveRelation(view)
p.copy(child = newChild)
  case _ => plan
}
```

Please also update the function 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 #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-09 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95300580
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -50,6 +50,36 @@ object SimpleAnalyzer extends Analyzer(
 new SimpleCatalystConf(caseSensitiveAnalysis = true))
 
 /**
+ * Provides a way to keep state during the analysis, this enables us to 
decouple the concerns
+ * of analysis environment from the catalog.
+ *
+ * Note this is thread local.
+ *
+ * @param defaultDatabase The default database used in the view 
resolution, this overrules the
+ *current catalog database.
+ * @param nestedViewLevel The nested level in the view resolution, this 
enables us to limit the
+ *depth of nested views.
--- End diff --

This is not being used, right? If so, could you update the comment? 


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

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



[GitHub] spark pull request #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-08 Thread jiangxb1987
Github user jiangxb1987 commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95098435
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.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.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.CatalystConf
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, 
View}
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+ * This file defines analysis rules related to views.
+ */
+
+/**
+ * Make sure that a view's child plan produces the view's output 
attributes. We wrap the child
+ * with a Project and add an alias for each output attribute. The 
attributes are resolved by
+ * name. This should be only done after the resolution batch, because the 
view attributes are
+ * not stable during resolution.
+ */
+case class AliasViewChild(conf: CatalystConf) extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan 
resolveOperators {
+case v @ View(_, output, child) if child.resolved =>
+  val resolver = conf.resolver
+  val newOutput = child.output.map { attr =>
+val newAttr = findAttributeByName(attr.name, output, resolver)
+// Check the dataType of the output attributes, throw an 
AnalysisException if they don't
+// match up.
+checkDataType(attr, newAttr)
+Alias(attr, attr.name)(exprId = newAttr.exprId, qualifier = 
newAttr.qualifier,
+  explicitMetadata = Some(newAttr.metadata))
+  }
+  v.copy(child = Project(newOutput, child))
+  }
+
+  /**
+   * Find the attribute that has the expected attribute name from an 
attribute list, the names
+   * are compared using conf.resolver.
+   * If the expected attribute is not found, throw an AnalysisException.
+   */
+  private def findAttributeByName(
+  name: String,
+  attrs: Seq[Attribute],
+  resolver: Resolver): Attribute = {
+attrs.collectFirst {
+  case attr if resolver(attr.name, name) => attr
+}.getOrElse(throw new AnalysisException(
+  s"Attribute with name '$name' is not found in " +
+s"'${attrs.map(_.name).mkString("(", ",", ")")}'"))
+  }
+
+  /**
+   * Check whether the dataType of `attr` could be casted to that of 
`other`, throw an
+   * AnalysisException if the both attributes don't match up.
+   */
+  private def checkDataType(attr: Attribute, other: Attribute): Unit = {
+if (!Cast.canCast(attr.dataType, other.dataType)) {
--- End diff --

I tested and realized the casting is needed. The dataType of child output 
attributes are deduced from the data, so it is possible that implict casting 
will be performed later.


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-08 Thread jiangxb1987
Github user jiangxb1987 commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95095659
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -767,7 +857,7 @@ class Analyzer(
 }
   }
 
- /**
+  /**
--- End diff --

Oops... Seems the format is wrong. :(


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

2017-01-08 Thread jiangxb1987
Github user jiangxb1987 commented on a diff in the pull request:

https://github.com/apache/spark/pull/16233#discussion_r95095587
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.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.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.CatalystConf
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, 
View}
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+ * This file defines analysis rules related to views.
+ */
+
+/**
+ * Make sure that a view's child plan produces the view's output 
attributes. We wrap the child
+ * with a Project and add an alias for each output attribute. The 
attributes are resolved by
+ * name. This should be only done after the resolution batch, because the 
view attributes are
+ * not stable during resolution.
+ */
+case class AliasViewChild(conf: CatalystConf) extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan 
resolveOperators {
+case v @ View(_, output, child) if child.resolved =>
+  val resolver = conf.resolver
+  val newOutput = child.output.map { attr =>
+val newAttr = findAttributeByName(attr.name, output, resolver)
+// Check the dataType of the output attributes, throw an 
AnalysisException if they don't
+// match up.
+checkDataType(attr, newAttr)
+Alias(attr, attr.name)(exprId = newAttr.exprId, qualifier = 
newAttr.qualifier,
+  explicitMetadata = Some(newAttr.metadata))
+  }
+  v.copy(child = Project(newOutput, child))
+  }
+
+  /**
+   * Find the attribute that has the expected attribute name from an 
attribute list, the names
+   * are compared using conf.resolver.
+   * If the expected attribute is not found, throw an AnalysisException.
+   */
+  private def findAttributeByName(
+  name: String,
+  attrs: Seq[Attribute],
+  resolver: Resolver): Attribute = {
+attrs.collectFirst {
+  case attr if resolver(attr.name, name) => attr
+}.getOrElse(throw new AnalysisException(
+  s"Attribute with name '$name' is not found in " +
+s"'${attrs.map(_.name).mkString("(", ",", ")")}'"))
+  }
+
+  /**
+   * Check whether the dataType of `attr` could be casted to that of 
`other`, throw an
+   * AnalysisException if the both attributes don't match up.
+   */
+  private def checkDataType(attr: Attribute, other: Attribute): Unit = {
+if (!Cast.canCast(attr.dataType, other.dataType)) {
--- End diff --

In fact I don't have a strong reason to do this. Let's just compare the 
equality of dataTypes.


---
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 #16233: [SPARK-18801][SQL] Support resolve a nested view

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

https://github.com/apache/spark/pull/16233#discussion_r95090120
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.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.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.CatalystConf
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, 
View}
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+ * This file defines analysis rules related to views.
+ */
+
+/**
+ * Make sure that a view's child plan produces the view's output 
attributes. We wrap the child
+ * with a Project and add an alias for each output attribute. The 
attributes are resolved by
+ * name. This should be only done after the resolution batch, because the 
view attributes are
+ * not stable during resolution.
+ */
+case class AliasViewChild(conf: CatalystConf) extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = plan 
resolveOperators {
+case v @ View(_, output, child) if child.resolved =>
+  val resolver = conf.resolver
+  val newOutput = child.output.map { attr =>
+val newAttr = findAttributeByName(attr.name, output, resolver)
+// Check the dataType of the output attributes, throw an 
AnalysisException if they don't
+// match up.
+checkDataType(attr, newAttr)
+Alias(attr, attr.name)(exprId = newAttr.exprId, qualifier = 
newAttr.qualifier,
+  explicitMetadata = Some(newAttr.metadata))
+  }
+  v.copy(child = Project(newOutput, child))
+  }
+
+  /**
+   * Find the attribute that has the expected attribute name from an 
attribute list, the names
+   * are compared using conf.resolver.
+   * If the expected attribute is not found, throw an AnalysisException.
+   */
+  private def findAttributeByName(
+  name: String,
+  attrs: Seq[Attribute],
+  resolver: Resolver): Attribute = {
+attrs.collectFirst {
+  case attr if resolver(attr.name, name) => attr
+}.getOrElse(throw new AnalysisException(
+  s"Attribute with name '$name' is not found in " +
+s"'${attrs.map(_.name).mkString("(", ",", ")")}'"))
+  }
+
+  /**
+   * Check whether the dataType of `attr` could be casted to that of 
`other`, throw an
+   * AnalysisException if the both attributes don't match up.
+   */
+  private def checkDataType(attr: Attribute, other: Attribute): Unit = {
+if (!Cast.canCast(attr.dataType, other.dataType)) {
--- End diff --

Casting seems a bit scary. Do you want to be this flexible?


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



  1   2   >