...
Inner join: remove all null-value triggers
ts | left | right | innerJoin (current) | innerJoin (suggested) |
1 | null | |||
2 | null | |||
3 | A | |||
4 | a | A - a | A - a | |
5 | B | B - a | B - a | |
6 | b | A - b B - b | A - b B - b | |
7 | null | null - a null - b | ||
8 | null | A - null B - null | ||
9 | C | C - a C - b | C - a C - b | |
10 | c | A - c B - c C - c | A - c B - c C - c | |
11 | null | A - null B - null C - null | ||
12 | null | null - a null - b null - c | ||
13 | null | A - null B - null C - null | ||
14 | d | A - d B - d C - d | A - d B - d C - d | |
15 | D | D - a D - b D - c D - d | D - a D - b D - c D - d |
Left join: remove all left null-value triggers; add trigger on right input (only for non-null-values)
ts | left | right | leftJoin (current) | leftJoin (suggested) |
1 | null | null - null | ||
2 | null | |||
3 | A | A - null | A - null | |
4 | a | A - a | ||
5 | B | B - a | B - a | |
6 | b | A - b B - B | ||
7 | null | null - a null - b | ||
8 | null | |||
9 | C | C - a C - b | C - a C - b | |
10 | c | A - c B - c C - c | ||
11 | null | |||
12 | null | null - a null - b null - c | ||
13 | null | |||
14 | d | A - d B - d C - d | ||
15 | D | D - a D - b D - c D - d | D - a D - b D - c D - d |
Outer join: remove null-value triggers
ts | left | right | outerJoin (current) | outerJoin (suggested) |
1 | null | null - null | ||
2 | null | null - null | ||
3 | A | A - null | A - null | |
4 | a | A - a | A - a | |
5 | B | B - a | B - a | |
6 | b | A - b B - b | A - b B - b | |
7 | null | null - a null - b | ||
8 | null | A - null B - null | ||
9 | C | C - a C - b | C - a C - b | |
10 | c | A - c B - c C - c | A - c B - c C - c | |
11 | null | A - null B - null | ||
12 | null | null - a null - b null - c | ||
13 | null | A - null B - null C - null | ||
14 | d | A - d B - d C - d | A - d B - d C - d | |
15 | D | D - a D - b D - c D - d | D - a D - b D - c D - d |
...
Inner join: add trigger for left non-null values
ts | left | right | leftJoin (current) | leftJoin (suggested) | innerJoin (suggested to add) |
1 | null | null - null | |||
2 | null | ||||
3 | A | A - null | A - null | ||
4 | a | ||||
5 | B | B - a | B - a | B - a | |
6 | b | ||||
7 | null | null - b | |||
8 | null | ||||
9 | C | C - null | C - null | ||
10 | c | ||||
11 | null | ||||
12 | null | null - null | |||
13 | null | ||||
14 | d | ||||
15 | D | D - d | D - d | D - d |
...
The semantics of the table below are as follows: It basically shows the current semantics if you ignore the red background color. As we only suggest to remove some null outputs, the cells with red background indicate those nulls we do suggest to remove.
ts | left | right | innerJoin | leftJoin | outerJoin |
1 | null | null | null | null | |
2 | null | null | null | null | |
3 | A | null | A - null | A - null | |
4 | a | A - a | A - a | A - a | |
5 | B | B - a | B - a | B - a | |
6 | b | B - b | B - b | B - b | |
7 | null | null | null | null - b | |
8 | null | null | null | null | |
9 | C | null | C - null | C - null | |
10 | c | C - c | C - c | C - c | |
11 | null | null | C - null | C - null | |
12 | null | null | null | null | |
13 | null | null | null | null | |
14 | d | null | null | null - d | |
15 | D | D - d | D - d | D - d |
Summary
With the suggested changes, we fix the following currently existing inconsistencies:
...
Handle <key:null> records for KStreams input as regular <key:value> records, rejected for the following reasons:
- in contrast to relational model, <key:null> records do have special semantics (there is actually nothing to be joined); this also relates to KTable tombstone (ie, delete) semantics of <key:null> records
- for inner joins, user would not expect that ValueJoiner is called with one parameter being null
- For for left/outer join, user cannot distinguish if the call to ValueJoiner is done because no key was found or because <key:null> record was found
- alternatively, we would need to introduce different ValueJoiner classes with different method, ie, #join(left, right), #joinLeft(left), #joinRight(right), but we want to keep API simple
- InnerValueJoin only offering #join()
LeftValueJoiner offering #join() and #leftJoin()
- OuterValueJoiner offering all three methods
add outer KStream-KTable join, rejected for the following reasons:
...