-
Notifications
You must be signed in to change notification settings - Fork 13k
/
CalciteConfig.scala
303 lines (271 loc) · 10.1 KB
/
CalciteConfig.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
/*
* 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.calcite
import java.util.Properties
import org.apache.calcite.config.{CalciteConnectionConfig, CalciteConnectionConfigImpl, CalciteConnectionProperty}
import org.apache.calcite.plan.RelOptRule
import org.apache.calcite.sql.SqlOperatorTable
import org.apache.calcite.sql.parser.SqlParser
import org.apache.calcite.sql.util.ChainedSqlOperatorTable
import org.apache.calcite.sql2rel.SqlToRelConverter
import org.apache.calcite.tools.{RuleSet, RuleSets}
import org.apache.flink.annotation.Internal
import org.apache.flink.table.api.PlannerConfig
import org.apache.flink.util.Preconditions
import scala.collection.JavaConverters._
/**
* Builder for creating a Calcite configuration.
*/
@Internal
class CalciteConfigBuilder {
/**
* Defines the normalization rule set. Normalization rules are dedicated for rewriting
* predicated logical plan before volcano optimization.
*/
private var replaceNormRules: Boolean = false
private var normRuleSets: List[RuleSet] = Nil
/**
* Defines the logical optimization rule set.
*/
private var replaceLogicalOptRules: Boolean = false
private var logicalOptRuleSets: List[RuleSet] = Nil
/**
* Defines the logical optimization rule set.
*/
private var replacePythonLogicalOptRules: Boolean = false
private var pythonLogicalOptRuleSets: List[RuleSet] = Nil
/**
* Defines the physical optimization rule set.
*/
private var replacePhysicalOptRules: Boolean = false
private var physicalOptRuleSets: List[RuleSet] = Nil
/**
* Defines the decoration rule set. Decoration rules are dedicated for rewriting predicated
* logical plan after volcano optimization.
*/
private var replaceDecoRules: Boolean = false
private var decoRuleSets: List[RuleSet] = Nil
/**
* Defines the SQL operator tables.
*/
private var replaceOperatorTable: Boolean = false
private var operatorTables: List[SqlOperatorTable] = Nil
/**
* Defines a SQL parser configuration.
*/
private var replaceSqlParserConfig: Option[SqlParser.Config] = None
/**
* Defines a configuration for SqlToRelConverter.
*/
private var replaceSqlToRelConverterConfig: Option[SqlToRelConverter.Config] = None
/**
* Replaces the built-in normalization rule set with the given rule set.
*/
def replaceNormRuleSet(replaceRuleSet: RuleSet): CalciteConfigBuilder = {
Preconditions.checkNotNull(replaceRuleSet)
normRuleSets = List(replaceRuleSet)
replaceNormRules = true
this
}
/**
* Appends the given normalization rule set to the built-in rule set.
*/
def addNormRuleSet(addedRuleSet: RuleSet): CalciteConfigBuilder = {
Preconditions.checkNotNull(addedRuleSet)
normRuleSets = addedRuleSet :: normRuleSets
this
}
/**
* Replaces the built-in optimization rule set with the given rule set.
*/
def replaceLogicalOptRuleSet(replaceRuleSet: RuleSet): CalciteConfigBuilder = {
Preconditions.checkNotNull(replaceRuleSet)
logicalOptRuleSets = List(replaceRuleSet)
replaceLogicalOptRules = true
this
}
/**
* Appends the given optimization rule set to the built-in rule set.
*/
def addLogicalOptRuleSet(addedRuleSet: RuleSet): CalciteConfigBuilder = {
Preconditions.checkNotNull(addedRuleSet)
logicalOptRuleSets = addedRuleSet :: logicalOptRuleSets
this
}
/**
* Replaces the built-in optimization rule set with the given rule set.
*/
def replacePhysicalOptRuleSet(replaceRuleSet: RuleSet): CalciteConfigBuilder = {
Preconditions.checkNotNull(replaceRuleSet)
physicalOptRuleSets = List(replaceRuleSet)
replacePhysicalOptRules = true
this
}
/**
* Appends the given optimization rule set to the built-in rule set.
*/
def addPhysicalOptRuleSet(addedRuleSet: RuleSet): CalciteConfigBuilder = {
Preconditions.checkNotNull(addedRuleSet)
physicalOptRuleSets = addedRuleSet :: physicalOptRuleSets
this
}
/**
* Replaces the built-in decoration rule set with the given rule set.
*
* The decoration rules are applied after the cost-based optimization phase.
* The decoration phase allows to rewrite the optimized plan and is not cost-based.
*
*/
def replaceDecoRuleSet(replaceRuleSet: RuleSet): CalciteConfigBuilder = {
Preconditions.checkNotNull(replaceRuleSet)
decoRuleSets = List(replaceRuleSet)
replaceDecoRules = true
this
}
/**
* Appends the given decoration rule set to the built-in rule set.
*
* The decoration rules are applied after the cost-based optimization phase.
* The decoration phase allows to rewrite the optimized plan and is not cost-based.
*/
def addDecoRuleSet(addedRuleSet: RuleSet): CalciteConfigBuilder = {
Preconditions.checkNotNull(addedRuleSet)
decoRuleSets = addedRuleSet :: decoRuleSets
this
}
/**
* Replaces the built-in SQL operator table with the given table.
*/
def replaceSqlOperatorTable(replaceSqlOperatorTable: SqlOperatorTable): CalciteConfigBuilder = {
Preconditions.checkNotNull(replaceSqlOperatorTable)
operatorTables = List(replaceSqlOperatorTable)
replaceOperatorTable = true
this
}
/**
* Appends the given table to the built-in SQL operator table.
*/
def addSqlOperatorTable(addedSqlOperatorTable: SqlOperatorTable): CalciteConfigBuilder = {
Preconditions.checkNotNull(addedSqlOperatorTable)
this.operatorTables = addedSqlOperatorTable :: this.operatorTables
this
}
/**
* Replaces the built-in SQL parser configuration with the given configuration.
*/
def replaceSqlParserConfig(sqlParserConfig: SqlParser.Config): CalciteConfigBuilder = {
Preconditions.checkNotNull(sqlParserConfig)
replaceSqlParserConfig = Some(sqlParserConfig)
this
}
/**
* Replaces the built-in SqlToRelConverter configuration with the given configuration.
*/
def replaceSqlToRelConverterConfig(config: SqlToRelConverter.Config)
: CalciteConfigBuilder = {
Preconditions.checkNotNull(config)
replaceSqlToRelConverterConfig = Some(config)
this
}
/**
* Convert the [[RuleSet]] List to [[Option]] type
*/
private def getRuleSet(inputRuleSet: List[RuleSet]): Option[RuleSet] = {
inputRuleSet match {
case Nil => None
case h :: Nil => Some(h)
case _ =>
// concat rule sets
val concatRules =
inputRuleSet.foldLeft(Nil: Iterable[RelOptRule])((c, r) => r.asScala ++ c)
Some(RuleSets.ofList(concatRules.asJava))
}
}
/**
* Builds a new [[CalciteConfig]].
*/
def build(): CalciteConfig = new CalciteConfig(
getRuleSet(normRuleSets),
replaceNormRules,
getRuleSet(logicalOptRuleSets),
replaceLogicalOptRules,
getRuleSet(pythonLogicalOptRuleSets),
replacePythonLogicalOptRules,
getRuleSet(physicalOptRuleSets),
replacePhysicalOptRules,
getRuleSet(decoRuleSets),
replaceDecoRules,
operatorTables match {
case Nil => None
case h :: Nil => Some(h)
case _ =>
// chain operator tables
Some(operatorTables.reduce((x, y) => ChainedSqlOperatorTable.of(x, y)))
},
this.replaceOperatorTable,
replaceSqlParserConfig,
replaceSqlToRelConverterConfig)
}
/**
* Calcite configuration for defining a custom Calcite configuration for Table and SQL API.
*/
@Internal
class CalciteConfig(
/** A custom normalization rule set. */
val normRuleSet: Option[RuleSet],
/** Whether this configuration replaces the built-in normalization rule set. */
val replacesNormRuleSet: Boolean,
/** A custom logical optimization rule set. */
val logicalOptRuleSet: Option[RuleSet],
/** Whether this configuration replaces the built-in logical optimization rule set. */
val replacesLogicalOptRuleSet: Boolean,
/** A custom Python logical optimization rule set. */
val pythonLogicalOptRuleSet: Option[RuleSet],
/** Whether this configuration replaces the built-in Python logical optimization rule set. */
val replacesPythonLogicalOptRuleSet: Boolean,
/** A custom physical optimization rule set. */
val physicalOptRuleSet: Option[RuleSet],
/** Whether this configuration replaces the built-in physical optimization rule set. */
val replacesPhysicalOptRuleSet: Boolean,
/** A custom decoration rule set. */
val decoRuleSet: Option[RuleSet],
/** Whether this configuration replaces the built-in decoration rule set. */
val replacesDecoRuleSet: Boolean,
/** A custom SQL operator table. */
val sqlOperatorTable: Option[SqlOperatorTable],
/** Whether this configuration replaces the built-in SQL operator table. */
val replacesSqlOperatorTable: Boolean,
/** A custom SQL parser configuration. */
val sqlParserConfig: Option[SqlParser.Config],
/** A custom configuration for SqlToRelConverter. */
val sqlToRelConverterConfig: Option[SqlToRelConverter.Config]) extends PlannerConfig
object CalciteConfig {
val DEFAULT: CalciteConfig = createBuilder().build()
/**
* Creates a new builder for constructing a [[CalciteConfig]].
*/
def createBuilder(): CalciteConfigBuilder = {
new CalciteConfigBuilder
}
def connectionConfig(parserConfig : SqlParser.Config): CalciteConnectionConfig = {
val prop = new Properties()
prop.setProperty(CalciteConnectionProperty.CASE_SENSITIVE.camelName,
String.valueOf(parserConfig.caseSensitive))
new CalciteConnectionConfigImpl(prop)
}
}