apache / parquet-java

Apache Parquet Java
https://parquet.apache.org/
Apache License 2.0
2.6k stars 1.41k forks source link

Path parsing error in ParquetWriter on Windows #3029

Open martin-frydl opened 1 week ago

martin-frydl commented 1 week ago

Describe the bug, including details regarding any error messages, version, and platform.

Hello,

I've found problem with parsing Windows paths in ParquestWriter since version 1.14.2. Given following code:

Path file = new Path(new File("out.parquet").toURI());
Configuration conf = new Configuration(false);
GroupWriteSupport gws = new GroupWriteSupport();
GroupWriteSupport.setSchema(new MessageType("name"), conf);
new ParquetWriter(file, conf, gws);

the exception is thrown:

java.nio.file.InvalidPathException: Illegal char <:> at index 4: file:/Z:/out.parquet
        at java.base/sun.nio.fs.WindowsPathParser.normalize(WindowsPathParser.java:182)
        at java.base/sun.nio.fs.WindowsPathParser.parse(WindowsPathParser.java:153)
        at java.base/sun.nio.fs.WindowsPathParser.parse(WindowsPathParser.java:77)
        at java.base/sun.nio.fs.WindowsPath.parse(WindowsPath.java:92)
        at java.base/sun.nio.fs.WindowsFileSystem.getPath(WindowsFileSystem.java:232)
        at java.base/java.nio.file.Path.of(Path.java:147)
        at java.base/java.nio.file.Paths.get(Paths.java:69)
        at org.apache.parquet.hadoop.ParquetWriter.<init>(ParquetWriter.java:392)
        at org.apache.parquet.hadoop.ParquetWriter.<init>(ParquetWriter.java:357)
        at org.apache.parquet.hadoop.ParquetWriter.<init>(ParquetWriter.java:332)
        at org.apache.parquet.hadoop.ParquetWriter.<init>(ParquetWriter.java:273)
        at org.apache.parquet.hadoop.ParquetWriter.<init>(ParquetWriter.java:228)
        at org.apache.parquet.hadoop.ParquetWriter.<init>(ParquetWriter.java:307)

The problem is caused by change in ParquetWriter:392. Here used to be new Path(path) where Path is from org.apache.hadoop.fs (see here) but now there is Paths.get(path) from java.nio.file. The problem is that path is in fact URI, i.e. something like file:/..., but java.nio.file.Path does not expect URI. It probably does not work correctly also on Linux but there is no exception since ":" is OK in path there.

Please note that the example code does not fully work. The code is part of larger application with complex initialization so right now I don't have the correct schema for the writer. Fortunately that exception is thrown before bad schema error.

Component(s)

Core

wgtmac commented 1 week ago

IIUC, file:// is the scheme used by Hadoop. Is it possible to remove the prefix in your case?

cc @dlvenable as this issue is related to https://github.com/apache/parquet-java/pull/1376

martin-frydl commented 4 days ago

I can remote it when creating Path in example above (first line) but ParquetWriter will add it again. Before calling Paths.get() it calls file.getPath() here -> OutputFile.getPath(). And this function will always add it to the returned string.