[GitHub] spark pull request #14537: [SPARK-16948][SQL] Querying empty partitioned orc...

2016-08-25 Thread yhuai
Github user yhuai commented on a diff in the pull request:

https://github.com/apache/spark/pull/14537#discussion_r76305079
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
@@ -237,21 +237,27 @@ private[hive] class 
HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
   new Path(metastoreRelation.catalogTable.storage.locationUri.get),
   partitionSpec)
 
-val inferredSchema = if (fileType.equals("parquet")) {
-  val inferredSchema =
-defaultSource.inferSchema(sparkSession, options, 
fileCatalog.allFiles())
-  inferredSchema.map { inferred =>
-ParquetFileFormat.mergeMetastoreParquetSchema(metastoreSchema, 
inferred)
-  }.getOrElse(metastoreSchema)
-} else {
-  defaultSource.inferSchema(sparkSession, options, 
fileCatalog.allFiles()).get
+val schema = fileType match {
+  case "parquet" =>
+val inferredSchema =
+  defaultSource.inferSchema(sparkSession, options, 
fileCatalog.allFiles())
+
+// For Parquet, get correct schema by merging Metastore schema 
data types
--- End diff --

I think we have a test. @liancheng should have more info. But, one 
clarification is that this merging is based on column name (we do take care the 
case sensitivity issue though). So, if you want to name a column with another 
name, I think it is not doable.


---
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 #14537: [SPARK-16948][SQL] Querying empty partitioned orc...

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

https://github.com/apache/spark/pull/14537#discussion_r76189706
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala ---
@@ -372,6 +373,40 @@ class OrcQuerySuite extends QueryTest with 
BeforeAndAfterAll with OrcTest {
 }
   }
 
+  test("SPARK-16948. Check empty orc tables in ORC") {
--- End diff --

how about `support empty orc table when converting hive serde table to data 
source table`


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

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



[GitHub] spark pull request #14537: [SPARK-16948][SQL] Querying empty partitioned orc...

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

https://github.com/apache/spark/pull/14537#discussion_r76189474
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala ---
@@ -54,10 +57,12 @@ class OrcFileFormat extends FileFormat with 
DataSourceRegister with Serializable
   sparkSession: SparkSession,
   options: Map[String, String],
   files: Seq[FileStatus]): Option[StructType] = {
-OrcFileOperator.readSchema(
-  files.map(_.getPath.toUri.toString),
-  Some(sparkSession.sessionState.newHadoopConf())
-)
+// Safe to ignore FileNotFoundException in case no files are found.
+val schema = Try(OrcFileOperator.readSchema(
--- End diff --

@rajeshbalamohan is this change unnecessary for this PR? If so, I'd like to 
revert it to make the PR as small as possible.


---
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 #14537: [SPARK-16948][SQL] Querying empty partitioned orc...

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

https://github.com/apache/spark/pull/14537#discussion_r76189247
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
@@ -237,21 +237,27 @@ private[hive] class 
HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
   new Path(metastoreRelation.catalogTable.storage.locationUri.get),
   partitionSpec)
 
-val inferredSchema = if (fileType.equals("parquet")) {
-  val inferredSchema =
-defaultSource.inferSchema(sparkSession, options, 
fileCatalog.allFiles())
-  inferredSchema.map { inferred =>
-ParquetFileFormat.mergeMetastoreParquetSchema(metastoreSchema, 
inferred)
-  }.getOrElse(metastoreSchema)
-} else {
-  defaultSource.inferSchema(sparkSession, options, 
fileCatalog.allFiles()).get
+val schema = fileType match {
+  case "parquet" =>
+val inferredSchema =
+  defaultSource.inferSchema(sparkSession, options, 
fileCatalog.allFiles())
+
+// For Parquet, get correct schema by merging Metastore schema 
data types
--- End diff --

Do we have a test for this feature? I think we should make them consistent, 
i.e. parquet conversions should also use the metastore schema.

cc @yhuai @liancheng 


---
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 #14537: [SPARK-16948][SQL] Querying empty partitioned orc...

2016-08-24 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/14537#discussion_r76180471
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala ---
@@ -372,6 +373,40 @@ class OrcQuerySuite extends QueryTest with 
BeforeAndAfterAll with OrcTest {
 }
   }
 
+  test("SPARK-16948. Check empty orc tables in ORC") {
+withSQLConf((HiveUtils.CONVERT_METASTORE_ORC.key, "true")) {
+  withTable("empty_orc_partitioned") {
+spark.sql(
--- End diff --

Nit: `spark.sql` -> `sql`


---
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 #14537: [SPARK-16948][SQL] Querying empty partitioned orc...

2016-08-24 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/14537#discussion_r76180414
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala ---
@@ -54,10 +57,12 @@ class OrcFileFormat extends FileFormat with 
DataSourceRegister with Serializable
   sparkSession: SparkSession,
   options: Map[String, String],
   files: Seq[FileStatus]): Option[StructType] = {
-OrcFileOperator.readSchema(
-  files.map(_.getPath.toUri.toString),
-  Some(sparkSession.sessionState.newHadoopConf())
-)
+// Safe to ignore FileNotFoundException in case no files are found.
+val schema = Try(OrcFileOperator.readSchema(
--- End diff --

You know, the changes could impact the behaviors of the callers of 
`inferSchema`. 

Could you write a test case to cover this scenario? 


---
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 #14537: [SPARK-16948][SQL] Querying empty partitioned orc...

2016-08-24 Thread rajeshbalamohan
Github user rajeshbalamohan commented on a diff in the pull request:

https://github.com/apache/spark/pull/14537#discussion_r76179877
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala ---
@@ -54,10 +57,12 @@ class OrcFileFormat extends FileFormat with 
DataSourceRegister with Serializable
   sparkSession: SparkSession,
   options: Map[String, String],
   files: Seq[FileStatus]): Option[StructType] = {
-OrcFileOperator.readSchema(
-  files.map(_.getPath.toUri.toString),
-  Some(sparkSession.sessionState.newHadoopConf())
-)
+// Safe to ignore FileNotFoundException in case no files are found.
+val schema = Try(OrcFileOperator.readSchema(
--- End diff --

Yes, in case this is referred anytime 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 #14537: [SPARK-16948][SQL] Querying empty partitioned orc...

2016-08-24 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/14537#discussion_r76170526
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
@@ -237,21 +237,27 @@ private[hive] class 
HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
   new Path(metastoreRelation.catalogTable.storage.locationUri.get),
   partitionSpec)
 
-val inferredSchema = if (fileType.equals("parquet")) {
-  val inferredSchema =
-defaultSource.inferSchema(sparkSession, options, 
fileCatalog.allFiles())
-  inferredSchema.map { inferred =>
-ParquetFileFormat.mergeMetastoreParquetSchema(metastoreSchema, 
inferred)
-  }.getOrElse(metastoreSchema)
-} else {
-  defaultSource.inferSchema(sparkSession, options, 
fileCatalog.allFiles()).get
+val schema = fileType match {
+  case "parquet" =>
+val inferredSchema =
+  defaultSource.inferSchema(sparkSession, options, 
fileCatalog.allFiles())
+
+// For Parquet, get correct schema by merging Metastore schema 
data types
+// and Parquet schema field names.
+inferredSchema.map { schema =>
+  
ParquetFileFormat.mergeMetastoreParquetSchema(metastoreSchema, schema)
+}.getOrElse(metastoreSchema)
+  case "orc" =>
+metastoreSchema
+  case _ =>
+throw new RuntimeException(s"Cannot convert a $fileType to a 
HadoopFsRelation")
--- End diff --

Spark SQL users do not know what is a `HadoopFsRelation`. 

`to a data source table`?


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

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



[GitHub] spark pull request #14537: [SPARK-16948][SQL] Querying empty partitioned orc...

2016-08-24 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/14537#discussion_r76170264
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala ---
@@ -372,6 +373,42 @@ class OrcQuerySuite extends QueryTest with 
BeforeAndAfterAll with OrcTest {
 }
   }
 
+  test("SPARK-16948. Check empty orc tables in ORC") {
+withSQLConf((HiveUtils.CONVERT_METASTORE_ORC.key, "true")) {
+  withTempPath { dir =>
+withTable("empty_orc_partitioned") {
+  spark.sql(
+s"""
+|CREATE TABLE empty_orc_partitioned(key INT, value STRING)
+|PARTITIONED BY (p INT) STORED AS ORC
+  """.stripMargin)
--- End diff --

Nit: 
```Scala
  sql(
"""
  |CREATE TABLE empty_orc_partitioned(key INT, value STRING)
  |PARTITIONED BY (p INT) STORED AS ORC
""".stripMargin)
```


---
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 #14537: [SPARK-16948][SQL] Querying empty partitioned orc...

2016-08-24 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/14537#discussion_r76170206
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala ---
@@ -372,6 +373,42 @@ class OrcQuerySuite extends QueryTest with 
BeforeAndAfterAll with OrcTest {
 }
   }
 
+  test("SPARK-16948. Check empty orc tables in ORC") {
+withSQLConf((HiveUtils.CONVERT_METASTORE_ORC.key, "true")) {
+  withTempPath { dir =>
+withTable("empty_orc_partitioned") {
+  spark.sql(
+s"""
+|CREATE TABLE empty_orc_partitioned(key INT, value STRING)
+|PARTITIONED BY (p INT) STORED AS ORC
+  """.stripMargin)
+
+  val emptyDF = Seq.empty[(Int, String)].toDF("key", 
"value").coalesce(1)
+
+  // Query empty table
+  checkAnswer(
+sql("SELECT key, value FROM empty_orc_partitioned WHERE key > 
10"),
+emptyDF)
+}
+
+withTable("empty_orc") {
+  spark.sql(
+s"""
+   |CREATE TABLE empty_orc(key INT, value STRING)
+   |STORED AS ORC
+  """.stripMargin)
--- End diff --

Nit: 
```
  sql(
"""
  |CREATE TABLE empty_orc(key INT, value STRING)
  |STORED AS ORC
""".stripMargin)
```


---
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 #14537: [SPARK-16948][SQL] Querying empty partitioned orc...

2016-08-24 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/14537#discussion_r76170101
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala ---
@@ -372,6 +373,42 @@ class OrcQuerySuite extends QueryTest with 
BeforeAndAfterAll with OrcTest {
 }
   }
 
+  test("SPARK-16948. Check empty orc tables in ORC") {
+withSQLConf((HiveUtils.CONVERT_METASTORE_ORC.key, "true")) {
+  withTempPath { dir =>
+withTable("empty_orc_partitioned") {
+  spark.sql(
+s"""
+|CREATE TABLE empty_orc_partitioned(key INT, value STRING)
+|PARTITIONED BY (p INT) STORED AS ORC
+  """.stripMargin)
+
+  val emptyDF = Seq.empty[(Int, String)].toDF("key", 
"value").coalesce(1)
+
+  // Query empty table
+  checkAnswer(
+sql("SELECT key, value FROM empty_orc_partitioned WHERE key > 
10"),
+emptyDF)
+}
+
+withTable("empty_orc") {
+  spark.sql(
+s"""
+   |CREATE TABLE empty_orc(key INT, value STRING)
+   |STORED AS ORC
+  """.stripMargin)
+
+  val emptyDF = Seq.empty[(Int, String)].toDF("key", 
"value").coalesce(1)
+
+  // Query empty table
+  checkAnswer(
+sql("SELECT key, value FROM empty_orc WHERE key > 10"),
--- End diff --

The same 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 #14537: [SPARK-16948][SQL] Querying empty partitioned orc...

2016-08-24 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/14537#discussion_r76170085
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala ---
@@ -372,6 +373,42 @@ class OrcQuerySuite extends QueryTest with 
BeforeAndAfterAll with OrcTest {
 }
   }
 
+  test("SPARK-16948. Check empty orc tables in ORC") {
+withSQLConf((HiveUtils.CONVERT_METASTORE_ORC.key, "true")) {
+  withTempPath { dir =>
+withTable("empty_orc_partitioned") {
+  spark.sql(
+s"""
+|CREATE TABLE empty_orc_partitioned(key INT, value STRING)
+|PARTITIONED BY (p INT) STORED AS ORC
+  """.stripMargin)
+
+  val emptyDF = Seq.empty[(Int, String)].toDF("key", 
"value").coalesce(1)
+
+  // Query empty table
+  checkAnswer(
+sql("SELECT key, value FROM empty_orc_partitioned WHERE key > 
10"),
--- End diff --

Do we need the where clause?


---
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 #14537: [SPARK-16948][SQL] Querying empty partitioned orc...

2016-08-24 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/14537#discussion_r76170033
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala ---
@@ -372,6 +373,42 @@ class OrcQuerySuite extends QueryTest with 
BeforeAndAfterAll with OrcTest {
 }
   }
 
+  test("SPARK-16948. Check empty orc tables in ORC") {
+withSQLConf((HiveUtils.CONVERT_METASTORE_ORC.key, "true")) {
+  withTempPath { dir =>
--- End diff --

Do we still need 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 #14537: [SPARK-16948][SQL] Querying empty partitioned orc...

2016-08-24 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/14537#discussion_r76170021
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala ---
@@ -54,10 +57,12 @@ class OrcFileFormat extends FileFormat with 
DataSourceRegister with Serializable
   sparkSession: SparkSession,
   options: Map[String, String],
   files: Seq[FileStatus]): Option[StructType] = {
-OrcFileOperator.readSchema(
-  files.map(_.getPath.toUri.toString),
-  Some(sparkSession.sessionState.newHadoopConf())
-)
+// Safe to ignore FileNotFoundException in case no files are found.
+val schema = Try(OrcFileOperator.readSchema(
--- End diff --

Do we still need 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 #14537: [SPARK-16948][SQL] Querying empty partitioned orc...

2016-08-23 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/14537#discussion_r75988035
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala ---
@@ -372,6 +372,29 @@ class OrcQuerySuite extends QueryTest with 
BeforeAndAfterAll with OrcTest {
 }
   }
 
+  test("SPARK-16948. Check empty orc partitioned tables in ORC") {
+withSQLConf((HiveUtils.CONVERT_METASTORE_ORC.key, "true")) {
+  withTempPath { dir =>
--- End diff --

Could you remove this line?


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

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



[GitHub] spark pull request #14537: [SPARK-16948][SQL] Querying empty partitioned orc...

2016-08-23 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/14537#discussion_r75987946
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala ---
@@ -372,6 +372,29 @@ class OrcQuerySuite extends QueryTest with 
BeforeAndAfterAll with OrcTest {
 }
   }
 
+  test("SPARK-16948. Check empty orc partitioned tables in ORC") {
+withSQLConf((HiveUtils.CONVERT_METASTORE_ORC.key, "true")) {
+  withTempPath { dir =>
+withTable("empty_orc_partitioned") {
+  spark.sql(
+s"""CREATE TABLE empty_orc_partitioned(key INT, value STRING)
+| PARTITIONED BY (p INT) STORED AS ORC
+  """.stripMargin)
--- End diff --

A comment about the style
```Scala
  sql(
"""
  |CREATE TABLE empty_orc_partitioned(key INT, value STRING)
  |PARTITIONED BY (p INT) STORED AS ORC
""".stripMargin)
```


---
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 #14537: [SPARK-16948][SQL] Querying empty partitioned orc...

2016-08-23 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/14537#discussion_r75987870
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala ---
@@ -372,6 +372,29 @@ class OrcQuerySuite extends QueryTest with 
BeforeAndAfterAll with OrcTest {
 }
   }
 
+  test("SPARK-16948. Check empty orc partitioned tables in ORC") {
+withSQLConf((HiveUtils.CONVERT_METASTORE_ORC.key, "true")) {
+  withTempPath { dir =>
+withTable("empty_orc_partitioned") {
+  spark.sql(
+s"""CREATE TABLE empty_orc_partitioned(key INT, value STRING)
+| PARTITIONED BY (p INT) STORED AS ORC
+  """.stripMargin)
+
+  val emptyDF = Seq.empty[(Int, String)].toDF("key", 
"value").coalesce(1)
+  emptyDF.createOrReplaceTempView("empty")
+
+  // Query empty table
+  val df = spark.sql(
+s"""SELECT key, value FROM empty_orc_partitioned
+| WHERE key > 10
+  """.stripMargin)
+  checkAnswer(df, emptyDF)
--- End diff --

A comment about the style. 
```Scala
  checkAnswer(
sql("SELECT key, value FROM empty_orc_partitioned WHERE key > 
10"),
emptyDF)
```


---
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 #14537: [SPARK-16948][SQL] Querying empty partitioned orc...

2016-08-23 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/14537#discussion_r75987730
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala ---
@@ -372,6 +372,29 @@ class OrcQuerySuite extends QueryTest with 
BeforeAndAfterAll with OrcTest {
 }
   }
 
+  test("SPARK-16948. Check empty orc partitioned tables in ORC") {
+withSQLConf((HiveUtils.CONVERT_METASTORE_ORC.key, "true")) {
+  withTempPath { dir =>
+withTable("empty_orc_partitioned") {
+  spark.sql(
+s"""CREATE TABLE empty_orc_partitioned(key INT, value STRING)
+| PARTITIONED BY (p INT) STORED AS ORC
+  """.stripMargin)
+
+  val emptyDF = Seq.empty[(Int, String)].toDF("key", 
"value").coalesce(1)
+  emptyDF.createOrReplaceTempView("empty")
--- End diff --

Could you remove this line?


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

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



[GitHub] spark pull request #14537: [SPARK-16948][SQL] Querying empty partitioned orc...

2016-08-23 Thread rajeshbalamohan
Github user rajeshbalamohan commented on a diff in the pull request:

https://github.com/apache/spark/pull/14537#discussion_r75967137
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
@@ -237,21 +237,26 @@ private[hive] class 
HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
   new Path(metastoreRelation.catalogTable.storage.locationUri.get),
   partitionSpec)
 
-val inferredSchema = if (fileType.equals("parquet")) {
-  val inferredSchema =
-defaultSource.inferSchema(sparkSession, options, 
fileCatalog.allFiles())
-  inferredSchema.map { inferred =>
-ParquetFileFormat.mergeMetastoreParquetSchema(metastoreSchema, 
inferred)
-  }.getOrElse(metastoreSchema)
-} else {
-  defaultSource.inferSchema(sparkSession, options, 
fileCatalog.allFiles()).get
+val inferredSchema =
+  defaultSource.inferSchema(sparkSession, options, 
fileCatalog.allFiles())
+val schema = fileType match {
+  case "parquet" =>
+// For Parquet, get correct schema by merging Metastore schema 
data types
+// and Parquet schema field names.
+inferredSchema.map { schema =>
+  
ParquetFileFormat.mergeMetastoreParquetSchema(metastoreSchema, schema)
+}.getOrElse(metastoreSchema)
+  case "orc" =>
+inferredSchema.getOrElse(metastoreSchema)
+  case _ =>
+inferredSchema.get
--- End diff --

Thanks @mallman . Addressed this in latest commit.


---
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 #14537: [SPARK-16948][SQL] Querying empty partitioned orc...

2016-08-23 Thread mallman
Github user mallman commented on a diff in the pull request:

https://github.com/apache/spark/pull/14537#discussion_r75904621
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
@@ -237,21 +237,26 @@ private[hive] class 
HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
   new Path(metastoreRelation.catalogTable.storage.locationUri.get),
   partitionSpec)
 
-val inferredSchema = if (fileType.equals("parquet")) {
-  val inferredSchema =
-defaultSource.inferSchema(sparkSession, options, 
fileCatalog.allFiles())
-  inferredSchema.map { inferred =>
-ParquetFileFormat.mergeMetastoreParquetSchema(metastoreSchema, 
inferred)
-  }.getOrElse(metastoreSchema)
-} else {
-  defaultSource.inferSchema(sparkSession, options, 
fileCatalog.allFiles()).get
+val inferredSchema =
+  defaultSource.inferSchema(sparkSession, options, 
fileCatalog.allFiles())
+val schema = fileType match {
+  case "parquet" =>
+// For Parquet, get correct schema by merging Metastore schema 
data types
+// and Parquet schema field names.
+inferredSchema.map { schema =>
+  
ParquetFileFormat.mergeMetastoreParquetSchema(metastoreSchema, schema)
+}.getOrElse(metastoreSchema)
+  case "orc" =>
+inferredSchema.getOrElse(metastoreSchema)
+  case _ =>
+inferredSchema.get
--- End diff --

IMHO returning `null` here would be worse than `inferredSchema.get`. Nobody 
likes debugging a `NullPointerException`. 😄

If we're going to assert that `fileType` must be either `parquet` or `orc`, 
I think throwing an exception like

throw new RuntimeException(s"Cannot convert a $fileType to a 
HadoopFsRelation")

here would be most appropriate.


---
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 #14537: [SPARK-16948][SQL] Querying empty partitioned orc...

2016-08-23 Thread rajeshbalamohan
Github user rajeshbalamohan commented on a diff in the pull request:

https://github.com/apache/spark/pull/14537#discussion_r75902767
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
@@ -237,21 +237,26 @@ private[hive] class 
HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
   new Path(metastoreRelation.catalogTable.storage.locationUri.get),
   partitionSpec)
 
-val inferredSchema = if (fileType.equals("parquet")) {
-  val inferredSchema =
-defaultSource.inferSchema(sparkSession, options, 
fileCatalog.allFiles())
-  inferredSchema.map { inferred =>
-ParquetFileFormat.mergeMetastoreParquetSchema(metastoreSchema, 
inferred)
-  }.getOrElse(metastoreSchema)
-} else {
-  defaultSource.inferSchema(sparkSession, options, 
fileCatalog.allFiles()).get
+val inferredSchema =
+  defaultSource.inferSchema(sparkSession, options, 
fileCatalog.allFiles())
+val schema = fileType match {
+  case "parquet" =>
+// For Parquet, get correct schema by merging Metastore schema 
data types
+// and Parquet schema field names.
+inferredSchema.map { schema =>
+  
ParquetFileFormat.mergeMetastoreParquetSchema(metastoreSchema, schema)
+}.getOrElse(metastoreSchema)
+  case "orc" =>
+inferredSchema.getOrElse(metastoreSchema)
+  case _ =>
+inferredSchema.get
--- End diff --

Not sure if exception has to be thrown in this case. Or just return null in 
this case?


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

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



[GitHub] spark pull request #14537: [SPARK-16948][SQL] Querying empty partitioned orc...

2016-08-23 Thread mallman
Github user mallman commented on a diff in the pull request:

https://github.com/apache/spark/pull/14537#discussion_r75884267
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
@@ -237,21 +237,26 @@ private[hive] class 
HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
   new Path(metastoreRelation.catalogTable.storage.locationUri.get),
   partitionSpec)
 
-val inferredSchema = if (fileType.equals("parquet")) {
-  val inferredSchema =
-defaultSource.inferSchema(sparkSession, options, 
fileCatalog.allFiles())
-  inferredSchema.map { inferred =>
-ParquetFileFormat.mergeMetastoreParquetSchema(metastoreSchema, 
inferred)
-  }.getOrElse(metastoreSchema)
-} else {
-  defaultSource.inferSchema(sparkSession, options, 
fileCatalog.allFiles()).get
+val inferredSchema =
+  defaultSource.inferSchema(sparkSession, options, 
fileCatalog.allFiles())
+val schema = fileType match {
+  case "parquet" =>
+// For Parquet, get correct schema by merging Metastore schema 
data types
+// and Parquet schema field names.
+inferredSchema.map { schema =>
+  
ParquetFileFormat.mergeMetastoreParquetSchema(metastoreSchema, schema)
+}.getOrElse(metastoreSchema)
+  case "orc" =>
+inferredSchema.getOrElse(metastoreSchema)
+  case _ =>
+inferredSchema.get
--- End diff --

@gatorsmile has asserted that we shouldn't get here. So shouldn't we throw 
some kind of 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 #14537: [SPARK-16948][SQL] Querying empty partitioned orc...

2016-08-23 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/14537#discussion_r75812657
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
@@ -237,21 +237,24 @@ private[hive] class 
HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
   new Path(metastoreRelation.catalogTable.storage.locationUri.get),
   partitionSpec)
 
-val inferredSchema = if (fileType.equals("parquet")) {
-  val inferredSchema =
-defaultSource.inferSchema(sparkSession, options, 
fileCatalog.allFiles())
-  inferredSchema.map { inferred =>
-ParquetFileFormat.mergeMetastoreParquetSchema(metastoreSchema, 
inferred)
+val inferredSchema =
+  defaultSource.inferSchema(sparkSession, options, 
fileCatalog.allFiles())
+val schema = if (fileType == "parquet") {
+  // For Parquet, get correct schema by merging Metastore schema 
data types
+  // and Parquet schema field names.
+  inferredSchema.map { schema =>
+ParquetFileFormat.mergeMetastoreParquetSchema(metastoreSchema, 
schema)
   }.getOrElse(metastoreSchema)
 } else {
-  defaultSource.inferSchema(sparkSession, options, 
fileCatalog.allFiles()).get
+  // For others (e.g orc), fall back to metastore schema if needed.
--- End diff --

If the fileType is not parquet, the only possible type is orc. To ensure 
this is always true, we should check `fileType == "orc"`.


---
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 #14537: [SPARK-16948][SQL] Querying empty partitioned orc...

2016-08-23 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/14537#discussion_r75812470
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala ---
@@ -372,6 +372,48 @@ class OrcQuerySuite extends QueryTest with 
BeforeAndAfterAll with OrcTest {
 }
   }
 
+  test("SPARK-16948. Check empty orc partitioned tables in ORC") {
+withSQLConf((HiveUtils.CONVERT_METASTORE_ORC.key, "true")) {
+  withTempPath { dir =>
+withTable("empty_orc_partitioned") {
+  spark.sql(
+s"""CREATE TABLE empty_orc_partitioned(key INT, value STRING)
+| PARTITIONED BY (p INT) STORED AS ORC
+  """.stripMargin)
+
+  val emptyDF = Seq.empty[(Int, String)].toDF("key", 
"value").coalesce(1)
+  emptyDF.createOrReplaceTempView("empty")
+
+  // Query empty table
+  val df = spark.sql(
+s"""SELECT key, value FROM empty_orc_partitioned
+| WHERE key > 10
+  """.stripMargin)
+  checkAnswer(df, emptyDF)
+}
+  }
+
+  withTempPath { dir =>
+withTable("empty_text_partitioned") {
+  spark.sql(
+s"""CREATE TABLE empty_text_partitioned(key INT, value STRING)
+| PARTITIONED BY (p INT) STORED AS TEXTFILE
--- End diff --

Testing the textfile format sounds useless. We do not convert it to 
`LogicalRelation`.


---
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 #14537: [SPARK-16948][SQL] Querying empty partitioned orc...

2016-08-17 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/14537#discussion_r75067717
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
@@ -287,21 +287,21 @@ private[hive] class 
HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
   new Path(metastoreRelation.catalogTable.storage.locationUri.get),
   partitionSpec)
 
-val inferredSchema = if (fileType.equals("parquet")) {
-  val inferredSchema =
-defaultSource.inferSchema(sparkSession, options, 
fileCatalog.allFiles())
-  inferredSchema.map { inferred =>
-ParquetFileFormat.mergeMetastoreParquetSchema(metastoreSchema, 
inferred)
+val inferredSchema =
+  defaultSource.inferSchema(sparkSession, options, 
fileCatalog.allFiles())
--- End diff --

can we be more specific here? e.g. doing it only if it is parquet or orc.



---
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 #14537: [SPARK-16948][SQL] Querying empty partitioned orc...

2016-08-17 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/14537#discussion_r75067639
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala ---
@@ -56,10 +59,11 @@ private[sql] class OrcFileFormat
   sparkSession: SparkSession,
   options: Map[String, String],
   files: Seq[FileStatus]): Option[StructType] = {
-OrcFileOperator.readSchema(
-  files.map(_.getPath.toUri.toString),
-  Some(sparkSession.sessionState.newHadoopConf())
-)
+val schema = Try(OrcFileOperator.readSchema(
+files.map(_.getPath.toUri.toString),
+Some(sparkSession.sessionState.newHadoopConf(
+  .recover { case _: FileNotFoundException => None }
--- End diff --

why are we ignoring file not found 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 #14537: [SPARK-16948][SQL] Querying empty partitioned orc...

2016-08-09 Thread mallman
Github user mallman commented on a diff in the pull request:

https://github.com/apache/spark/pull/14537#discussion_r74092780
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
@@ -287,14 +287,14 @@ private[hive] class 
HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
   new Path(metastoreRelation.catalogTable.storage.locationUri.get),
   partitionSpec)
 
+val schema =
+  defaultSource.inferSchema(sparkSession, options, 
fileCatalog.allFiles())
 val inferredSchema = if (fileType.equals("parquet")) {
--- End diff --

I just noticed the boolean expression should be `fileType == "parquet"` to 
make it idiomatic Scala. Can you make that change?


---
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 #14537: [SPARK-16948][SQL] Querying empty partitioned orc...

2016-08-09 Thread mallman
Github user mallman commented on a diff in the pull request:

https://github.com/apache/spark/pull/14537#discussion_r74092457
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
@@ -287,14 +287,14 @@ private[hive] class 
HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
   new Path(metastoreRelation.catalogTable.storage.locationUri.get),
   partitionSpec)
 
+val schema =
--- End diff --

Thanks for refactoring this.

I think it makes more sense if

defaultSource.inferSchema(sparkSession, options, fileCatalog.allFiles())

is called `inferredSchema` and the value of the `if 
(fileType.equals("parquet"))` expression is called `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 #14537: [SPARK-16948][SQL] Querying empty partitioned orc...

2016-08-09 Thread mallman
Github user mallman commented on a diff in the pull request:

https://github.com/apache/spark/pull/14537#discussion_r74003788
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
@@ -294,7 +294,9 @@ private[hive] class HiveMetastoreCatalog(sparkSession: 
SparkSession) extends Log
 ParquetFileFormat.mergeMetastoreParquetSchema(metastoreSchema, 
inferred)
   }.getOrElse(metastoreSchema)
 } else {
-  defaultSource.inferSchema(sparkSession, options, 
fileCatalog.allFiles()).get
+  val inferredSchema =
--- End diff --

There's some code duplicated in both branches of this `if` expression. Can 
you refactor it to remove the duplication, please?


---
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 #14537: [SPARK-16948][SQL] Querying empty partitioned orc...

2016-08-08 Thread rajeshbalamohan
GitHub user rajeshbalamohan opened a pull request:

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

[SPARK-16948][SQL] Querying empty partitioned orc tables throws excep…

## What changes were proposed in this pull request?
Querying empty partitioned ORC tables from spark-sql throws exception with 
`spark.sql.hive.convertMetastoreOrc=true`.  This is due to the fact that the 
inferschema() would end up throwing `FileNotFoundException` when no files are 
present in partitioned orc tables.  Patch attempts to fix it, wherein it would 
fall back to metastore based schema information. 

## How was this patch tested?
Included unit tests and also tested it in small scale cluster.


(If this patch involves UI changes, please attach a screenshot; otherwise, 
remove this)


…tion

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

$ git pull https://github.com/rajeshbalamohan/spark SPARK-16948

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

https://github.com/apache/spark/pull/14537.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 #14537


commit 5721b88c7c816f57ef39374ac9b335d870543628
Author: Rajesh Balamohan 
Date:   2016-08-08T08:28:23Z

[SPARK-16948][SQL] Querying empty partitioned orc tables throws exception




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