forked from apache/flink
-
Notifications
You must be signed in to change notification settings - Fork 7
/
StreamExecChangelogNormalize.scala
139 lines (121 loc) · 5.33 KB
/
StreamExecChangelogNormalize.scala
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.flink.table.planner.plan.nodes.physical.stream
import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
import org.apache.calcite.rel.`type`.RelDataType
import org.apache.calcite.rel.{RelNode, RelWriter, SingleRel}
import org.apache.flink.api.dag.Transformation
import org.apache.flink.streaming.api.operators.KeyedProcessOperator
import org.apache.flink.streaming.api.transformations.OneInputTransformation
import org.apache.flink.table.api.config.ExecutionConfigOptions
import org.apache.flink.table.data.RowData
import org.apache.flink.table.planner.delegation.StreamPlanner
import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, StreamExecNode}
import org.apache.flink.table.planner.plan.utils.{AggregateUtil, ChangelogPlanUtils, KeySelectorUtil}
import org.apache.flink.table.runtime.operators.bundle.KeyedMapBundleOperator
import org.apache.flink.table.runtime.operators.deduplicate.{DeduplicateKeepLastRowFunction, MiniBatchDeduplicateKeepLastRowFunction}
import org.apache.flink.table.runtime.typeutils.InternalTypeInfo
import java.util
import scala.collection.JavaConversions._
/**
* Stream physical RelNode which normalizes a changelog stream which maybe an upsert stream or
* a changelog stream containing duplicate events. This node normalize such stream into a regular
* changelog stream that contains INSERT/UPDATE_BEFORE/UPDATE_AFTER/DELETE records without
* duplication.
*/
class StreamExecChangelogNormalize(
cluster: RelOptCluster,
traitSet: RelTraitSet,
input: RelNode,
val uniqueKeys: Array[Int])
extends SingleRel(cluster, traitSet, input)
with StreamPhysicalRel
with StreamExecNode[RowData] {
override def requireWatermark: Boolean = false
override def deriveRowType(): RelDataType = getInput.getRowType
override def copy(traitSet: RelTraitSet, inputs: util.List[RelNode]): RelNode = {
new StreamExecChangelogNormalize(
cluster,
traitSet,
inputs.get(0),
uniqueKeys)
}
override def explainTerms(pw: RelWriter): RelWriter = {
val fieldNames = getRowType.getFieldNames
super.explainTerms(pw)
.item("key", uniqueKeys.map(fieldNames.get).mkString(", "))
}
//~ ExecNode methods -----------------------------------------------------------
override def getInputNodes: util.List[ExecNode[StreamPlanner, _]] = {
List(getInput.asInstanceOf[ExecNode[StreamPlanner, _]])
}
override def replaceInputNode(
ordinalInParent: Int,
newInputNode: ExecNode[StreamPlanner, _]): Unit = {
replaceInput(ordinalInParent, newInputNode.asInstanceOf[RelNode])
}
override protected def translateToPlanInternal(
planner: StreamPlanner): Transformation[RowData] = {
val inputTransform = getInputNodes.get(0).translateToPlan(planner)
.asInstanceOf[Transformation[RowData]]
val rowTypeInfo = inputTransform.getOutputType.asInstanceOf[InternalTypeInfo[RowData]]
val generateUpdateBefore = ChangelogPlanUtils.generateUpdateBefore(this)
val tableConfig = planner.getTableConfig
val isMiniBatchEnabled = tableConfig.getConfiguration.getBoolean(
ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED)
val operator = if (isMiniBatchEnabled) {
val exeConfig = planner.getExecEnv.getConfig
val rowSerializer = rowTypeInfo.createSerializer(exeConfig)
val processFunction = new MiniBatchDeduplicateKeepLastRowFunction(
rowTypeInfo,
generateUpdateBefore,
true, // generateInsert
false, // inputInsertOnly
rowSerializer,
// disable state ttl, the changelog normalize should keep all state to have data integrity
// we can enable state ttl if this is really needed in some cases
-1)
val trigger = AggregateUtil.createMiniBatchTrigger(tableConfig)
new KeyedMapBundleOperator(
processFunction,
trigger)
} else {
val processFunction = new DeduplicateKeepLastRowFunction(
-1, // disable state ttl
rowTypeInfo,
generateUpdateBefore,
true, // generateInsert
false) // inputInsertOnly
new KeyedProcessOperator[RowData, RowData, RowData](processFunction)
}
val ret = new OneInputTransformation(
inputTransform,
getRelDetailedDescription,
operator,
rowTypeInfo,
inputTransform.getParallelism)
if (inputsContainSingleton()) {
ret.setParallelism(1)
ret.setMaxParallelism(1)
}
val selector = KeySelectorUtil.getRowDataSelector(uniqueKeys, rowTypeInfo)
ret.setStateKeySelector(selector)
ret.setStateKeyType(selector.getProducedType)
ret
}
}