stream2000 opened a new issue, #11776:
URL: https://github.com/apache/hudi/issues/11776

   **_Tips before filing an issue_**
   
   - Have you gone through our [FAQs](https://hudi.apache.org/learn/faq/)?
   
   - Join the mailing list to engage in conversations and get faster support at 
[email protected].
   
   - If you have triaged this as a bug, then file an 
[issue](https://issues.apache.org/jira/projects/HUDI/issues) directly.
   
   **Describe the problem you faced**
   
   8350 [task-result-getter-0] WARN  org.apache.spark.scheduler.TaskSetManager 
[] - Lost task 0.0 in stage 9.0 (TID 8) (30.221.115.93 executor driver): 
org.apache.hudi.exception.HoodieKeyException: recordKey value: "null" for 
field: "ID" cannot be null or empty.
   
   the primary key config must be in lower case now.
   
   
   **To Reproduce**
   
   ```scala
     test("Test primary key case sensitive") {
       withTempDir { tmp =>
         val tableName = generateTableName
         // Create a partitioned table
         spark.sql(
           s"""
              |create table $tableName (
              |  id int,
              |  name string,
              |  price double,
              |  ts long,
              |  dt string
              |) using hudi
              | tblproperties (primaryKey = 'ID'
              | )
              | partitioned by (dt)
              | location '${tmp.getCanonicalPath}'
          """.stripMargin)
         spark.sql(
           s"""
              | insert into $tableName
              | select 1 as id, 'a1' as name, 10 as price, 1000 as ts, 
'2021-01-05' as dt
           """.stripMargin)
         checkAnswer(s"select id, name, price, ts, dt from $tableName")(
           Seq(1, "a1", 10.0, 1000 , "2021-01-05")
         )
       }
     }
   ```
   
   **Expected behavior**
   
   the primary key config should be case - insensitive
   
   **Environment Description**
   
   * Hudi version :
   * master branch
   
   
   * Spark version :
   
   3.2.0 
   
   * Hive version :
   
   * Hadoop version :
   
   * Storage (HDFS/S3/GCS..) :
   
   * Running on Docker? (yes/no) :
   
   **Additional context**
   
   Add any other context about the problem here.
   
   **Stacktrace**
   
   ```Add the stacktrace of the error.```
   8350 [task-result-getter-0] WARN  org.apache.spark.scheduler.TaskSetManager 
[] - Lost task 0.0 in stage 9.0 (TID 8) (30.221.115.93 executor driver): 
org.apache.hudi.exception.HoodieKeyException: recordKey value: "null" for 
field: "ID" cannot be null or empty.
        at org.apache.hudi.keygen.KeyGenUtils.getRecordKey(KeyGenUtils.java:205)
        at 
org.apache.hudi.keygen.SimpleAvroKeyGenerator.getRecordKey(SimpleAvroKeyGenerator.java:50)
        at 
org.apache.hudi.keygen.SimpleKeyGenerator.getRecordKey(SimpleKeyGenerator.java:64)
        at 
org.apache.hudi.keygen.BaseKeyGenerator.getKey(BaseKeyGenerator.java:70)
        at 
org.apache.spark.sql.hudi.command.SqlKeyGenerator.$anonfun$getRecordKey$1(SqlKeyGenerator.scala:79)
        at scala.Option.map(Option.scala:230)
        at 
org.apache.spark.sql.hudi.command.SqlKeyGenerator.getRecordKey(SqlKeyGenerator.scala:79)
        at 
org.apache.hudi.HoodieCreateRecordUtils$.getHoodieKeyAndMaybeLocationFromAvroRecord(HoodieCreateRecordUtils.scala:206)
        at 
org.apache.hudi.HoodieCreateRecordUtils$.$anonfun$createHoodieRecordRdd$5(HoodieCreateRecordUtils.scala:133)
        at scala.collection.Iterator$$anon$10.next(Iterator.scala:461)
        at 
org.apache.spark.storage.memory.MemoryStore.putIterator(MemoryStore.scala:224)
        at 
org.apache.spark.storage.memory.MemoryStore.putIteratorAsBytes(MemoryStore.scala:352)
        at 
org.apache.spark.storage.BlockManager.$anonfun$doPutIterator$1(BlockManager.scala:1508)
        at 
org.apache.spark.storage.BlockManager.org$apache$spark$storage$BlockManager$$doPut(BlockManager.scala:1418)
        at 
org.apache.spark.storage.BlockManager.doPutIterator(BlockManager.scala:1482)
        at 
org.apache.spark.storage.BlockManager.getOrElseUpdate(BlockManager.scala:1305)
        at org.apache.spark.rdd.RDD.getOrCompute(RDD.scala:384)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:335)
        at 
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
        at 
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
        at 
org.apache.spark.shuffle.ShuffleWriteProcessor.write(ShuffleWriteProcessor.scala:59)
        at 
org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:99)
        at 
org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:52)
        at org.apache.spark.scheduler.Task.run(Task.scala:131)
        at 
org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:506)
        at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1491)
        at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:509)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:750
   


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

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to