Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SUPPORT]Unable to read hudi table and got an IllegalArgumentException: For input string: "null" #8061

Closed
wolf8334 opened this issue Feb 27, 2023 · 8 comments
Assignees
Labels
priority:major degraded perf; unable to move forward; potential bugs schema-and-data-types spark-sql version-compatibility

Comments

@wolf8334
Copy link

wolf8334 commented Feb 27, 2023

Describe the problem you faced

I use java and spark 3.3 to read hudi 0.13.0 table following the guide on offical website.
The guide says this will work,but I got an IllegalArgumentException: For input string: "null".

To Reproduce

Steps to reproduce the behavior:

1.generate one hudi COW table from mysql table.
2.get access to the COW table through spark sql
3.the IllegalArgumentException: For input string: "null" shows.
4.I have already changed the datasource and the table structure,It has no relationship with this.
5 I use this command line and I am sure there are datas in my parquet file.
./hadoop jar ~/parquet-tools-1.9.0.jar cat hdfs://192.168.5.128:9000/user/spark/hudi/1/2.parquet

Expected behavior

the data is shown.

Environment Description

  • Hudi version :
    0.12.2,0.13.0

  • Spark version :
    3.3.2

  • Hive version :
    none

  • Hadoop version :
    3.3.4

  • Storage (HDFS/S3/GCS..) :
    HDFS

  • Running on Docker? (yes/no) :
    no.my local laptop

Additional context
JDK 1.8

Add any other context about the problem here.
`Map<String, String> hudiConf = new HashMap<>();
hudiConf.put(HoodieWriteConfig.TBL_NAME.key(), "t_yklc_info");

        Dataset<Row> demods = getActiveSession().read().options(hudiConf).format("org.apache.hudi").load("/user/spark/hudi/*/*");

        demods.createOrReplaceTempView("lcinfo");
        demods.printSchema();

        logger.info(getActiveSession().conf().get(SQLConf.LEGACY_PARQUET_NANOS_AS_LONG().key()).toString());
        logger.info(getActiveSession().conf().get(SQLConf.PARQUET_BINARY_AS_STRING().key()).toString());
        logger.info(getActiveSession().conf().get(SQLConf.PARQUET_INT96_AS_TIMESTAMP().key()).toString());
        logger.info(getActiveSession().conf().get(SQLConf.CASE_SENSITIVE().key()).toString());


        Dataset<Row> ds = getActiveSession().sql("select APP_NO from lcinfo where APP_NO = '1' and STAT_CYCLE = '2'");
        ds.printSchema();
        ds.show();`

Stacktrace
INFO 18:45:03.183 | org.apache.spark.sql.execution.datasources.FileScanRDD | Reading File path: hdfs://192.168.5.128:9000/user/spark/hudi/2/1.parquet, range: 0-3964741, partition values: [empty row]
ERROR 18:45:03.420 | org.apache.spark.executor.Executor | Exception in task 3.0 in stage 1.0 (TID 60)
java.lang.IllegalArgumentException: For input string: "null"
at scala.collection.immutable.StringLike.parseBoolean(StringLike.scala:330) ~[scala-library-2.12.15.jar:?]
at scala.collection.immutable.StringLike.toBoolean(StringLike.scala:289) ~[scala-library-2.12.15.jar:?]
at scala.collection.immutable.StringLike.toBoolean$(StringLike.scala:289) ~[scala-library-2.12.15.jar:?]
at scala.collection.immutable.StringOps.toBoolean(StringOps.scala:33) ~[scala-library-2.12.15.jar:?]
at org.apache.spark.sql.execution.datasources.parquet.ParquetToSparkSchemaConverter.(ParquetSchemaConverter.scala:70) ~[spark-sql_2.12-3.3.2.jar:3.3.2]
at org.apache.spark.sql.execution.datasources.parquet.HoodieParquetFileFormatHelper$.buildImplicitSchemaChangeInfo(HoodieParquetFileFormatHelper.scala:30) ~[hudi-spark3.3-bundle_2.12-0.13.0.jar:3.3.2]
at org.apache.spark.sql.execution.datasources.parquet.Spark32PlusHoodieParquetFileFormat.$anonfun$buildReaderWithPartitionValues$2(Spark32PlusHoodieParquetFileFormat.scala:231) ~[hudi-spark3.3-bundle_2.12-0.13.0.jar:3.3.2]
at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.org$apache$spark$sql$execution$datasources$FileScanRDD$$anon$$readCurrentFile(FileScanRDD.scala:209) ~[spark-sql_2.12-3.3.2.jar:3.3.2]
at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.nextIterator(FileScanRDD.scala:270) ~[spark-sql_2.12-3.3.2.jar:3.3.2]
at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.hasNext(FileScanRDD.scala:116) ~[spark-sql_2.12-3.3.2.jar:3.3.2]
at org.apache.spark.sql.execution.FileSourceScanExec$$anon$1.hasNext(DataSourceScanExec.scala:561) ~[spark-sql_2.12-3.3.2.jar:3.3.2]
at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.columnartorow_nextBatch_0$(Unknown Source) ~[?:?]
at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(Unknown Source) ~[?:?]
at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43) ~[spark-sql_2.12-3.3.2.jar:3.3.2]
at org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:760) ~[spark-sql_2.12-3.3.2.jar:3.3.2]
at org.apache.spark.sql.execution.SparkPlan.$anonfun$getByteArrayRdd$1(SparkPlan.scala:364) ~[spark-sql_2.12-3.3.2.jar:3.3.2]
at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:890) ~[spark-core_2.12-3.3.2.jar:3.3.2]
at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:890) ~[spark-core_2.12-3.3.2.jar:3.3.2]
at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52) ~[spark-core_2.12-3.3.2.jar:3.3.2]
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:365) ~[spark-core_2.12-3.3.2.jar:3.3.2]
at org.apache.spark.rdd.RDD.iterator(RDD.scala:329) ~[spark-core_2.12-3.3.2.jar:3.3.2]
at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90) ~[spark-core_2.12-3.3.2.jar:3.3.2]
at org.apache.spark.scheduler.Task.run(Task.scala:136) ~[spark-core_2.12-3.3.2.jar:3.3.2]
at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:548) ~[spark-core_2.12-3.3.2.jar:3.3.2]
at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1504) ~[spark-core_2.12-3.3.2.jar:3.3.2]
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:551) ~[spark-core_2.12-3.3.2.jar:3.3.2]
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) ~[?:1.8.0_362]
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) ~[?:1.8.0_362]
at java.lang.Thread.run(Thread.java:750) ~[?:1.8.0_362]

