Path parsing error in ParquetWriter on Windows
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
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
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.
Hello, Please, do you have any estimate when this issue will be fixed? Do you have enough inputs for the fix?
This is a regression introduced by https://github.com/apache/parquet-java/pull/1376. I don't think we can work around it easily. Should we revert that commit? @ggershinsky @gszadovszky?
I don't think we should revert the original change.
Since EncryptionPropertiesHelper uses Hadoop Path anyway, we can simply extend the API to pass the path as a String and convert it directly inside. Then, we avoid the error of creating a nio Path from the string path...
@martin-frydl Have you tried to use Path file = new Path(out.parquet"); instead of Path file = new Path(new File("out.parquet").toURI());. I debugged it a little bit. It seems that File.toURI() prepends the file:/ prefix.
@wgtmac Yes, but it does not help. The problem is that Path is "converted" into OutputFile here: HadoopOutputFile.fromPath(file, conf) and then file.getPath() is called which adds "file:" back. I see no way how to bypass this.
Could you please have a try on https://github.com/apache/parquet-java/pull/3038? @martin-frydl
@wgtmac Yes, that fixes the problem. Thank you.
@wgtmac thank you for the fix. It will help us.