[GitHub] spark pull request #14957: [SPARK-4502][SQL]Support parquet nested struct pr...

2017-06-26 Thread xuanyuanking
Github user xuanyuanking closed the pull request at:

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


---
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 #14957: [SPARK-4502][SQL]Support parquet nested struct pr...

2016-10-30 Thread xuanyuanking
Github user xuanyuanking commented on a diff in the pull request:

https://github.com/apache/spark/pull/14957#discussion_r85656841
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala
 ---
@@ -442,6 +443,79 @@ class FileSourceStrategySuite extends QueryTest with 
SharedSQLContext with Predi
 }
   }
 
+  test("[SPARK-4502] pruning nested schema by projects correctly") {
+val testFunc = 
PrivateMethod[Seq[StructField]]('generateStructFieldsContainsNesting)
+// Construct fullSchema like below:
+//root
+//|-- col: struct (nullable = true)
+//||-- s1: struct (nullable = true)
+//|||-- s1_1: long (nullable = true)
+//|||-- s1_2: long (nullable = true)
+//||-- str: string (nullable = true)
+//||-- info_list: array (nullable = true)
+//|||-- element: struct (containsNull = true)
+//||||-- s1: struct (nullable = true)
+//|||||-- s1_1: long (nullable = true)
+//|||||-- s1_2: long (nullable = true)
+//|-- num: long (nullable = true)
+//|-- str: string (nullable = true)
+val nested_s1 = StructField("s1",
+  StructType(
+Seq(
+  StructField("s1_1", LongType, true),
+  StructField("s1_2", LongType, true)
+)
+  ), true)
+val flat_str = StructField("str", StringType, true)
+val nested_arr = StructField("info_list", 
ArrayType(StructType(Seq(nested_s1))), true)
+
+val fullSchema = StructType(
+  Seq(
+StructField("col", StructType(Seq(nested_s1, flat_str, 
nested_arr)), true),
+StructField("num", LongType, true),
+flat_str
+  ))
+
+// Attr of struct col
+val colAttr = AttributeReference("col", StructType(
+  Seq(nested_s1, flat_str, nested_arr)), true)()
+// Child expression of col.s1.s1_1
+val childExp = GetStructField(
+  GetStructField(colAttr, 0, Some("s1")), 0, Some("s1_1"))
+// Child expression of col.info_list[0].s1.s1_1
+val arrayChildExp = GetStructField(
+  GetStructField(
+GetArrayItem(
+  GetStructField(colAttr, 0, Some("info_list")),
+  Literal(0)
+), 0, Some("s1")
+  ), 0, Some("s1_1")
+)
+// Project list of "select num, col.s1.s1_1 as s1_1, 
col.info_list[0].s1.s1_1 as complex_get"
+val projects = Seq(
+  AttributeReference("num", LongType, true)(),
+  Alias(childExp, "s1_1")(),
+  Alias(arrayChildExp, "complex_get")()
+  )
+val expextResult =
+  Seq(
+StructField("num", LongType, true),
+StructField("col", StructType(
+  Seq(
+StructField(
+  "s1",
+  StructType(Seq(StructField("s1_1", LongType, true))),
+  true)
+  )
+), true),
+StructField("col", StructType(Seq(nested_arr)))
+  )
+// Call the function generateStructFieldsContainsNesting
+val result = 
FileSourceStrategy.invokePrivate[Seq[StructField]](testFunc(projects,
+  fullSchema))
+assert(result == expextResult)
+  }
--- End diff --

fix done. Thanks for liancheng's remind.
Here I considered the CreateStruct(Unsafe) and CreateNamedStruct(Unsafe), 
other expressions in complexTypeCreator(CreateArray, CreateMap) just ignore.


---
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 #14957: [SPARK-4502][SQL]Support parquet nested struct pr...

2016-10-30 Thread xuanyuanking
Github user xuanyuanking commented on a diff in the pull request:

https://github.com/apache/spark/pull/14957#discussion_r85656729
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala
 ---
@@ -126,4 +136,52 @@ object FileSourceStrategy extends Strategy with 
Logging {
 
 case _ => Nil
   }
+
+  private def generateStructFieldsContainsNesting(projects: 
Seq[Expression],
+  totalSchema: StructType) : 
Seq[StructField] = {
+def generateStructField(curField: List[String],
+ node: Expression) : Seq[StructField] = {
+  node match {
+case ai: GetArrayItem =>
+  // Here we drop the previous for simplify array and map support.
+  // Same strategy in GetArrayStructFields and GetMapValue
+  generateStructField(List.empty[String], ai.child)
+case asf: GetArrayStructFields =>
+  generateStructField(List.empty[String], asf.child)
+case mv: GetMapValue =>
+  generateStructField(List.empty[String], mv.child)
+case attr: AttributeReference =>
+  Seq(getFieldRecursively(totalSchema, attr.name :: curField))
+case sf: GetStructField =>
+  generateStructField(sf.name.get :: curField, sf.child)
--- End diff --

fix done.
a little question here, all projects parse from sql must have the name 
while projects from dataframe api may not   , 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 #14957: [SPARK-4502][SQL]Support parquet nested struct pr...

2016-10-28 Thread liancheng
Github user liancheng commented on a diff in the pull request:

https://github.com/apache/spark/pull/14957#discussion_r85611295
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala
 ---
@@ -126,4 +140,59 @@ object FileSourceStrategy extends Strategy with 
Logging {
 
 case _ => Nil
   }
+
+  private def generateStructFieldsContainsNesting(
--- End diff --

You may make this method `private[sql]` so that you don't need to rely on 
the `PrivateMethod` ScalaTest trick to test 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 #14957: [SPARK-4502][SQL]Support parquet nested struct pr...

2016-10-28 Thread liancheng
Github user liancheng commented on a diff in the pull request:

https://github.com/apache/spark/pull/14957#discussion_r85610578
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala
 ---
@@ -97,7 +99,19 @@ object FileSourceStrategy extends Strategy with Logging {
 dataColumns
   .filter(requiredAttributes.contains)
   .filterNot(partitionColumns.contains)
-  val outputSchema = readDataColumns.toStructType
+  val outputSchema = if (
+  fsRelation.sqlContext.conf.parquetNestedColumnPruningEnabled &&
+  fsRelation.fileFormat.isInstanceOf[ParquetFileFormat]
--- End diff --

Use two space indentation 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 #14957: [SPARK-4502][SQL]Support parquet nested struct pr...

2016-10-28 Thread liancheng
Github user liancheng commented on a diff in the pull request:

https://github.com/apache/spark/pull/14957#discussion_r84562093
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala
 ---
@@ -126,4 +136,52 @@ object FileSourceStrategy extends Strategy with 
Logging {
 
 case _ => Nil
   }
+
+  private def generateStructFieldsContainsNesting(projects: 
Seq[Expression],
+  totalSchema: StructType) : 
Seq[StructField] = {
+def generateStructField(curField: List[String],
+ node: Expression) : Seq[StructField] = {
+  node match {
+case ai: GetArrayItem =>
+  // Here we drop the previous for simplify array and map support.
+  // Same strategy in GetArrayStructFields and GetMapValue
+  generateStructField(List.empty[String], ai.child)
+case asf: GetArrayStructFields =>
+  generateStructField(List.empty[String], asf.child)
+case mv: GetMapValue =>
+  generateStructField(List.empty[String], mv.child)
+case attr: AttributeReference =>
+  Seq(getFieldRecursively(totalSchema, attr.name :: curField))
+case sf: GetStructField =>
+  generateStructField(sf.name.get :: curField, sf.child)
--- End diff --

This name is optional and might not be set. We should retrieve the actual 
field name using the ordinal of `sf`.


---
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 #14957: [SPARK-4502][SQL]Support parquet nested struct pr...

2016-10-23 Thread xuanyuanking
Github user xuanyuanking commented on a diff in the pull request:

https://github.com/apache/spark/pull/14957#discussion_r84592883
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
@@ -661,6 +666,8 @@ private[sql] class SQLConf extends Serializable with 
CatalystConf with Logging {
 
   def isParquetINT96AsTimestamp: Boolean = 
getConf(PARQUET_INT96_AS_TIMESTAMP)
 
+  def isParquetNestColumnPruning: Boolean = 
getConf(PARQUET_NEST_COLUMN_PRUNING)
--- End diff --

rename done


---
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 #14957: [SPARK-4502][SQL]Support parquet nested struct pr...

2016-10-23 Thread xuanyuanking
Github user xuanyuanking commented on a diff in the pull request:

https://github.com/apache/spark/pull/14957#discussion_r84592876
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
@@ -212,6 +212,11 @@ object SQLConf {
 .booleanConf
 .createWithDefault(true)
 
+  val PARQUET_NEST_COLUMN_PRUNING = 
SQLConfigBuilder("spark.sql.parquet.nestColumnPruning")
--- End diff --

rename done


---
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 #14957: [SPARK-4502][SQL]Support parquet nested struct pr...

2016-10-23 Thread xuanyuanking
Github user xuanyuanking commented on a diff in the pull request:

https://github.com/apache/spark/pull/14957#discussion_r84592888
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala
 ---
@@ -571,6 +571,37 @@ class ParquetQuerySuite extends QueryTest with 
ParquetTest with SharedSQLContext
 }
   }
 
+  test("SPARK-4502 parquet nested fields pruning") {
+// Schema of "test-data/nested-array-struct.parquet":
+//root
+//|-- col: struct (nullable = true)
+//||-- s1: struct (nullable = true)
+//|||-- s1_1: long (nullable = true)
+//|||-- s1_2: long (nullable = true)
+//||-- str: string (nullable = true)
+//|-- num: long (nullable = true)
+//|-- str: string (nullable = true)
+val df = 
readResourceParquetFile("test-data/nested-struct.snappy.parquet")
+df.createOrReplaceTempView("tmp_table")
--- End diff --

fix done


---
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 #14957: [SPARK-4502][SQL]Support parquet nested struct pr...

2016-10-23 Thread xuanyuanking
Github user xuanyuanking commented on a diff in the pull request:

https://github.com/apache/spark/pull/14957#discussion_r84592865
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala
 ---
@@ -126,4 +136,52 @@ object FileSourceStrategy extends Strategy with 
Logging {
 
 case _ => Nil
   }
+
+  private def generateStructFieldsContainsNesting(projects: 
Seq[Expression],
+  totalSchema: StructType) : 
Seq[StructField] = {
+def generateStructField(curField: List[String],
+ node: Expression) : Seq[StructField] = {
+  node match {
+case ai: GetArrayItem =>
+  // Here we drop the previous for simplify array and map support.
+  // Same strategy in GetArrayStructFields and GetMapValue
+  generateStructField(List.empty[String], ai.child)
+case asf: GetArrayStructFields =>
+  generateStructField(List.empty[String], asf.child)
+case mv: GetMapValue =>
+  generateStructField(List.empty[String], mv.child)
+case attr: AttributeReference =>
+  Seq(getFieldRecursively(totalSchema, attr.name :: curField))
+case sf: GetStructField =>
+  generateStructField(sf.name.get :: curField, sf.child)
+case _ =>
+  if (node.children.nonEmpty) {
+node.children.flatMap(child => generateStructField(curField, 
child))
+  } else {
+Seq.empty[StructField]
+  }
+  }
+}
+
+def getFieldRecursively(totalSchema: StructType,
+name: List[String]): StructField = {
+  if (name.length > 1) {
+val curField = name.head
+val curFieldType = totalSchema(curField)
+curFieldType.dataType match {
+  case st: StructType =>
+val newField = getFieldRecursively(StructType(st.fields), 
name.drop(1))
+StructField(curFieldType.name, StructType(Seq(newField)),
+  curFieldType.nullable, curFieldType.metadata)
+  case _ =>
+throw new IllegalArgumentException(s"""Field "$curField" is 
not struct field.""")
+}
+  } else {
+totalSchema(name.head)
+  }
+}
--- End diff --

The func getFieldRecursively here need the return value which is a 
StructField contains all nested relation in path. For example:
The fullSchema is:
```
root
 |-- col: struct (nullable = true)
 ||-- s1: struct (nullable = true)
 |||-- s1_1: long (nullable = true)
 |||-- s1_2: long (nullable = true)
 ||-- str: string (nullable = true)
 |-- num: long (nullable = true)
 |-- str: string (nullable = true)
```
the func should return:
```

StructField(col,StructType(StructField(s1,StructType(StructField(s1_1,LongType,true)),true)),true)
```
So maybe I can't use the simplified func getnestedField because it returns 
only the last StructField:
```
StructField(s1_1,LongType,true)
```




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

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



[GitHub] spark pull request #14957: [SPARK-4502][SQL]Support parquet nested struct pr...

2016-10-23 Thread xuanyuanking
Github user xuanyuanking commented on a diff in the pull request:

https://github.com/apache/spark/pull/14957#discussion_r84592821
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala
 ---
@@ -126,4 +136,52 @@ object FileSourceStrategy extends Strategy with 
Logging {
 
 case _ => Nil
   }
+
+  private def generateStructFieldsContainsNesting(projects: 
Seq[Expression],
+  totalSchema: StructType) : 
Seq[StructField] = {
+def generateStructField(curField: List[String],
+ node: Expression) : Seq[StructField] = {
+  node match {
+case ai: GetArrayItem =>
+  // Here we drop the previous for simplify array and map support.
+  // Same strategy in GetArrayStructFields and GetMapValue
+  generateStructField(List.empty[String], ai.child)
+case asf: GetArrayStructFields =>
+  generateStructField(List.empty[String], asf.child)
+case mv: GetMapValue =>
+  generateStructField(List.empty[String], mv.child)
+case attr: AttributeReference =>
+  Seq(getFieldRecursively(totalSchema, attr.name :: curField))
+case sf: GetStructField =>
+  generateStructField(sf.name.get :: curField, sf.child)
+case _ =>
+  if (node.children.nonEmpty) {
+node.children.flatMap(child => generateStructField(curField, 
child))
+  } else {
+Seq.empty[StructField]
+  }
+  }
+}
+
+def getFieldRecursively(totalSchema: StructType,
+name: List[String]): StructField = {
--- End diff --

fix done


---
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 #14957: [SPARK-4502][SQL]Support parquet nested struct pr...

2016-10-23 Thread xuanyuanking
Github user xuanyuanking commented on a diff in the pull request:

https://github.com/apache/spark/pull/14957#discussion_r84592816
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala
 ---
@@ -126,4 +136,52 @@ object FileSourceStrategy extends Strategy with 
Logging {
 
 case _ => Nil
   }
+
+  private def generateStructFieldsContainsNesting(projects: 
Seq[Expression],
+  totalSchema: StructType) : 
Seq[StructField] = {
--- End diff --

fix code style done.
No problem, I'll add tests for the private func 
generateStructFieldsContainsNesting next patch, this patch fix all code style 
and naming problem.


---
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 #14957: [SPARK-4502][SQL]Support parquet nested struct pr...

2016-10-23 Thread xuanyuanking
Github user xuanyuanking commented on a diff in the pull request:

https://github.com/apache/spark/pull/14957#discussion_r84592818
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala
 ---
@@ -126,4 +136,52 @@ object FileSourceStrategy extends Strategy with 
Logging {
 
 case _ => Nil
   }
+
+  private def generateStructFieldsContainsNesting(projects: 
Seq[Expression],
+  totalSchema: StructType) : 
Seq[StructField] = {
+def generateStructField(curField: List[String],
+ node: Expression) : Seq[StructField] = {
--- End diff --

fix done


---
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 #14957: [SPARK-4502][SQL]Support parquet nested struct pr...

2016-10-23 Thread xuanyuanking
Github user xuanyuanking commented on a diff in the pull request:

https://github.com/apache/spark/pull/14957#discussion_r84592806
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala
 ---
@@ -97,7 +99,15 @@ object FileSourceStrategy extends Strategy with Logging {
 dataColumns
   .filter(requiredAttributes.contains)
   .filterNot(partitionColumns.contains)
-  val outputSchema = readDataColumns.toStructType
+  val outputSchema = if 
(fsRelation.sqlContext.conf.isParquetNestColumnPruning
+&& fsRelation.fileFormat.isInstanceOf[ParquetFileFormat]) {
+val totalSchema = readDataColumns.toStructType
+val prunedSchema = StructType(
+  generateStructFieldsContainsNesting(projects, totalSchema))
+// Merge schema in same StructType and merge with filterAttributes
+prunedSchema.fields.map(f => StructType(Array(f))).reduceLeft(_ 
merge _)
+  .merge(filterAttributes.toSeq.toStructType)
+  } else readDataColumns.toStructType
--- End diff --

fix done


---
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 #14957: [SPARK-4502][SQL]Support parquet nested struct pr...

2016-10-23 Thread xuanyuanking
Github user xuanyuanking commented on a diff in the pull request:

https://github.com/apache/spark/pull/14957#discussion_r84592805
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala
 ---
@@ -97,7 +99,15 @@ object FileSourceStrategy extends Strategy with Logging {
 dataColumns
   .filter(requiredAttributes.contains)
   .filterNot(partitionColumns.contains)
-  val outputSchema = readDataColumns.toStructType
+  val outputSchema = if 
(fsRelation.sqlContext.conf.isParquetNestColumnPruning
+&& fsRelation.fileFormat.isInstanceOf[ParquetFileFormat]) {
+val totalSchema = readDataColumns.toStructType
--- End diff --

fix done


---
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 #14957: [SPARK-4502][SQL]Support parquet nested struct pr...

2016-10-21 Thread liancheng
Github user liancheng commented on a diff in the pull request:

https://github.com/apache/spark/pull/14957#discussion_r84422346
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala
 ---
@@ -126,4 +136,52 @@ object FileSourceStrategy extends Strategy with 
Logging {
 
 case _ => Nil
   }
+
+  private def generateStructFieldsContainsNesting(projects: 
Seq[Expression],
+  totalSchema: StructType) : 
Seq[StructField] = {
--- End diff --

Please check [Spark code style guide][1] and re-format this one.

[1]: 
https://cwiki.apache.org/confluence/display/SPARK/Spark+Code+Style+Guide


---
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 #14957: [SPARK-4502][SQL]Support parquet nested struct pr...

2016-10-21 Thread liancheng
Github user liancheng commented on a diff in the pull request:

https://github.com/apache/spark/pull/14957#discussion_r84422485
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
@@ -212,6 +212,11 @@ object SQLConf {
 .booleanConf
 .createWithDefault(true)
 
+  val PARQUET_NEST_COLUMN_PRUNING = 
SQLConfigBuilder("spark.sql.parquet.nestColumnPruning")
+.doc("When set this to true, we will tell parquet only read the nest 
column`s leaf fields ")
--- End diff --

Please reword this doc string to:

> When true, Parquet column pruning also works for nested fields.


---
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 #14957: [SPARK-4502][SQL]Support parquet nested struct pr...

2016-10-21 Thread liancheng
Github user liancheng commented on a diff in the pull request:

https://github.com/apache/spark/pull/14957#discussion_r84422636
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
@@ -661,6 +666,8 @@ private[sql] class SQLConf extends Serializable with 
CatalystConf with Logging {
 
   def isParquetINT96AsTimestamp: Boolean = 
getConf(PARQUET_INT96_AS_TIMESTAMP)
 
+  def isParquetNestColumnPruning: Boolean = 
getConf(PARQUET_NEST_COLUMN_PRUNING)
--- End diff --

`parquetNestedColumnPruningEnabled`


---
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 #14957: [SPARK-4502][SQL]Support parquet nested struct pr...

2016-10-21 Thread liancheng
Github user liancheng commented on a diff in the pull request:

https://github.com/apache/spark/pull/14957#discussion_r84558190
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala
 ---
@@ -126,4 +136,52 @@ object FileSourceStrategy extends Strategy with 
Logging {
 
 case _ => Nil
   }
+
+  private def generateStructFieldsContainsNesting(projects: 
Seq[Expression],
+  totalSchema: StructType) : 
Seq[StructField] = {
--- End diff --

Would you please add comments and test cases for testing this method, which 
is basically the essential part of this PR?


---
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 #14957: [SPARK-4502][SQL]Support parquet nested struct pr...

2016-10-21 Thread liancheng
Github user liancheng commented on a diff in the pull request:

https://github.com/apache/spark/pull/14957#discussion_r84422606
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
@@ -212,6 +212,11 @@ object SQLConf {
 .booleanConf
 .createWithDefault(true)
 
+  val PARQUET_NEST_COLUMN_PRUNING = 
SQLConfigBuilder("spark.sql.parquet.nestColumnPruning")
--- End diff --

Please rename to `PARQUET_NESTED_COLUMN_PRUNING` and 
`spark.sql.parquet.nestedColumnPruning` respectively.


---
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 #14957: [SPARK-4502][SQL]Support parquet nested struct pr...

2016-10-21 Thread liancheng
Github user liancheng commented on a diff in the pull request:

https://github.com/apache/spark/pull/14957#discussion_r84422353
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala
 ---
@@ -126,4 +136,52 @@ object FileSourceStrategy extends Strategy with 
Logging {
 
 case _ => Nil
   }
+
+  private def generateStructFieldsContainsNesting(projects: 
Seq[Expression],
+  totalSchema: StructType) : 
Seq[StructField] = {
+def generateStructField(curField: List[String],
+ node: Expression) : Seq[StructField] = {
--- End diff --

And this one.


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

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



[GitHub] spark pull request #14957: [SPARK-4502][SQL]Support parquet nested struct pr...

2016-10-21 Thread liancheng
Github user liancheng commented on a diff in the pull request:

https://github.com/apache/spark/pull/14957#discussion_r84406104
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala
 ---
@@ -97,7 +99,15 @@ object FileSourceStrategy extends Strategy with Logging {
 dataColumns
   .filter(requiredAttributes.contains)
   .filterNot(partitionColumns.contains)
-  val outputSchema = readDataColumns.toStructType
+  val outputSchema = if 
(fsRelation.sqlContext.conf.isParquetNestColumnPruning
+&& fsRelation.fileFormat.isInstanceOf[ParquetFileFormat]) {
+val totalSchema = readDataColumns.toStructType
+val prunedSchema = StructType(
+  generateStructFieldsContainsNesting(projects, totalSchema))
+// Merge schema in same StructType and merge with filterAttributes
+prunedSchema.fields.map(f => StructType(Array(f))).reduceLeft(_ 
merge _)
+  .merge(filterAttributes.toSeq.toStructType)
+  } else readDataColumns.toStructType
--- End diff --

Please re-format the above change to the following format:

```scala
if (
  ... &&
  ...
) {
  ...
} else {
  ...
}
```


---
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 #14957: [SPARK-4502][SQL]Support parquet nested struct pr...

2016-10-21 Thread liancheng
Github user liancheng commented on a diff in the pull request:

https://github.com/apache/spark/pull/14957#discussion_r84559521
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala
 ---
@@ -126,4 +136,52 @@ object FileSourceStrategy extends Strategy with 
Logging {
 
 case _ => Nil
   }
+
+  private def generateStructFieldsContainsNesting(projects: 
Seq[Expression],
+  totalSchema: StructType) : 
Seq[StructField] = {
+def generateStructField(curField: List[String],
+ node: Expression) : Seq[StructField] = {
+  node match {
+case ai: GetArrayItem =>
+  // Here we drop the previous for simplify array and map support.
+  // Same strategy in GetArrayStructFields and GetMapValue
+  generateStructField(List.empty[String], ai.child)
+case asf: GetArrayStructFields =>
+  generateStructField(List.empty[String], asf.child)
+case mv: GetMapValue =>
+  generateStructField(List.empty[String], mv.child)
+case attr: AttributeReference =>
+  Seq(getFieldRecursively(totalSchema, attr.name :: curField))
+case sf: GetStructField =>
+  generateStructField(sf.name.get :: curField, sf.child)
+case _ =>
+  if (node.children.nonEmpty) {
+node.children.flatMap(child => generateStructField(curField, 
child))
+  } else {
+Seq.empty[StructField]
+  }
+  }
+}
+
+def getFieldRecursively(totalSchema: StructType,
+name: List[String]): StructField = {
+  if (name.length > 1) {
+val curField = name.head
+val curFieldType = totalSchema(curField)
+curFieldType.dataType match {
+  case st: StructType =>
+val newField = getFieldRecursively(StructType(st.fields), 
name.drop(1))
+StructField(curFieldType.name, StructType(Seq(newField)),
+  curFieldType.nullable, curFieldType.metadata)
+  case _ =>
+throw new IllegalArgumentException(s"""Field "$curField" is 
not struct field.""")
+}
+  } else {
+totalSchema(name.head)
+  }
+}
--- End diff --

This function can be simplified to:

```scala
def getNestedField(schema: StructType, path: Seq[String]): StructField = {
  require(path.nonEmpty, "")

  path.tail.foldLeft(schema(path.head)) { (field, name) =>
field.dataType match {
  case t: StructType => t(name)
  case _ => ??? // Throw exception 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 #14957: [SPARK-4502][SQL]Support parquet nested struct pr...

2016-10-21 Thread liancheng
Github user liancheng commented on a diff in the pull request:

https://github.com/apache/spark/pull/14957#discussion_r84436528
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala
 ---
@@ -97,7 +99,15 @@ object FileSourceStrategy extends Strategy with Logging {
 dataColumns
   .filter(requiredAttributes.contains)
   .filterNot(partitionColumns.contains)
-  val outputSchema = readDataColumns.toStructType
+  val outputSchema = if 
(fsRelation.sqlContext.conf.isParquetNestColumnPruning
+&& fsRelation.fileFormat.isInstanceOf[ParquetFileFormat]) {
+val totalSchema = readDataColumns.toStructType
--- End diff --

Maybe `fullSchema`?


---
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 #14957: [SPARK-4502][SQL]Support parquet nested struct pr...

2016-10-21 Thread liancheng
Github user liancheng commented on a diff in the pull request:

https://github.com/apache/spark/pull/14957#discussion_r84422762
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala
 ---
@@ -571,6 +571,37 @@ class ParquetQuerySuite extends QueryTest with 
ParquetTest with SharedSQLContext
 }
   }
 
+  test("SPARK-4502 parquet nested fields pruning") {
+// Schema of "test-data/nested-array-struct.parquet":
+//root
+//|-- col: struct (nullable = true)
+//||-- s1: struct (nullable = true)
+//|||-- s1_1: long (nullable = true)
+//|||-- s1_2: long (nullable = true)
+//||-- str: string (nullable = true)
+//|-- num: long (nullable = true)
+//|-- str: string (nullable = true)
+val df = 
readResourceParquetFile("test-data/nested-struct.snappy.parquet")
+df.createOrReplaceTempView("tmp_table")
--- End diff --

You may use `SQLTestUtils.withTempView` to wrap this test so that you don't 
need to drop the temporary view manually.


---
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 #14957: [SPARK-4502][SQL]Support parquet nested struct pr...

2016-10-21 Thread liancheng
Github user liancheng commented on a diff in the pull request:

https://github.com/apache/spark/pull/14957#discussion_r84422376
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala
 ---
@@ -126,4 +136,52 @@ object FileSourceStrategy extends Strategy with 
Logging {
 
 case _ => Nil
   }
+
+  private def generateStructFieldsContainsNesting(projects: 
Seq[Expression],
+  totalSchema: StructType) : 
Seq[StructField] = {
+def generateStructField(curField: List[String],
+ node: Expression) : Seq[StructField] = {
+  node match {
+case ai: GetArrayItem =>
+  // Here we drop the previous for simplify array and map support.
+  // Same strategy in GetArrayStructFields and GetMapValue
+  generateStructField(List.empty[String], ai.child)
+case asf: GetArrayStructFields =>
+  generateStructField(List.empty[String], asf.child)
+case mv: GetMapValue =>
+  generateStructField(List.empty[String], mv.child)
+case attr: AttributeReference =>
+  Seq(getFieldRecursively(totalSchema, attr.name :: curField))
+case sf: GetStructField =>
+  generateStructField(sf.name.get :: curField, sf.child)
+case _ =>
+  if (node.children.nonEmpty) {
+node.children.flatMap(child => generateStructField(curField, 
child))
+  } else {
+Seq.empty[StructField]
+  }
+  }
+}
+
+def getFieldRecursively(totalSchema: StructType,
+name: List[String]): StructField = {
--- End diff --

And 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 #14957: [SPARK-4502][SQL]Support parquet nested struct pr...

2016-09-07 Thread xuanyuanking
Github user xuanyuanking commented on a diff in the pull request:

https://github.com/apache/spark/pull/14957#discussion_r77934764
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala
 ---
@@ -97,7 +98,16 @@ object FileSourceStrategy extends Strategy with Logging {
 dataColumns
   .filter(requiredAttributes.contains)
   .filterNot(partitionColumns.contains)
-  val outputSchema = readDataColumns.toStructType
+  val outputSchema = if 
(fsRelation.sqlContext.conf.isParquetNestColumnPruning) {
--- End diff --

I ran the command 
```
./build/sbt "test-only org.apache.spark.sql.execution.datasources.*"
```
locally and three test suits failed
```
[error] Failed tests:
[error] org.apache.spark.sql.execution.datasources.csv.CSVSuite
[error] org.apache.spark.sql.execution.datasources.json.JsonSuite
[error] 
org.apache.spark.sql.execution.datasources.parquet.ParquetPartitionDiscoverySuite
```
But I run them separately, all three pass. Also I run 
```
./build/sbt "test-only org.apache.spark.sql.execution.csv.*"
./build/sbt "test-only org.apache.spark.sql.execution.json.*"
```
all tests pass.


---
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 #14957: [SPARK-4502][SQL]Support parquet nested struct pr...

2016-09-07 Thread xuanyuanking
Github user xuanyuanking commented on a diff in the pull request:

https://github.com/apache/spark/pull/14957#discussion_r77934557
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala
 ---
@@ -97,7 +98,16 @@ object FileSourceStrategy extends Strategy with Logging {
 dataColumns
   .filter(requiredAttributes.contains)
   .filterNot(partitionColumns.contains)
-  val outputSchema = readDataColumns.toStructType
+  val outputSchema = if 
(fsRelation.sqlContext.conf.isParquetNestColumnPruning) {
--- End diff --

It's my mistake here, I can only make sure this patch work for parquet,so I 
should check the fileFormat here, also like the config 
namespace(`spark.sql.parquet.nestColumnPruning`), it can only work for parquet. 
I add a patch to fix 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 #14957: [SPARK-4502][SQL]Support parquet nested struct pr...

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

https://github.com/apache/spark/pull/14957#discussion_r77849862
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala
 ---
@@ -97,7 +98,16 @@ object FileSourceStrategy extends Strategy with Logging {
 dataColumns
   .filter(requiredAttributes.contains)
   .filterNot(partitionColumns.contains)
-  val outputSchema = readDataColumns.toStructType
+  val outputSchema = if 
(fsRelation.sqlContext.conf.isParquetNestColumnPruning) {
--- End diff --

Also, do you mind if I ask which tests were failed? I will try to reproduce 
by myself.


---
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 #14957: [SPARK-4502][SQL]Support parquet nested struct pr...

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

https://github.com/apache/spark/pull/14957#discussion_r77849592
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala
 ---
@@ -97,7 +98,16 @@ object FileSourceStrategy extends Strategy with Logging {
 dataColumns
   .filter(requiredAttributes.contains)
   .filterNot(partitionColumns.contains)
-  val outputSchema = readDataColumns.toStructType
+  val outputSchema = if 
(fsRelation.sqlContext.conf.isParquetNestColumnPruning) {
--- End diff --

Oh, I meant enabling/disabling affects the other data sources. I see it's 
disabled by default. I ran ,for example, `JsonSuite` after enabling this option 
(after leaving the comment above) and saw some failures related with nested 
structures.


---
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 #14957: [SPARK-4502][SQL]Support parquet nested struct pr...

2016-09-07 Thread xuanyuanking
Github user xuanyuanking commented on a diff in the pull request:

https://github.com/apache/spark/pull/14957#discussion_r77845789
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala
 ---
@@ -97,7 +98,16 @@ object FileSourceStrategy extends Strategy with Logging {
 dataColumns
   .filter(requiredAttributes.contains)
   .filterNot(partitionColumns.contains)
-  val outputSchema = readDataColumns.toStructType
+  val outputSchema = if 
(fsRelation.sqlContext.conf.isParquetNestColumnPruning) {
--- End diff --

All three tests failed because a datetime check error, correct answer is 
like '2015-05-23' but the spark answer is '2015-05-22', I don't think this 
error is made by my patch.
Do anybody have the same problem before? it really confusing me and running 
test suit seperate it will pass!


---
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 #14957: [SPARK-4502][SQL]Support parquet nested struct pr...

2016-09-07 Thread xuanyuanking
Github user xuanyuanking commented on a diff in the pull request:

https://github.com/apache/spark/pull/14957#discussion_r77844084
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala
 ---
@@ -97,7 +98,16 @@ object FileSourceStrategy extends Strategy with Logging {
 dataColumns
   .filter(requiredAttributes.contains)
   .filterNot(partitionColumns.contains)
-  val outputSchema = readDataColumns.toStructType
+  val outputSchema = if 
(fsRelation.sqlContext.conf.isParquetNestColumnPruning) {
--- End diff --

I run all datasource tests by 
`./build/sbt "test-only org.apache.spark.sql.execution.datasources.*"`
three test failed, but run them seperately, all tests can passwd


---
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 #14957: [SPARK-4502][SQL]Support parquet nested struct pr...

2016-09-06 Thread xuanyuanking
Github user xuanyuanking commented on a diff in the pull request:

https://github.com/apache/spark/pull/14957#discussion_r77762907
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala ---
@@ -280,6 +280,29 @@ case class StructType(fields: Array[StructField]) 
extends DataType with Seq[Stru
   }
 
   /**
+   * Extracts the [[StructField]] with the given name recursively.
+   *
+   * @throws IllegalArgumentException if the parent field's type is not 
StructType
+   */
+  def getFieldRecursively(name: String): StructField = {
--- End diff --

I think there's another way to solve this problem, maybe generate the final 
structType in FileSourceStrategy better.I'll try it and give another pr 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 #14957: [SPARK-4502][SQL]Support parquet nested struct pr...

2016-09-06 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/14957#discussion_r77762149
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala
 ---
@@ -571,6 +571,44 @@ class ParquetQuerySuite extends QueryTest with 
ParquetTest with SharedSQLContext
 }
   }
 
+  test("SPARK-4502 parquet nested fields pruning") {
+// Schema of "test-data/nested-array-struct.parquet":
+//root
+//|-- primitive: integer (nullable = true)
+//|-- myComplex: array (nullable = true)
+//||-- element: struct (containsNull = true)
+//|||-- id: integer (nullable = true)
+//|||-- repeatedMessage: array (nullable = true)
+//||||-- element: struct (containsNull = true)
+//|||||-- someId: integer (nullable = true)
+val df = 
readResourceParquetFile("test-data/nested-array-struct.parquet")
--- End diff --

Ah, I missed. Sorry.


---
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 #14957: [SPARK-4502][SQL]Support parquet nested struct pr...

2016-09-06 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/14957#discussion_r77761381
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala ---
@@ -280,6 +280,29 @@ case class StructType(fields: Array[StructField]) 
extends DataType with Seq[Stru
   }
 
   /**
+   * Extracts the [[StructField]] with the given name recursively.
+   *
+   * @throws IllegalArgumentException if the parent field's type is not 
StructType
+   */
+  def getFieldRecursively(name: String): StructField = {
--- End diff --

I think I understood how it works. My point is, this is a Parquet-specific 
problem not related with Catalyst module. I don't see any reason that this 
method should be exposed.

I believe we can do this not by modifying the column names (not even for a 
temporary use).



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

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



[GitHub] spark pull request #14957: [SPARK-4502][SQL]Support parquet nested struct pr...

2016-09-06 Thread xuanyuanking
Github user xuanyuanking commented on a diff in the pull request:

https://github.com/apache/spark/pull/14957#discussion_r77760397
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala ---
@@ -280,6 +280,29 @@ case class StructType(fields: Array[StructField]) 
extends DataType with Seq[Stru
   }
 
   /**
+   * Extracts the [[StructField]] with the given name recursively.
+   *
+   * @throws IllegalArgumentException if the parent field's type is not 
StructType
+   */
+  def getFieldRecursively(name: String): StructField = {
--- End diff --

The mark of nested fields is one kind of tmp data, finally it will convert 
to a pruned StructType and pass to 
`org.apache.spark.sql.parquet.row.requested_schema`


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

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



[GitHub] spark pull request #14957: [SPARK-4502][SQL]Support parquet nested struct pr...

2016-09-06 Thread xuanyuanking
Github user xuanyuanking commented on a diff in the pull request:

https://github.com/apache/spark/pull/14957#discussion_r77760264
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala
 ---
@@ -571,6 +571,44 @@ class ParquetQuerySuite extends QueryTest with 
ParquetTest with SharedSQLContext
 }
   }
 
+  test("SPARK-4502 parquet nested fields pruning") {
+// Schema of "test-data/nested-array-struct.parquet":
+//root
+//|-- primitive: integer (nullable = true)
+//|-- myComplex: array (nullable = true)
+//||-- element: struct (containsNull = true)
+//|||-- id: integer (nullable = true)
+//|||-- repeatedMessage: array (nullable = true)
+//||||-- element: struct (containsNull = true)
+//|||||-- someId: integer (nullable = true)
+val df = 
readResourceParquetFile("test-data/nested-array-struct.parquet")
--- End diff --


https://github.com/apache/spark/blob/master/sql/core/src/test/resources/test-data/nested-array-struct.parquet
I reuse this file to test nested struct in paruqet, this file in 
sql/core/src/test/resources/test-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 #14957: [SPARK-4502][SQL]Support parquet nested struct pr...

2016-09-06 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/14957#discussion_r77757859
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala
 ---
@@ -571,6 +571,44 @@ class ParquetQuerySuite extends QueryTest with 
ParquetTest with SharedSQLContext
 }
   }
 
+  test("SPARK-4502 parquet nested fields pruning") {
+// Schema of "test-data/nested-array-struct.parquet":
+//root
+//|-- primitive: integer (nullable = true)
+//|-- myComplex: array (nullable = true)
+//||-- element: struct (containsNull = true)
+//|||-- id: integer (nullable = true)
+//|||-- repeatedMessage: array (nullable = true)
+//||||-- element: struct (containsNull = true)
+//|||||-- someId: integer (nullable = true)
+val df = 
readResourceParquetFile("test-data/nested-array-struct.parquet")
+df.createOrReplaceTempView("tmp_table")
+// normal test
+val query1 = "select primitive,myComplex[0].id from tmp_table"
+val result1 = sql(query1)
+withSQLConf(SQLConf.PARQUET_NEST_COLUMN_PRUNING.key -> "true") {
+  checkAnswer(sql(query1), result1)
--- End diff --

Does this really test if the nested fields are pruned? I think this test 
will pass regardless of the newly added option.


---
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 #14957: [SPARK-4502][SQL]Support parquet nested struct pr...

2016-09-06 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/14957#discussion_r77757667
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala ---
@@ -280,6 +280,29 @@ case class StructType(fields: Array[StructField]) 
extends DataType with Seq[Stru
   }
 
   /**
+   * Extracts the [[StructField]] with the given name recursively.
+   *
+   * @throws IllegalArgumentException if the parent field's type is not 
StructType
+   */
+  def getFieldRecursively(name: String): StructField = {
--- End diff --

Isn't this Parquet-specific problem? I wonder adding this method is 
appropriate.

Also, I am not too sure if it is appropriate to mark nested fields by 
modifying field names with a character.


---
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 #14957: [SPARK-4502][SQL]Support parquet nested struct pr...

2016-09-06 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/14957#discussion_r77757611
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala
 ---
@@ -571,6 +571,44 @@ class ParquetQuerySuite extends QueryTest with 
ParquetTest with SharedSQLContext
 }
   }
 
+  test("SPARK-4502 parquet nested fields pruning") {
+// Schema of "test-data/nested-array-struct.parquet":
+//root
+//|-- primitive: integer (nullable = true)
+//|-- myComplex: array (nullable = true)
+//||-- element: struct (containsNull = true)
+//|||-- id: integer (nullable = true)
+//|||-- repeatedMessage: array (nullable = true)
+//||||-- element: struct (containsNull = true)
+//|||||-- someId: integer (nullable = true)
+val df = 
readResourceParquetFile("test-data/nested-array-struct.parquet")
--- End diff --

It seems we don't have this file in this PR. So running tests will fail.


---
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 #14957: [SPARK-4502][SQL]Support parquet nested struct pr...

2016-09-06 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/14957#discussion_r77757552
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala
 ---
@@ -97,7 +98,16 @@ object FileSourceStrategy extends Strategy with Logging {
 dataColumns
   .filter(requiredAttributes.contains)
   .filterNot(partitionColumns.contains)
-  val outputSchema = readDataColumns.toStructType
+  val outputSchema = if 
(fsRelation.sqlContext.conf.isParquetNestColumnPruning) {
--- End diff --

It will affect all other data sources. I am pretty sure any tests related 
with this will pass.


---
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 #14957: [SPARK-4502][SQL]Support parquet nested struct pr...

2016-09-06 Thread xuanyuanking
Github user xuanyuanking commented on a diff in the pull request:

https://github.com/apache/spark/pull/14957#discussion_r77753006
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala ---
@@ -259,8 +259,23 @@ case class StructType(fields: Array[StructField]) 
extends DataType with Seq[Stru
* @throws IllegalArgumentException if a field with the given name does 
not exist
*/
   def apply(name: String): StructField = {
-nameToField.getOrElse(name,
-  throw new IllegalArgumentException(s"""Field "$name" does not 
exist."""))
+if (name.contains('.')) {
--- End diff --

@HyukjinKwon Thanks for your review, mix the recursively get with the 
default apply has this problem, I fixed it in next patch and use ',' which is a 
invalid character in Parquet schema


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

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



[GitHub] spark pull request #14957: [SPARK-4502][SQL]Support parquet nested struct pr...

2016-09-05 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/14957#discussion_r77561307
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala ---
@@ -259,8 +259,23 @@ case class StructType(fields: Array[StructField]) 
extends DataType with Seq[Stru
* @throws IllegalArgumentException if a field with the given name does 
not exist
*/
   def apply(name: String): StructField = {
-nameToField.getOrElse(name,
-  throw new IllegalArgumentException(s"""Field "$name" does not 
exist."""))
+if (name.contains('.')) {
--- End diff --

IIUU, this will drop the support to access the field name containing `.` 
(e.g. `"a.b"`) which can be accessed via `` "`a.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 #14957: [SPARK-4502][SQL]Support parquet nested struct pr...

2016-09-05 Thread xuanyuanking
GitHub user xuanyuanking opened a pull request:

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

[SPARK-4502][SQL]Support parquet nested struct pruning and add releva…

## What changes were proposed in this pull request?

Like the description in  
[SPARK-4502](https://issues.apache.org/jira/browse/SPARK-4502), we have the 
same problem in Baidu and our user's parquet file has complex nested parquet 
struct(400+ fields and 4 layer nested) so this problem brings unnecessary data 
read and time spend. This pr fixed the problem and main fix ideas list as 
follows:

1.  Add booleanConf `spark.sql.parquet.nestColumnPruning`, when it’s 
closed, same logical with before

2.  In `FileSourceStrategy`, traverse `projects[NamedExpression]` and 
generate a map which key is attributeName, value is a Seq[String] of 
corresponding nested fields.
For example:  [“people” -> (“people.age”, “people.addr.city”)] 

3. Replace the attributeName in origin requiredColumns.
For example:  origin requiredColumns is [“people”, “consume”], 
replace it to [“people.age”, “people.addr.city”, “consume”]

4. Merge structType of fields in same structType and merge filter attributes
For example:  the json format of struct type [“people.addr.city”, 
“people.addr.zip_code”] will merge to [“people.addr.[city,zip_code]”]

5.  `StructType.apply` read the columns contains “.” Recursively


## How was this patch tested?

add new test in `ParquetQuerySuite`

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

$ git pull https://github.com/xuanyuanking/spark SPARK-4502

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

https://github.com/apache/spark/pull/14957.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #14957


commit 7eaa4287a5c112a192cca388863324ffd855203e
Author: liyuanjian 
Date:   2016-09-05T08:27:02Z

[SPARK-4502][SQL]Support parquet nested struct pruning and add relevant 
tests




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

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