-
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
Out-of-core state backend for JDBC databases #1305
Merged
Merged
Changes from 1 commit
Commits
Show all changes
7 commits
Select commit
Hold shift + click to select a range
ad6f826
[FLINK-2916] [streaming] Expose operator and task information to Stat…
gyfora 75a7c4b
[FLINK-2924] [streaming] Out-of-core state backend for JDBC databases
gyfora 347e6f7
[FLINK-2924] [streaming] Improved sharding logic
gyfora c254bda
[FLINK-2924] [streaming] Use timestamps to store checkpoints so it su…
gyfora 43b8e57
[FLINK-2924] [streaming] Improve compacting logic
gyfora cd8be0b
[FLINK-2924] [streaming] Use short job id for table names
gyfora db2a964
[FLINK-2924] [streaming] Execute compactions in background thread + k…
gyfora File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Next
Next commit
[FLINK-2916] [streaming] Expose operator and task information to Stat…
…eBackend
- Loading branch information
commit ad6f826584be7527c58e2126e2828f82afc97875
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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 would like to get rid of this change and simply let the state backend create a UID for the state name.
This method is called one per proper creation of a state (so it should not need deterministic state naming). Recovery happens from the state handle, which can store all required info.
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 am not completely sure what you mean here.
Multiple different states can have the same name in different tasks. As far as I know we dont assume unique state names. This gets worse if the chained tasks have states with the same name then they actually go to the same backend as well.
I dont see how to go around this without an operator id. Could you please clarify your idea?
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 suggest to not let the operator supply an ID and name, but simply leave the naming of the state to the state backend. The SqlStateBackend could just use
UUID.randomUUID().toString()
instead ofoperatorId+stateName
.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 point is that all parallel instances write to the same set of tables. This will way sharding is transparently handled and the job parallelism can actually change without affecting the state. (No need to repartition it)
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.
Otherwise you will have to create p*numShards tables and you wont even know what state is in it from looking at the table names
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.
What you mention depends on the parallel subtask ID (which is already given in the initialize() method). The operatorId and name are the same for all parallel instances anyways.
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 "name" (as a string) of the state is a very API specific thing that no other part of the runtime is concerned with. The operator ID is something specific to the StreamGraphBuilder and not to the streaming tasks at all. I think we are tying things together here that should not be tied together.
I still do not understand how this affects sharding. Does the shard assignment depend on the state name (rather than the parallel subtask / JobVertexId) ?
I only see that the table names will have the task name instead of the name of the state.
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.
Let me first describe how sharding works than I will give a concrete example.
Key-Value pairs are sharded by key not by the subtask. This means that each parallel subtask maintains a connection to all the shards and partitions the states before writing them to the appropriate shards according to the user defined partitioner (in the backend config). This is much better than sharding by subtask because we can later change the parallelism of the job without affecting the state and also lets us defined a more elaborate sharding strategy through the partitioner.
This means, when a kv state is created we create a table for that kvstate in each shard. If we would do it according to your suggestion we would need to create numShards number of tables for each parallel instance (total of p*ns) for each kvstate. Furthermore this makes the fancy sharding useless because we cannot change the job parallelism. So we need to make sure that parallel subtasks of a given operator write to the same state tables (so we only have ns number of tables regardless of the parallelism).
In order to do this we need something that uniqely identifies a given state in the streaming program (and parallel instances should have the same id).
The information required to create such unique state id is an identifier for the operator that has the state + the name of the state. (The information obtained from the environment is not enough because chained operators have the same environment, therefore if they have conflicting state names the id is not unique). The only thing that identifies an operator in the logical streaming program is the operator id assigned by the jobgraphbuilder (thats the whole point of having it).
An example job with p=2 and numshards = 3:
chained map -> filter, both the mapper and filter has a state named "count", and let's assume that mapper has opid 1 and filter 2.
In this case the mapper would create 3 db tables (1 on each shard) with the same name kvstate_count_1_jobId. The filter would also create 3 tables with names: kvstate_count_2_jobId
All mapper instances would write to all three database shards, and the same goes for all the filters.
I hope you get what I am trying to say.
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.
You might be right that state name and operator id is too api specific, but we will need ways to globally identify states which is impossible without that I think currently.
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.
Thanks for the description of the sharding. The issue is that you need a deterministic table name that each KeyValueState can create independently.