@wolf8334
Copy link
Author

I add these code and it works.But still wander why.
sc.set("spark.sql.legacy.parquet.nanosAsLong", "false");
sc.set("spark.sql.parquet.binaryAsString", "false");
sc.set("spark.sql.parquet.int96AsTimestamp", "true");
sc.set("spark.sql.caseSensitive", "false");

@caokz
Copy link

caokz commented Mar 8, 2023

I also encountered this problem when using hudi 0.13.0 on spark 3.3.2 and found that an exception was thrown when querying the mor table and the merge type was "REALTIME_PAYLOAD_COMBINE". The reason for this is that spark 3.3.2 is not compatible with the ParquetToSparkSchemaConverter class of spark 3.3.1. The constructor method of the ParquetToSparkSchemaConverterl class in spark 3.3.2 requires the "LEGACY_PARQUET_NANOS_AS_LONG" configuration parameter, whereas in The buildReaderWithPartitionValues method of the Spark32PlusHoodieParquetFileFormatl class does not initialize the value of this parameter. So my conclusion is that hudi 0.13.0 is currently not compatible with spark 3.3.2.

@danny0405
Copy link
Contributor

danny0405 commented Mar 8, 2023

Thanks for the feedback, found that there seems already a fixing PR: #8082,
let's move the discussions there and it's great if you guys can help the review.

@cmanning-arcadia
Copy link

cmanning-arcadia commented Apr 10, 2023

If anyone finds this and has the issue from spark-shell, try adding the parameters as command line args. Example:

spark-shell --packages org.apache.hudi:hudi-spark3.3-bundle_2.12:0.12.2 \
--conf 'spark.serializer=org.apache.spark.serializer.KryoSerializer' \
--conf 'spark.sql.extensions=org.apache.spark.sql.hudi.HoodieSparkSessionExtension' \
--conf 'spark.sql.catalog.spark_catalog=org.apache.spark.sql.hudi.catalog.HoodieCatalog' \
--conf 'spark.hadoop.spark.sql.legacy.parquet.nanosAsLong=false' \
--conf 'spark.hadoop.spark.sql.parquet.binaryAsString=false' \
--conf 'spark.hadoop.spark.sql.parquet.int96AsTimestamp=true' \
--conf 'spark.hadoop.spark.sql.caseSensitive=false'

@codope codope added the priority:major degraded perf; unable to move forward; potential bugs label Apr 12, 2023
@codope
Copy link
Member

codope commented Apr 12, 2023

Closing as we have the PR and we will followup there.

@codope codope closed this as completed Apr 12, 2023
@bigdata-spec
Copy link

Thanks for the feedback, found that there seems already a fixing PR: #8082, let's move the discussions there and it's great if you guys can help the review.

Hi,I will try Spark3.3.2 and Hudi 0.13,does it mean master can fix this problem?

@bigdata-spec
Copy link

bigdata-spec commented Apr 21, 2023

spark.hadoop.spark.sql.parquet.binaryAsString

@cmanning-arcadia Hi, I have some doubt,
I find

--conf 'spark.hadoop.spark.sql.legacy.parquet.nanosAsLong=false' \
--conf 'spark.hadoop.spark.sql.parquet.binaryAsString=false' \
--conf 'spark.hadoop.spark.sql.parquet.int96AsTimestamp=true' \
--conf 'spark.hadoop.spark.sql.caseSensitive=false'

in Apache Spark3.3.2 source , spark.hadoop.spark.sql.legacy.parquet.nanosAsLong is false originally and so on.

@yihua
Copy link
Contributor

yihua commented Jul 21, 2023

Hi @bigdata-spec Have you tried Hudi 0.13.1 release which is compatible with Spark 3.3.2 release, without adding additional spark configs above?

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
priority:major degraded perf; unable to move forward; potential bugs schema-and-data-types spark-sql version-compatibility
Projects
Archived in project
Development

No branches or pull requests

7 participants