[GitHub] spark pull request #16430: [SPARK-17077] [SQL] Cardinality estimation for pr...

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

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


---
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 #16430: [SPARK-17077] [SQL] Cardinality estimation for pr...

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

https://github.com/apache/spark/pull/16430#discussion_r95098331
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/ProjectEstimation.scala
 ---
@@ -0,0 +1,44 @@
+/*
+ * 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.plans.logical.statsEstimation
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, 
AttributeMap}
+import org.apache.spark.sql.catalyst.plans.logical.{Project, Statistics}
+
+object ProjectEstimation {
+  import EstimationUtils._
+
+  def estimate(project: Project): Option[Statistics] = {
+if (rowCountsExist(project.child)) {
+  val childStats = project.child.statistics
+  val inputAttrStats = childStats.attributeStats
+  // Match alias with its child's column stat
+  val aliasStats = project.expressions.collect {
+case alias @ Alias(attr: Attribute, _) if 
inputAttrStats.contains(attr) =>
--- End diff --

In the long run, we should define a `statistics` interface in `Expression`, 
so that we can propagate the column stats more naturally, for more cases(not 
only `Alias`, but also `Add`, `Mod`, etc.). But currently catalyst doesn't 
propagate attributes correctly, e.g. 
https://issues.apache.org/jira/browse/SPARK-17995 (`Union`, `Except`, etc. has 
the same problem), we may need to hack a lot of places to propagate column 
stats correctly.

According to @wzhfy 's benchmark, it turns out we can speed up most of the 
cases if we take care of `Alias`, so I'm ok with the current approach.


---
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 #16430: [SPARK-17077] [SQL] Cardinality estimation for pr...

2017-01-06 Thread wzhfy
Github user wzhfy commented on a diff in the pull request:

https://github.com/apache/spark/pull/16430#discussion_r95048331
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/ProjectEstimationSuite.scala
 ---
@@ -0,0 +1,51 @@
+/*
+ * 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.statsEstimation
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeMap, 
AttributeReference}
+import org.apache.spark.sql.catalyst.plans.logical._
+import 
org.apache.spark.sql.catalyst.plans.logical.statsEstimation.EstimationUtils._
+import org.apache.spark.sql.types.IntegerType
+
+
+class ProjectEstimationSuite extends StatsEstimationTestBase {
+
+  test("estimate project with alias") {
+val ar1 = AttributeReference("key1", IntegerType)()
+val ar2 = AttributeReference("key2", IntegerType)()
+val colStat1 = ColumnStat(2, Some(1), Some(2), 0, 4, 4)
+val colStat2 = ColumnStat(1, Some(10), Some(10), 0, 4, 4)
+
+val child = StatsTestPlan(
+  outputList = Seq(ar1, ar2),
+  stats = Statistics(
+sizeInBytes = 2 * (4 + 4),
+rowCount = Some(2),
+attributeStats = AttributeMap(Seq(ar1 -> colStat1, ar2 -> 
colStat2
+
+val project = Project(Seq(ar1, Alias(ar2, "abc")()), child)
+val expectedColStats = Seq("key1" -> colStat1, "abc" -> colStat2)
+val expectedAttrStats = toAttributeMap(expectedColStats, project)
+// The number of rows won't change for project.
+val expectedStats = Statistics(
+  sizeInBytes = 2 * getRowSize(project.output, expectedAttrStats),
--- End diff --

I tested getRowSize for int type. But yes, we should have a separate test 
for this method.


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

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



[GitHub] spark pull request #16430: [SPARK-17077] [SQL] Cardinality estimation for pr...

2017-01-06 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/16430#discussion_r95040736
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/ProjectEstimation.scala
 ---
@@ -0,0 +1,44 @@
+/*
+ * 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.plans.logical.statsEstimation
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, 
AttributeMap}
+import org.apache.spark.sql.catalyst.plans.logical.{Project, Statistics}
+
+object ProjectEstimation {
+  import EstimationUtils._
+
+  def estimate(project: Project): Option[Statistics] = {
+if (rowCountsExist(project.child)) {
+  val childStats = project.child.statistics
+  val inputAttrStats = childStats.attributeStats
+  // Match alias with its child's column stat
+  val aliasStats = project.expressions.collect {
+case alias @ Alias(attr: Attribute, _) if 
inputAttrStats.contains(attr) =>
--- End diff --

my question from before was really whether we need to match on other things 
as well (that are not just Alias - e.g. can an attribute be other 
NamedExpression?)

cc @cloud-fan 


---
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 #16430: [SPARK-17077] [SQL] Cardinality estimation for pr...

2017-01-06 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/16430#discussion_r95040478
  
--- Diff: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/ProjectEstimationSuite.scala
 ---
@@ -0,0 +1,51 @@
+/*
+ * 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.statsEstimation
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeMap, 
AttributeReference}
+import org.apache.spark.sql.catalyst.plans.logical._
+import 
org.apache.spark.sql.catalyst.plans.logical.statsEstimation.EstimationUtils._
+import org.apache.spark.sql.types.IntegerType
+
+
+class ProjectEstimationSuite extends StatsEstimationTestBase {
+
+  test("estimate project with alias") {
+val ar1 = AttributeReference("key1", IntegerType)()
+val ar2 = AttributeReference("key2", IntegerType)()
+val colStat1 = ColumnStat(2, Some(1), Some(2), 0, 4, 4)
+val colStat2 = ColumnStat(1, Some(10), Some(10), 0, 4, 4)
+
+val child = StatsTestPlan(
+  outputList = Seq(ar1, ar2),
+  stats = Statistics(
+sizeInBytes = 2 * (4 + 4),
+rowCount = Some(2),
+attributeStats = AttributeMap(Seq(ar1 -> colStat1, ar2 -> 
colStat2
+
+val project = Project(Seq(ar1, Alias(ar2, "abc")()), child)
+val expectedColStats = Seq("key1" -> colStat1, "abc" -> colStat2)
+val expectedAttrStats = toAttributeMap(expectedColStats, project)
+// The number of rows won't change for project.
+val expectedStats = Statistics(
+  sizeInBytes = 2 * getRowSize(project.output, expectedAttrStats),
--- End diff --

the way this test is written getRowSize is completely untested. We can 
almost change getRowSize to always return 0 and all the tests would pass. Can 
you have test cases covering 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 #16430: [SPARK-17077] [SQL] Cardinality estimation for pr...

2017-01-05 Thread wzhfy
Github user wzhfy commented on a diff in the pull request:

https://github.com/apache/spark/pull/16430#discussion_r94909141
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/estimation/ProjectEstimation.scala
 ---
@@ -0,0 +1,44 @@
+/*
+ * 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.plans.logical.estimation
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, 
AttributeMap}
+import org.apache.spark.sql.catalyst.plans.logical.{Project, Statistics}
+
+object ProjectEstimation {
+  import EstimationUtils._
+
+  def estimate(project: Project): Option[Statistics] = {
+if (rowCountsExist(project.child)) {
+  val childStats = project.child.statistics
+  val inputAttrStats = childStats.attributeStats
+  // Match alias with its child's column stat
+  val aliasStats = project.expressions.collect {
+case alias@Alias(attr: Attribute, _) if 
inputAttrStats.contains(attr) =>
--- End diff --

I extract `attr: Attribute` because inputAttrStats is a AttributeMap and 
only accepts Attribute


---
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 #16430: [SPARK-17077] [SQL] Cardinality estimation for pr...

2017-01-05 Thread wzhfy
Github user wzhfy commented on a diff in the pull request:

https://github.com/apache/spark/pull/16430#discussion_r94906707
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/estimation/ProjectEstimation.scala
 ---
@@ -0,0 +1,44 @@
+/*
+ * 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.plans.logical.estimation
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, 
AttributeMap}
+import org.apache.spark.sql.catalyst.plans.logical.{Project, Statistics}
+
+object ProjectEstimation {
+  import EstimationUtils._
+
+  def estimate(project: Project): Option[Statistics] = {
+if (rowCountsExist(project.child)) {
+  val childStats = project.child.statistics
+  val inputAttrStats = childStats.attributeStats
+  // Match alias with its child's column stat
+  val aliasStats = project.expressions.collect {
+case alias@Alias(attr: Attribute, _) if 
inputAttrStats.contains(attr) =>
--- End diff --

We don't need to deal with AttributeReference here, we can get it directly 
from 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 #16430: [SPARK-17077] [SQL] Cardinality estimation for pr...

2017-01-05 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/16430#discussion_r94905442
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/estimation/EstimationSuite.scala 
---
@@ -0,0 +1,81 @@
+/*
+ * 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.estimation
+
+import org.apache.spark.sql.QueryTest
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap}
+import org.apache.spark.sql.catalyst.plans.logical._
+import 
org.apache.spark.sql.catalyst.plans.logical.estimation.EstimationUtils._
+import org.apache.spark.sql.test.SharedSQLContext
+
+
+/**
+ * End-to-end suite testing statistics estimation for logical operators.
+ */
+class EstimationSuite extends QueryTest with SharedSQLContext {
+  import testImplicits._
+
+  /** Table info: all estimation tests are conducted on these tables. */
+  private val estimationTestData = Seq(
+("estimation_test1", Seq("key11", "key12"), Seq[(Int, Int)]((1, 10), 
(2, 10
+
+  /** Original column stats */
+  val colStatForKey11 = ColumnStat(2, Some(1), Some(2), 0, 4, 4)
+  val colStatForKey12 = ColumnStat(1, Some(10), Some(10), 0, 4, 4)
+
+  override def beforeAll(): Unit = {
--- End diff --

(the way you can fix this is to create a leaf logical plan node with 
statistics you can pass in)


---
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 #16430: [SPARK-17077] [SQL] Cardinality estimation for pr...

2017-01-05 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/16430#discussion_r94903216
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/estimation/EstimationSuite.scala 
---
@@ -0,0 +1,81 @@
+/*
+ * 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.estimation
+
+import org.apache.spark.sql.QueryTest
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap}
+import org.apache.spark.sql.catalyst.plans.logical._
+import 
org.apache.spark.sql.catalyst.plans.logical.estimation.EstimationUtils._
+import org.apache.spark.sql.test.SharedSQLContext
+
+
+/**
+ * End-to-end suite testing statistics estimation for logical operators.
+ */
+class EstimationSuite extends QueryTest with SharedSQLContext {
--- End diff --

rename this ProjectEstimationSuite?


---
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 #16430: [SPARK-17077] [SQL] Cardinality estimation for pr...

2017-01-05 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/16430#discussion_r94903188
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/estimation/EstimationSuite.scala 
---
@@ -0,0 +1,81 @@
+/*
+ * 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.estimation
+
+import org.apache.spark.sql.QueryTest
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap}
+import org.apache.spark.sql.catalyst.plans.logical._
+import 
org.apache.spark.sql.catalyst.plans.logical.estimation.EstimationUtils._
+import org.apache.spark.sql.test.SharedSQLContext
+
+
+/**
+ * End-to-end suite testing statistics estimation for logical operators.
+ */
+class EstimationSuite extends QueryTest with SharedSQLContext {
+  import testImplicits._
+
+  /** Table info: all estimation tests are conducted on these tables. */
+  private val estimationTestData = Seq(
+("estimation_test1", Seq("key11", "key12"), Seq[(Int, Int)]((1, 10), 
(2, 10
+
+  /** Original column stats */
+  val colStatForKey11 = ColumnStat(2, Some(1), Some(2), 0, 4, 4)
+  val colStatForKey12 = ColumnStat(1, Some(10), Some(10), 0, 4, 4)
+
+  override def beforeAll(): Unit = {
--- End diff --

Basically it would be great to make this actually a unit test suite, rather 
than an end-to-end test suite.



---
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 #16430: [SPARK-17077] [SQL] Cardinality estimation for pr...

2017-01-05 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/16430#discussion_r94902185
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/estimation/ProjectEstimation.scala
 ---
@@ -0,0 +1,44 @@
+/*
+ * 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.plans.logical.estimation
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, 
AttributeMap}
+import org.apache.spark.sql.catalyst.plans.logical.{Project, Statistics}
+
+object ProjectEstimation {
+  import EstimationUtils._
+
+  def estimate(project: Project): Option[Statistics] = {
+if (rowCountsExist(project.child)) {
+  val childStats = project.child.statistics
+  val inputAttrStats = childStats.attributeStats
+  // Match alias with its child's column stat
+  val aliasStats = project.expressions.collect {
+case alias@Alias(attr: Attribute, _) if 
inputAttrStats.contains(attr) =>
--- End diff --

also is it possible that we are seeing other NamedExpression like 
AttriuteReference 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 #16430: [SPARK-17077] [SQL] Cardinality estimation for pr...

2017-01-05 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/16430#discussion_r94902161
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/estimation/ProjectEstimation.scala
 ---
@@ -0,0 +1,44 @@
+/*
+ * 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.plans.logical.estimation
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, 
AttributeMap}
+import org.apache.spark.sql.catalyst.plans.logical.{Project, Statistics}
+
+object ProjectEstimation {
+  import EstimationUtils._
+
+  def estimate(project: Project): Option[Statistics] = {
+if (rowCountsExist(project.child)) {
+  val childStats = project.child.statistics
+  val inputAttrStats = childStats.attributeStats
+  // Match alias with its child's column stat
+  val aliasStats = project.expressions.collect {
+case alias@Alias(attr: Attribute, _) if 
inputAttrStats.contains(attr) =>
--- End diff --

```
case a: Alias if inputAttrStats.contains(a.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 #16430: [SPARK-17077] [SQL] Cardinality estimation for pr...

2017-01-05 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/16430#discussion_r94902084
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/estimation/EstimationUtils.scala
 ---
@@ -0,0 +1,54 @@
+/*
+ * 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.plans.logical.estimation
+
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap}
+import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, 
LogicalPlan}
+import org.apache.spark.sql.types.StringType
+
+
+object EstimationUtils {
+
+  /** Check if each plan has rowCount in its statistics. */
+  def rowCountsExist(plans: LogicalPlan*): Boolean =
+plans.forall(_.statistics.rowCount.isDefined)
+
+  /** Get column stats for output attributes. */
+  def getOutputMap(inputMap: AttributeMap[ColumnStat], output: 
Seq[Attribute])
+: AttributeMap[ColumnStat] = {
+AttributeMap(output.flatMap(a => inputMap.get(a).map(a -> _)))
+  }
+
+  def getRowSize(attributes: Seq[Attribute], attrStats: 
AttributeMap[ColumnStat]): Long = {
+// We assign a generic overhead for a Row object, the actual overhead 
is different for different
+// Row format.
+8 + attributes.map { attr =>
--- End diff --

we can update to use this algorithm in a separate pr. we can merge this pr 
if we fix the issue with test.



---
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 #16430: [SPARK-17077] [SQL] Cardinality estimation for pr...

2017-01-05 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/16430#discussion_r94902048
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/estimation/EstimationUtils.scala
 ---
@@ -0,0 +1,54 @@
+/*
+ * 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.plans.logical.estimation
+
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap}
+import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, 
LogicalPlan}
+import org.apache.spark.sql.types.StringType
+
+
+object EstimationUtils {
+
+  /** Check if each plan has rowCount in its statistics. */
+  def rowCountsExist(plans: LogicalPlan*): Boolean =
+plans.forall(_.statistics.rowCount.isDefined)
+
+  /** Get column stats for output attributes. */
+  def getOutputMap(inputMap: AttributeMap[ColumnStat], output: 
Seq[Attribute])
+: AttributeMap[ColumnStat] = {
+AttributeMap(output.flatMap(a => inputMap.get(a).map(a -> _)))
+  }
+
+  def getRowSize(attributes: Seq[Attribute], attrStats: 
AttributeMap[ColumnStat]): Long = {
+// We assign a generic overhead for a Row object, the actual overhead 
is different for different
+// Row format.
+8 + attributes.map { attr =>
--- End diff --

so rather than estimating like this, can we get the data size of the child 
node, and use that to estimate the data size of the parent?

for fixed length types, we know the size; for variable length types, we 
assume the size is evenly distributed.

e.g. if the total length is 1000, and we have rowcount 10, and we have 3 
fields: a int, b long, c string

then we assume the avg length per row is 100, and the avg length of c would 
be 100 - 4 - 8 = 88?



---
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 #16430: [SPARK-17077] [SQL] Cardinality estimation for pr...

2017-01-05 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/16430#discussion_r94901694
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/estimation/EstimationSuite.scala 
---
@@ -0,0 +1,81 @@
+/*
+ * 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.estimation
+
+import org.apache.spark.sql.QueryTest
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap}
+import org.apache.spark.sql.catalyst.plans.logical._
+import 
org.apache.spark.sql.catalyst.plans.logical.estimation.EstimationUtils._
+import org.apache.spark.sql.test.SharedSQLContext
+
+
+/**
+ * End-to-end suite testing statistics estimation for logical operators.
+ */
+class EstimationSuite extends QueryTest with SharedSQLContext {
+  import testImplicits._
+
+  /** Table info: all estimation tests are conducted on these tables. */
+  private val estimationTestData = Seq(
+("estimation_test1", Seq("key11", "key12"), Seq[(Int, Int)]((1, 10), 
(2, 10
+
+  /** Original column stats */
+  val colStatForKey11 = ColumnStat(2, Some(1), Some(2), 0, 4, 4)
+  val colStatForKey12 = ColumnStat(1, Some(10), Some(10), 0, 4, 4)
+
+  override def beforeAll(): Unit = {
--- End diff --

in order to write a unit test, can we create a logical plan node with a 
some fake statistics that's passed in? that way we don't need everything end to 
end and can even put this in the catalyst package.



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

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