isburmistrov commented on code in PR #24079: URL: https://github.com/apache/flink/pull/24079#discussion_r1450246875
########## flink-runtime/src/main/java/org/apache/flink/runtime/state/CompressibleFSDataOutputStream.java: ########## @@ -41,6 +41,10 @@ public CompressibleFSDataOutputStream( @Override public long getPos() throws IOException { + // Underlying compression involves buffering, so the only way to report correct position is + // to flush the underlying stream. This lowers the effectivity of compression, but there is + // no other way, since the position is often used as a split point. + flush(); Review Comment: Do you think it's the best way to fix it? I think this approach has a few drawbacks: - As mentioned, lower efficiency of compression. Actually for some compression algorithms this may make it useless, if not harmful (if there is some significant amount of metadata that gets written on `flush`) - It's somewhat strange that the result is different whether we call `getPos()` in the process of compressing the output, or we don't. This may later come as a big surprise for someone using `CompressibleFSDataOutputStream`. - `Seek` operation in the corresponding `CompressibleFSDataInputStream` doesn't make sense for any position, only for some specific positions. Which is also not super bad, but confusing. IMO, a better - albeit more involved - approach may be to change the semantics of `getPos()` for `CompressibleFSDataOutputStream`. This can represent the position in the **uncompressed** data. Correspondingly, `seek` operation in `CompressibleFSDataInputStream` can be implemented having this semantics in mind: one can read the `compressingDelegate` until the desired position is reached. This approach also doesn't have the problem of `seek` being valid for specific positions only, which is a nice side effect. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org