Hi Chesnay! That is an interesting problem, though hard to judge with the information we have.
Can you elaborate a bit on the following points: - When putting the objects from the Java Flink side into the shared memory, you need to serialize them. How do you do that? Into a buffer, then copy that into the shared memory ByteBuffer? Directly? - Shared memory access has to be somehow controlled. The pipes give you flow control for free (blocking write calls when the stream consumer is busy). What do you do for the shared memory? Usually, one uses semaphores, or, in java File(Range)Locks to coordinate access and block until memory regions are made available. Can you check if there are some busy waiting parts in you code? - More general: The code is slower, but does it burn CPU cycles in its slowness or is it waiting for locks / monitors / conditions ? Stephan On Wed, Aug 27, 2014 at 8:34 PM, Chesnay Schepler < [email protected]> wrote: > Hello everyone, > > This will be some kind of brainstorming question. > > As some of you may know I am currently working on the Python API. The most > crucial part here is how the data is exchanged between Java and Python. > Up to this point we used pipes for this, but switched recently to memory > mapped files in hopes of increasing the (lacking) performance. > > Early (simplified) prototypes (outside of Flink) showed that this would > yield a significant increase. yet when i added the code to flink and ran a > job, there was > no effect. like at all. two radically different schemes ran in /exactly/ > the same time. > > my conclusion was that code already in place (and not part of the > prototypes) is responsible for this. > so i went ahead and modified the prototypes to use all relevant code from > the Python API in order to narrow down the culprit. but this time, the > performance increase was there. > > Now here's the question: How can the /very same code/ perform so much > worse when integrated into flink? if the code is not the problem, what > could be it? > > i spent a lot of time looking for that one line of code that cripples the > performance, but I'm pretty much out of places to look. > >
