[GitHub] [spark] AngersZhuuuu commented on a change in pull request #29739: [SPARK-32867][SQL] When explain, HiveTableRelation show limited message

2020-09-17 Thread GitBox


AngersZh commented on a change in pull request #29739:
URL: https://github.com/apache/spark/pull/29739#discussion_r490138867



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala
##
@@ -693,4 +694,29 @@ case class HiveTableRelation(
   override def newInstance(): HiveTableRelation = copy(
 dataCols = dataCols.map(_.newInstance()),
 partitionCols = partitionCols.map(_.newInstance()))
+
+  override def simpleString(maxFields: Int): String = {
+val catalogTable = tableMeta.storage.serde match {
+  case Some(serde) => tableMeta.identifier :: serde :: Nil
+  case _ => tableMeta.identifier :: Nil
+}
+
+val metadataEntries = Map(
+  "CatalogTable" -> catalogTable.mkString(", "),
+  "Data Cols" -> truncatedString(dataCols, "[", ", ", "]", maxFields),
+  "Partition Cols" -> truncatedString(partitionCols, "[", ", ", "]", 
maxFields),
+  "Statistic" -> tableStats.map(_.simpleString).getOrElse(""),
+  "Pruned Partitions" -> prunedPartitions.map { partitions =>

Review comment:
   > maybe omit the partition serde if it's the same as table serde?
   
   Updated





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] AngersZhuuuu commented on a change in pull request #29739: [SPARK-32867][SQL] When explain, HiveTableRelation show limited message

2020-09-17 Thread GitBox


AngersZh commented on a change in pull request #29739:
URL: https://github.com/apache/spark/pull/29739#discussion_r490132577



##
File path: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala
##
@@ -187,6 +189,47 @@ class HiveTableScanSuite extends HiveComparisonTest with 
SQLTestUtils with TestH
 }
   }
 
+  test("SPARK-32867: When explain, HiveTableRelation show limited message") {
+withSQLConf(HiveUtils.CONVERT_METASTORE_ORC.key -> "false",

Review comment:
   > Is this config related to this test? Looks we can write tests without 
it though.
   
   Just a  habit about writing UT of hive table. Remove it, thanks for correct 
this.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] AngersZhuuuu commented on a change in pull request #29739: [SPARK-32867][SQL] When explain, HiveTableRelation show limited message

2020-09-17 Thread GitBox


AngersZh commented on a change in pull request #29739:
URL: https://github.com/apache/spark/pull/29739#discussion_r490049331



##
File path: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala
##
@@ -187,6 +189,47 @@ class HiveTableScanSuite extends HiveComparisonTest with 
SQLTestUtils with TestH
 }
   }
 
