Re: [I] Failed to assign splits due to the serialized split size [iceberg]

2024-01-22 Thread via GitHub
javrasya commented on issue #9410: URL: https://github.com/apache/iceberg/issues/9410#issuecomment-1904744066 @pvary , thank you for that. You are right. It is all immutable so it makes sense that rewrite operation would create another snapshot and I should be using that not a prior one to

Re: [I] Failed to assign splits due to the serialized split size [iceberg]

2024-01-16 Thread via GitHub
pvary commented on issue #9410: URL: https://github.com/apache/iceberg/issues/9410#issuecomment-1894100365 @javrasya: The Spark RewriteDataFiles should create a new snapshot in the table. If the query reads this new snapshot, then it should not read the old delete files anymore. If ExpireS

Re: [I] Failed to assign splits due to the serialized split size [iceberg]

2024-01-13 Thread via GitHub
javrasya commented on issue #9410: URL: https://github.com/apache/iceberg/issues/9410#issuecomment-1890599978 Tried `[rewrite_data_files](https://iceberg.apache.org/docs/1.4.0/spark-procedures/#rewrite_data_files)` via Spark, not really sure if it would do the same with `RewriteDataFilesAc

Re: [I] Failed to assign splits due to the serialized split size [iceberg]

2024-01-12 Thread via GitHub
javrasya commented on issue #9410: URL: https://github.com/apache/iceberg/issues/9410#issuecomment-1890349330 Thank you both. You are right @stevenzwu , it is sad that there is no implementation yet for `ConvertEqualityDeleteFiles`. I will give `RewriteDataFilesAction` a try and let you kn

Re: [I] Failed to assign splits due to the serialized split size [iceberg]

2024-01-12 Thread via GitHub
pvary commented on issue #9410: URL: https://github.com/apache/iceberg/issues/9410#issuecomment-1890336927 > But I still have too many equality deletes which is still causing the limit excess. Did I use the wrong spark procedure? I think RewriteDataFilesAction could help you there. If

Re: [I] Failed to assign splits due to the serialized split size [iceberg]

2024-01-12 Thread via GitHub
stevenzwu commented on issue #9410: URL: https://github.com/apache/iceberg/issues/9410#issuecomment-1890332322 > Yes I did use Spark ([rewrite_position_delete_files](https://iceberg.apache.org/docs/1.3.1/spark-procedures/#rewrite_position_delete_files)) to clean up positional deletes. Maybe

Re: [I] Failed to assign splits due to the serialized split size [iceberg]

2024-01-12 Thread via GitHub
javrasya commented on issue #9410: URL: https://github.com/apache/iceberg/issues/9410#issuecomment-1890244814 I took the liberty and created the PR since I had the changes locally. Hope you guys don't mind 🙏 -- This is an automated message from the Apache Git Service. To respond to the m

Re: [I] Failed to assign splits due to the serialized split size [iceberg]

2024-01-12 Thread via GitHub
javrasya commented on issue #9410: URL: https://github.com/apache/iceberg/issues/9410#issuecomment-1890232951 I had the changes and test working in my local. It a bit tricky to just use `writeBytes`. DataOutputSerializer.writeBytes is buggy and increasing the buffer position twice. Do you

Re: [I] Failed to assign splits due to the serialized split size [iceberg]

2024-01-12 Thread via GitHub
stevenzwu commented on issue #9410: URL: https://github.com/apache/iceberg/issues/9410#issuecomment-1890156116 ah. I didn't know it is a batch read mode using `asOfSnapshotId`. note that they are `delete` (not `deleted`) files to capture the row-level deletes. the actual files are not load

Re: [I] Failed to assign splits due to the serialized split size [iceberg]

2024-01-12 Thread via GitHub
javrasya commented on issue #9410: URL: https://github.com/apache/iceberg/issues/9410#issuecomment-1889832176 It still feels weird to allow that big of a split to be created. Wouldn't it possible to make the deleted files lazy and rather be loaded in the respective task node, instead of the

Re: [I] Failed to assign splits due to the serialized split size [iceberg]

2024-01-12 Thread via GitHub
javrasya commented on issue #9410: URL: https://github.com/apache/iceberg/issues/9410#issuecomment-1889782916 Thanks for the answer @stevenzwu, no you are right. I know we shouldn't should streaming on Iceberg other than append only tables. But we don't do stream in this case. Every day we

Re: [I] Failed to assign splits due to the serialized split size [iceberg]

2024-01-12 Thread via GitHub
stevenzwu commented on issue #9410: URL: https://github.com/apache/iceberg/issues/9410#issuecomment-1889639566 first of all, current Flink Iceberg source (FLIP-27 or old) doesn't support streaming read with row-level deletes. It only read append-only snapshots/commits. > is it possi

Re: [I] Failed to assign splits due to the serialized split size [iceberg]

2024-01-12 Thread via GitHub
javrasya commented on issue #9410: URL: https://github.com/apache/iceberg/issues/9410#issuecomment-1889413720 Thanks for your input. I tried [rewrite_position_delete_files](https://iceberg.apache.org/docs/1.3.1/spark-procedures/#rewrite_position_delete_files) but no luck. -- This is an a

Re: [I] Failed to assign splits due to the serialized split size [iceberg]

