sivabalan narayanan created HUDI-3041:
-----------------------------------------

             Summary: drop partitions fails with spark.read
                 Key: HUDI-3041
                 URL: https://issues.apache.org/jira/browse/HUDI-3041
             Project: Apache Hudi
          Issue Type: Bug
          Components: Spark Integration
    Affects Versions: 0.9.0
            Reporter: sivabalan narayanan


With 0.9.0, we added support for dropping partition columns after generating 
the partition path for hoodie records. but looks like we have some gaps in end 
to end flow. 

 

hudi serializes partition path fields to table properties and re-uses when in 
need. So, while querying the field may not be part of table schema since hudi 
would have removed the field (along with trimming the schema) completely. 

Stacktrace for Issue (1)
{code:java}
scala> val tripsSnapshotDF = spark.read.format("hudi").load(basePath)
java.lang.IllegalArgumentException: Cannot find column: 'partitionId' in the 
schema[StructField(_hoodie_commit_time,StringType,true),StructField(_hoodie_commit_seqno,StringType,true),StructField(_hoodie_record_key,StringType,true),StructField(_hoodie_partition_path,StringType,true),StructField(_hoodie_file_name,StringType,true),StructField(rowId,StringType,true),StructField(preComb,LongType,true),StructField(name,StringType,true),StructField(versionId,StringType,true),StructField(toBeDeletedStr,StringType,true),StructField(intToLong,IntegerType,true),StructField(longToInt,LongType,true)]
  at 
org.apache.hudi.HoodieFileIndex$$anonfun$5$$anonfun$apply$1.apply(HoodieFileIndex.scala:106)
  at 
org.apache.hudi.HoodieFileIndex$$anonfun$5$$anonfun$apply$1.apply(HoodieFileIndex.scala:106)
  at scala.collection.MapLike$class.getOrElse(MapLike.scala:128)
  at scala.collection.AbstractMap.getOrElse(Map.scala:59)
  at org.apache.hudi.HoodieFileIndex$$anonfun$5.apply(HoodieFileIndex.scala:106)
  at org.apache.hudi.HoodieFileIndex$$anonfun$5.apply(HoodieFileIndex.scala:105)
  at 
scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
  at 
scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
  at 
scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33)
  at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:186)
  at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
  at scala.collection.mutable.ArrayOps$ofRef.map(ArrayOps.scala:186)
  at 
org.apache.hudi.HoodieFileIndex._partitionSchemaFromProperties$lzycompute(HoodieFileIndex.scala:105)
  at 
org.apache.hudi.HoodieFileIndex._partitionSchemaFromProperties(HoodieFileIndex.scala:99)
  at 
org.apache.hudi.HoodieFileIndex.getAllQueryPartitionPaths(HoodieFileIndex.scala:348)
  at 
org.apache.hudi.HoodieFileIndex.loadPartitionPathFiles(HoodieFileIndex.scala:420)
  at org.apache.hudi.HoodieFileIndex.refresh0(HoodieFileIndex.scala:214)
  at org.apache.hudi.HoodieFileIndex.<init>(HoodieFileIndex.scala:149)
  at org.apache.hudi.DefaultSource.getBaseFileOnlyView(DefaultSource.scala:199)
  at org.apache.hudi.DefaultSource.createRelation(DefaultSource.scala:116)
  at org.apache.hudi.DefaultSource.createRelation(DefaultSource.scala:67)
  at 
org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:318)
  at 
org.apache.spark.sql.DataFrameReader.loadV1Source(DataFrameReader.scala:223)
  at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:211)
  at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:178)
  ... 63 elided {code}
Steps to reproduce: 

[https://gist.github.com/nsivabalan/570a96004e41f84565c99d8994b12d57]

 



--
This message was sent by Atlassian Jira
(v8.20.1#820001)

Reply via email to