[GitHub] [spark] cloud-fan commented on a change in pull request #29045: [SPARK-32234][SQL] Spark sql commands are failing on selecting the orc tables

2020-07-15 Thread GitBox


cloud-fan commented on a change in pull request #29045:
URL: https://github.com/apache/spark/pull/29045#discussion_r455190828



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala
##
@@ -112,25 +112,26 @@ case class OrcPartitionReaderFactory(
   override def buildColumnarReader(file: PartitionedFile): 
PartitionReader[ColumnarBatch] = {
 val conf = broadcastedConf.value.value
 
-val resultSchemaString = OrcUtils.orcTypeDescriptionString(resultSchema)
-OrcConf.MAPRED_INPUT_SCHEMA.setString(conf, resultSchemaString)
 OrcConf.IS_SCHEMA_EVOLUTION_CASE_SENSITIVE.setBoolean(conf, 
isCaseSensitive)
 
 val filePath = new Path(new URI(file.filePath))
 
 val fs = filePath.getFileSystem(conf)
 val readerOptions = OrcFile.readerOptions(conf).filesystem(fs)
-val requestedColIdsOrEmptyFile =
+val resultedColPruneInfo =
   Utils.tryWithResource(OrcFile.createReader(filePath, readerOptions)) { 
reader =>
 OrcUtils.requestedColumnIds(
   isCaseSensitive, dataSchema, readDataSchema, reader, conf)
   }
 
-if (requestedColIdsOrEmptyFile.isEmpty) {
+if (resultedColPruneInfo.isEmpty) {
   new EmptyPartitionReader
 } else {
-  val requestedColIds = requestedColIdsOrEmptyFile.get ++ 
Array.fill(partitionSchema.length)(-1)
-  assert(requestedColIds.length == resultSchema.length,
+  val (requestedColIds, canPruneCols) = resultedColPruneInfo.get
+  val resultSchemaString = OrcUtils.orcResultSchemaString(canPruneCols,
+dataSchema, resultSchema, partitionSchema, conf)
+  val requestedDataColIds = requestedColIds ++ 
Array.fill(partitionSchema.length)(-1)

Review comment:
   I think we should switch the name. Here we add the partition column IDs 
and better to call it `requestedColIds`. The former one can be called 
`requestedDataColIds` as it doesn't contain partition columns.





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] cloud-fan commented on a change in pull request #29045: [SPARK-32234][SQL] Spark sql commands are failing on selecting the orc tables

2020-07-15 Thread GitBox


cloud-fan commented on a change in pull request #29045:
URL: https://github.com/apache/spark/pull/29045#discussion_r455043130



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala
##
@@ -66,24 +66,28 @@ case class OrcPartitionReaderFactory(
   override def buildReader(file: PartitionedFile): 
PartitionReader[InternalRow] = {
 val conf = broadcastedConf.value.value
 
-val resultSchemaString = OrcUtils.orcTypeDescriptionString(resultSchema)
-OrcConf.MAPRED_INPUT_SCHEMA.setString(conf, resultSchemaString)
 OrcConf.IS_SCHEMA_EVOLUTION_CASE_SENSITIVE.setBoolean(conf, 
isCaseSensitive)
 
 val filePath = new Path(new URI(file.filePath))
 
 val fs = filePath.getFileSystem(conf)
 val readerOptions = OrcFile.readerOptions(conf).filesystem(fs)
-val requestedColIdsOrEmptyFile =
+val resultedColPruneInfo =
   Utils.tryWithResource(OrcFile.createReader(filePath, readerOptions)) { 
reader =>
 OrcUtils.requestedColumnIds(
   isCaseSensitive, dataSchema, readDataSchema, reader, conf)
   }
 
-if (requestedColIdsOrEmptyFile.isEmpty) {
+if (resultedColPruneInfo.isEmpty) {
   new EmptyPartitionReader[InternalRow]
 } else {
-  val requestedColIds = requestedColIdsOrEmptyFile.get
+  val (requestedColIds, canPruneCols) = resultedColPruneInfo.get
+  val resultSchemaString = if (canPruneCols) {
+OrcUtils.orcTypeDescriptionString(resultSchema)
+  } else {
+OrcUtils.orcTypeDescriptionString(StructType(dataSchema.fields ++ 
partitionSchema.fields))
+  }
+  OrcConf.MAPRED_INPUT_SCHEMA.setString(conf, resultSchemaString)

Review comment:
   adding a new helper method is also good.





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] cloud-fan commented on a change in pull request #29045: [SPARK-32234][SQL] Spark sql commands are failing on selecting the orc tables

2020-07-15 Thread GitBox


cloud-fan commented on a change in pull request #29045:
URL: https://github.com/apache/spark/pull/29045#discussion_r454989126



##
File path: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcQuerySuite.scala
##
@@ -288,4 +288,33 @@ class HiveOrcQuerySuite extends OrcQueryTest with 
TestHiveSingleton {
   }
 }
   }
+
+  test("SPARK-32234: orc data created by the hive tables having _col fields 
name" +

Review comment:
   we can shorten the test name: `SPARK-32234: read ORC table with column 
names all starting with '_col'`





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] cloud-fan commented on a change in pull request #29045: [SPARK-32234][SQL] Spark sql commands are failing on selecting the orc tables

2020-07-15 Thread GitBox


cloud-fan commented on a change in pull request #29045:
URL: https://github.com/apache/spark/pull/29045#discussion_r454988428



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala
##
@@ -66,24 +66,28 @@ case class OrcPartitionReaderFactory(
   override def buildReader(file: PartitionedFile): 
PartitionReader[InternalRow] = {
 val conf = broadcastedConf.value.value
 
-val resultSchemaString = OrcUtils.orcTypeDescriptionString(resultSchema)
-OrcConf.MAPRED_INPUT_SCHEMA.setString(conf, resultSchemaString)
 OrcConf.IS_SCHEMA_EVOLUTION_CASE_SENSITIVE.setBoolean(conf, 
isCaseSensitive)
 
 val filePath = new Path(new URI(file.filePath))
 
 val fs = filePath.getFileSystem(conf)
 val readerOptions = OrcFile.readerOptions(conf).filesystem(fs)
-val requestedColIdsOrEmptyFile =
+val resultedColPruneInfo =
   Utils.tryWithResource(OrcFile.createReader(filePath, readerOptions)) { 
reader =>
 OrcUtils.requestedColumnIds(
   isCaseSensitive, dataSchema, readDataSchema, reader, conf)
   }
 
-if (requestedColIdsOrEmptyFile.isEmpty) {
+if (resultedColPruneInfo.isEmpty) {
   new EmptyPartitionReader[InternalRow]
 } else {
-  val requestedColIds = requestedColIdsOrEmptyFile.get
+  val (requestedColIds, canPruneCols) = resultedColPruneInfo.get
+  val resultSchemaString = if (canPruneCols) {
+OrcUtils.orcTypeDescriptionString(resultSchema)
+  } else {
+OrcUtils.orcTypeDescriptionString(StructType(dataSchema.fields ++ 
partitionSchema.fields))
+  }
+  OrcConf.MAPRED_INPUT_SCHEMA.setString(conf, resultSchemaString)

Review comment:
   To avoid duplicated code, we can move these to the `requestedColumnIds` 
method
   ```
   def requestedColumnIds(..., partitionSchema: StructType): Option[Array[Int]] 
= {
 ...
 if (orcFieldNames.isEmpty) {
   None
 } else if (orcFieldNames.forall(_.startsWith("_col"))) {
   OrcConf.MAPRED_INPUT_SCHEMA.setString(conf,
 OrcUtils.orcTypeDescriptionString(StructType(dataSchema.fields ++ 
partitionSchema.fields)))
   ...
 } else {
   OrcConf.MAPRED_INPUT_SCHEMA.setString(conf,
 OrcUtils.orcTypeDescriptionString(StructType(requiredSchema.fields ++ 
partitionSchema.fields)))
   ...
 }
   }
   ```

##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala
##
@@ -66,24 +66,28 @@ case class OrcPartitionReaderFactory(
   override def buildReader(file: PartitionedFile): 
PartitionReader[InternalRow] = {
 val conf = broadcastedConf.value.value
 
-val resultSchemaString = OrcUtils.orcTypeDescriptionString(resultSchema)
-OrcConf.MAPRED_INPUT_SCHEMA.setString(conf, resultSchemaString)
 OrcConf.IS_SCHEMA_EVOLUTION_CASE_SENSITIVE.setBoolean(conf, 
isCaseSensitive)
 
 val filePath = new Path(new URI(file.filePath))
 
 val fs = filePath.getFileSystem(conf)
 val readerOptions = OrcFile.readerOptions(conf).filesystem(fs)
-val requestedColIdsOrEmptyFile =
+val resultedColPruneInfo =
   Utils.tryWithResource(OrcFile.createReader(filePath, readerOptions)) { 
reader =>
 OrcUtils.requestedColumnIds(
   isCaseSensitive, dataSchema, readDataSchema, reader, conf)
   }
 
-if (requestedColIdsOrEmptyFile.isEmpty) {
+if (resultedColPruneInfo.isEmpty) {
   new EmptyPartitionReader[InternalRow]
 } else {
-  val requestedColIds = requestedColIdsOrEmptyFile.get
+  val (requestedColIds, canPruneCols) = resultedColPruneInfo.get
+  val resultSchemaString = if (canPruneCols) {
+OrcUtils.orcTypeDescriptionString(resultSchema)
+  } else {
+OrcUtils.orcTypeDescriptionString(StructType(dataSchema.fields ++ 
partitionSchema.fields))
+  }
+  OrcConf.MAPRED_INPUT_SCHEMA.setString(conf, resultSchemaString)

Review comment:
   To avoid duplicated code, we can move these code to the 
`requestedColumnIds` method
   ```
   def requestedColumnIds(..., partitionSchema: StructType): Option[Array[Int]] 
= {
 ...
 if (orcFieldNames.isEmpty) {
   None
 } else if (orcFieldNames.forall(_.startsWith("_col"))) {
   OrcConf.MAPRED_INPUT_SCHEMA.setString(conf,
 OrcUtils.orcTypeDescriptionString(StructType(dataSchema.fields ++ 
partitionSchema.fields)))
   ...
 } else {
   OrcConf.MAPRED_INPUT_SCHEMA.setString(conf,
 OrcUtils.orcTypeDescriptionString(StructType(requiredSchema.fields ++ 
partitionSchema.fields)))
   ...
 }
   }
   ```





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




[GitHub] [spark] cloud-fan commented on a change in pull request #29045: [SPARK-32234][SQL] Spark sql commands are failing on selecting the orc tables

2020-07-15 Thread GitBox


cloud-fan commented on a change in pull request #29045:
URL: https://github.com/apache/spark/pull/29045#discussion_r454986263



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala
##
@@ -116,15 +116,16 @@ object OrcUtils extends Logging {
   }
 
   /**
-   * Returns the requested column ids from the given ORC file. Column id can 
be -1, which means the
-   * requested column doesn't exist in the ORC file. Returns None if the given 
ORC file is empty.
+   * @return Returns the requested column ids from the given ORC file and 
Boolean flag to use actual

Review comment:
   can we update the comment a little bit?





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] cloud-fan commented on a change in pull request #29045: [SPARK-32234][SQL] Spark sql commands are failing on selecting the orc tables

2020-07-15 Thread GitBox


cloud-fan commented on a change in pull request #29045:
URL: https://github.com/apache/spark/pull/29045#discussion_r454837031



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala
##
@@ -116,47 +116,53 @@ object OrcUtils extends Logging {
   }
 
   /**
-   * Returns the requested column ids from the given ORC file. Column id can 
be -1, which means the
-   * requested column doesn't exist in the ORC file. Returns None if the given 
ORC file is empty.
+   * @return Returns the requested column ids from the given ORC file and 
Boolean flag to use actual
+   * schema or result schema. Column id can be -1, which means the requested 
column doesn't
+   * exist in the ORC file. Returns None if the given ORC file is empty.
*/
   def requestedColumnIds(
   isCaseSensitive: Boolean,
   dataSchema: StructType,
   requiredSchema: StructType,
   reader: Reader,
-  conf: Configuration): Option[Array[Int]] = {
+  conf: Configuration): (Option[Array[Int]], Boolean) = {

Review comment:
   how about
   ```
   def requestedColumnIds(...): Option[(Array[Int], Boolean)]
   ...
   val result = ... { OrcUtils.requestedColumnIds... }
   if (result.isEmpty) {
 Iterator.empty
   } else {
 val (requestedColIds, canPruneCols) = result.get
 ...
   }
   ```





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] cloud-fan commented on a change in pull request #29045: [SPARK-32234][SQL] Spark sql commands are failing on selecting the orc tables

2020-07-15 Thread GitBox


cloud-fan commented on a change in pull request #29045:
URL: https://github.com/apache/spark/pull/29045#discussion_r454835573



##
File path: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcQuerySuite.scala
##
@@ -288,4 +288,35 @@ class HiveOrcQuerySuite extends OrcQueryTest with 
TestHiveSingleton {
   }
 }
   }
+
+  test("SPARK-32234: orc data created by the hive tables having _col fields 
name" +
+" for ORC_IMPLEMENTATION") {
+Seq("native", "hive").foreach { orcImpl =>
+  Seq("false", "true").foreach { vectorized =>
+withSQLConf(
+  SQLConf.ORC_IMPLEMENTATION.key -> orcImpl,
+  SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> vectorized) {
+  withTempPath { dir =>

Review comment:
   nit: we don't need to provide a custom location. CREATE TABLE without 
LOCATION clause can also reproduce it.





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] cloud-fan commented on a change in pull request #29045: [SPARK-32234][SQL] Spark sql commands are failing on selecting the orc tables

2020-07-15 Thread GitBox


cloud-fan commented on a change in pull request #29045:
URL: https://github.com/apache/spark/pull/29045#discussion_r454835082



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala
##
@@ -116,47 +116,53 @@ object OrcUtils extends Logging {
   }
 
   /**
-   * Returns the requested column ids from the given ORC file. Column id can 
be -1, which means the
-   * requested column doesn't exist in the ORC file. Returns None if the given 
ORC file is empty.
+   * @return Returns the requested column ids from the given ORC file and 
Boolean flag to use actual
+   * schema or result schema. Column id can be -1, which means the requested 
column doesn't
+   * exist in the ORC file. Returns None if the given ORC file is empty.
*/
   def requestedColumnIds(
   isCaseSensitive: Boolean,
   dataSchema: StructType,
   requiredSchema: StructType,
   reader: Reader,
-  conf: Configuration): Option[Array[Int]] = {
+  conf: Configuration): (Option[Array[Int]], Boolean) = {
+var canPruneCols = true

Review comment:
   do we really need it? We can just use boolean literal in the places that 
return the value.





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] cloud-fan commented on a change in pull request #29045: [SPARK-32234][SQL] Spark sql commands are failing on selecting the orc tables

2020-07-15 Thread GitBox


cloud-fan commented on a change in pull request #29045:
URL: https://github.com/apache/spark/pull/29045#discussion_r454834518



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala
##
@@ -179,12 +179,17 @@ class OrcFileFormat
 
   val fs = filePath.getFileSystem(conf)
   val readerOptions = OrcFile.readerOptions(conf).filesystem(fs)
-  val requestedColIdsOrEmptyFile =
+  val (requestedColIdsOrEmptyFile, canPruneCols) =
 Utils.tryWithResource(OrcFile.createReader(filePath, readerOptions)) { 
reader =>
   OrcUtils.requestedColumnIds(
 isCaseSensitive, dataSchema, requiredSchema, reader, conf)
 }
 
+  if (!canPruneCols) {

Review comment:
   nit: we can simplify the code a bit
   ```
   val resultSchemaString = if (canPruneCols) {
 OrcUtils.orcTypeDescriptionString(resultSchema)
   } else {
 OrcUtils.orcTypeDescriptionString(StructType(dataSchema.fields ++ 
partitionSchema.fields))
   }
   ```
   
   Then we don't need to keep the `val actualSchema =...` and `var 
resultSchemaString =...` at the beginning.





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] cloud-fan commented on a change in pull request #29045: [SPARK-32234][SQL] Spark sql commands are failing on selecting the orc tables

2020-07-14 Thread GitBox


cloud-fan commented on a change in pull request #29045:
URL: https://github.com/apache/spark/pull/29045#discussion_r454539775



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala
##
@@ -74,7 +74,7 @@ case class OrcPartitionReaderFactory(
 
 val fs = filePath.getFileSystem(conf)
 val readerOptions = OrcFile.readerOptions(conf).filesystem(fs)
-val requestedColIdsOrEmptyFile =
+val (requestedColIdsOrEmptyFile, _) =

Review comment:
   We should fix the file source v2 code path as well.





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] cloud-fan commented on a change in pull request #29045: [SPARK-32234][SQL] Spark sql commands are failing on selecting the orc tables

2020-07-14 Thread GitBox


cloud-fan commented on a change in pull request #29045:
URL: https://github.com/apache/spark/pull/29045#discussion_r454539148



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala
##
@@ -179,12 +179,17 @@ class OrcFileFormat
 
   val fs = filePath.getFileSystem(conf)
   val readerOptions = OrcFile.readerOptions(conf).filesystem(fs)
-  val requestedColIdsOrEmptyFile =
+  val (requestedColIdsOrEmptyFile, sendActualSchema) =

Review comment:
   `sendActualSchema` is a bit confusing, how about `canPruneCols`?





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] cloud-fan commented on a change in pull request #29045: [SPARK-32234][SQL] Spark sql commands are failing on selecting the orc tables

2020-07-14 Thread GitBox


cloud-fan commented on a change in pull request #29045:
URL: https://github.com/apache/spark/pull/29045#discussion_r454538915



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala
##
@@ -116,47 +116,53 @@ object OrcUtils extends Logging {
   }
 
   /**
-   * Returns the requested column ids from the given ORC file. Column id can 
be -1, which means the
-   * requested column doesn't exist in the ORC file. Returns None if the given 
ORC file is empty.
+   * @return Returns the requested column ids from the given ORC file and 
Boolean flag to use actual
+   * schema or result schema. Column id can be -1, which means the requested 
column doesn't
+   * exist in the ORC file. Returns None if the given ORC file is empty.
*/
   def requestedColumnIds(
   isCaseSensitive: Boolean,
   dataSchema: StructType,
   requiredSchema: StructType,
   reader: Reader,
-  conf: Configuration): Option[Array[Int]] = {
+  conf: Configuration): (Option[Array[Int]], Boolean) = {

Review comment:
   I think the return type should be `Option[(Array[Int], Boolean)]`





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] cloud-fan commented on a change in pull request #29045: [SPARK-32234][SQL] Spark sql commands are failing on selecting the orc tables

2020-07-14 Thread GitBox


cloud-fan commented on a change in pull request #29045:
URL: https://github.com/apache/spark/pull/29045#discussion_r454538413



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala
##
@@ -116,47 +116,53 @@ object OrcUtils extends Logging {
   }
 
   /**
-   * Returns the requested column ids from the given ORC file. Column id can 
be -1, which means the
-   * requested column doesn't exist in the ORC file. Returns None if the given 
ORC file is empty.
+   * @return Returns the requested column ids from the given ORC file and 
Boolean flag to use actual
+   * schema or result schema. Column id can be -1, which means the requested 
column doesn't
+   * exist in the ORC file. Returns None if the given ORC file is empty.
*/
   def requestedColumnIds(
   isCaseSensitive: Boolean,
   dataSchema: StructType,
   requiredSchema: StructType,
   reader: Reader,
-  conf: Configuration): Option[Array[Int]] = {
+  conf: Configuration): (Option[Array[Int]], Boolean) = {
+var sendActualSchema = false
 val orcFieldNames = reader.getSchema.getFieldNames.asScala
 if (orcFieldNames.isEmpty) {
   // SPARK-8501: Some old empty ORC files always have an empty schema 
stored in their footer.
-  None
+  (None, sendActualSchema)
 } else {
   if (orcFieldNames.forall(_.startsWith("_col"))) {
 // This is a ORC file written by Hive, no field names in the physical 
schema, assume the
 // physical schema maps to the data scheme by index.
 assert(orcFieldNames.length <= dataSchema.length, "The given data 
schema " +
   s"${dataSchema.catalogString} has less fields than the actual ORC 
physical schema, " +
   "no idea which columns were dropped, fail to read.")
-Some(requiredSchema.fieldNames.map { name =>
+(Some(requiredSchema.fieldNames.map { name =>
   val index = dataSchema.fieldIndex(name)
   if (index < orcFieldNames.length) {
+// for ORC file written by Hive, no field names
+// in the physical schema, there is a need to send the
+// entire dataSchema instead of required schema
+sendActualSchema = true
 index
   } else {
 -1

Review comment:
   I tried the test locally, and saw warning messages like
   ```
   10:45:52.783 WARN org.apache.orc.impl.SchemaEvolution: Column names are 
missing from this file. This is caused by a writer earlier than HIVE-4243. The 
reader will reconcile schemas based on index. File type: 
struct<_col1:int,_col2:string,_col3:int>, reader type: struct<_col2:string>
   {9}
   ```
   
   I think we can't do column pruning anyway if the physical file schema is 
`_col0`, ... We can always return true in this branch.





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] cloud-fan commented on a change in pull request #29045: [SPARK-32234][SQL] Spark sql commands are failing on selecting the orc tables

2020-07-14 Thread GitBox


cloud-fan commented on a change in pull request #29045:
URL: https://github.com/apache/spark/pull/29045#discussion_r454515540



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala
##
@@ -116,47 +116,53 @@ object OrcUtils extends Logging {
   }
 
   /**
-   * Returns the requested column ids from the given ORC file. Column id can 
be -1, which means the
-   * requested column doesn't exist in the ORC file. Returns None if the given 
ORC file is empty.
+   * @return Returns the requested column ids from the given ORC file and 
Boolean flag to use actual
+   * schema or result schema. Column id can be -1, which means the requested 
column doesn't
+   * exist in the ORC file. Returns None if the given ORC file is empty.
*/
   def requestedColumnIds(
   isCaseSensitive: Boolean,
   dataSchema: StructType,
   requiredSchema: StructType,
   reader: Reader,
-  conf: Configuration): Option[Array[Int]] = {
+  conf: Configuration): (Option[Array[Int]], Boolean) = {
+var sendActualSchema = false
 val orcFieldNames = reader.getSchema.getFieldNames.asScala
 if (orcFieldNames.isEmpty) {

Review comment:
   If we can't do column pruning for the _col case, shall we simply return 
None? e.g.
   ```
   if (orcFieldNames.isEmpty || orcFieldNames.forall(_.startsWith("_col"))) {
 None
   }
   ```





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] cloud-fan commented on a change in pull request #29045: [SPARK-32234][SQL] Spark sql commands are failing on selecting the orc tables

2020-07-14 Thread GitBox


cloud-fan commented on a change in pull request #29045:
URL: https://github.com/apache/spark/pull/29045#discussion_r454515540



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala
##
@@ -116,47 +116,53 @@ object OrcUtils extends Logging {
   }
 
   /**
-   * Returns the requested column ids from the given ORC file. Column id can 
be -1, which means the
-   * requested column doesn't exist in the ORC file. Returns None if the given 
ORC file is empty.
+   * @return Returns the requested column ids from the given ORC file and 
Boolean flag to use actual
+   * schema or result schema. Column id can be -1, which means the requested 
column doesn't
+   * exist in the ORC file. Returns None if the given ORC file is empty.
*/
   def requestedColumnIds(
   isCaseSensitive: Boolean,
   dataSchema: StructType,
   requiredSchema: StructType,
   reader: Reader,
-  conf: Configuration): Option[Array[Int]] = {
+  conf: Configuration): (Option[Array[Int]], Boolean) = {
+var sendActualSchema = false
 val orcFieldNames = reader.getSchema.getFieldNames.asScala
 if (orcFieldNames.isEmpty) {

Review comment:
   If we can't do column pruning for the _col case, shall we simply return 
None? e.g.
   ```
   if (orcFieldNames.isEmpty || orcFieldNames.forall(_.startsWith("_col"))) {
 None
   }
   ```





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] cloud-fan commented on a change in pull request #29045: [SPARK-32234][SQL] Spark sql commands are failing on selecting the orc tables

2020-07-14 Thread GitBox


cloud-fan commented on a change in pull request #29045:
URL: https://github.com/apache/spark/pull/29045#discussion_r454476253



##
File path: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcQuerySuite.scala
##
@@ -288,4 +288,35 @@ class HiveOrcQuerySuite extends OrcQueryTest with 
TestHiveSingleton {
   }
 }
   }
+
+  test("SPARK-32234: orc data created by the hive tables having _col fields 
name" +
+" for ORC_IMPLEMENTATION") {
+Seq("native", "hive").foreach { orcImpl =>
+  Seq("false", "true").foreach { vectorized =>
+withSQLConf(
+  SQLConf.ORC_IMPLEMENTATION.key -> orcImpl,
+  SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> vectorized) {
+  withTempPath { dir =>
+withTable("test_hive_orc_impl") {
+  spark.sql(
+s"""
+   | CREATE TABLE test_hive_orc_impl
+   | (_col1 INT, _col2 STRING, _col3 INT)

Review comment:
   can this test reproduce the bug? the table schema matches the physical 
file schema.





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] cloud-fan commented on a change in pull request #29045: [SPARK-32234][SQL] Spark sql commands are failing on selecting the orc tables

2020-07-14 Thread GitBox


cloud-fan commented on a change in pull request #29045:
URL: https://github.com/apache/spark/pull/29045#discussion_r454475026



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala
##
@@ -160,12 +160,12 @@ class OrcFileFormat
 }
 
 val resultSchema = StructType(requiredSchema.fields ++ 
partitionSchema.fields)
+val actualSchema = StructType(dataSchema.fields ++ partitionSchema.fields)
 val sqlConf = sparkSession.sessionState.conf
 val enableVectorizedReader = supportBatch(sparkSession, resultSchema)
 val capacity = sqlConf.orcVectorizedReaderBatchSize
 
-val resultSchemaString = OrcUtils.orcTypeDescriptionString(resultSchema)
-OrcConf.MAPRED_INPUT_SCHEMA.setString(hadoopConf, resultSchemaString)

Review comment:
   After reading the code more, I think this is the real problem. When the 
physical file schema doesn't match the table schema, this `resultSchemaString` 
is wrong. Then the ORC lib does column pruning wrongly and causes the index out 
of bound error.
   
   I think we should set this config in the executor side, where we know the 
physical file schema. The `OrcUtils.requestedColumnIds` should not only report 
the column indices, but also the actual `requiredSchema`.





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] cloud-fan commented on a change in pull request #29045: [SPARK-32234][SQL] Spark sql commands are failing on selecting the orc tables

2020-07-14 Thread GitBox


cloud-fan commented on a change in pull request #29045:
URL: https://github.com/apache/spark/pull/29045#discussion_r454475026



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala
##
@@ -160,12 +160,12 @@ class OrcFileFormat
 }
 
 val resultSchema = StructType(requiredSchema.fields ++ 
partitionSchema.fields)
+val actualSchema = StructType(dataSchema.fields ++ partitionSchema.fields)
 val sqlConf = sparkSession.sessionState.conf
 val enableVectorizedReader = supportBatch(sparkSession, resultSchema)
 val capacity = sqlConf.orcVectorizedReaderBatchSize
 
-val resultSchemaString = OrcUtils.orcTypeDescriptionString(resultSchema)
-OrcConf.MAPRED_INPUT_SCHEMA.setString(hadoopConf, resultSchemaString)

Review comment:
   After reading the code more, I think this is the real problem. When the 
physical file schema doesn't match the table schema, this `resultSchemaString` 
is wrong.
   
   I think we should set this config in the executor side, where we know the 
physical file schema. The `OrcUtils.requestedColumnIds` should not only report 
the column indices, but also the actual `requiredSchema`.





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] cloud-fan commented on a change in pull request #29045: [SPARK-32234][SQL] Spark sql commands are failing on selecting the orc tables

2020-07-14 Thread GitBox


cloud-fan commented on a change in pull request #29045:
URL: https://github.com/apache/spark/pull/29045#discussion_r454396181



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala
##
@@ -116,47 +116,53 @@ object OrcUtils extends Logging {
   }
 
   /**
-   * Returns the requested column ids from the given ORC file. Column id can 
be -1, which means the
-   * requested column doesn't exist in the ORC file. Returns None if the given 
ORC file is empty.
+   * @return Returns the requested column ids from the given ORC file and 
Boolean flag to use actual
+   * schema or result schema. Column id can be -1, which means the requested 
column doesn't
+   * exist in the ORC file. Returns None if the given ORC file is empty.
*/
   def requestedColumnIds(
   isCaseSensitive: Boolean,
   dataSchema: StructType,
   requiredSchema: StructType,
   reader: Reader,
-  conf: Configuration): Option[Array[Int]] = {
+  conf: Configuration): (Option[Array[Int]], Boolean) = {
+var sendActualSchema = false
 val orcFieldNames = reader.getSchema.getFieldNames.asScala

Review comment:
   following your previous explanation in 
https://github.com/apache/spark/pull/29045#discussion_r454234413
   
   1. the code is creating a wrap VectorizedRowBatchWrap using the result 
schema which is ``
   2. `orcVectorWrappers[i] = new OrcColumnVector(dt, 
wrap.batch().cols[colId]);` accesses the 6th column of the orc batch.
   
   Why it doesn't fail when the physical orc file schema matches the table 
schema in metastore?





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] cloud-fan commented on a change in pull request #29045: [SPARK-32234][SQL] Spark sql commands are failing on selecting the orc tables

2020-07-14 Thread GitBox


cloud-fan commented on a change in pull request #29045:
URL: https://github.com/apache/spark/pull/29045#discussion_r454318909



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala
##
@@ -116,47 +116,53 @@ object OrcUtils extends Logging {
   }
 
   /**
-   * Returns the requested column ids from the given ORC file. Column id can 
be -1, which means the
-   * requested column doesn't exist in the ORC file. Returns None if the given 
ORC file is empty.
+   * @return Returns the requested column ids from the given ORC file and 
Boolean flag to use actual
+   * schema or result schema. Column id can be -1, which means the requested 
column doesn't
+   * exist in the ORC file. Returns None if the given ORC file is empty.
*/
   def requestedColumnIds(
   isCaseSensitive: Boolean,
   dataSchema: StructType,
   requiredSchema: StructType,
   reader: Reader,
-  conf: Configuration): Option[Array[Int]] = {
+  conf: Configuration): (Option[Array[Int]], Boolean) = {
+var sendActualSchema = false
 val orcFieldNames = reader.getSchema.getFieldNames.asScala

Review comment:
   Does the failure go away if the physical orc file schema matches the 
table schema in metastore?





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] cloud-fan commented on a change in pull request #29045: [SPARK-32234][SQL] Spark sql commands are failing on selecting the orc tables

2020-07-14 Thread GitBox


cloud-fan commented on a change in pull request #29045:
URL: https://github.com/apache/spark/pull/29045#discussion_r454201305



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala
##
@@ -116,47 +116,53 @@ object OrcUtils extends Logging {
   }
 
   /**
-   * Returns the requested column ids from the given ORC file. Column id can 
be -1, which means the
-   * requested column doesn't exist in the ORC file. Returns None if the given 
ORC file is empty.
+   * @return Returns the requested column ids from the given ORC file and 
Boolean flag to use actual
+   * schema or result schema. Column id can be -1, which means the requested 
column doesn't
+   * exist in the ORC file. Returns None if the given ORC file is empty.
*/
   def requestedColumnIds(
   isCaseSensitive: Boolean,
   dataSchema: StructType,
   requiredSchema: StructType,
   reader: Reader,
-  conf: Configuration): Option[Array[Int]] = {
+  conf: Configuration): (Option[Array[Int]], Boolean) = {
+var sendActualSchema = false
 val orcFieldNames = reader.getSchema.getFieldNames.asScala

Review comment:
   Please correct me if I'm wrong:
   1. the physical orc file schema is `_col0`, ...
   2. the table schema in metastore is `d_date_sk`, ...
   3. the query only requires only `d_year`
   
   I don't know why the query fails. The `requestedColumnIds` will be `[6]` and 
the orc reader will read the `_col6` column. Everything should be fine.





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] cloud-fan commented on a change in pull request #29045: [SPARK-32234][SQL] Spark sql commands are failing on selecting the orc tables

2020-07-14 Thread GitBox


cloud-fan commented on a change in pull request #29045:
URL: https://github.com/apache/spark/pull/29045#discussion_r454122245



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala
##
@@ -116,47 +116,53 @@ object OrcUtils extends Logging {
   }
 
   /**
-   * Returns the requested column ids from the given ORC file. Column id can 
be -1, which means the
-   * requested column doesn't exist in the ORC file. Returns None if the given 
ORC file is empty.
+   * @return Returns the requested column ids from the given ORC file and 
Boolean flag to use actual
+   * schema or result schema. Column id can be -1, which means the requested 
column doesn't
+   * exist in the ORC file. Returns None if the given ORC file is empty.
*/
   def requestedColumnIds(
   isCaseSensitive: Boolean,
   dataSchema: StructType,
   requiredSchema: StructType,
   reader: Reader,
-  conf: Configuration): Option[Array[Int]] = {
+  conf: Configuration): (Option[Array[Int]], Boolean) = {
+var sendActualSchema = false
 val orcFieldNames = reader.getSchema.getFieldNames.asScala

Review comment:
   let's say we are running the test `CREATE TABLE test_hive_orc_impl ...`, 
what's the value of `dataSchema`, `requiredSchema` and `orcFieldNames` when we 
reach here?





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] cloud-fan commented on a change in pull request #29045: [SPARK-32234][SQL] Spark sql commands are failing on selecting the orc tables

2020-07-13 Thread GitBox


cloud-fan commented on a change in pull request #29045:
URL: https://github.com/apache/spark/pull/29045#discussion_r454119852



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala
##
@@ -179,12 +179,17 @@ class OrcFileFormat
 
   val fs = filePath.getFileSystem(conf)
   val readerOptions = OrcFile.readerOptions(conf).filesystem(fs)
-  val requestedColIdsOrEmptyFile =
+  val (requestedColIdsOrEmptyFile, sendActualSchema) =
 Utils.tryWithResource(OrcFile.createReader(filePath, readerOptions)) { 
reader =>
   OrcUtils.requestedColumnIds(
 isCaseSensitive, dataSchema, requiredSchema, reader, conf)
 }
 
+  if (sendActualSchema) {
+resultSchemaString = OrcUtils.orcTypeDescriptionString(actualSchema)

Review comment:
   do you mean we can't do column pruning in this case?





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