Giangblackk opened a new issue, #11113: URL: https://github.com/apache/iceberg/issues/11113
### Apache Iceberg version 1.6.1 (latest release) ### Query engine Spark ### Please describe the bug 🐞 I faced the error when streaming read from an Iceberg table in Minio S3 with Nessie catalog. This error only happen to streaming read, the streaming write part is totally work. My detail code is in this github repository: https://github.com/Giangblackk/streaming_benchmark/tree/feature/minio-nessie Context: - Run Pyspark to connect to Spark Connect server 3.5.2 with configured Nessie Catalog - Streaming read from Iceberg table in Minio S3 and write to Kafka Spark Connect server command: ```bash bash /opt/spark/sbin/start-connect-server.sh \ --packages org.apache.spark:spark-connect_2.12:3.5.2,org.apache.iceberg:iceberg-spark-runtime-3.5_2.12:1.6.1,org.apache.spark:spark-sql-kafka-0-10_2.12:3.5.2,org.projectnessie.nessie-integrations:nessie-spark-extensions-3.5_2.12:0.96.0 \ --jars /opt/spark/jars/postgresql-42.7.4.jar,/opt/spark/jars/iceberg-aws-bundle-1.6.1.jar \ --driver-class-path /opt/spark/jars/postgresql-42.7.4.jar,/opt/spark/jars/iceberg-aws-bundle-1.6.1.jar \ --conf spark.sql.extensions=org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions,org.projectnessie.spark.extensions.NessieSparkSessionExtensions \ --conf spark.sql.catalog.nessie=org.apache.iceberg.spark.SparkCatalog \ --conf spark.sql.catalog.nessie.warehouse=s3://minio/warehouse \ --conf spark.sql.catalog.nessie.s3.endpoint=http://minio:9000 \ --conf spark.sql.catalog.nessie.catalog-impl=org.apache.iceberg.nessie.NessieCatalog \ --conf spark.sql.catalog.nessie.io-impl=org.apache.iceberg.aws.s3.S3FileIO \ --conf spark.sql.catalog.nessie.uri=http://nessie:19120/api/v1 \ --conf spark.sql.catalog.nessie.ref=main \ --conf spark.sql.catalog.nessie.cache-enabled=false ``` Pyspark code: ```python from pyspark.sql import SparkSession spark: SparkSession = ( SparkSession.builder.appName("iceberg_stream_read") .remote("sc://localhost:15003") .getOrCreate() ) spark.catalog.setCurrentCatalog("nessie") df = spark.readStream.format("iceberg").load("kafka_topic") df.selectExpr("id as key", "value").writeStream.format("kafka").option( "kafka.bootstrap.servers", "redpanda-0:9092" ).option("checkpointLocation", "/opt/spark/checkpoint2").option( "topic", "sink-topic" ).start() ``` I met the error that seem like iceberg handle checkpoint path incorrectly: ``` org.apache.iceberg.exceptions.ValidationException: Invalid S3 URI, cannot determine scheme: file:/opt/spark/checkpoint2/sources/0/offsets/0 at org.apache.iceberg.exceptions.ValidationException.check(ValidationException.java:49) at org.apache.iceberg.aws.s3.S3URI.<init>(S3URI.java:72) at org.apache.iceberg.aws.s3.S3InputFile.fromLocation(S3InputFile.java:39) at org.apache.iceberg.aws.s3.S3FileIO.newInputFile(S3FileIO.java:125) at org.apache.iceberg.spark.source.SparkMicroBatchStream$InitialOffsetStore.initialOffset(SparkMicroBatchStream.java:486) at org.apache.iceberg.spark.source.SparkMicroBatchStream.<init>(SparkMicroBatchStream.java:110) at org.apache.iceberg.spark.source.SparkScan.toMicroBatchStream(SparkScan.java:158) at org.apache.spark.sql.execution.streaming.MicroBatchExecution$$anonfun$1.$anonfun$applyOrElse$4(MicroBatchExecution.scala:107) at scala.collection.mutable.HashMap.getOrElseUpdate(HashMap.scala:86) at org.apache.spark.sql.execution.streaming.MicroBatchExecution$$anonfun$1.applyOrElse(MicroBatchExecution.scala:100) at org.apache.spark.sql.execution.streaming.MicroBatchExecution$$anonfun$1.applyOrElse(MicroBatchExecution.scala:84) at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$1(TreeNode.scala:461) at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(origin.scala:76) at org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:461) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:32) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning(AnalysisHelper.scala:267) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning$(AnalysisHelper.scala:263) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:32) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:32) at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$3(TreeNode.scala:466) at org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren(TreeNode.scala:1216) at org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren$(TreeNode.scala:1215) at org.apache.spark.sql.catalyst.plans.logical.Project.mapChildren(basicLogicalOperators.scala:71) at org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:466) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:32) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning(AnalysisHelper.scala:267) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning$(AnalysisHelper.scala:263) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:32) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:32) at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:437) at org.apache.spark.sql.catalyst.trees.TreeNode.transform(TreeNode.scala:405) at org.apache.spark.sql.execution.streaming.MicroBatchExecution.logicalPlan$lzycompute(MicroBatchExecution.scala:84) at org.apache.spark.sql.execution.streaming.MicroBatchExecution.logicalPlan(MicroBatchExecution.scala:64) at org.apache.spark.sql.execution.streaming.StreamExecution.$anonfun$runStream$1(StreamExecution.scala:303) at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23) at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:900) at org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$runStream(StreamExecution.scala:289) at org.apache.spark.sql.execution.streaming.StreamExecution$$anon$1.$anonfun$run$1(StreamExecution.scala:211) at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23) at org.apache.spark.JobArtifactSet$.withActiveJobArtifactState(JobArtifactSet.scala:94) at org.apache.spark.sql.execution.streaming.StreamExecution$$anon$1.run(StreamExecution.scala:211) ``` ### Willingness to contribute - [ ] I can contribute a fix for this bug independently - [X] I would be willing to contribute a fix for this bug with guidance from the Iceberg community - [ ] I cannot contribute a fix for this bug at this time -- 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: issues-unsubscr...@iceberg.apache.org.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org