venkata91 commented on PR #26592: URL: https://github.com/apache/flink/pull/26592#issuecomment-2913387581
> > Thanks for the patch. It is a good catch. > > I understand it might be a little tricky to do unit test. but we may be able to do integration test. We can do the following: > > > > 1. start a session cluster with normal size JM/TM ( < 32GB ) > > 2. when launch the job on the command line, use an additional JVM arg of `-XX:-UseCompressedOops` to force a 24-bytes object header with JVM memory > 4 GB. > > > > This way we should be able to test it. > > Thanks for reviewing the patch ! > > Yes, I'm working on e-2-e test with `FlinkContainers`. In this case, we need 2 different JVMs (JM and TM) one with `UseCompressedOops` and one without that. Still working on the e-2-e test, will post it once it is ready. I was trying to reproduce the issue locally with test datasets and Flink SQL with inner join, but it is not reproducible locally. Note: with remote debugging I could confirm the issue still exists - basically JM codegens `HashPartitioner` with `arrayBaseOffset` = 24 while on the TM side it is 16. What happens is the following: 1. Bytes [16 ... 32] is where the key is present and length = 16 2. Bytes [32 ... 40] is filled with JVM garbage which looks something like [1, 0, 0, 0, ... 0] 3. Hash function reads bytes from [24 ... 40] where first 8 bytes are valid bytes of the key and the remaining 8 bytes is just JVM garbage. 4. The same pattern exists for both the occurrences of the same key (in fact for all the keys). 5. Due to the above, the hashcode computed is consistent b/w both the occurrences of the same key and gets shuffled to the appropriate task, even though the hashcode computation is wrong. I'm still trying to construct a dataset where [32 ... 40] bytes are different for the same key. Note: even in our internal environment, this issue is reproducible only when `nested fields projection pushdown` is enabled. This optimization is implemented in our internal connector. -- 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