+  test("SPARK-32867: When explain, HiveTableRelation show limited message") {
+withSQLConf(HiveUtils.CONVERT_METASTORE_ORC.key -> "false",
+  "hive.exec.dynamic.partition.mode" -> "nonstrict") {
+  withTable("df") {
+spark.range(1000)
+  .select(col("id"), col("id").as("k"))
+  .write
+  .partitionBy("k")
+  .format("hive")
+  .mode("overwrite")
+  .saveAsTable("df")
+
+val scan1 = getHiveTableScanExec("SELECT * FROM df WHERE df.k < 3")
+assert(scan1.simpleString(100).replaceAll("#\\d+L", "") ==
+  "Scan hive default.df [id, k]," +
+" HiveTableRelation [" +
+"`default`.`df`," +
+" org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe," +
+" Data Cols: [id]," +
+" Partition Cols: [k]," +
+" Pruned Partitions: [k=0, k=1, k=2]," +
+" Statistic: sizeInBytes=8.0 EiB" +

Review comment:
   > Printing `8.0 EiB` looks weird and we need `Statistic` here?
   
   Not so important, IMO, we can remove





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] AngersZhuuuu commented on a change in pull request #29739: [SPARK-32867][SQL] When explain, HiveTableRelation show limited message

2020-09-17 Thread GitBox


AngersZh commented on a change in pull request #29739:
URL: https://github.com/apache/spark/pull/29739#discussion_r490049054



##
File path: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala
##
@@ -187,6 +189,47 @@ class HiveTableScanSuite extends HiveComparisonTest with 
SQLTestUtils with TestH
 }
   }
 
+  test("SPARK-32867: When explain, HiveTableRelation show limited message") {
+withSQLConf(HiveUtils.CONVERT_METASTORE_ORC.key -> "false",
+  "hive.exec.dynamic.partition.mode" -> "nonstrict") {
+  withTable("df") {
+spark.range(1000)

Review comment:
   > I think we don't need to make many partitions, so `1000` -> `4`?
   
   how about 30? since I want to show `` for HiveTableRelation..





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] AngersZhuuuu commented on a change in pull request #29739: [SPARK-32867][SQL] When explain, HiveTableRelation show limited message

2020-09-17 Thread GitBox


AngersZh commented on a change in pull request #29739:
URL: https://github.com/apache/spark/pull/29739#discussion_r490009980



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala
##
@@ -693,4 +694,29 @@ case class HiveTableRelation(
   override def newInstance(): HiveTableRelation = copy(
 dataCols = dataCols.map(_.newInstance()),
 partitionCols = partitionCols.map(_.newInstance()))
+
+  override def simpleString(maxFields: Int): String = {
+val catalogTable = tableMeta.storage.serde match {
+  case Some(serde) => tableMeta.identifier :: serde :: Nil
+  case _ => tableMeta.identifier :: Nil
+}
+
+val metadataEntries = Map(
+  "CatalogTable" -> catalogTable.mkString(", "),
+  "Data Cols" -> truncatedString(dataCols, "[", ", ", "]", maxFields),
+  "Partition Cols" -> truncatedString(partitionCols, "[", ", ", "]", 
maxFields),
+  "Statistic" -> tableStats.map(_.simpleString).getOrElse(""),
+  "Pruned Partitions" -> prunedPartitions.map { partitions =>

Review comment:
   > maybe omit the partition serde if it's the same as table serde?
   
   Strictly speaking, the field information for the partition may also be 
different





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] AngersZhuuuu commented on a change in pull request #29739: [SPARK-32867][SQL] When explain, HiveTableRelation show limited message

2020-09-17 Thread GitBox


AngersZh commented on a change in pull request #29739:
URL: https://github.com/apache/spark/pull/29739#discussion_r490008431



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala
##
@@ -693,4 +694,29 @@ case class HiveTableRelation(
   override def newInstance(): HiveTableRelation = copy(
 dataCols = dataCols.map(_.newInstance()),
 partitionCols = partitionCols.map(_.newInstance()))
+
+  override def simpleString(maxFields: Int): String = {
+val catalogTable = tableMeta.storage.serde match {
+  case Some(serde) => tableMeta.identifier :: serde :: Nil
+  case _ => tableMeta.identifier :: Nil
+}
+
+val metadataEntries = Map(
+  "CatalogTable" -> catalogTable.mkString(", "),
+  "Data Cols" -> truncatedString(dataCols, "[", ", ", "]", maxFields),
+  "Partition Cols" -> truncatedString(partitionCols, "[", ", ", "]", 
maxFields),
+  "Statistic" -> tableStats.map(_.simpleString).getOrElse(""),
+  "Pruned Partitions" -> prunedPartitions.map { partitions =>

Review comment:
   > maybe omit the partition serde if it's the same as table serde?
   
   Looks good to me. update later





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] AngersZhuuuu commented on a change in pull request #29739: [SPARK-32867][SQL] When explain, HiveTableRelation show limited message

2020-09-17 Thread GitBox


AngersZh commented on a change in pull request #29739:
URL: https://github.com/apache/spark/pull/29739#discussion_r490003795



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala
##
@@ -693,4 +694,29 @@ case class HiveTableRelation(
   override def newInstance(): HiveTableRelation = copy(
 dataCols = dataCols.map(_.newInstance()),
 partitionCols = partitionCols.map(_.newInstance()))
+
+  override def simpleString(maxFields: Int): String = {
+val catalogTable = tableMeta.storage.serde match {
+  case Some(serde) => tableMeta.identifier :: serde :: Nil
+  case _ => tableMeta.identifier :: Nil
+}
+
+val metadataEntries = Map(
+  "CatalogTable" -> catalogTable.mkString(", "),
+  "Data Cols" -> truncatedString(dataCols, "[", ", ", "]", maxFields),
+  "Partition Cols" -> truncatedString(partitionCols, "[", ", ", "]", 
maxFields),
+  "Statistic" -> tableStats.map(_.simpleString).getOrElse(""),
+  "Pruned Partitions" -> prunedPartitions.map { partitions =>

Review comment:
   > AFAIK, in Hive each partition can have a different serde. Shall we 
display partition serde like before?
   
   Since table level `serde` wan shown, show partition's `serde` is ok for me.
   
   But be honest, default meta value length is 100, one partition `serde` 
length always more than 30. It won't show too much info
   





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] AngersZhuuuu commented on a change in pull request #29739: [SPARK-32867][SQL] When explain, HiveTableRelation show limited message

2020-09-16 Thread GitBox


AngersZh commented on a change in pull request #29739:
URL: https://github.com/apache/spark/pull/29739#discussion_r489285126



##
File path: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala
##
@@ -187,6 +189,47 @@ class HiveTableScanSuite extends HiveComparisonTest with 
SQLTestUtils with TestH
 }
   }
 
+  test("SPARK-32867: When explain, HiveTableRelation show limited message") {
+withSQLConf(HiveUtils.CONVERT_METASTORE_ORC.key -> "false",
+  "hive.exec.dynamic.partition.mode" -> "nonstrict") {
+  withTable("df") {
+spark.range(1000)
+  .select(col("id"), col("id").as("k"))
+  .write
+  .partitionBy("k")
+  .format("hive")
+  .mode("overwrite")
+  .saveAsTable("df")
+
+val scan1 = getHiveTableScanExec("SELECT * FROM df WHERE df.k < 3")
+assert(scan1.simpleString(100).replaceAll("#\\d+L", "") ==
+  "Scan hive default.df [id, k]," +
+" HiveTableRelation [" +
+"`default`.`df`," +

Review comment:
   > let's leave it, as it's a trivial problem.
   
   Ok.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] AngersZhuuuu commented on a change in pull request #29739: [SPARK-32867][SQL] When explain, HiveTableRelation show limited message

2020-09-16 Thread GitBox


AngersZh commented on a change in pull request #29739:
URL: https://github.com/apache/spark/pull/29739#discussion_r489273824



##
File path: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala
##
@@ -187,6 +189,47 @@ class HiveTableScanSuite extends HiveComparisonTest with 
SQLTestUtils with TestH
 }
   }
 
+  test("SPARK-32867: When explain, HiveTableRelation show limited message") {
+withSQLConf(HiveUtils.CONVERT_METASTORE_ORC.key -> "false",
+  "hive.exec.dynamic.partition.mode" -> "nonstrict") {
+  withTable("df") {
+spark.range(1000)
+  .select(col("id"), col("id").as("k"))
+  .write
+  .partitionBy("k")
+  .format("hive")
+  .mode("overwrite")
+  .saveAsTable("df")
+
+val scan1 = getHiveTableScanExec("SELECT * FROM df WHERE df.k < 3")
+assert(scan1.simpleString(100).replaceAll("#\\d+L", "") ==
+  "Scan hive default.df [id, k]," +
+" HiveTableRelation [" +
+"`default`.`df`," +

Review comment:
   > This seems duplicated to `Scan hive default.df`
   
   Yea, in origin plan, also have this problem. we can't drop this in 
`HiveTableRelation`  since we need to show table in Optimized plan and  
analyzed plan. 
   
   How about  implement HiveTableScanExec's `simpleString` to filter  
HiveTableRelation's table info?





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] AngersZhuuuu commented on a change in pull request #29739: [SPARK-32867][SQL] When explain, HiveTableRelation show limited message

2020-09-16 Thread GitBox


AngersZh commented on a change in pull request #29739:
URL: https://github.com/apache/spark/pull/29739#discussion_r489202822



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala
##
@@ -693,4 +694,29 @@ case class HiveTableRelation(
   override def newInstance(): HiveTableRelation = copy(
 dataCols = dataCols.map(_.newInstance()),
 partitionCols = partitionCols.map(_.newInstance()))
+
+  override def simpleString(maxFields: Int): String = {
+val catalogTable = tableMeta.storage.serde match {
+  case Some(serde) => tableMeta.identifier :: serde :: Nil
+  case _ => tableMeta.identifier :: Nil
+}
+
+val metadataEntries = Map(
+  "CatalogTable" -> catalogTable.mkString(", "),

Review comment:
   > is it also how file source tables display table info?
   
   File source tables won't show this.
   ```
   == Parsed Logical Plan ==
   'Project [*]
   +- 'Filter ('df.k < 3)
  +- 'UnresolvedRelation [df], []
   
   == Analyzed Logical Plan ==
   id: bigint, k: bigint
   Project [id#7L, k#8L]
   +- Filter (k#8L < cast(3 as bigint))
  +- SubqueryAlias spark_catalog.default.df
 +- Relation[id#7L,k#8L] parquet
   
   == Optimized Logical Plan ==
   Filter (isnotnull(k#8L) AND (k#8L < 3))
   +- Relation[id#7L,k#8L] parquet
   
   == Physical Plan ==
   *(1) ColumnarToRow
   +- FileScan parquet default.df[id#7L,k#8L] Batched: true, DataFilters: [], 
Format: Parquet, Location: 
InMemoryFileIndex[file:/Users/angerszhu/Documents/project/AngersZhu/spark/sql/hive/target/tmp/hiv...,
 PartitionFilters: [isnotnull(k#8L), (k#8L < 3)], PushedFilters: [], 
ReadSchema: struct
   
   ```





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] AngersZhuuuu commented on a change in pull request #29739: [SPARK-32867][SQL] When explain, HiveTableRelation show limited message

2020-09-15 Thread GitBox


AngersZh commented on a change in pull request #29739:
URL: https://github.com/apache/spark/pull/29739#discussion_r489126362



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala
##
@@ -693,4 +694,29 @@ case class HiveTableRelation(
   override def newInstance(): HiveTableRelation = copy(
 dataCols = dataCols.map(_.newInstance()),
 partitionCols = partitionCols.map(_.newInstance()))
+
+  override def simpleString(maxFields: Int): String = {

Review comment:
   > The explain output of `HiveTableScanExec` does not have the same issue?
   
   No, only caused. by `HiveTableRelation`, Optimized Plan also show too many 
information too.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] AngersZhuuuu commented on a change in pull request #29739: [SPARK-32867][SQL] When explain, HiveTableRelation show limited message

2020-09-15 Thread GitBox


AngersZh commented on a change in pull request #29739:
URL: https://github.com/apache/spark/pull/29739#discussion_r489126362



##
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala
##
@@ -693,4 +694,29 @@ case class HiveTableRelation(
   override def newInstance(): HiveTableRelation = copy(
 dataCols = dataCols.map(_.newInstance()),
 partitionCols = partitionCols.map(_.newInstance()))
+
+  override def simpleString(maxFields: Int): String = {

Review comment:
   > The explain output of `HiveTableScanExec` does not have the same issue?
   
   No, only caused. by `HiveTableRelation`





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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



[GitHub] [spark] AngersZhuuuu commented on a change in pull request #29739: [SPARK-32867][SQL] When explain, HiveTableRelation show limited message

2020-09-14 Thread GitBox


AngersZh commented on a change in pull request #29739:
URL: https://github.com/apache/spark/pull/29739#discussion_r488340103



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala
##
@@ -55,7 +55,7 @@ trait DataSourceScanExec extends LeafExecNode {
   // Metadata that describes more details of this scan.
   protected def metadata: Map[String, String]
 
-  protected val maxMetadataValueLength = 100
+  protected val maxMetadataValueLength = conf.maxMetadataValueLength

Review comment:
   > Have you checked the related PR? 
https://github.com/apache/spark/pull/29688/files#diff-2a91a9a59953aa82fa132aaf45bd731bR58
   
   Sorry, don't notice that pr. I have similar ideal like that pr too.  I'm 
going to restore this change and deal with the two PR conflicts  finally.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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