Class JoinHelper<STATE_VIEW,OUTER_STATE_VIEW extends STATE_VIEW>
-
Constructor Summary
ConstructorsConstructorDescriptionJoinHelper(boolean leftIsOuter, boolean rightIsOuter, org.apache.flink.table.data.utils.JoinedRowData outRow, org.apache.flink.table.data.RowData leftNullRow, org.apache.flink.table.data.RowData rightNullRow, org.apache.flink.streaming.api.operators.TimestampedCollector<org.apache.flink.table.data.RowData> collector) -
Method Summary
Modifier and TypeMethodDescriptionabstract voidaddRecord(STATE_VIEW stateView, org.apache.flink.table.data.RowData record) abstract voidaddRecordInOuterSide(OUTER_STATE_VIEW stateView, org.apache.flink.table.data.RowData record, int numOfAssociations) voidprocessJoin(org.apache.flink.table.data.RowData input, STATE_VIEW inputSideAsyncStateView, STATE_VIEW otherSideAsyncStateView, boolean inputIsLeft, AssociatedRecords otherSideAssociatedRecords, boolean isSuppress) Process an input element and output incremental joined records, retraction messages will be sent in some scenarios.abstract voidretractRecord(STATE_VIEW stateView, org.apache.flink.table.data.RowData record) abstract voidupdateNumOfAssociationsInOuterSide(OUTER_STATE_VIEW stateView, org.apache.flink.table.data.RowData record, int numOfAssociations)
-
Constructor Details
-
JoinHelper
public JoinHelper(boolean leftIsOuter, boolean rightIsOuter, org.apache.flink.table.data.utils.JoinedRowData outRow, org.apache.flink.table.data.RowData leftNullRow, org.apache.flink.table.data.RowData rightNullRow, org.apache.flink.streaming.api.operators.TimestampedCollector<org.apache.flink.table.data.RowData> collector)
-
-
Method Details
-
processJoin
public void processJoin(org.apache.flink.table.data.RowData input, STATE_VIEW inputSideAsyncStateView, STATE_VIEW otherSideAsyncStateView, boolean inputIsLeft, AssociatedRecords otherSideAssociatedRecords, boolean isSuppress) throws Exception Process an input element and output incremental joined records, retraction messages will be sent in some scenarios.Following is the pseudo code to describe the core logic of this method. The logic of this method is too complex, so we provide the pseudo code to help understand the logic. We should keep sync the following pseudo code with the real logic of the method.
Note: "+I" represents "INSERT", "-D" represents "DELETE", "+U" represents "UPDATE_AFTER", "-U" represents "UPDATE_BEFORE". We forward input RowKind if it is inner join, otherwise, we always send insert and delete for simplification. We can optimize this to send -U & +U instead of D & I in the future (see FLINK-17337). They are equivalent in this join case. It may need some refactoring if we want to send -U & +U, so we still keep -D & +I for now for simplification. See
FlinkChangelogModeInferenceProgram.SatisfyModifyKindSetTraitVisitor.if input record is accumulate | if input side is outer | | if there is no matched rows on the other side, send +I[record+null], state.add(record, 0) | | if there are matched rows on the other side | | | if other side is outer | | | | if the matched num in the matched rows == 0, send -D[null+other] | | | | if the matched num in the matched rows > 0, skip | | | | otherState.update(other, old + 1) | | | endif | | | send +I[record+other]s, state.add(record, other.size) | | endif | endif | if input side not outer | | state.add(record) | | if there is no matched rows on the other side, skip | | if there are matched rows on the other side | | | if other side is outer | | | | if the matched num in the matched rows == 0, send -D[null+other] | | | | if the matched num in the matched rows > 0, skip | | | | otherState.update(other, old + 1) | | | | send +I[record+other]s | | | else | | | | send +I/+U[record+other]s (using input RowKind) | | | endif | | endif | endif endif if input record is retract | state.retract(record) | if there is no matched rows on the other side | | if input side is outer, send -D[record+null] | endif | if there are matched rows on the other side, send -D[record+other]s if outer, send -D/-U[record+other]s if inner. | | if other side is outer | | | if the matched num in the matched rows == 0, this should never happen! | | | if the matched num in the matched rows == 1, send +I[null+other] | | | if the matched num in the matched rows > 1, skip | | | otherState.update(other, old - 1) | | endif | endif endif
- Parameters:
input- the input elementinputSideAsyncStateView- state of input sideotherSideAsyncStateView- state of other sideinputIsLeft- whether input side is left sideotherSideAssociatedRecords- associated records in the state of the other side- Throws:
Exception
-
addRecord
public abstract void addRecord(STATE_VIEW stateView, org.apache.flink.table.data.RowData record) throws Exception - Throws:
Exception
-
retractRecord
public abstract void retractRecord(STATE_VIEW stateView, org.apache.flink.table.data.RowData record) throws Exception - Throws:
Exception
-
addRecordInOuterSide
public abstract void addRecordInOuterSide(OUTER_STATE_VIEW stateView, org.apache.flink.table.data.RowData record, int numOfAssociations) throws Exception - Throws:
Exception
-
updateNumOfAssociationsInOuterSide
public abstract void updateNumOfAssociationsInOuterSide(OUTER_STATE_VIEW stateView, org.apache.flink.table.data.RowData record, int numOfAssociations) throws Exception - Throws:
Exception
-