/
CarbonSessionState.scala
314 lines (286 loc) · 11.6 KB
/
CarbonSessionState.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
/*
* 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.spark.sql.hive
import org.apache.hadoop.conf.Configuration
import org.apache.spark.sql.catalyst.analysis.{Analyzer, FunctionRegistry}
import org.apache.spark.sql.catalyst.catalog.{ExternalCatalogUtils, FunctionResourceLoader, GlobalTempViewManager, SessionCatalog}
import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, BoundReference, Expression, InterpretedPredicate, PredicateSubquery, ScalarSubquery}
import org.apache.spark.sql.catalyst.optimizer.Optimizer
import org.apache.spark.sql.catalyst.parser.ParserInterface
import org.apache.spark.sql.catalyst.parser.ParserUtils._
import org.apache.spark.sql.catalyst.parser.SqlBaseParser.CreateTableContext
import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, SubqueryAlias}
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.catalyst.{CatalystConf, TableIdentifier}
import org.apache.spark.sql.execution.datasources._
import org.apache.spark.sql.execution.strategy.{CarbonLateDecodeStrategy, DDLStrategy, StreamingTableStrategy}
import org.apache.spark.sql.execution.{SparkOptimizer, SparkSqlAstBuilder}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.optimizer.{CarbonIUDRule, CarbonLateDecodeRule, CarbonUDFTransformRule}
import org.apache.spark.sql.parser.{CarbonHelperSqlAstBuilder, CarbonSpark2SqlParser, CarbonSparkSqlParser}
import org.apache.spark.sql.{CarbonDatasourceHadoopRelation, CarbonEnv, ExperimentalMethods, SparkSession, Strategy}
import org.apache.carbondata.core.datamap.DataMapStoreManager
import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
import org.apache.carbondata.core.util.CarbonProperties
/**
* This class will have carbon catalog and refresh the relation from cache if the carbontable in
* carbon catalog is not same as cached carbon relation's carbon table
*
* @param externalCatalog
* @param globalTempViewManager
* @param sparkSession
* @param functionResourceLoader
* @param functionRegistry
* @param conf
* @param hadoopConf
*/
class CarbonSessionCatalog(
externalCatalog: HiveExternalCatalog,
globalTempViewManager: GlobalTempViewManager,
sparkSession: SparkSession,
functionResourceLoader: FunctionResourceLoader,
functionRegistry: FunctionRegistry,
conf: SQLConf,
hadoopConf: Configuration)
extends HiveSessionCatalog(
externalCatalog,
globalTempViewManager,
sparkSession,
functionResourceLoader,
functionRegistry,
conf,
hadoopConf) {
lazy val carbonEnv = {
val env = new CarbonEnv
env.init(sparkSession)
env
}
/**
* This method will invalidate carbonrelation from cache if carbon table is updated in
* carbon catalog
*
* @param name
* @param alias
* @return
*/
override def lookupRelation(name: TableIdentifier,
alias: Option[String]): LogicalPlan = {
val rtnRelation = super.lookupRelation(name, alias)
var toRefreshRelation = false
rtnRelation match {
case SubqueryAlias(_,
LogicalRelation(carbonDatasourceHadoopRelation: CarbonDatasourceHadoopRelation, _, _), _) =>
toRefreshRelation = refreshRelationFromCache(name, alias, carbonDatasourceHadoopRelation)
case LogicalRelation(carbonDatasourceHadoopRelation: CarbonDatasourceHadoopRelation, _, _) =>
toRefreshRelation = refreshRelationFromCache(name, alias, carbonDatasourceHadoopRelation)
case _ =>
}
if (toRefreshRelation) {
super.lookupRelation(name, alias)
} else {
rtnRelation
}
}
private def refreshRelationFromCache(identifier: TableIdentifier,
alias: Option[String],
carbonDatasourceHadoopRelation: CarbonDatasourceHadoopRelation): Boolean = {
var isRefreshed = false
val storePath = CarbonProperties.getStorePath
carbonEnv.carbonMetastore.
checkSchemasModifiedTimeAndReloadTable(identifier)
val table = carbonEnv.carbonMetastore.getTableFromMetadataCache(
carbonDatasourceHadoopRelation.carbonTable.getDatabaseName,
carbonDatasourceHadoopRelation.carbonTable.getTableName)
if (table.isEmpty || (table.isDefined &&
table.get.getTableLastUpdatedTime !=
carbonDatasourceHadoopRelation.carbonTable.getTableLastUpdatedTime)) {
refreshTable(identifier)
DataMapStoreManager.getInstance().
clearDataMaps(AbsoluteTableIdentifier.from(storePath,
identifier.database.getOrElse("default"), identifier.table))
isRefreshed = true
logInfo(s"Schema changes have been detected for table: $identifier")
}
isRefreshed
}
/**
* returns hive client from session state
*
* @return
*/
def getClient(): org.apache.spark.sql.hive.client.HiveClient = {
sparkSession.sessionState.asInstanceOf[CarbonSessionState].metadataHive
}
/**
* This is alternate way of getting partition information. It first fetches all partitions from
* hive and then apply filter instead of querying hive along with filters.
* @param partitionFilters
* @param sparkSession
* @param identifier
* @return
*/
def getPartitionsAlternate(partitionFilters: Seq[Expression],
sparkSession: SparkSession,
identifier: TableIdentifier) = {
val allPartitions = sparkSession.sessionState.catalog.listPartitions(identifier)
val catalogTable = sparkSession.sessionState.catalog.getTableMetadata(identifier)
val partitionSchema = catalogTable.partitionSchema
if (partitionFilters.nonEmpty) {
val boundPredicate =
InterpretedPredicate.create(partitionFilters.reduce(And).transform {
case att: AttributeReference =>
val index = partitionSchema.indexWhere(_.name == att.name)
BoundReference(index, partitionSchema(index).dataType, nullable = true)
})
allPartitions.filter { p => boundPredicate(p.toRow(partitionSchema)) }
} else {
allPartitions
}
}
}
/**
* Session state implementation to override sql parser and adding strategies
* @param sparkSession
*/
class CarbonSessionState(sparkSession: SparkSession) extends HiveSessionState(sparkSession) {
override lazy val sqlParser: ParserInterface = new CarbonSparkSqlParser(conf, sparkSession)
experimentalMethods.extraStrategies = extraStrategies
experimentalMethods.extraOptimizations = extraOptimizations
def extraStrategies: Seq[Strategy] = {
Seq(
new StreamingTableStrategy(sparkSession),
new CarbonLateDecodeStrategy,
new DDLStrategy(sparkSession)
)
}
def extraOptimizations: Seq[Rule[LogicalPlan]] = {
Seq(new CarbonIUDRule,
new CarbonUDFTransformRule,
new CarbonLateDecodeRule)
}
override lazy val optimizer: Optimizer = new CarbonOptimizer(catalog, conf, experimentalMethods)
def extendedAnalyzerRules: Seq[Rule[LogicalPlan]] = Nil
def internalAnalyzerRules: Seq[Rule[LogicalPlan]] = {
catalog.ParquetConversions ::
catalog.OrcConversions ::
CarbonPreInsertionCasts(sparkSession) ::
CarbonIUDAnalysisRule(sparkSession) ::
AnalyzeCreateTable(sparkSession) ::
PreprocessTableInsertion(conf) ::
DataSourceAnalysis(conf) ::
(if (conf.runSQLonFile) {
new ResolveDataSource(sparkSession) :: Nil
} else { Nil })
}
override lazy val analyzer: Analyzer =
new CarbonAnalyzer(catalog, conf, sparkSession,
new Analyzer(catalog, conf) {
override val extendedResolutionRules =
if (extendedAnalyzerRules.nonEmpty) {
extendedAnalyzerRules ++ internalAnalyzerRules
} else {
internalAnalyzerRules
}
override val extendedCheckRules = Seq(
PreWriteCheck(conf, catalog))
}
)
/**
* Internal catalog for managing table and database states.
*/
override lazy val catalog = {
new CarbonSessionCatalog(
sparkSession.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog],
sparkSession.sharedState.globalTempViewManager,
sparkSession,
functionResourceLoader,
functionRegistry,
conf,
newHadoopConf())
}
}
class CarbonAnalyzer(catalog: SessionCatalog,
conf: CatalystConf,
sparkSession: SparkSession,
analyzer: Analyzer) extends Analyzer(catalog, conf) {
override def execute(plan: LogicalPlan): LogicalPlan = {
var logicalPlan = analyzer.execute(plan)
logicalPlan = CarbonPreAggregateDataLoadingRules(logicalPlan)
CarbonPreAggregateQueryRules(sparkSession).apply(logicalPlan)
}
}
class CarbonOptimizer(
catalog: SessionCatalog,
conf: SQLConf,
experimentalMethods: ExperimentalMethods)
extends SparkOptimizer(catalog, conf, experimentalMethods) {
override def execute(plan: LogicalPlan): LogicalPlan = {
val transFormedPlan: LogicalPlan = CarbonOptimizerUtil.transformForScalarSubQuery(plan)
super.execute(transFormedPlan)
}
}
object CarbonOptimizerUtil {
def transformForScalarSubQuery(plan: LogicalPlan) : LogicalPlan = {
// In case scalar subquery add flag in relation to skip the decoder plan in optimizer rule,
// And optimize whole plan at once.
val transFormedPlan = plan.transform {
case filter: Filter =>
filter.transformExpressions {
case s: ScalarSubquery =>
val tPlan = s.plan.transform {
case lr: LogicalRelation
if lr.relation.isInstanceOf[CarbonDatasourceHadoopRelation] =>
lr.relation.asInstanceOf[CarbonDatasourceHadoopRelation].isSubquery += true
lr
}
ScalarSubquery(tPlan, s.children, s.exprId)
case p: PredicateSubquery =>
val tPlan = p.plan.transform {
case lr: LogicalRelation
if lr.relation.isInstanceOf[CarbonDatasourceHadoopRelation] =>
lr.relation.asInstanceOf[CarbonDatasourceHadoopRelation].isSubquery += true
lr
}
PredicateSubquery(tPlan, p.children, p.nullAware, p.exprId)
}
}
transFormedPlan
}
}
class CarbonSqlAstBuilder(conf: SQLConf, parser: CarbonSpark2SqlParser, sparkSession: SparkSession)
extends SparkSqlAstBuilder(conf) {
val helper = new CarbonHelperSqlAstBuilder(conf, parser, sparkSession)
override def visitCreateTable(ctx: CreateTableContext): LogicalPlan = {
val fileStorage = helper.getFileStorage(ctx.createFileFormat)
if (fileStorage.equalsIgnoreCase("'carbondata'") ||
fileStorage.equalsIgnoreCase("'org.apache.carbondata.format'")) {
helper.createCarbonTable(
tableHeader = ctx.createTableHeader,
skewSpecContext = ctx.skewSpec,
bucketSpecContext = ctx.bucketSpec,
partitionColumns = ctx.partitionColumns,
columns = ctx.columns,
tablePropertyList = ctx.tablePropertyList,
locationSpecContext = ctx.locationSpec(),
tableComment = Option(ctx.STRING()).map(string),
ctas = ctx.AS,
query = ctx.query)
} else {
super.visitCreateTable(ctx)
}
}
}