Uploaded image for project: 'Spark'
  1. Spark
  2. SPARK-15248

Partition added with ALTER TABLE to a hive partitioned table is not read while querying

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • None
    • 2.0.0
    • SQL
    • None

    Description

      Table partitions can be added with locations different from default warehouse location of a hive table.

      CREATE TABLE parquetTable (a int) PARTITIONED BY (b int) STORED AS parquet
      ALTER TABLE parquetTable ADD PARTITION (b=1) LOCATION '/path/1'

      Querying such a table throws error as the MetastoreFileCatalog does not list the added partition directory, it only lists the default base location.

      asd
      asd

      [info] - SPARK-15248: explicitly added partitions should be readable *** FAILED *** (1 second, 8 milliseconds)
      [info]   java.util.NoSuchElementException: key not found: file:/Users/tdas/Projects/Spark/spark2/target/tmp/spark-b39ad224-c5d1-4966-8981-fb45a2066d61/non-default-partition
      [info]   at scala.collection.MapLike$class.default(MapLike.scala:228)
      [info]   at scala.collection.AbstractMap.default(Map.scala:59)
      [info]   at scala.collection.MapLike$class.apply(MapLike.scala:141)
      [info]   at scala.collection.AbstractMap.apply(Map.scala:59)
      [info]   at org.apache.spark.sql.execution.datasources.PartitioningAwareFileCatalog$$anonfun$listFiles$1.apply(PartitioningAwareFileCatalog.scala:59)
      [info]   at org.apache.spark.sql.execution.datasources.PartitioningAwareFileCatalog$$anonfun$listFiles$1.apply(PartitioningAwareFileCatalog.scala:55)
      [info]   at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
      [info]   at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
      [info]   at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
      [info]   at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
      [info]   at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
      [info]   at scala.collection.AbstractTraversable.map(Traversable.scala:104)
      [info]   at org.apache.spark.sql.execution.datasources.PartitioningAwareFileCatalog.listFiles(PartitioningAwareFileCatalog.scala:55)
      [info]   at org.apache.spark.sql.execution.datasources.FileSourceStrategy$.apply(FileSourceStrategy.scala:93)
      [info]   at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:59)
      [info]   at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:59)
      [info]   at scala.collection.Iterator$$anon$12.nextCur(Iterator.scala:434)
      [info]   at scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:440)
      [info]   at org.apache.spark.sql.catalyst.planning.QueryPlanner.plan(QueryPlanner.scala:60)
      [info]   at org.apache.spark.sql.catalyst.planning.QueryPlanner.planLater(QueryPlanner.scala:55)
      [info]   at org.apache.spark.sql.execution.SparkStrategies$SpecialLimits$.apply(SparkStrategies.scala:55)
      [info]   at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:59)
      [info]   at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:59)
      [info]   at scala.collection.Iterator$$anon$12.nextCur(Iterator.scala:434)
      [info]   at scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:440)
      [info]   at org.apache.spark.sql.catalyst.planning.QueryPlanner.plan(QueryPlanner.scala:60)
      [info]   at org.apache.spark.sql.execution.QueryExecution.sparkPlan$lzycompute(QueryExecution.scala:77)
      [info]   at org.apache.spark.sql.execution.QueryExecution.sparkPlan(QueryExecution.scala:75)
      [info]   at org.apache.spark.sql.execution.QueryExecution.executedPlan$lzycompute(QueryExecution.scala:82)
      [info]   at org.apache.spark.sql.execution.QueryExecution.executedPlan(QueryExecution.scala:82)
      [info]   at org.apache.spark.sql.QueryTest.assertEmptyMissingInput(QueryTest.scala:330)
      [info]   at org.apache.spark.sql.QueryTest.checkAnswer(QueryTest.scala:146)
      [info]   at org.apache.spark.sql.QueryTest.checkAnswer(QueryTest.scala:159)
      [info]   at org.apache.spark.sql.hive.ParquetMetastoreSuite$$anonfun$12$$anonfun$apply$mcV$sp$7$$anonfun$apply$mcV$sp$25.apply(parquetSuites.scala:554)
      [info]   at org.apache.spark.sql.hive.ParquetMetastoreSuite$$anonfun$12$$anonfun$apply$mcV$sp$7$$anonfun$apply$mcV$sp$25.apply(parquetSuites.scala:535)
      [info]   at org.apache.spark.sql.test.SQLTestUtils$class.withTempDir(SQLTestUtils.scala:125)
      [info]   at org.apache.spark.sql.hive.ParquetPartitioningTest.withTempDir(parquetSuites.scala:726)
      [info]   at org.apache.spark.sql.hive.ParquetMetastoreSuite$$anonfun$12$$anonfun$apply$mcV$sp$7.apply$mcV$sp(parquetSuites.scala:535)
      [info]   at org.apache.spark.sql.test.SQLTestUtils$class.withTable(SQLTestUtils.scala:166)
      [info]   at org.apache.spark.sql.hive.ParquetPartitioningTest.withTable(parquetSuites.scala:726)
      [info]   at org.apache.spark.sql.hive.ParquetMetastoreSuite$$anonfun$12.apply$mcV$sp(parquetSuites.scala:534)
      [info]   at org.apache.spark.sql.hive.ParquetMetastoreSuite$$anonfun$12.apply(parquetSuites.scala:534)
      [info]   at org.apache.spark.sql.hive.ParquetMetastoreSuite$$anonfun$12.apply(parquetSuites.scala:534)
      

      Attachments

        Activity

          People

            tdas Tathagata Das
            tdas Tathagata Das
            Votes:
            0 Vote for this issue
            Watchers:
            3 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: