-
Notifications
You must be signed in to change notification settings - Fork 702
/
MVAnalyzerRule.scala
167 lines (154 loc) · 6.34 KB
/
MVAnalyzerRule.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
/*
* 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.carbondata.mv.datamap
import scala.collection.JavaConverters._
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.analysis.{UnresolvedAlias, UnresolvedAttribute}
import org.apache.spark.sql.catalyst.catalog.CatalogTable
import org.apache.spark.sql.catalyst.expressions.{Alias, ScalaUDF}
import org.apache.spark.sql.catalyst.plans.logical.{Command, DeserializeToObject, LogicalPlan}
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.execution.datasources.LogicalRelation
import org.apache.carbondata.common.logging.LogServiceFactory
import org.apache.carbondata.core.constants.CarbonCommonConstants
import org.apache.carbondata.core.datamap.DataMapStoreManager
import org.apache.carbondata.core.metadata.schema.datamap.DataMapClassProvider
import org.apache.carbondata.core.metadata.schema.table.DataMapSchema
import org.apache.carbondata.core.util.ThreadLocalSessionInfo
import org.apache.carbondata.datamap.DataMapManager
import org.apache.carbondata.mv.rewrite.{SummaryDataset, SummaryDatasetCatalog}
/**
* Analyzer rule to rewrite the query for MV datamap
*
* @param sparkSession
*/
class MVAnalyzerRule(sparkSession: SparkSession) extends Rule[LogicalPlan] {
// TODO Find way better way to get the provider.
private val dataMapProvider =
DataMapManager.get().getDataMapProvider(null,
new DataMapSchema("", DataMapClassProvider.MV.getShortName), sparkSession)
private val LOGGER = LogServiceFactory.getLogService(classOf[MVAnalyzerRule].getName)
override def apply(plan: LogicalPlan): LogicalPlan = {
var needAnalysis = true
plan.transformAllExpressions {
// first check if any preAgg scala function is applied it is present is in plan
// then call is from create preaggregate table class so no need to transform the query plan
// TODO Add different UDF name
case al@Alias(udf: ScalaUDF, name) if name.equalsIgnoreCase("preAgg") =>
needAnalysis = false
al
// in case of query if any unresolve alias is present then wait for plan to be resolved
// return the same plan as we can tranform the plan only when everything is resolved
case unresolveAlias@UnresolvedAlias(_, _) =>
needAnalysis = false
unresolveAlias
case attr@UnresolvedAttribute(_) =>
needAnalysis = false
attr
}
val catalog = DataMapStoreManager.getInstance().getDataMapCatalog(dataMapProvider,
DataMapClassProvider.MV.getShortName).asInstanceOf[SummaryDatasetCatalog]
if (needAnalysis && catalog != null && isValidPlan(plan, catalog)) {
val modularPlan = catalog.mvSession.sessionState.rewritePlan(plan).withMVTable
if (modularPlan.find(_.rewritten).isDefined) {
val compactSQL = modularPlan.asCompactSQL
val analyzed = sparkSession.sql(compactSQL).queryExecution.analyzed
analyzed
} else {
plan
}
} else {
plan
}
}
/**
* Whether the plan is valid for doing modular plan matching and datamap replacing.
*/
def isValidPlan(plan: LogicalPlan, catalog: SummaryDatasetCatalog): Boolean = {
if (!plan.isInstanceOf[Command] && !plan.isInstanceOf[DeserializeToObject]) {
val catalogs = extractCatalogs(plan)
!isDataMapReplaced(catalog.listAllValidSchema(), catalogs) &&
isDataMapExists(catalog.listAllValidSchema(), catalogs) &&
!isSegmentSetForMainTable(catalogs)
} else {
false
}
}
/**
* Check whether datamap table already updated in the query.
*
* @param mvdataSetArray Array of available mvdataset which include modular plans
* @return Boolean whether already datamap replaced in the plan or not
*/
def isDataMapReplaced(
mvdataSetArray: Array[SummaryDataset],
catalogs: Seq[Option[CatalogTable]]): Boolean = {
catalogs.exists { c =>
mvdataSetArray.exists { mv =>
val identifier = mv.dataMapSchema.getRelationIdentifier
identifier.getTableName.equals(c.get.identifier.table) &&
identifier.getDatabaseName.equals(c.get.database)
}
}
}
/**
* Check whether any suitable datamaps(like datamap which parent tables are present in the plan)
* exists for this plan.
*
* @param mvs
* @return
*/
def isDataMapExists(mvs: Array[SummaryDataset], catalogs: Seq[Option[CatalogTable]]): Boolean = {
catalogs.exists { c =>
mvs.exists { mv =>
mv.dataMapSchema.getParentTables.asScala.exists { identifier =>
identifier.getTableName.equals(c.get.identifier.table) &&
identifier.getDatabaseName.equals(c.get.database)
}
}
}
}
private def extractCatalogs(plan: LogicalPlan): Seq[Option[CatalogTable]] = {
val catalogs = plan collect {
case l: LogicalRelation => l.catalogTable
}
catalogs
}
/**
* Check if any segments are set for main table for Query. If any segments are set, then
* skip mv datamap table for query
*/
def isSegmentSetForMainTable(catalogs: Seq[Option[CatalogTable]]): Boolean = {
catalogs.foreach { c =>
val carbonSessionInfo = ThreadLocalSessionInfo.getCarbonSessionInfo
if (carbonSessionInfo != null) {
val segmentsToQuery = carbonSessionInfo.getSessionParams
.getProperty(CarbonCommonConstants.CARBON_INPUT_SEGMENTS +
c.get.identifier.database.get + "." +
c.get.identifier.table, "")
if (segmentsToQuery.isEmpty || segmentsToQuery.equalsIgnoreCase("*")) {
return false
} else {
return true
}
} else {
return false
}
}
false
}
}