Uploaded image for project: 'Apache Hudi'
  1. Apache Hudi
  2. HUDI-4944

The encoded slash (%2F) in partition path is not properly decoded during Spark read

    XMLWordPrintableJSON

Details

    Description

      When the source partitioned parquet table of the bootstrap operation has the encoded slash (%2F) in the partition path, e.g., "partition_path=2015%2F03%2F17", after the metadata-only bootstrap with the bootstrap indexing storing the data file path containing the partition path with the encoded slash (%2F), the target bootstrapped Hudi table cannot be read due to FileNotFound exception.  The root cause is that the encoding of the slash is lost when creating the new Path instance with the URI (see below, that "partition_path=2015/03/17" instead of "partition_path=2015%2F03%2F17").

      Caused by: java.io.FileNotFoundException: File does not exist: hdfs://localhost:62738/user/ethan/test_dataset_bootstrapped/partition_path=2015/03/17/e0fa3466-d3bc-43f7-b586-2f95d8745095_3-161-675_00000000000001.parquet
          at org.apache.hadoop.hdfs.DistributedFileSystem$29.doCall(DistributedFileSystem.java:1528)
          at org.apache.hadoop.hdfs.DistributedFileSystem$29.doCall(DistributedFileSystem.java:1521)
          at org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
          at org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:1521)
          at org.apache.parquet.hadoop.util.HadoopInputFile.fromPath(HadoopInputFile.java:39)
          at org.apache.parquet.hadoop.ParquetFileReader.readFooter(ParquetFileReader.java:448)
          at org.apache.spark.sql.execution.datasources.parquet.Spark24HoodieParquetFileFormat$$anonfun$buildReaderWithPartitionValues$1.footerFileMetaData$lzycompute$1(Spark24HoodieParquetFileFormat.scala:131)
          at org.apache.spark.sql.execution.datasources.parquet.Spark24HoodieParquetFileFormat$$anonfun$buildReaderWithPartitionValues$1.footerFileMetaData$1(Spark24HoodieParquetFileFormat.scala:130)
          at org.apache.spark.sql.execution.datasources.parquet.Spark24HoodieParquetFileFormat$$anonfun$buildReaderWithPartitionValues$1.apply(Spark24HoodieParquetFileFormat.scala:134)
          at org.apache.spark.sql.execution.datasources.parquet.Spark24HoodieParquetFileFormat$$anonfun$buildReaderWithPartitionValues$1.apply(Spark24HoodieParquetFileFormat.scala:111)
          at org.apache.hudi.HoodieDataSourceHelper$$anonfun$buildHoodieParquetReader$1.apply(HoodieDataSourceHelper.scala:71)
          at org.apache.hudi.HoodieDataSourceHelper$$anonfun$buildHoodieParquetReader$1.apply(HoodieDataSourceHelper.scala:70)
          at org.apache.hudi.HoodieBootstrapRDD.compute(HoodieBootstrapRDD.scala:60)
          at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324) 

      The path conversion that causes the problem is in the code below.  "new URI(file.filePath)" decodes the "%2F" and converts the slash.

      Spark24HoodieParquetFileFormat (same for Spark32PlusHoodieParquetFileFormat)

      val fileSplit =
        new FileSplit(new Path(new URI(file.filePath)), file.start, file.length, Array.empty) 

      This fails the tests below and we need to use a partition path without slashes in the value for now: 

      TestHoodieDeltaStreamer#testBulkInsertsAndUpsertsWithBootstrap

      ITTestHoodieDemo#testParquetDemo

      Attachments

        1. Untitled
          2 kB
          Jonathan Vexler

        Issue Links

          Activity

            People

              jonvex Jonathan Vexler
              guoyihua Ethan Guo (this is the old account; please use "yihua")
              Votes:
              0 Vote for this issue
              Watchers:
              2 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: