-
Notifications
You must be signed in to change notification settings - Fork 13.1k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[FLINK-986] [FLINK-25] [Distributed runtime] Add initial support for intermediate results #254
Conversation
+1 for merging it asap. |
Very nice description! We should add this to the Internals section in the docs. |
decodedMsg = new ErrorResponse(); | ||
} | ||
else { | ||
// Exception: unknown message |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Really throw an exception here? ;-)
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes ;)
This will take a bit to double-check. All in all: nice code! You serialize Strings in most places by writing their byte length and their bytes: The encoding logic in the |
} | ||
|
||
public boolean isRecycled() { | ||
return referenceCount.get() > 0; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This seems like a bug, it should be return referenceCount.get() <= 0;
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yeah definitely, see above.
The I think it does not hurt there, as it only happens once per partition at startup and once at cleanup time. |
for (int i = 0; i < inGates.size(); i++) { | ||
this.inputGates.get(i).initializeChannels(inGates.get(i)); | ||
catch (Throwable t) { | ||
LOG.error(ExceptionUtils.stringifyException(t), t); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think that the stringifyException
call is unnecessary and even hindering proper logging. I would remove it, construct a proper error message for the log and pass the exception as the second parameter.
Tagging buffers in the memory segment as "data frames" vs "serialized events" seems dangerous. If someone uses the |
import java.nio.ByteBuffer; | ||
|
||
// TODO We have to rethink this. Currently this limits the event size to the page size | ||
public class EventSerializer { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The restriction to one buffer may be fine here, when the buffer is not part of the pool anyways and can be of arbitrary size. Does event deserialization of arbitrary size work on the receiver side?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
No, I think this was simply not a good idea to do this way ;) See above. It's also related to your next note...
I would like to understand more about the BufferFutures. Is there ever a case where you need a buffer, but not immediately, so you can work with a future action and keep on going? If yes, then the futures may be a good idea. If not, they may be a bit of a complicated way to replace a blocking call. |
maybeNotifyConsumers(partitionType.isPipelined()); | ||
} | ||
|
||
public void addToAllPartitions(Buffer buffer) throws IOException { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think this does not work for buffers that contain data frames. The buffers are queued without incrementing the reference counter, so they will be released as soon as the first queue discards them.
In general, I am skeptical if it is a good idea to have the same buffer referenced from various queues. If we start spilling one of the queues, we may not get any memory back, because references are held from other queues.
I think for the time being, broadcasts should really write disjoint results. As a next step, we should create a single queue that is requested by all consumers. That way, we need not duplicate data in memory, spilled, and ease the reasoning about how much we get back when spilling data.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes, this is a bug...
It seems that we do not need multiple references to buffers in order to guarantee recycling only when the latest consumer is done. Why not simplify the logic in the buffer? (it seems a bit broken right now anyways) |
else { | ||
Throwable error = result.getError(); | ||
if (error != null) { | ||
throw new IOException(error.getMessage()); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This should properly chain the root exception, will probably be crucial for debugging errors.
The |
Also, we could rename it to |
Thanks for the feedback so far. :-) I replied inline where possible. NettyMessage: that's a very good idea. I will also use the String serialization utilities you suggested. IntermediateResultManager: I agree, but left it as it is as it doesn't hurt yet. I will change it. BufferFuture: I initially thought that we might use them in a case like that. But not at the moment and therefore it indeed complicates things. I will revert that as well. ;) Buffer references: we don't use it at the moment, but we will need that in any case for the "adaptive" spilling or multiple consumers. It would be OK to remove it for time being. It would be easy to add back. Exceptions: I will remove the stringify calls and add proper messages. Maybe we should completely remove them from the other code as well? PipelinedPartitionQueue: your proposed name is better. I have a simple synchronized version as well, which should work fine for the single consumer case. Can you have a special look at the partition queue interface? We need some kind of asynchrony (subscribe-notify). Maybe you have another idea about how to expose this. |
PS: regarding the queue iterator interface. The |
55e5d4b
to
e469296
Compare
I've squashed all commits and rebased this on the current master. |
e469296
to
ad250bc
Compare
Are there any open, known issues with this pull request? |
I am trying this out right now. Hope to be able to +1 it soon... |
…intermediate results
93a8521
to
70b87d2
Compare
Just rebased on the current master again. Locally, all tests pass. Let's see what Travis says. |
I am excited! |
Nice. +1 to merge this now! |
+1 |
YES!!! 👏 👏 👏 |
FLINK-25: Offer buffer-oriented runtime abstraction
BufferWriter
). With this pull request, the input side offers a similar interface:BufferReader
.AbstractInvokable
) involves two methods:registerInputOutput()
andinvoke()
. The I/O setup was coupled with the creation of readers/writers inregisterInputOutput()
although the required information is independent of this Therefore, they are now directly instantiated by the runtime environment and can be accessed viagetEnvironment().getReader(int)
andgetEnvironment().getWriter(int)
respectively.FLINK-986: Add initial support for intermediate results
Source => Task i => ... => Sink
. With Stephan's rework, they already look like this:Task:Intermediate Result <= Task i:Intermediate Result <= ... <= Task:Intermediate Result
. Tasks produce intermediate results and attach to them for consumption. These changes logically decoupled produced results and in the job graph. At the runtime level, the intermediate results were still tightly coupled to the producing and consuming task. With this pull request, the producing task, the produced intermediate result, and the consuming task are decoupled as well.Changes
Buffer management/The life of a Buffer
NetworkBufferPool
(previouslyGlobalBufferPool
). These network buffers are divided uniformly at runtime among the tasks. The current default for this pool is set to 2048 buffers (each 32k bytes), resulting in a total of 64 MB. Intermediate results get their buffers from this buffer pool. Therefore, for any in-memory persistent data set to make sense, we would have to give the network pool a lot more memory. I think that this is currently not desirable as our memory management is static, e.g. memory given to the network pool would be missing at operators like sorting or hashing. In my opinion, adaptive memory management (FLINK-1101) is therefore a requirement before continuing with more runtime intermediate result variants (see below).BufferWriter => ChannelManager => NETWORK xor local InputChannel
. Now it isBufferWriter => IntermediateResult
. What happens at the intermediate result is flexible depending on theIntermediateResultType
. Early discussions with @StephanEwen suggested the following three dimensions:The current state offers an ephemeral-pipelined-back pressure implementation, which is what we currently have. (I have removed code for a persistent-blocking-no back pressure variant for now.)
Intermediate results, intermediate result partitions and intermediate result partition queues
Scheduling of consumers and receiver-initiated sending
If the partitioning is not known or a task has no known consumers, the partition will consist of a single queue (or the default degree of parallelism). When it is consumed, a re-partitioning task needs to be inserted.
number of produced partitions
RPC calls to the job manager (seeConsumerNotificationProtocol
). The consumer tasks can then be deployed with the necessary information to request the respective queues. This results innumber of consumer sub tasks
task deployment RPC calls. The first notification from a task manager results in the deployment of the receiver. It is possible that not all producer task locations are known at deployment time of a consumer task. At the moment, we follow the naive approach of sending task update RPC calls with information about the location of the producer for every partition. This currently results in a total ofN + N * M
RPC calls per produced intermediate result (whereN
is the number of produced partitions andM
the number of consumer tasks).2*N
as we are guaranteed to know all producer locations when all partitions are finished. It should also be possible to further reduce this number to the number of task managers by collecting the notifications per task manager and then sending them to the job manager at once.More robust shuffle
ChannelManager
, which kept track of all channels. This component was huge and responsible for many different things (network buffers, channel lookup, buffer routing, TCP connections). This has been re-factored to more localized components, where resource acquisition and release can be handled in the same local component, which is in my opinion much easier to follow and understand.Next steps
The goal is to merge the current state as soon as possible and then start to extend (I've removed some code, which was already in place). I could have done this way earlier and intend to not make that mistake again.
I intend to address these points over the course of this week and work on the branch of this pull request.
After I've addressed the above points, I find it reasonable to work on reducing the number of redundant RPC calls and introducing the persistent-blocking-no back pressure intermediate result partition variant.