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

Path parsing error in ParquetWriter on Windows #3029

Closed
martin-frydl opened this issue Oct 9, 2024 · 10 comments · Fixed by #3038
Closed

Path parsing error in ParquetWriter on Windows #3029

martin-frydl opened this issue Oct 9, 2024 · 10 comments · Fixed by #3038

Comments

@martin-frydl
Copy link

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
Copy link
Member

wgtmac commented Oct 12, 2024

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 #1376

@martin-frydl
Copy link
Author

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.

@pavelchuchma
Copy link

Hello,
Please, do you have any estimate when this issue will be fixed? Do you have enough inputs for the fix?

@wgtmac
Copy link
Member

wgtmac commented Oct 22, 2024

This is a regression introduced by #1376. I don't think we can work around it easily. Should we revert that commit? @ggershinsky @gszadovszky?

@gszadovszky
Copy link
Contributor

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...

@wgtmac
Copy link
Member

wgtmac commented Oct 26, 2024

@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.

@martin-frydl
Copy link
Author

@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.

@wgtmac
Copy link
Member

wgtmac commented Oct 30, 2024

Could you please have a try on #3038? @martin-frydl

@martin-frydl
Copy link
Author

@wgtmac Yes, that fixes the problem. Thank you.

@pavelchuchma
Copy link

@wgtmac thank you for the fix. It will help us.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging a pull request may close this issue.

4 participants