forked from apache/flink
-
Notifications
You must be signed in to change notification settings - Fork 7
/
FlinkRelMdModifiedMonotonicity.scala
627 lines (544 loc) · 23.3 KB
/
FlinkRelMdModifiedMonotonicity.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
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
/*
* 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.metadata
import org.apache.flink.table.planner.calcite.FlinkTypeFactory
import org.apache.flink.table.planner.functions.utils.ScalarSqlFunction
import org.apache.flink.table.planner.plan.`trait`.RelModifiedMonotonicity
import org.apache.flink.table.planner.plan.metadata.FlinkMetadata.ModifiedMonotonicity
import org.apache.flink.table.planner.plan.nodes.calcite.{Expand, Rank, TableAggregate, WindowAggregate, WindowTableAggregate}
import org.apache.flink.table.planner.plan.nodes.logical._
import org.apache.flink.table.planner.plan.nodes.physical.batch.{BatchExecCorrelate, BatchExecGroupAggregateBase}
import org.apache.flink.table.planner.plan.nodes.physical.stream._
import org.apache.flink.table.planner.plan.schema.{FlinkPreparingTableBase, TableSourceTable}
import org.apache.flink.table.planner.plan.stats.{WithLower, WithUpper}
import org.apache.flink.table.planner.{JByte, JDouble, JFloat, JList, JLong, JShort}
import org.apache.calcite.plan.hep.HepRelVertex
import org.apache.calcite.plan.volcano.RelSubset
import org.apache.calcite.rel.core._
import org.apache.calcite.rel.metadata._
import org.apache.calcite.rel.{RelCollation, RelFieldCollation, RelNode}
import org.apache.calcite.rex.{RexCall, RexCallBinding, RexInputRef, RexNode}
import org.apache.calcite.sql.fun.{SqlCountAggFunction, SqlMinMaxAggFunction, SqlSumAggFunction, SqlSumEmptyIsZeroAggFunction}
import org.apache.calcite.sql.validate.SqlMonotonicity
import org.apache.calcite.sql.validate.SqlMonotonicity._
import org.apache.calcite.sql.{SqlKind, SqlOperatorBinding}
import org.apache.calcite.util.Util
import org.apache.flink.table.connector.source.ScanTableSource
import org.apache.flink.types.RowKind
import java.math.{BigDecimal => JBigDecimal}
import java.sql.{Date, Time, Timestamp}
import java.util.Collections
import scala.collection.JavaConversions._
/**
* FlinkRelMdModifiedMonotonicity supplies a default implementation of
* [[FlinkRelMetadataQuery#getRelModifiedMonotonicity]] for logical algebra.
*/
class FlinkRelMdModifiedMonotonicity private extends MetadataHandler[ModifiedMonotonicity] {
override def getDef: MetadataDef[ModifiedMonotonicity] = FlinkMetadata.ModifiedMonotonicity.DEF
def getRelModifiedMonotonicity(rel: TableScan, mq: RelMetadataQuery): RelModifiedMonotonicity = {
val monotonicity: RelModifiedMonotonicity = rel match {
case _: FlinkLogicalDataStreamTableScan | _: StreamExecDataStreamScan =>
val table = rel.getTable.unwrap(classOf[FlinkPreparingTableBase])
table.getStatistic.getRelModifiedMonotonicity
case _: FlinkLogicalTableSourceScan | _: StreamExecTableSourceScan =>
val table = rel.getTable.unwrap(classOf[TableSourceTable])
table.tableSource match {
case sts: ScanTableSource if !sts.getChangelogMode.containsOnly(RowKind.INSERT) =>
// changelog source can't produce CONSTANT ModifiedMonotonicity
new RelModifiedMonotonicity(Array.fill(rel.getRowType.getFieldCount)(NOT_MONOTONIC))
case _ => null
}
case _ => null
}
if (monotonicity != null) {
monotonicity
} else {
new RelModifiedMonotonicity(Array.fill(rel.getRowType.getFieldCount)(CONSTANT))
}
}
def getRelModifiedMonotonicity(rel: Project, mq: RelMetadataQuery): RelModifiedMonotonicity = {
getProjectMonotonicity(rel.getProjects, rel.getInput, mq)
}
def getRelModifiedMonotonicity(rel: Calc, mq: RelMetadataQuery): RelModifiedMonotonicity = {
val projects = rel.getProgram.getProjectList.map(rel.getProgram.expandLocalRef)
getProjectMonotonicity(projects, rel.getInput, mq)
}
private def getProjectMonotonicity(
projects: JList[RexNode],
input: RelNode,
mq: RelMetadataQuery): RelModifiedMonotonicity = {
val fmq = FlinkRelMetadataQuery.reuseOrCreate(mq)
// contains delete
if (containsDelete(mq, input)) {
return null
}
// all append
if (allAppend(fmq, input)) {
return new RelModifiedMonotonicity(Array.fill(projects.size)(CONSTANT))
}
// contains update
// init field monotonicities
val fieldMonotonicities = Array.fill(projects.size())(NOT_MONOTONIC)
val inputFieldMonotonicities = fmq.getRelModifiedMonotonicity(input).fieldMonotonicities
def getInputFieldIndex(node: RexNode, indexInProject: Int): Int = {
node match {
case ref: RexInputRef =>
fieldMonotonicities(indexInProject) = inputFieldMonotonicities(ref.getIndex)
ref.getIndex
case a: RexCall if a.getKind == SqlKind.AS || a.getKind == SqlKind.CAST =>
getInputFieldIndex(a.getOperands.get(0), indexInProject)
case c: RexCall if c.getOperands.size() == 1 =>
c.getOperator match {
case ssf: ScalarSqlFunction =>
val inputIndex = getInputFieldIndex(c.getOperands.get(0), indexInProject)
// collations of stream node are empty currently.
val binding = RexCallBinding.create(
input.getCluster.getTypeFactory, c, Collections.emptyList[RelCollation])
val udfMonotonicity = getUdfMonotonicity(ssf, binding)
val inputMono = if (inputIndex > -1) {
inputFieldMonotonicities(inputIndex)
} else {
NOT_MONOTONIC
}
if (inputMono == udfMonotonicity) {
fieldMonotonicities(indexInProject) = inputMono
} else {
fieldMonotonicities(indexInProject) = NOT_MONOTONIC
}
inputIndex
case _ => -1
}
case _ => -1
}
}
// copy child mono
projects.zipWithIndex.foreach { case (expr, idx) =>
getInputFieldIndex(expr, idx)
}
new RelModifiedMonotonicity(fieldMonotonicities)
}
def getRelModifiedMonotonicity(rel: Expand, mq: RelMetadataQuery): RelModifiedMonotonicity = {
getMonotonicity(rel.getInput(0), mq, rel.getRowType.getFieldCount)
}
def getRelModifiedMonotonicity(rel: Rank, mq: RelMetadataQuery): RelModifiedMonotonicity = {
val fmq = FlinkRelMetadataQuery.reuseOrCreate(mq)
val inputMonotonicity = fmq.getRelModifiedMonotonicity(rel.getInput)
// If child monotonicity is null, we should return early.
if (inputMonotonicity == null) {
return null
}
// if partitionBy a update field or partitionBy a field whose mono is null, just return null
if (rel.partitionKey.exists(e => inputMonotonicity.fieldMonotonicities(e) != CONSTANT)) {
return null
}
val fieldCount = rel.getRowType.getFieldCount
// init current mono
val currentMonotonicity = notMonotonic(fieldCount)
// 1. partitionBy field is CONSTANT
rel.partitionKey.foreach(e => currentMonotonicity.fieldMonotonicities(e) = CONSTANT)
// 2. row number filed is CONSTANT
if (rel.outputRankNumber) {
currentMonotonicity.fieldMonotonicities(fieldCount - 1) = CONSTANT
}
// 3. time attribute field is increasing
(0 until fieldCount).foreach(e => {
if (FlinkTypeFactory.isTimeIndicatorType(rel.getRowType.getFieldList.get(e).getType)) {
inputMonotonicity.fieldMonotonicities(e) = INCREASING
}
})
val fieldCollations = rel.orderKey.getFieldCollations
if (fieldCollations.nonEmpty) {
// 4. process the first collation field, we can only deduce the first collation field
val firstCollation = fieldCollations.get(0)
// Collation field index in child node will be same with Rank node,
// see ProjectToLogicalProjectAndWindowRule for details.
val fieldMonotonicity = inputMonotonicity.fieldMonotonicities(firstCollation.getFieldIndex)
val result = fieldMonotonicity match {
case SqlMonotonicity.INCREASING | SqlMonotonicity.CONSTANT
if firstCollation.direction == RelFieldCollation.Direction.DESCENDING => INCREASING
case SqlMonotonicity.DECREASING | SqlMonotonicity.CONSTANT
if firstCollation.direction == RelFieldCollation.Direction.ASCENDING => DECREASING
case _ => NOT_MONOTONIC
}
currentMonotonicity.fieldMonotonicities(firstCollation.getFieldIndex) = result
}
currentMonotonicity
}
def getRelModifiedMonotonicity(
rel: StreamExecDeduplicate,
mq: RelMetadataQuery): RelModifiedMonotonicity = {
if (allAppend(mq, rel.getInput)) {
val mono = new RelModifiedMonotonicity(
Array.fill(rel.getRowType.getFieldCount)(NOT_MONOTONIC))
rel.getUniqueKeys.foreach(e => mono.fieldMonotonicities(e) = CONSTANT)
mono
} else {
null
}
}
def getRelModifiedMonotonicity(
rel: StreamExecChangelogNormalize,
mq: RelMetadataQuery): RelModifiedMonotonicity = {
val mono = new RelModifiedMonotonicity(Array.fill(rel.getRowType.getFieldCount)(NOT_MONOTONIC))
rel.uniqueKeys.foreach(e => mono.fieldMonotonicities(e) = CONSTANT)
mono
}
def getRelModifiedMonotonicity(
rel: StreamExecWatermarkAssigner,
mq: RelMetadataQuery): RelModifiedMonotonicity = {
getMonotonicity(rel.getInput, mq, rel.getRowType.getFieldCount)
}
def getRelModifiedMonotonicity(
rel: StreamExecMiniBatchAssigner,
mq: RelMetadataQuery): RelModifiedMonotonicity = {
getMonotonicity(rel.getInput, mq, rel.getRowType.getFieldCount)
}
def getRelModifiedMonotonicity(rel: Exchange, mq: RelMetadataQuery): RelModifiedMonotonicity = {
// for exchange, get correspond from input
val fmq = FlinkRelMetadataQuery.reuseOrCreate(mq)
fmq.getRelModifiedMonotonicity(rel.getInput)
}
def getRelModifiedMonotonicity(rel: Aggregate, mq: RelMetadataQuery): RelModifiedMonotonicity = {
getRelModifiedMonotonicityOnAggregate(rel.getInput, mq, rel.getAggCallList.toList,
rel.getGroupSet.toArray)
}
def getRelModifiedMonotonicity(
rel: WindowTableAggregate,
mq: RelMetadataQuery): RelModifiedMonotonicity = {
if (allAppend(mq, rel.getInput)) {
constants(rel.getRowType.getFieldCount)
} else {
null
}
}
def getRelModifiedMonotonicity(
rel: TableAggregate, mq: RelMetadataQuery): RelModifiedMonotonicity = {
getRelModifiedMonotonicityOnTableAggregate(
rel.getInput, rel.getGroupSet.toArray, rel.getRowType.getFieldCount, mq)
}
def getRelModifiedMonotonicity(
rel: BatchExecGroupAggregateBase,
mq: RelMetadataQuery): RelModifiedMonotonicity = null
def getRelModifiedMonotonicity(
rel: StreamExecGroupAggregate,
mq: RelMetadataQuery): RelModifiedMonotonicity = {
getRelModifiedMonotonicityOnAggregate(rel.getInput, mq, rel.aggCalls.toList, rel.grouping)
}
def getRelModifiedMonotonicity(
rel: StreamExecGroupTableAggregate,
mq: RelMetadataQuery): RelModifiedMonotonicity = {
getRelModifiedMonotonicityOnTableAggregate(
rel.getInput, rel.grouping, rel.getRowType.getFieldCount, mq)
}
def getRelModifiedMonotonicity(
rel: StreamExecGlobalGroupAggregate,
mq: RelMetadataQuery): RelModifiedMonotonicity = {
// global and local agg should have same update monotonicity
val fmq = FlinkRelMetadataQuery.reuseOrCreate(mq)
fmq.getRelModifiedMonotonicity(rel.getInput)
}
def getRelModifiedMonotonicity(
rel: StreamExecLocalGroupAggregate,
mq: RelMetadataQuery): RelModifiedMonotonicity = {
getRelModifiedMonotonicityOnAggregate(rel.getInput, mq, rel.aggCalls.toList, rel.grouping)
}
def getRelModifiedMonotonicity(
rel: StreamExecIncrementalGroupAggregate,
mq: RelMetadataQuery): RelModifiedMonotonicity = {
getRelModifiedMonotonicityOnAggregate(
rel.getInput, mq, rel.finalAggCalls.toList, rel.finalAggGrouping)
}
def getRelModifiedMonotonicity(
rel: WindowAggregate,
mq: RelMetadataQuery): RelModifiedMonotonicity = null
def getRelModifiedMonotonicity(
rel: StreamExecGroupWindowAggregate,
mq: RelMetadataQuery): RelModifiedMonotonicity = {
if (allAppend(mq, rel.getInput) && !rel.emitStrategy.produceUpdates) {
constants(rel.getRowType.getFieldCount)
} else {
null
}
}
def getRelModifiedMonotonicity(
rel: StreamExecGroupWindowTableAggregate,
mq: RelMetadataQuery): RelModifiedMonotonicity = {
if (allAppend(mq, rel.getInput)) {
constants(rel.getRowType.getFieldCount)
} else {
null
}
}
def getRelModifiedMonotonicity(
rel: FlinkLogicalOverAggregate,
mq: RelMetadataQuery): RelModifiedMonotonicity = constants(rel.getRowType.getFieldCount)
def getRelModifiedMonotonicity(
rel: StreamExecOverAggregate,
mq: RelMetadataQuery): RelModifiedMonotonicity = constants(rel.getRowType.getFieldCount)
def getRelModifiedMonotonicityOnTableAggregate(
input: RelNode,
grouping: Array[Int],
rowSize: Int,
mq: RelMetadataQuery): RelModifiedMonotonicity = {
val fmq = FlinkRelMetadataQuery.reuseOrCreate(mq)
val inputMonotonicity = fmq.getRelModifiedMonotonicity(input)
// if group by an update field or group by a field mono is null, just return null
if (inputMonotonicity == null ||
grouping.exists(e => inputMonotonicity.fieldMonotonicities(e) != CONSTANT)) {
return null
}
val groupCnt = grouping.length
val fieldMonotonicity =
Array.fill(groupCnt)(CONSTANT) ++ Array.fill(rowSize - grouping.length)(NOT_MONOTONIC)
new RelModifiedMonotonicity(fieldMonotonicity)
}
def getRelModifiedMonotonicityOnAggregate(
input: RelNode,
mq: RelMetadataQuery,
aggCallList: List[AggregateCall],
grouping: Array[Int]): RelModifiedMonotonicity = {
val fmq = FlinkRelMetadataQuery.reuseOrCreate(mq)
val inputMonotonicity = fmq.getRelModifiedMonotonicity(input)
// if group by a update field or group by a field mono is null, just return null
if (inputMonotonicity == null ||
grouping.exists(e => inputMonotonicity.fieldMonotonicities(e) != CONSTANT)) {
return null
}
val groupCnt = grouping.length
// init monotonicity for group keys and agg calls
val fieldMonotonicities =
Array.fill(groupCnt)(CONSTANT) ++ Array.fill(aggCallList.size)(NOT_MONOTONIC)
// get original monotonicity ignore input
aggCallList.zipWithIndex.foreach { case (aggCall, idx) =>
val aggCallMonotonicity = getMonotonicityOnAggCall(aggCall, fmq, input)
fieldMonotonicities(idx + groupCnt) = aggCallMonotonicity
}
// need to re-calculate monotonicity if child contains update
if (containsUpdate(fmq, input)) {
aggCallList.zipWithIndex.foreach { case (aggCall, idx) =>
val index = groupCnt + idx
if (aggCall.getArgList.size() > 1) {
fieldMonotonicities(index) = NOT_MONOTONIC
} else if (aggCall.getArgList.size() == 1) {
val childMono = inputMonotonicity.fieldMonotonicities(aggCall.getArgList.head)
val currentMono = fieldMonotonicities(index)
if (childMono != currentMono &&
!aggCall.getAggregation.isInstanceOf[SqlCountAggFunction]) {
// count will Increasing even child is NOT_MONOTONIC
fieldMonotonicities(index) = NOT_MONOTONIC
}
}
}
}
new RelModifiedMonotonicity(fieldMonotonicities)
}
def getMonotonicityOnAggCall(
aggCall: AggregateCall,
mq: RelMetadataQuery,
input: RelNode): SqlMonotonicity = {
val fmq = FlinkRelMetadataQuery.reuseOrCreate(mq)
aggCall.getAggregation match {
case _: SqlCountAggFunction => INCREASING
case minMax: SqlMinMaxAggFunction => minMax.kind match {
case SqlKind.MAX => INCREASING
case SqlKind.MIN => DECREASING
case _ => NOT_MONOTONIC
}
case _: SqlSumAggFunction | _: SqlSumEmptyIsZeroAggFunction =>
val valueInterval = fmq.getFilteredColumnInterval(
input, aggCall.getArgList.head, aggCall.filterArg)
if (valueInterval == null) {
NOT_MONOTONIC
} else {
valueInterval match {
case n1: WithLower =>
val compare = isValueGreaterThanZero(n1.lower)
if (compare >= 0) {
INCREASING
} else {
NOT_MONOTONIC
}
case n2: WithUpper =>
val compare = isValueGreaterThanZero(n2.upper)
if (compare <= 0) {
DECREASING
} else {
NOT_MONOTONIC
}
case _ =>
// value range has no lower end
NOT_MONOTONIC
}
}
case _ => NOT_MONOTONIC
}
}
def getRelModifiedMonotonicity(rel: Join, mq: RelMetadataQuery): RelModifiedMonotonicity = {
val joinType = rel.getJoinType
if (joinType.equals(JoinRelType.ANTI)) {
return null
}
val left = rel.getLeft
val right = rel.getRight
val joinInfo = rel.analyzeCondition
val leftKeys = joinInfo.leftKeys
val rightKeys = joinInfo.rightKeys
val fmq = FlinkRelMetadataQuery.reuseOrCreate(mq)
// if group set contains update return null
val containDelete = containsDelete(fmq, left) || containsDelete(fmq, right)
val containUpdate = containsUpdate(fmq, left) || containsUpdate(fmq, right)
def isAllConstantOnKeys(rel: RelNode, keys: Array[Int]): Boolean = {
val mono = fmq.getRelModifiedMonotonicity(rel)
keys.forall(mono != null && mono.fieldMonotonicities(_) == CONSTANT)
}
val isKeyAllAppend = isAllConstantOnKeys(left, leftKeys.toIntArray) &&
isAllConstantOnKeys(right, rightKeys.toIntArray)
if (!containDelete && isKeyAllAppend && (containUpdate && joinInfo.isEqui || !containUpdate)) {
// output rowtype of semi equals to the rowtype of left child
if (joinType.equals(JoinRelType.SEMI)) {
fmq.getRelModifiedMonotonicity(left)
} else {
val leftFieldMonotonicities = fmq.getRelModifiedMonotonicity(left).fieldMonotonicities
val rightFieldMonotonicities = fmq.getRelModifiedMonotonicity(right).fieldMonotonicities
new RelModifiedMonotonicity(leftFieldMonotonicities ++ rightFieldMonotonicities)
}
} else {
null
}
}
def getRelModifiedMonotonicity(
rel: StreamExecIntervalJoin,
mq: RelMetadataQuery): RelModifiedMonotonicity = {
// window join won't have update
constants(rel.getRowType.getFieldCount)
}
def getRelModifiedMonotonicity(rel: Correlate, mq: RelMetadataQuery): RelModifiedMonotonicity = {
getMonotonicity(rel.getInput(0), mq, rel.getRowType.getFieldCount)
}
def getRelModifiedMonotonicity(
rel: BatchExecCorrelate,
mq: RelMetadataQuery): RelModifiedMonotonicity = null
def getRelModifiedMonotonicity(
rel: StreamExecCorrelate,
mq: RelMetadataQuery): RelModifiedMonotonicity = {
getMonotonicity(rel.getInput(0), mq, rel.getRowType.getFieldCount)
}
// TODO supports temporal table function join
def getRelModifiedMonotonicity(rel: Union, mq: RelMetadataQuery): RelModifiedMonotonicity = {
val fmq = FlinkRelMetadataQuery.reuseOrCreate(mq)
if (rel.getInputs.exists(p => containsDelete(fmq, p))) {
null
} else {
val inputMonotonicities = rel.getInputs.map(fmq.getRelModifiedMonotonicity)
val head = inputMonotonicities.head
if (inputMonotonicities.forall(head.equals(_))) {
head
} else {
notMonotonic(rel.getRowType.getFieldCount)
}
}
}
def getRelModifiedMonotonicity(
hepRelVertex: HepRelVertex,
mq: RelMetadataQuery): RelModifiedMonotonicity = {
val fmq = FlinkRelMetadataQuery.reuseOrCreate(mq)
fmq.getRelModifiedMonotonicity(hepRelVertex.getCurrentRel)
}
def getRelModifiedMonotonicity(
subset: RelSubset,
mq: RelMetadataQuery): RelModifiedMonotonicity = {
val fmq = FlinkRelMetadataQuery.reuseOrCreate(mq)
val rel = Util.first(subset.getBest, subset.getOriginal)
fmq.getRelModifiedMonotonicity(rel)
}
def getRelModifiedMonotonicity(rel: RelNode, mq: RelMetadataQuery): RelModifiedMonotonicity = null
/**
* Utility to create a RelModifiedMonotonicity which all fields is modified constant which
* means all the field's value will not be modified.
*/
def constants(fieldCount: Int): RelModifiedMonotonicity = {
new RelModifiedMonotonicity(Array.fill(fieldCount)(CONSTANT))
}
def notMonotonic(fieldCount: Int): RelModifiedMonotonicity = {
new RelModifiedMonotonicity(Array.fill(fieldCount)(NOT_MONOTONIC))
}
/**
* These operator won't generate update itself
*/
def getMonotonicity(
input: RelNode,
mq: RelMetadataQuery,
fieldCount: Int): RelModifiedMonotonicity = {
val fmq = FlinkRelMetadataQuery.reuseOrCreate(mq)
if (containsDelete(fmq, input)) {
null
} else if (allAppend(fmq, input)) {
new RelModifiedMonotonicity(Array.fill(fieldCount)(CONSTANT))
} else {
new RelModifiedMonotonicity(Array.fill(fieldCount)(NOT_MONOTONIC))
}
}
def containsDelete(mq: RelMetadataQuery, node: RelNode): Boolean = {
val fmq = FlinkRelMetadataQuery.reuseOrCreate(mq)
fmq.getRelModifiedMonotonicity(node) == null
}
def containsUpdate(mq: RelMetadataQuery, node: RelNode): Boolean = {
val fmq = FlinkRelMetadataQuery.reuseOrCreate(mq)
if (containsDelete(fmq, node)) {
false
} else {
val monotonicity = fmq.getRelModifiedMonotonicity(node)
monotonicity.fieldMonotonicities.exists(_ != CONSTANT)
}
}
def allAppend(mq: RelMetadataQuery, node: RelNode): Boolean = {
val fmq = FlinkRelMetadataQuery.reuseOrCreate(mq)
if (containsDelete(fmq, node)) {
false
} else {
val monotonicity = fmq.getRelModifiedMonotonicity(node)
monotonicity.fieldMonotonicities.forall(_ == CONSTANT)
}
}
def getUdfMonotonicity(udf: ScalarSqlFunction, binding: SqlOperatorBinding): SqlMonotonicity = {
// get monotonicity info from ScalarSqlFunction directly.
udf.getMonotonicity(binding)
}
private def isValueGreaterThanZero[T](value: Comparable[T]): Int = {
value match {
case i: Integer => i.compareTo(0)
case l: JLong => l.compareTo(0L)
case db: JDouble => db.compareTo(0d)
case f: JFloat => f.compareTo(0f)
case s: JShort => s.compareTo(0.toShort)
case b: JByte => b.compareTo(0.toByte)
case dec: JBigDecimal => dec.compareTo(JBigDecimal.ZERO)
case _: Date | _: Time | _: Timestamp | _: String =>
//not interested here, just return negative
-1
case _ =>
// other numeric types
value.asInstanceOf[Comparable[Any]].compareTo(0.asInstanceOf[Comparable[Any]])
}
}
}
object FlinkRelMdModifiedMonotonicity {
private val INSTANCE = new FlinkRelMdModifiedMonotonicity
val SOURCE: RelMetadataProvider = ReflectiveRelMetadataProvider.reflectiveSource(
FlinkMetadata.ModifiedMonotonicity.METHOD, INSTANCE)
}