2024-01-11 Thread via GitHub
javrasya commented on issue #9410: URL: https://github.com/apache/iceberg/issues/9410#issuecomment-1888101057 Hi again @pvary. I managed to run it in debug mode and the JSON which is being deserialized is crammed with delete files (`EQUALITY_DELETES` and `POSITION_DELETES`). It is a gigan

Re: [I] Failed to assign splits due to the serialized split size [iceberg]

2024-01-10 Thread via GitHub
javrasya commented on issue #9410: URL: https://github.com/apache/iceberg/issues/9410#issuecomment-1884525429 I couldn't do this @pvary , the split is far ahead and some time is needed to get there in the application. My local environment is not able to run the app on real data and hit this

Re: [I] Failed to assign splits due to the serialized split size [iceberg]

2024-01-08 Thread via GitHub
pvary commented on issue #9410: URL: https://github.com/apache/iceberg/issues/9410#issuecomment-1881459496 @javrasya: I do not see yet, how any of the above changes could create delete files. For debugging: Could you use conditional breakpoints, or you could put a breakpoint where th

Re: [I] Failed to assign splits due to the serialized split size [iceberg]

2024-01-08 Thread via GitHub
javrasya commented on issue #9410: URL: https://github.com/apache/iceberg/issues/9410#issuecomment-1880874873 Hi @pvary , I failed to debug that locally, Couldn't reproduce it since it is one file of many and it takes time to hit that and can't really do that in a debug session. We don't do

Re: [I] Failed to assign splits due to the serialized split size [iceberg]

2024-01-08 Thread via GitHub
pvary commented on issue #9410: URL: https://github.com/apache/iceberg/issues/9410#issuecomment-1880601867 @javrasya: I think we should fix the serialization issue, but I would like to understand the root cause before jumping to solutions. The deletes seems to be an issue anyway - which sho

Re: [I] Failed to assign splits due to the serialized split size [iceberg]

2024-01-06 Thread via GitHub
javrasya commented on issue #9410: URL: https://github.com/apache/iceberg/issues/9410#issuecomment-1879638660 We don't do any deletes actually. I will try to debug it locally somehow on that single file it was failing on. I can why it is big. But regardless, what do you think would be a rem

Re: [I] Failed to assign splits due to the serialized split size [iceberg]

2024-01-05 Thread via GitHub
pvary commented on issue #9410: URL: https://github.com/apache/iceberg/issues/9410#issuecomment-1879580902 @javrasya: Table with 14 columns should not cause any issues. The default stats also could not cause issues. I made a mistake reading the code, and combined splits also could not

Re: [I] Failed to assign splits due to the serialized split size [iceberg]

2024-01-05 Thread via GitHub
javrasya commented on issue #9410: URL: https://github.com/apache/iceberg/issues/9410#issuecomment-1879135064 @pvary I wasn't aware of `splitOpenFileCost`, thank you for sharing that. The way how I did it is that I introduced my own SplitAssignerFactory and SplitAssigner and pass tha

Re: [I] Failed to assign splits due to the serialized split size [iceberg]

2024-01-05 Thread via GitHub
pvary commented on issue #9410: URL: https://github.com/apache/iceberg/issues/9410#issuecomment-1878817631 @javrasya: This is table should not be too wide, and the statistics should be limited as well (unless you did some specific tweaking there). My best guess is your first suggestion:

Re: [I] Failed to assign splits due to the serialized split size [iceberg]

2024-01-05 Thread via GitHub
javrasya commented on issue #9410: URL: https://github.com/apache/iceberg/issues/9410#issuecomment-1878646665 No idea tbh since I run into this on production and there I don't have the ability to go deep and debug. I wouldn't say it is a wide table in terms of number of columns and I

Re: [I] Failed to assign splits due to the serialized split size [iceberg]

2024-01-04 Thread via GitHub
javrasya commented on issue #9410: URL: https://github.com/apache/iceberg/issues/9410#issuecomment-1878085557 It seems this bug has been introduced by version 1.4.0 which is kid of new. Tried fixing it by tweaking the SplitAssignerFactory I pass down to the IcebergSource but even though I r

Re: [I] Failed to assign splits due to the serialized split size [iceberg]

2024-01-04 Thread via GitHub
pvary commented on issue #9410: URL: https://github.com/apache/iceberg/issues/9410#issuecomment-1877233309 @stevenzwu: After a quick check, I have found this: https://github.com/apache/flink/blob/c6997c97c575d334679915c328792b8a3067cfb5/flink-core/src/main/java/org/apache/flink/core/memor

Re: [I] Failed to assign splits due to the serialized split size [iceberg]

2024-01-04 Thread via GitHub
nastra commented on issue #9410: URL: https://github.com/apache/iceberg/issues/9410#issuecomment-1876875354 @stevenzwu @pvary could you guys take a look please? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL a

[I] Failed to assign splits due to the serialized split size [iceberg]

2024-01-04 Thread via GitHub
javrasya opened a new issue, #9410: URL: https://github.com/apache/iceberg/issues/9410 ### Apache Iceberg version 1.4.2 (latest release) ### Query engine Flink ### Please describe the bug 🐞 Hi there, I am trying to consume records from an Iceberg table in my