[ 
https://issues.apache.org/jira/browse/BEAM-13493?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17472864#comment-17472864
 ] 

Javier Domingo Cansino commented on BEAM-13493:
-----------------------------------------------

The reason for this is that when using hive partitioning format, where the path 
to the file is using folder for date partitioning, such as 
{{gs://my-bucket/EventV1/dt=2021-12-31/event.avro}}, the folder it goes into 
depends on the data that WriteToFiles receives.

It would break both native Hive partitioning and Hive-style partitioning in 
Bigquery GCS external tables. Although it doesn't break it in BQ/GCS because it 
doesn't rely on {{mkdir}}

> fileio calls mkdirs with basepath and not dirname(full_file_name)
> -----------------------------------------------------------------
>
>                 Key: BEAM-13493
>                 URL: https://issues.apache.org/jira/browse/BEAM-13493
>             Project: Beam
>          Issue Type: Bug
>          Components: io-py-files
>    Affects Versions: 2.34.0, 2.35.0
>            Reporter: Javier Domingo Cansino
>            Priority: P2
>
> When calling {{apache_beam.io.fileio.WriteToFiles}} with a {{file_naming}} 
> argument that adds a directory to the path, the current implementation fails 
> to write files if a {{mkdirs}} or analogous call is needed in the underlying 
> file storage.
> Example, 
> {code}
> apache_beam.io.fileio.WriteToFiles(
>     path="some/base/dir", sink=..., destination=lambda x: "events",
>     file_naming=lambda *x: "subdir/file.txt"
> ) 
> {code}
> the current fileio implementation will call {{mkdirs}} with {{some/base/dir}} 
> instead of {{some/base/dir/subdir}}.
> The bug is currently at 
> https://github.com/apache/beam/blob/67bcf1e16e3fdf68cdea7a4b42b9c003e4b8948c/sdks/python/apache_beam/io/fileio.py#L605.
> ====
> Personally, I would recommend changing the FileSystems interface to have 
> `open` call `mkdirs` in storages that require root parent directory creation.



--
This message was sent by Atlassian Jira
(v8.20.1#820001)

Reply via email to