You signed in with another tab or window. Reload to refresh your session.You signed out in another tab or window. Reload to refresh your session.You switched accounts on another tab or window. Reload to refresh your session.Dismiss alert
Is your feature request related to a problem or challenge? Please describe what you are trying to do.
In https://github.com/ArroyoSystems/arroyo we are building a distributed, stateful stream processing engine. It is able to tolerate failures by taking periodic checkpoints, using a variant of the Chandry-Lamport algorithm that is also used by Flink. In order for this to work you need to be able to reliably snapshot the state of operators.
In the case of writing files to an object store such as S3, we'd like to be able to use the multi-part upload (MPU) capabilities so that we can write large files across checkpoint barriers, without having to separately store all of the file contents. This can be done by introducing direct APIs to initiate a MPU, add a new part, and complete it.
Additionally, for writing structured formats such as Parquet, you may need to store some additional state in order to be able to finish the file. The current arrow-rs APIs don't provide any way to checkpoint the internal state of a writer, so if the process dies you can't recover.
Describe the solution you'd like
I'd like to be able to directly control the multipart upload, so I can know which parts have been uploaded, and with which IDs.
I'd also like some way to finish a parquet file, either by restoring the internal state or storing the final bytes in case of interruption.
Currently I've implemented ways to do both of these in a branch off of 43.0.0, whose diff you can see here. The parquet stuff is very clearly a hack, and the ObjectStore APIs are very S3 specific.
Describe alternatives you've considered
We'll likely maintain our fork in the absence of these capabilities being provided in the main library.
The text was updated successfully, but these errors were encountered:
I'd like to be able to directly control the multipart upload
Most cloud stores have a minimum part size, this means you can't always flush as you may not have enough data
Some stores, such as Cloudflare R2, have fixed chunk sizes that you must write in exact blocks of
Multipart upload would be tricky to expose in a manner that could be consistent across multiple stores
I'd also like some way to finish a parquet file, either by restoring the internal state
I think the best we could get is row group level granularity on this, the nature of parquet's encoding necessitates buffering a substantial amount of data in various places internally per row group. Providing a way to reliably snapshot this state would be inordinately complicated, and likely incredibly fragile.
can write large files across checkpoint barriers
Taking a step back I wonder if you've considered simply persisting multiple smaller files? If the state is sufficiently large to warrant being committed in multiple parts, perhaps the parts are large enough to stand on their own? This is what systems like iceberg or deltalake do, writing multiple files and then committing them as part of the same transaction. Would this work for your use-case?
Is your feature request related to a problem or challenge? Please describe what you are trying to do.
In https://github.com/ArroyoSystems/arroyo we are building a distributed, stateful stream processing engine. It is able to tolerate failures by taking periodic checkpoints, using a variant of the Chandry-Lamport algorithm that is also used by Flink. In order for this to work you need to be able to reliably snapshot the state of operators.
In the case of writing files to an object store such as S3, we'd like to be able to use the multi-part upload (MPU) capabilities so that we can write large files across checkpoint barriers, without having to separately store all of the file contents. This can be done by introducing direct APIs to initiate a MPU, add a new part, and complete it.
Additionally, for writing structured formats such as Parquet, you may need to store some additional state in order to be able to finish the file. The current arrow-rs APIs don't provide any way to checkpoint the internal state of a writer, so if the process dies you can't recover.
Describe the solution you'd like
I'd like to be able to directly control the multipart upload, so I can know which parts have been uploaded, and with which IDs.
I'd also like some way to finish a parquet file, either by restoring the internal state or storing the final bytes in case of interruption.
Currently I've implemented ways to do both of these in a branch off of 43.0.0, whose diff you can see here. The parquet stuff is very clearly a hack, and the ObjectStore APIs are very S3 specific.
Describe alternatives you've considered
We'll likely maintain our fork in the absence of these capabilities being provided in the main library.
The text was updated successfully, but these errors were encountered: