From 2bb5281c99b5745d343baa6c8665dab6ff9340b2 Mon Sep 17 00:00:00 2001 From: Xteen Date: Thu, 15 Sep 2022 14:51:09 +0800 Subject: [PATCH 1/7] add log parser --- .../log-parser/pom.xml | 239 +++++++++++++ .../spark/deploy/history/LogsParser.scala | 323 ++++++++++++++++++ .../spark/conf/OmniCachePluginConfig.scala | 8 + .../rules/AbstractMaterializedViewRule.scala | 24 +- .../optimizer/rules/MVRewriteRule.scala | 22 +- .../rules/MaterializedViewJoinRule.scala | 11 +- 6 files changed, 609 insertions(+), 18 deletions(-) create mode 100644 omnicache/omnicache-spark-extension/log-parser/pom.xml create mode 100644 omnicache/omnicache-spark-extension/log-parser/src/main/scala/org/apache/spark/deploy/history/LogsParser.scala diff --git a/omnicache/omnicache-spark-extension/log-parser/pom.xml b/omnicache/omnicache-spark-extension/log-parser/pom.xml new file mode 100644 index 000000000..ce88dbc91 --- /dev/null +++ b/omnicache/omnicache-spark-extension/log-parser/pom.xml @@ -0,0 +1,239 @@ + + + + + com.huawei.kunpeng + boostkit-omnicache-spark-parent + 3.1.1-1.0.0 + + 4.0.0 + + boostkit-omnicache-spark-log-parser + jar + 3.1.1-1.0.0 + + log-parser + + + 14.0.1 + + + + + com.huawei.kunpeng + boostkit-omnicache-spark + 3.1.1-1.0.0 + + + com.google.guava + guava + ${guava.version} + + + org.apache.spark + spark-sql_${scala.binary.version} + + + org.apache.spark + spark-catalyst_${scala.binary.version} + + + org.apache.spark + spark-hive_${scala.binary.version} + + + + + org.apache.spark + spark-core_${scala.binary.version} + test-jar + test + + + org.apache.hadoop + hadoop-client + + + org.apache.curator + curator-recipes + + + + + junit + junit + test + + + org.scalatest + scalatest_${scala.binary.version} + test + + + org.scala-lang + scala-library + + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + org.antlr + antlr4-maven-plugin + + + + antlr4 + + + + + true + src/main/antlr4 + true + + + + org.apache.maven.plugins + maven-compiler-plugin + + ${java.version} + ${java.version} + + + + compile + + compile + + + + + + net.alchim31.maven + scala-maven-plugin + + ${scala.recompile.mode} + + + + scala-compile-first + process-resources + + add-source + compile + + + + scala-test-compile + process-test-resources + + testCompile + + + + + + org.scalastyle + scalastyle-maven-plugin + + false + true + true + false + ${project.basedir}/src/main/scala + ${project.basedir}/src/test/scala + ${user.dir}/scalastyle-config.xml + ${project.basedir}/target/scalastyle-output.xml + ${project.build.sourceEncoding} + ${project.reporting.outputEncoding} + + + + + check + + + + + + org.apache.maven.plugins + maven-checkstyle-plugin + + false + true + + ${project.basedir}/src/main/java + ${project.basedir}/src/main/scala + + + ${project.basedir}/src/test/java + ${project.basedir}/src/test/scala + + dev/checkstyle.xml + ${project.basedir}/target/checkstyle-output.xml + ${project.build.sourceEncoding} + ${project.reporting.outputEncoding} + + + + com.puppycrawl.tools + checkstyle + ${puppycrawl.checkstyle.version} + + + + + + check + + + + + + + + org.scalatest + scalatest-maven-plugin + + false + + ${project.build.directory}/surefire-reports + . + TestSuite.txt + + + + test + + test + + + + + + org.scoverage + scoverage-maven-plugin + + + test + test + + report + + + + + true + true + ${project.build.sourceEncoding} + true + + + + + \ No newline at end of file diff --git a/omnicache/omnicache-spark-extension/log-parser/src/main/scala/org/apache/spark/deploy/history/LogsParser.scala b/omnicache/omnicache-spark-extension/log-parser/src/main/scala/org/apache/spark/deploy/history/LogsParser.scala new file mode 100644 index 000000000..c6e194da0 --- /dev/null +++ b/omnicache/omnicache-spark-extension/log-parser/src/main/scala/org/apache/spark/deploy/history/LogsParser.scala @@ -0,0 +1,323 @@ +/* + * 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.deploy.history + +import java.util.ServiceLoader +import java.util.regex.Pattern + +import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} +import org.json4s.DefaultFormats +import org.json4s.jackson.Json +import scala.collection.JavaConverters.iterableAsScalaIterableConverter +import scala.collection.mutable +import scala.util.control.Breaks + +import org.apache.spark.{JobExecutionStatus, SparkConf} +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.Status.ASYNC_TRACKING_ENABLED +import org.apache.spark.scheduler.ReplayListenerBus +import org.apache.spark.scheduler.ReplayListenerBus.{ReplayEventsFilter, SELECT_ALL_FILTER} +import org.apache.spark.sql.catalyst.optimizer.rules._ +import org.apache.spark.sql.execution.ui._ +import org.apache.spark.status.{AppHistoryServerPlugin, AppStatusListener, ElementTrackingStore} +import org.apache.spark.util.Utils +import org.apache.spark.util.kvstore.{InMemoryStore, KVStore} + +class LogsParser(conf: SparkConf, eventLogDir: String, outPutDir: String) extends Logging { + + private val LINE_SEPARATOR = "\n" + private val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) + // Visible for testing + private[history] val fs: FileSystem = new Path(eventLogDir).getFileSystem(hadoopConf) + + /** + * parseAppHistoryLog + * + * @param appId appId + * @param fileName fileName + */ + def parseAppHistoryLog(appId: String, fileName: String): Unit = { + val inMemoryStore = new InMemoryStore() + val reader = EventLogFileReader(fs, new Path(eventLogDir, appId)) + rebuildAppStore(inMemoryStore, reader.get) + + val sqlStatusStore = new SQLAppStatusStore(inMemoryStore) + val mvRewriteSuccessInfo = getMVRewriteSuccessInfo(inMemoryStore) + + // create OutputDir + if (!fs.exists(new Path(outPutDir))) { + fs.mkdirs(new Path(outPutDir)) + } + + // continue for curLoop + val curLoop = new Breaks + + var jsons = Seq.empty[Map[String, String]] + sqlStatusStore.executionsList().foreach { execution => + curLoop.breakable { + // skip unNormal execution + val isRunning = execution.completionTime.isEmpty || + execution.jobs.exists { case (_, status) => status == JobExecutionStatus.RUNNING } + val isFailed = execution + .jobs.exists { case (_, status) => status == JobExecutionStatus.FAILED } + if (isRunning || isFailed) { + curLoop.break() + } + + val uiData = execution + val executionId = uiData.executionId + val planDesc = uiData.physicalPlanDescription + val query = uiData.description + var mvs = mvRewriteSuccessInfo.getOrElse(query, "") + if (mvs.nonEmpty) { + mvs.split(";").foreach { mv => + if (!planDesc.contains(mv)) { + mvs = "" + } + } + } + + // write dot + val graph: SparkPlanGraph = sqlStatusStore.planGraph(executionId) + sqlStatusStore.planGraph(executionId) + val metrics = sqlStatusStore.executionMetrics(executionId) + val node = getNodeInfo(graph) + + val jsonMap = Map( + "original query" -> query, + "materialized views" -> mvs, + "physical plan" -> planDesc, + "dot metrics" -> graph.makeDotFile(metrics), + "node metrics" -> node) + jsons :+= jsonMap + } + } + // write json file into hdfs + val jsonFile: String = Json(DefaultFormats).write(jsons) + writeFiles(fs, outPutDir + "/" + fileName + ".json", jsonFile) + } + + /** + * getMVRewriteSuccessInfo + * + * @param store KVStore + * @return {sql:mvs} + */ + def getMVRewriteSuccessInfo(store: KVStore): mutable.Map[String, String] = { + val infos = mutable.Map.empty[String, String] + try { + // The ApplicationInfo may not be available when Spark is starting up + Utils.tryWithResource( + store.view(classOf[SparkListenerMVRewriteSuccess]) + .closeableIterator() + ) { it => + while (it.hasNext) { + val info = it.next() + infos += (info.sql, info.usingMvs) + } + } + } catch { + case e: NoSuchElementException => + logWarning("getMVRewriteSuccessInfo is failed for ", e) + } + infos + } + + /** + * getNodeInfo from graph + * + * @param graph SparkPlanGraph + * @return NodeInfo + */ + def getNodeInfo(graph: SparkPlanGraph): String = { + // write node + val tmpContext = new StringBuilder + tmpContext.append("[PlanMetric]") + nextLine(tmpContext) + graph.allNodes.foreach { node => + tmpContext.append(s"id:${node.id} name:${node.name} desc:${node.desc}") + nextLine(tmpContext) + node.metrics.foreach { metric => + metric.toString + tmpContext.append("SQLPlanMetric(") + tmpContext.append(metric.name) + tmpContext.append(",") + if (metric.metricType == "timing") { + tmpContext.append(s"${metric.accumulatorId * 1000000} ns, ") + } else if (metric.metricType == "nsTiming") { + tmpContext.append(s"${metric.accumulatorId} ns, ") + } else { + tmpContext.append(s"${metric.accumulatorId}, ") + } + tmpContext.append(metric.metricType) + tmpContext.append(")") + nextLine(tmpContext) + } + nextLine(tmpContext) + nextLine(tmpContext) + nextLine(tmpContext) + } + + graph.edges.foreach { edges => + tmpContext.append(edges.makeDotEdge) + nextLine(tmpContext) + } + + tmpContext.append("[SubGraph]") + nextLine(tmpContext) + graph.allNodes.foreach { + case cluster: SparkPlanGraphCluster => + tmpContext.append(s"cluster ${cluster.id} : ") + for (i <- cluster.nodes.indices) { + tmpContext.append(s"${cluster.nodes(i).id} ") + } + nextLine(tmpContext) + case node => + } + nextLine(tmpContext) + tmpContext.toString() + } + + def nextLine(context: StringBuilder): Unit = { + context.append(LINE_SEPARATOR) + } + + /** + * rebuildAppStore + * + * @param store KVStore + * @param reader EventLogFileReader + */ + private[spark] def rebuildAppStore(store: KVStore, reader: EventLogFileReader): Unit = { + // Disable async updates, since they cause higher memory usage, and it's ok to take longer + // to parse the event logs in the SHS + val replayConf = conf.clone().set(ASYNC_TRACKING_ENABLED, false) + val trackingStore = new ElementTrackingStore(store, replayConf) + val replayBus = new ReplayListenerBus() + val listener = new AppStatusListener(trackingStore, replayConf, false) + replayBus.addListener(listener) + replayBus.addListener(new MVRewriteSuccessListener(trackingStore)) + + for { + plugin <- loadPlugins() + listener <- plugin.createListeners(conf, trackingStore) + } replayBus.addListener(listener) + + try { + val eventLogFiles = reader.listEventLogFiles + + // parse event log + parseAppEventLogs(eventLogFiles, replayBus, !reader.completed) + trackingStore.close(false) + } catch { + case e: Exception => + Utils.tryLogNonFatalError { + trackingStore.close() + } + throw e + } + } + + /** + * loadPlugins + * + * @return Plugins + */ + private def loadPlugins(): Iterable[AppHistoryServerPlugin] = { + val plugins = ServiceLoader.load(classOf[AppHistoryServerPlugin], + Utils.getContextOrSparkClassLoader).asScala + plugins + } + + /** + * parseAppEventLogs + * + * @param logFiles Seq[FileStatus] + * @param replayBus ReplayListenerBus + * @param maybeTruncated Boolean + * @param eventsFilter ReplayEventsFilter + */ + private def parseAppEventLogs(logFiles: Seq[FileStatus], + replayBus: ReplayListenerBus, + maybeTruncated: Boolean, + eventsFilter: ReplayEventsFilter = SELECT_ALL_FILTER): Unit = { + // stop replaying next log files if ReplayListenerBus indicates some error or halt + var continueReplay = true + logFiles.foreach { file => + if (continueReplay) { + Utils.tryWithResource(EventLogFileReader.openEventLog(file.getPath, fs)) { in => + continueReplay = replayBus.replay(in, file.getPath.toString, + maybeTruncated = maybeTruncated, eventsFilter = eventsFilter) + } + } + } + } + + /** + * write parsed logInfo to logPath + * + * @param fs FileSystem + * @param logPath logPath + * @param context logInfo + */ + private def writeFiles(fs: FileSystem, logPath: String, context: String): Unit = { + val os = fs.create(new Path(logPath)) + os.write(context.getBytes()) + os.close() + } +} + +/* +arg0: spark.eventLog.dir, eg. hdfs://server1:9000/spark2-history +arg1: output dir in hdfs, eg. hdfs://server1:9000/logParser +arg2: log file to be parsed, eg. application_1646816941391_0115.lz4 + */ +object ParseLog extends Logging { + def main(args: Array[String]): Unit = { + if (args == null || args.length != 3) { + logWarning("input params is invalid,such as below\n" + + "arg0: spark.eventLog.dir, eg. hdfs://server1:9000/spark2-history\n" + + "arg1: output dir in hdfs, eg. hdfs://server1:9000/logParser\n" + + "arg2: log file to be parsed, eg. application_1646816941391_0115.lz4\n") + } + val sparkEventLogDir = args(0) + val outputDir = args(1) + val logName = args(2) + + val conf = new SparkConf + // spark.eventLog.dir + conf.set("spark.eventLog.dir", sparkEventLogDir) + // spark.eventLog.compress + conf.set("spark.eventLog.compress", "true") + // fs.hdfs.impl + conf.set("fs.hdfs.impl", "org.apache.hadoop.hdfs.DistributedFileSystem") + val logParser = new LogsParser(conf, sparkEventLogDir, outputDir) + + // file pattern + val regex = "application_[0-9]+._[0-9]+.lz4" + val pattern = Pattern.compile(regex) + val matcher = pattern.matcher(logName) + if (matcher.find) { + val appId = matcher.group + logParser.parseAppHistoryLog(appId, logName) + } else { + logWarning(logName + " is illegal") + } + } +} diff --git a/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/conf/OmniCachePluginConfig.scala b/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/conf/OmniCachePluginConfig.scala index d9fc9b317..dbb9b78eb 100644 --- a/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/conf/OmniCachePluginConfig.scala +++ b/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/conf/OmniCachePluginConfig.scala @@ -20,6 +20,8 @@ package com.huawei.boostkit.spark.conf import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.CatalogTable +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.internal.SQLConf class OmniCachePluginConfig(conf: SQLConf) { @@ -86,4 +88,10 @@ object OmniCachePluginConfig { val catalogTable = spark.sessionState.catalog.getTableMetadata(mv) !catalogTable.properties.getOrElse(MV_UPDATE_REWRITE_ENABLED, "true").toBoolean } + + def isMVInUpdate(viewTablePlan: LogicalPlan): Boolean = { + val logicalPlan = viewTablePlan.asInstanceOf[LogicalRelation] + !logicalPlan.catalogTable.get + .properties.getOrElse(MV_UPDATE_REWRITE_ENABLED, "true").toBoolean + } } diff --git a/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/AbstractMaterializedViewRule.scala b/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/AbstractMaterializedViewRule.scala index df26d192f..3bf2c2f2b 100644 --- a/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/AbstractMaterializedViewRule.scala +++ b/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/AbstractMaterializedViewRule.scala @@ -61,7 +61,7 @@ abstract class AbstractMaterializedViewRule(sparkSession: SparkSession) // 3.use all tables to fetch views(may match) from ViewMetaData val candidateViewPlans = getApplicableMaterializations(queryTables.map(t => t.tableName)) - .filter(x => !OmniCachePluginConfig.isMVInUpdate(sparkSession, x._1)) + .filter(x => !OmniCachePluginConfig.isMVInUpdate(x._2)) if (candidateViewPlans.isEmpty) { return finalPlan } @@ -112,6 +112,9 @@ abstract class AbstractMaterializedViewRule(sparkSession: SparkSession) val (newViewTablePlan, newViewQueryPlan, newTopViewProject) = newViewPlans.get viewTablePlan = newViewTablePlan viewQueryPlan = newViewQueryPlan + if (newTopViewProject.isEmpty) { + viewQueryExpr = newViewQueryPlan + } topViewProject = newTopViewProject } @@ -284,6 +287,7 @@ abstract class AbstractMaterializedViewRule(sparkSession: SparkSession) } } JavaConverters.asScalaIteratorConverter(result.iterator()).asScala.toSeq + .sortWith((map1, map2) => map1.toString < map2.toString) } /** @@ -594,18 +598,6 @@ abstract class AbstractMaterializedViewRule(sparkSession: SparkSession) if (columnsEquiPredictsResult.isEmpty) { return None } - val viewTableAttrSet = viewTableAttrs.map(ExpressionEqual).toSet - columnsEquiPredictsResult.get.foreach { expr => - expr.foreach { - case attr: AttributeReference => - if (!viewTableAttrSet.contains(ExpressionEqual(attr))) { - logDebug(s"attr:%s cannot found in view:%s" - .format(attr, OmniCachePluginConfig.getConf.curMatchMV)) - return None - } - case _ => - } - } // 5.rewrite rangeCompensation,residualCompensation by viewTableAttrs val otherPredictsResult = rewriteExpressions(Seq(compensationRangePredicts.get, @@ -642,7 +634,7 @@ abstract class AbstractMaterializedViewRule(sparkSession: SparkSession) val swapProjectList = if (swapTableColumn) { swapTableColumnReferences(viewProjectList, tableMapping, columnMapping) } else { - swapColumnTableReferences(viewProjectList, tableMapping, columnMapping) + swapTableColumnReferences(viewProjectList, tableMapping, columnMapping) } val swapTableAttrs = swapTableReferences(viewTableAttrs, tableMapping) @@ -665,11 +657,11 @@ abstract class AbstractMaterializedViewRule(sparkSession: SparkSession) }.asInstanceOf[T] // 4.iterate result and dfs check every AttributeReference in ViewTableAttributeReference - val viewTableAttrSet = swapTableAttrs.map(ExpressionEqual).toSet + val viewTableAttrSet = swapTableAttrs.map(_.exprId).toSet result.foreach { expr => expr.foreach { case attr: AttributeReference => - if (!viewTableAttrSet.contains(ExpressionEqual(attr))) { + if (!viewTableAttrSet.contains(attr.exprId)) { logDebug(s"attr:%s cannot found in view:%s" .format(attr, OmniCachePluginConfig.getConf.curMatchMV)) return None diff --git a/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/MVRewriteRule.scala b/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/MVRewriteRule.scala index 98989c460..888d5af02 100644 --- a/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/MVRewriteRule.scala +++ b/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/MVRewriteRule.scala @@ -17,17 +17,21 @@ package org.apache.spark.sql.catalyst.optimizer.rules +import com.fasterxml.jackson.annotation.JsonIgnore import com.huawei.boostkit.spark.conf.OmniCachePluginConfig import com.huawei.boostkit.spark.util.RewriteHelper import scala.collection.mutable import org.apache.spark.SparkContext +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.internal.Logging -import org.apache.spark.scheduler.SparkListenerEvent +import org.apache.spark.scheduler.{SparkListener, SparkListenerEvent} import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LogicalPlan, Project} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.command.OmniCacheCreateMvCommand +import org.apache.spark.status.ElementTrackingStore +import org.apache.spark.util.kvstore.KVIndex class MVRewriteRule(session: SparkSession) extends Rule[LogicalPlan] with Logging { val omniCacheConf: OmniCachePluginConfig = OmniCachePluginConfig.getConf @@ -84,5 +88,21 @@ class MVRewriteRule(session: SparkSession) extends Rule[LogicalPlan] with Loggin } } +@DeveloperApi case class SparkListenerMVRewriteSuccess(sql: String, usingMvs: String) extends SparkListenerEvent { + @JsonIgnore + @KVIndex + def id: String = (System.currentTimeMillis() + "%s%s".format(sql, usingMvs).hashCode).toString +} + +class MVRewriteSuccessListener( + kvStore: ElementTrackingStore) extends SparkListener with Logging { + + override def onOtherEvent(event: SparkListenerEvent): Unit = { + event match { + case _: SparkListenerMVRewriteSuccess => + kvStore.write(event) + case _ => + } + } } diff --git a/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/MaterializedViewJoinRule.scala b/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/MaterializedViewJoinRule.scala index 15717fa16..99248e01c 100644 --- a/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/MaterializedViewJoinRule.scala +++ b/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/MaterializedViewJoinRule.scala @@ -66,11 +66,20 @@ class MaterializedViewJoinRule(sparkSession: SparkSession) topViewProject.get } + var projectList: Seq[NamedExpression] = newViewQueryPlan match { + case p: Project => + p.projectList + case _ => + newViewQueryPlan.output + } + needTables.foreach { needTable => newViewQueryPlan = Join(newViewQueryPlan, needTable.logicalPlan, Inner, None, JoinHint.NONE) + projectList ++= needTable.logicalPlan.output } - Some(newViewTablePlan, viewQueryPlan, None) + newViewQueryPlan = Project(projectList, newViewQueryPlan) + Some(newViewTablePlan, newViewQueryPlan, None) } /** -- Gitee From d68eb51696bbb31958e2dc84f57f2f29142b4a32 Mon Sep 17 00:00:00 2001 From: Xteen Date: Thu, 15 Sep 2022 19:37:44 +0800 Subject: [PATCH 2/7] add some comment and adjust code style --- omnicache/omnicache-spark-extension/build.sh | 3 +- .../log-parser/pom.xml | 3 +- .../spark/deploy/history/LogsParser.scala | 13 +-- .../omnicache-spark-extension/plugin/pom.xml | 1 + .../spark/conf/OmniCachePluginConfig.scala | 4 +- .../boostkit/spark/util/ExprSimplifier.scala | 5 +- .../boostkit/spark/util/RewriteHelper.scala | 57 ++++++++++--- .../boostkit/spark/util/ViewMetadata.scala | 38 +++------ .../rules/AbstractMaterializedViewRule.scala | 85 +++++++++++++------ .../optimizer/rules/MVRewriteRule.scala | 18 ++-- .../rules/MaterializedViewAggregateRule.scala | 5 +- .../rules/MaterializedViewJoinRule.scala | 6 +- .../parser/OmniCacheExtensionAstBuilder.scala | 20 +++-- .../parser/OmniCacheExtensionSqlParser.scala | 2 +- .../execution/command/OmniCacheCommand.scala | 25 ++++-- omnicache/omnicache-spark-extension/pom.xml | 1 + 16 files changed, 186 insertions(+), 100 deletions(-) diff --git a/omnicache/omnicache-spark-extension/build.sh b/omnicache/omnicache-spark-extension/build.sh index ad436cddd..a319895de 100644 --- a/omnicache/omnicache-spark-extension/build.sh +++ b/omnicache/omnicache-spark-extension/build.sh @@ -1,2 +1,3 @@ #!/bin/bash -mvn clean package \ No newline at end of file +cpu_name=$(lscpu | grep Architecture | awk '{print $2}') +mvn clean package -Ddep.os.arch="-${cpu_name}" \ No newline at end of file diff --git a/omnicache/omnicache-spark-extension/log-parser/pom.xml b/omnicache/omnicache-spark-extension/log-parser/pom.xml index ce88dbc91..75f93ca23 100644 --- a/omnicache/omnicache-spark-extension/log-parser/pom.xml +++ b/omnicache/omnicache-spark-extension/log-parser/pom.xml @@ -10,7 +10,7 @@ 4.0.0 - boostkit-omnicache-spark-log-parser + boostkit-omnicache-logparser-spark jar 3.1.1-1.0.0 @@ -78,6 +78,7 @@ + ${artifactId}-${version}${dep.os.arch} target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes diff --git a/omnicache/omnicache-spark-extension/log-parser/src/main/scala/org/apache/spark/deploy/history/LogsParser.scala b/omnicache/omnicache-spark-extension/log-parser/src/main/scala/org/apache/spark/deploy/history/LogsParser.scala index c6e194da0..ce2d438a7 100644 --- a/omnicache/omnicache-spark-extension/log-parser/src/main/scala/org/apache/spark/deploy/history/LogsParser.scala +++ b/omnicache/omnicache-spark-extension/log-parser/src/main/scala/org/apache/spark/deploy/history/LogsParser.scala @@ -35,7 +35,7 @@ import org.apache.spark.scheduler.ReplayListenerBus import org.apache.spark.scheduler.ReplayListenerBus.{ReplayEventsFilter, SELECT_ALL_FILTER} import org.apache.spark.sql.catalyst.optimizer.rules._ import org.apache.spark.sql.execution.ui._ -import org.apache.spark.status.{AppHistoryServerPlugin, AppStatusListener, ElementTrackingStore} +import org.apache.spark.status._ import org.apache.spark.util.Utils import org.apache.spark.util.kvstore.{InMemoryStore, KVStore} @@ -100,6 +100,7 @@ class LogsParser(conf: SparkConf, eventLogDir: String, outPutDir: String) extend val node = getNodeInfo(graph) val jsonMap = Map( + "executionId" -> executionId.toString, "original query" -> query, "materialized views" -> mvs, "physical plan" -> planDesc, @@ -110,7 +111,7 @@ class LogsParser(conf: SparkConf, eventLogDir: String, outPutDir: String) extend } // write json file into hdfs val jsonFile: String = Json(DefaultFormats).write(jsons) - writeFiles(fs, outPutDir + "/" + fileName + ".json", jsonFile) + writeFile(fs, outPutDir + "/" + fileName + ".json", jsonFile) } /** @@ -122,14 +123,14 @@ class LogsParser(conf: SparkConf, eventLogDir: String, outPutDir: String) extend def getMVRewriteSuccessInfo(store: KVStore): mutable.Map[String, String] = { val infos = mutable.Map.empty[String, String] try { - // The ApplicationInfo may not be available when Spark is starting up + // The ApplicationInfo may not be available when Spark is starting up. Utils.tryWithResource( store.view(classOf[SparkListenerMVRewriteSuccess]) .closeableIterator() ) { it => while (it.hasNext) { val info = it.next() - infos += (info.sql, info.usingMvs) + infos += (info.sql -> info.usingMvs) } } } catch { @@ -206,7 +207,7 @@ class LogsParser(conf: SparkConf, eventLogDir: String, outPutDir: String) extend */ private[spark] def rebuildAppStore(store: KVStore, reader: EventLogFileReader): Unit = { // Disable async updates, since they cause higher memory usage, and it's ok to take longer - // to parse the event logs in the SHS + // to parse the event logs in the SHS. val replayConf = conf.clone().set(ASYNC_TRACKING_ENABLED, false) val trackingStore = new ElementTrackingStore(store, replayConf) val replayBus = new ReplayListenerBus() @@ -276,7 +277,7 @@ class LogsParser(conf: SparkConf, eventLogDir: String, outPutDir: String) extend * @param logPath logPath * @param context logInfo */ - private def writeFiles(fs: FileSystem, logPath: String, context: String): Unit = { + private def writeFile(fs: FileSystem, logPath: String, context: String): Unit = { val os = fs.create(new Path(logPath)) os.write(context.getBytes()) os.close() diff --git a/omnicache/omnicache-spark-extension/plugin/pom.xml b/omnicache/omnicache-spark-extension/plugin/pom.xml index 3879b778d..721879bdd 100644 --- a/omnicache/omnicache-spark-extension/plugin/pom.xml +++ b/omnicache/omnicache-spark-extension/plugin/pom.xml @@ -73,6 +73,7 @@ + ${artifactId}-${version}${dep.os.arch} target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes diff --git a/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/conf/OmniCachePluginConfig.scala b/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/conf/OmniCachePluginConfig.scala index dbb9b78eb..43214d6ae 100644 --- a/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/conf/OmniCachePluginConfig.scala +++ b/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/conf/OmniCachePluginConfig.scala @@ -90,8 +90,8 @@ object OmniCachePluginConfig { } def isMVInUpdate(viewTablePlan: LogicalPlan): Boolean = { - val logicalPlan = viewTablePlan.asInstanceOf[LogicalRelation] - !logicalPlan.catalogTable.get + val logicalRelation = viewTablePlan.asInstanceOf[LogicalRelation] + !logicalRelation.catalogTable.get .properties.getOrElse(MV_UPDATE_REWRITE_ENABLED, "true").toBoolean } } diff --git a/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/util/ExprSimplifier.scala b/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/util/ExprSimplifier.scala index 13b1a607d..ae0040823 100644 --- a/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/util/ExprSimplifier.scala +++ b/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/util/ExprSimplifier.scala @@ -30,7 +30,6 @@ import org.apache.spark.sql.catalyst.optimizer._ import org.apache.spark.sql.catalyst.plans.logical.{Filter, Join, LogicalPlan, OneRowRelation} import org.apache.spark.sql.types.{BooleanType, DataType, NullType} - case class ExprSimplifier(unknownAsFalse: Boolean, pulledUpPredicates: Set[Expression]) { @@ -476,7 +475,7 @@ case class ExprSimplifier(unknownAsFalse: Boolean, for (term <- terms) { // Excluding self-simplification if (!term.eq(orOp)) { - // Simplification between a orExpression and a orExpression. + // Simplification between a OrExpression and a OrExpression. if (term.isInstanceOf[Or]) { if (containsAllSql(ors, decomposeDisjunctions(term).toSet)) { terms.-=(orOp) @@ -615,7 +614,7 @@ case class ExprSimplifier(unknownAsFalse: Boolean, case c@Not(c1@EqualNullSafe(_, Literal.TrueLiteral)) => Not(c1.copy(left = child2)) case c => - throw new RuntimeException("unSupport type is predict simplify :%s".format(c)) + throw new RuntimeException("unSupport type is predicate simplify :%s".format(c)) } return condition2 } diff --git a/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/util/RewriteHelper.scala b/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/util/RewriteHelper.scala index 38b7d40a3..18f2db275 100644 --- a/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/util/RewriteHelper.scala +++ b/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/util/RewriteHelper.scala @@ -28,14 +28,14 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.internal.SQLConf -class RewriteHelper extends PredicateHelper { +trait RewriteHelper extends PredicateHelper { val SESSION_CATALOG_NAME: String = "spark_catalog" - val EMPTY_BITMAP: HashBiMap[String, String] = HashBiMap.create[String, String]() + val EMPTY_BIMAP: HashBiMap[String, String] = HashBiMap.create[String, String]() val EMPTY_MAP: Map[ExpressionEqual, mutable.Set[ExpressionEqual]] = Map[ExpressionEqual, mutable.Set[ExpressionEqual]]() - val EMPTY_MULTIMAP: Multimap[Int, Int] = ArrayListMultimap.create[Int, Int] + val EMPTY_MULTIMAP: Multimap[Int, Int] = ArrayListMultimap.create[Int, Int]() def mergeConjunctiveExpressions(e: Seq[Expression]): Expression = { if (e.isEmpty) { @@ -81,8 +81,9 @@ class RewriteHelper extends PredicateHelper { for ((project, column) <- topProjectList.zip(viewTablePlan.output)) { project match { // only map attr - case _@Alias(attr@AttributeReference(_, _, _, _), _) => - exprIdToQualifier += (column.exprId -> attr) + case a@Alias(attr@AttributeReference(_, _, _, _), _) => + exprIdToQualifier += (column.exprId -> + attr.copy(name = a.name)(exprId = attr.exprId, qualifier = attr.qualifier)) case a@AttributeReference(_, _, _, _) => exprIdToQualifier += (column.exprId -> a) // skip function @@ -204,11 +205,11 @@ class RewriteHelper extends PredicateHelper { (mappedQuery, mappedTables) } - def swapTableColumnReferences[T <: Iterable[Expression]](expression: T, + def swapTableColumnReferences[T <: Iterable[Expression]](expressions: T, tableMapping: BiMap[String, String], columnMapping: Map[ExpressionEqual, mutable.Set[ExpressionEqual]]): T = { - var result: T = expression + var result: T = expressions if (!tableMapping.isEmpty) { result = result.map { expr => expr.transform { @@ -243,24 +244,24 @@ class RewriteHelper extends PredicateHelper { result } - def swapColumnTableReferences[T <: Iterable[Expression]](expression: T, + def swapColumnTableReferences[T <: Iterable[Expression]](expressions: T, tableMapping: BiMap[String, String], columnMapping: Map[ExpressionEqual, mutable.Set[ExpressionEqual]]): T = { - var result = swapTableColumnReferences(expression, EMPTY_BITMAP, columnMapping) + var result = swapTableColumnReferences(expressions, EMPTY_BIMAP, columnMapping) result = swapTableColumnReferences(result, tableMapping, EMPTY_MAP) result } - def swapTableReferences[T <: Iterable[Expression]](expression: T, + def swapTableReferences[T <: Iterable[Expression]](expressions: T, tableMapping: BiMap[String, String]): T = { - swapTableColumnReferences(expression, tableMapping, EMPTY_MAP) + swapTableColumnReferences(expressions, tableMapping, EMPTY_MAP) } - def swapColumnReferences[T <: Iterable[Expression]](expression: T, + def swapColumnReferences[T <: Iterable[Expression]](expressions: T, columnMapping: Map[ExpressionEqual, mutable.Set[ExpressionEqual]]): T = { - swapTableColumnReferences(expression, EMPTY_BITMAP, columnMapping) + swapTableColumnReferences(expressions, EMPTY_BIMAP, columnMapping) } } @@ -431,6 +432,36 @@ object RewriteHelper extends PredicateHelper { def disableCachePlugin(): Unit = { SQLConf.get.setConfString("spark.sql.omnicache.enable", "false") } + + def checkAttrsValid(logicalPlan: LogicalPlan): Boolean = { + logicalPlan.foreachUp { + case _: LeafNode => + case plan => + val attributeSets = plan.expressions.map { expression => + AttributeSet.fromAttributeSets( + expression.collect { + case s: SubqueryExpression => + var res = s.references + s.plan.transformAllExpressions { + case e@OuterReference(ar) => + res ++= AttributeSet(ar.references) + e + case e => e + } + res + case e => e.references + }) + } + val request = AttributeSet.fromAttributeSets(attributeSets) + val input = plan.inputSet + val missing = request -- input + if (missing.nonEmpty) { + logWarning("checkAttrsValid failed for missing:%s".format(missing)) + return false + } + } + true + } } case class ExpressionEqual(expression: Expression) { diff --git a/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/util/ViewMetadata.scala b/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/util/ViewMetadata.scala index 807418400..07534fa38 100644 --- a/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/util/ViewMetadata.scala +++ b/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/util/ViewMetadata.scala @@ -17,10 +17,9 @@ package com.huawei.boostkit.spark.util -import com.google.common.collect.Lists import com.huawei.boostkit.spark.conf.OmniCachePluginConfig._ import java.util.concurrent.ConcurrentHashMap -import org.apache.calcite.util.graph._ +import scala.collection.mutable import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.TableIdentifier @@ -35,9 +34,7 @@ object ViewMetadata extends RewriteHelper { val viewToContainsTables = new ConcurrentHashMap[String, Set[TableEqual]]() - val usesGraph: DirectedGraph[String, DefaultEdge] = DefaultDirectedGraph.create() - - var frozenGraph: Graphs.FrozenGraph[String, DefaultEdge] = Graphs.makeImmutable(usesGraph) + val tableToViews = new ConcurrentHashMap[String, mutable.Set[String]]() var spark: SparkSession = _ @@ -52,10 +49,6 @@ object ViewMetadata extends RewriteHelper { status = STATUS_LOADING } - def usesGraphTopologicalOrderIterator: java.lang.Iterable[String] = { - TopologicalOrderIterator.of[String, DefaultEdge](usesGraph) - } - def saveViewMetadataToMap(catalogTable: CatalogTable): Unit = this.synchronized { // if QUERY_REWRITE_ENABLED is false, doesn't load ViewMetadata if (!catalogTable.properties.getOrElse(MV_REWRITE_ENABLED, "false").toBoolean) { @@ -91,15 +84,15 @@ object ViewMetadata extends RewriteHelper { // spark_catalog.db.table val viewName = catalogTable.identifier.toString() - // mappedViewQueryPlan and mappedViewContainsTable + // mappedViewQueryPlan and mappedViewContainsTables val (mappedViewQueryPlan, mappedViewContainsTables) = extractTables(viewQueryPlan) - usesGraph.addVertex(viewName) mappedViewContainsTables .foreach { mappedViewContainsTable => val name = mappedViewContainsTable.tableName - usesGraph.addVertex(name) - usesGraph.addEdge(name, viewName) + val views = tableToViews.getOrDefault(name, mutable.Set.empty) + views += viewName + tableToViews.put(name, views) } // extract view query project's Attr and replace view table's Attr by query project's Attr @@ -113,7 +106,7 @@ object ViewMetadata extends RewriteHelper { viewToTablePlan.putIfAbsent(viewName, mappedViewTablePlan) } catch { case e: Throwable => - logDebug(s"Failed to saveViewMetadataToMap. errmsg: ${e.getMessage}") + logDebug(s"Failed to saveViewMetadataToMap,errmsg: ${e.getMessage}") // reset preDatabase spark.sessionState.catalogManager.setCurrentNamespace(Array(preDatabase)) } @@ -129,21 +122,19 @@ object ViewMetadata extends RewriteHelper { def addCatalogTableToCache(table: CatalogTable): Unit = this.synchronized { saveViewMetadataToMap(table) - rebuildGraph() - } - - def rebuildGraph(): Unit = { - frozenGraph = Graphs.makeImmutable(usesGraph) } def removeMVCache(tableName: TableIdentifier): Unit = this.synchronized { val viewName = tableName.toString() - usesGraph.removeAllVertices(Lists.newArrayList(viewName)) viewToContainsTables.remove(viewName) viewToViewQueryPlan.remove(viewName) viewToTablePlan.remove(viewName) - viewToContainsTables.remove(viewName) - rebuildGraph() + tableToViews.forEach { (key, value) => + if (value.contains(viewName)) { + value -= viewName + tableToViews.put(key, value) + } + } } def init(sparkSession: SparkSession): Unit = { @@ -165,7 +156,6 @@ object ViewMetadata extends RewriteHelper { val tables = omniCacheFilter(catalog, db) tables.foreach(tableData => saveViewMetadataToMap(tableData)) } - rebuildGraph() } def omniCacheFilter(catalog: SessionCatalog, @@ -176,7 +166,7 @@ object ViewMetadata extends RewriteHelper { tableData.properties.contains(MV_QUERY_ORIGINAL_SQL) } } catch { - // if db exists a table hive materialized view, will throw annalysis exception + // if db exists a table hive materialized view, will throw analysis exception case e: Throwable => logDebug(s"Failed to listTables in $mvDataBase, errmsg: ${e.getMessage}") Seq.empty[CatalogTable] diff --git a/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/AbstractMaterializedViewRule.scala b/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/AbstractMaterializedViewRule.scala index 3bf2c2f2b..81c979c6f 100644 --- a/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/AbstractMaterializedViewRule.scala +++ b/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/AbstractMaterializedViewRule.scala @@ -119,12 +119,12 @@ abstract class AbstractMaterializedViewRule(sparkSession: SparkSession) } // 4.5.extractPredictExpressions from viewQueryPlan and mappedQueryPlan - val queryPredictExpression = extractPredictExpressions(queryExpr, EMPTY_BITMAP) + val queryPredictExpression = extractPredictExpressions(queryExpr, EMPTY_BIMAP) val viewProjectList = extractTopProjectList(viewQueryExpr) val viewTableAttrs = viewTablePlan.output - // 4.6.if a table emps used >=2 times in s sql (query and view) + // 4.6.if a table emps used >=2 times in a sql (query and view) // we should try the combination,switch the seq // view:SELECT V1.locationid,V2.empname FROM emps V1 JOIN emps V2 // ON V1.deptno='1' AND V2.deptno='2' AND V1.empname = V2.empname; @@ -143,7 +143,7 @@ abstract class AbstractMaterializedViewRule(sparkSession: SparkSession) queryPredictExpression, viewPredictExpression, inverseTableMapping, viewPredictExpression._1.getEquivalenceClassesMap, viewProjectList, viewTableAttrs) - // 4.8.compensationPredicates isEmpty, because view's row data cannot satify query + // 4.8.compensationPredicates isEmpty, because view's row data cannot satisfy query if (newViewTablePlan.isEmpty) { mappingLoop.break() } @@ -168,13 +168,19 @@ abstract class AbstractMaterializedViewRule(sparkSession: SparkSession) } /** - * cehck plan if match current rule + * check plan if match current rule * * @param logicalPlan LogicalPlan * @return true:matched ; false:unMatched */ def isValidPlan(logicalPlan: LogicalPlan): Boolean + /** + * basic check for all rule + * + * @param logicalPlan LogicalPlan + * @return true:matched ; false:unMatched + */ def isValidLogicalPlan(logicalPlan: LogicalPlan): Boolean = { logicalPlan.foreach { case _: LogicalRelation => @@ -202,19 +208,19 @@ abstract class AbstractMaterializedViewRule(sparkSession: SparkSession) LogicalPlan, LogicalPlan)] = { // viewName, viewTablePlan, viewQueryPlan var viewPlans = Seq.empty[(String, LogicalPlan, LogicalPlan)] + val viewNames = mutable.Set.empty[String] // 1.topological iterate graph - ViewMetadata.usesGraphTopologicalOrderIterator.forEach { viewName => - // 2.check this node is mv - if (ViewMetadata.viewToTablePlan.containsKey(viewName) - // 3.iterate mv used tables and check edge of (table,mv) in graph - && usesTable(viewName, ViewMetadata - .viewToContainsTables.get(viewName), ViewMetadata.frozenGraph)) { - // 4.add plan info - val viewQueryPlan = ViewMetadata.viewToViewQueryPlan.get(viewName) - val viewTablePlan = ViewMetadata.viewToTablePlan.get(viewName) - viewPlans +:= (viewName, viewTablePlan, viewQueryPlan) + tableNames.foreach { tableName => + if (ViewMetadata.tableToViews.containsKey(tableName)) { + viewNames ++= ViewMetadata.tableToViews.get(tableName) } } + viewNames.foreach { viewName => + // 4.add plan info + val viewQueryPlan = ViewMetadata.viewToViewQueryPlan.get(viewName) + val viewTablePlan = ViewMetadata.viewToTablePlan.get(viewName) + viewPlans +:= (viewName, viewTablePlan, viewQueryPlan) + } viewPlans } @@ -242,7 +248,7 @@ abstract class AbstractMaterializedViewRule(sparkSession: SparkSession) * @param viewTablePlan viewTablePlan * @param viewQueryPlan viewQueryPlan * @param topViewProject topViewProject - * @param needTables needTables + * @param needTables need join compensate tables * @return join compensated viewTablePlan */ def compensateViewPartial(viewTablePlan: LogicalPlan, @@ -251,6 +257,14 @@ abstract class AbstractMaterializedViewRule(sparkSession: SparkSession) needTables: Set[TableEqual]): Option[(LogicalPlan, LogicalPlan, Option[Project])] = None + /** + * We map every table in the query to a table with the same qualified + * name (all query tables are contained in the view, thus this is equivalent + * to mapping every table in the query to a view table). + * + * @param queryTables queryTables + * @return + */ def generateTableMappings(queryTables: Set[TableEqual]): Seq[BiMap[String, String]] = { val multiMapTables: Multimap[String, String] = ArrayListMultimap.create() for (t1 <- queryTables) { @@ -272,7 +286,7 @@ abstract class AbstractMaterializedViewRule(sparkSession: SparkSession) } // continue } else { - // Multiple reference: flatten + // Multiple references: flatten val newResult: ImmutableList.Builder[BiMap[String, String]] = ImmutableList.builder() t2s.forEach { target => result.forEach { m => @@ -350,10 +364,17 @@ abstract class AbstractMaterializedViewRule(sparkSession: SparkSession) Some(compensationPredicate) } + /** + * extractPossibleMapping {queryEquivalenceClasses:[contained viewEquivalenceClasses]} + * + * @param queryEquivalenceClasses queryEquivalenceClasses + * @param viewEquivalenceClasses viewEquivalenceClasses + * @return {queryEquivalenceClasses:[contained viewEquivalenceClasses]} + */ def extractPossibleMapping(queryEquivalenceClasses: List[mutable.Set[ExpressionEqual]], viewEquivalenceClasses: List[mutable.Set[ExpressionEqual]]): Option[Multimap[Int, Int]] = { // extractPossibleMapping {queryEquivalenceClasses:[contained viewEquivalenceClasses]} - // query:c1=c2=c3=c4 view:c1=c2 ,c3=c4 + // query:c1=c2=c3=c4 view:c1=c2 , c3=c4 val mapping = ArrayListMultimap.create[Int, Int]() val breakLoop = new Breaks @@ -385,6 +406,12 @@ abstract class AbstractMaterializedViewRule(sparkSession: SparkSession) Some(mapping) } + /** + * + * @param queryExpression queryExpression + * @param viewExpression viewExpression + * @return compensate Expression + */ def splitFilter(queryExpression: Expression, viewExpression: Expression): Option[Expression] = { // 1.canonicalize expression,main for reorder val queryExpression2 = RewriteHelper.canonicalize(ExprSimplifier.simplify(queryExpression)) @@ -438,7 +465,7 @@ abstract class AbstractMaterializedViewRule(sparkSession: SparkSession) val queries = ExprOptUtil.disjunctions(queryExpression) val views = ExprOptUtil.disjunctions(viewExpression) - // 1.compare difference which queries residue + // 1.compute difference which queries residue val difference = queries.map(ExpressionEqual) -- views.map(ExpressionEqual) // 2.1.queries equal to views,just return true @@ -537,9 +564,8 @@ abstract class AbstractMaterializedViewRule(sparkSession: SparkSession) true } - /** - * compute compensationPredicts between viewQueryPlan and mappedQueryPlan + * compute compensationPredicates between viewQueryPlan and mappedQueryPlan * * @param viewTablePlan viewTablePlan * @param queryPredict queryPredict @@ -619,10 +645,10 @@ abstract class AbstractMaterializedViewRule(sparkSession: SparkSession) * @param swapTableColumn true:swapTableColumn;false:swapColumnTable * @param tableMapping tableMapping * @param columnMapping columnMapping - * @param viewProjectList viewProjectList + * @param viewProjectList viewProjectList/viewAggExpression * @param viewTableAttrs viewTableAttrs * @tparam T T <: Iterable[Expression] - * @return rewriteExprs + * @return rewritedExprs */ def rewriteExpressions[T <: Iterable[Expression]]( exprsToRewrite: T, swapTableColumn: Boolean, @@ -657,11 +683,11 @@ abstract class AbstractMaterializedViewRule(sparkSession: SparkSession) }.asInstanceOf[T] // 4.iterate result and dfs check every AttributeReference in ViewTableAttributeReference - val viewTableAttrSet = swapTableAttrs.map(_.exprId).toSet + val viewTableAttrsSet = swapTableAttrs.map(_.exprId).toSet result.foreach { expr => expr.foreach { case attr: AttributeReference => - if (!viewTableAttrSet.contains(attr.exprId)) { + if (!viewTableAttrsSet.contains(attr.exprId)) { logDebug(s"attr:%s cannot found in view:%s" .format(attr, OmniCachePluginConfig.getConf.curMatchMV)) return None @@ -672,7 +698,6 @@ abstract class AbstractMaterializedViewRule(sparkSession: SparkSession) Some(result) } - /** * if the rewrite expression exprId != origin expression exprId, * replace by Alias(rewrite expression,origin.name)(exprId=origin.exprId) @@ -698,6 +723,16 @@ abstract class AbstractMaterializedViewRule(sparkSession: SparkSession) /** * replace and alias expression or attr by viewTableAttr + * + * @param exprsToRewrite exprsToRewrite + * @param swapTableColumn true:swapTableColumn;false:swapColumnTable + * @param tableMapping tableMapping + * @param columnMapping columnMapping + * @param viewProjectList viewProjectList/viewAggExpression + * @param viewTableAttrs viewTableAttrs + * @param originExpressions originExpressions + * @tparam T T <: Iterable[Expression] + * @return rewrited and alias expression */ def rewriteAndAliasExpressions[T <: Iterable[Expression]]( exprsToRewrite: T, swapTableColumn: Boolean, diff --git a/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/MVRewriteRule.scala b/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/MVRewriteRule.scala index 888d5af02..1cb701fdc 100644 --- a/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/MVRewriteRule.scala +++ b/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/MVRewriteRule.scala @@ -27,14 +27,14 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.internal.Logging import org.apache.spark.scheduler.{SparkListener, SparkListenerEvent} import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LogicalPlan, Project} +import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.command.OmniCacheCreateMvCommand import org.apache.spark.status.ElementTrackingStore import org.apache.spark.util.kvstore.KVIndex class MVRewriteRule(session: SparkSession) extends Rule[LogicalPlan] with Logging { - val omniCacheConf: OmniCachePluginConfig = OmniCachePluginConfig.getConf + val omniCacheConf: OmniCachePluginConfig = OmniCachePluginConfig.getSessionConf val joinRule = new MaterializedViewJoinRule(session) val aggregateRule = new MaterializedViewAggregateRule(session) @@ -59,6 +59,7 @@ class MVRewriteRule(session: SparkSession) extends Rule[LogicalPlan] with Loggin def tryRewritePlan(plan: LogicalPlan): LogicalPlan = { val usingMvs = mutable.Set.empty[String] + val rewriteStartSecond = System.currentTimeMillis() val res = plan.transformDown { case p: Project => joinRule.perform(Some(p), p.child, usingMvs) @@ -70,17 +71,24 @@ class MVRewriteRule(session: SparkSession) extends Rule[LogicalPlan] with Loggin RewriteHelper.extractAllAttrsFromExpression(a.aggregateExpressions).toSeq, a.child) val rewritedChild = joinRule.perform(Some(child), child.child, usingMvs) if (rewritedChild != child) { - rewritedPlan = a.copy(child = rewritedChild) + val projectChild = rewritedChild.asInstanceOf[Project] + rewritedPlan = a.copy(child = Project( + projectChild.projectList ++ projectChild.child.output, projectChild.child)) } } rewritedPlan case p => p } if (usingMvs.nonEmpty) { + if (!RewriteHelper.checkAttrsValid(res)) { + return plan + } val sql = session.sparkContext.getLocalProperty(SparkContext.SPARK_JOB_DESCRIPTION) val mvs = usingMvs.mkString(";").replaceAll("`", "") - val log = "logicalPlan MVRewrite success,using materialized view:[%s],original sql:%s" - .format(mvs, sql) + val costSecond = (System.currentTimeMillis() - rewriteStartSecond).toString + val log = ("logicalPlan MVRewrite success," + + "using materialized view:[%s],cost %s milliseconds,original sql:%s") + .format(mvs, costSecond, sql) logDebug(log) session.sparkContext.listenerBus.post(SparkListenerMVRewriteSuccess(sql, mvs)) } diff --git a/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/MaterializedViewAggregateRule.scala b/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/MaterializedViewAggregateRule.scala index fe82c4744..6dda86891 100644 --- a/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/MaterializedViewAggregateRule.scala +++ b/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/MaterializedViewAggregateRule.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ class MaterializedViewAggregateRule(sparkSession: SparkSession) extends AbstractMaterializedViewRule(sparkSession: SparkSession) { /** - * cehck plan if match current rule + * check plan if match current rule * * @param logicalPlan LogicalPlan * @return true:matched ; false:unMatched @@ -43,7 +43,6 @@ class MaterializedViewAggregateRule(sparkSession: SparkSession) logicalPlan.children.forall(isValidLogicalPlan) } - /** * queryTableInfo!=viewTableInfo , need do join compensate * @@ -208,7 +207,6 @@ class MaterializedViewAggregateRule(sparkSession: SparkSession) // 4.rewrite and alias queryAggExpressions // if the rewrite expression exprId != origin expression exprId, // replace by Alias(rewrite expression,origin.name)(exprId=origin.exprId) - val rewritedQueryAggExpressions = rewriteAndAliasExpressions(newQueryAggExpressions, swapTableColumn = true, tableMapping, columnMapping, viewProjectList, viewTableAttrs, queryAgg.aggregateExpressions) @@ -240,5 +238,4 @@ class MaterializedViewAggregateRule(sparkSession: SparkSession) qualifier = qualifier, explicitMetadata = alias.explicitMetadata, nonInheritableMetadataKeys = alias.nonInheritableMetadataKeys) } - } diff --git a/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/MaterializedViewJoinRule.scala b/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/MaterializedViewJoinRule.scala index 99248e01c..474972c3b 100644 --- a/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/MaterializedViewJoinRule.scala +++ b/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/MaterializedViewJoinRule.scala @@ -22,14 +22,14 @@ import com.huawei.boostkit.spark.util.{ExpressionEqual, TableEqual} import scala.collection.mutable import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, NamedExpression} +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.Inner -import org.apache.spark.sql.catalyst.plans.logical.{Join, JoinHint, LogicalPlan, Project} +import org.apache.spark.sql.catalyst.plans.logical._ class MaterializedViewJoinRule(sparkSession: SparkSession) extends AbstractMaterializedViewRule(sparkSession: SparkSession) { /** - * cehck plan if match current rule + * check plan if match current rule * * @param logicalPlan LogicalPlan * @return true:matched ; false:unMatched diff --git a/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/parser/OmniCacheExtensionAstBuilder.scala b/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/parser/OmniCacheExtensionAstBuilder.scala index a41680244..414afa602 100644 --- a/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/parser/OmniCacheExtensionAstBuilder.scala +++ b/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/parser/OmniCacheExtensionAstBuilder.scala @@ -35,6 +35,11 @@ import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.execution.datasources._ +/** + * Parse extended AST to LogicalPlan + * + * @param delegate Spark default ParserInterface + */ class OmniCacheExtensionAstBuilder(spark: SparkSession, delegate: ParserInterface) extends OmniCacheSqlExtensionsBaseVisitor[AnyRef] with SQLConfHelper with Logging { @@ -61,8 +66,7 @@ class OmniCacheExtensionAstBuilder(spark: SparkSession, delegate: ParserInterfac case Seq(mv) => (None, mv) case Seq(database, mv) => (Some(database), mv) case _ => throw new AnalysisException( - "The mv name is not valid: %s".format(identifier.mkString(".")) - ) + "The mv name is not valid: %s".format(identifier.mkString("."))) } try { @@ -102,8 +106,7 @@ class OmniCacheExtensionAstBuilder(spark: SparkSession, delegate: ParserInterfac case Seq(mv) => (None, mv) case Seq(database, mv) => (Some(database), mv) case _ => throw new AnalysisException( - "The mv name is not valid: %s".format(tableIdent.mkString(".")) - ) + "The mv name is not valid: %s".format(tableIdent.mkString("."))) } val tableIdentifier = TableIdentifier(name, databaseName) @@ -119,7 +122,7 @@ class OmniCacheExtensionAstBuilder(spark: SparkSession, delegate: ParserInterfac throw new RuntimeException("cannot refresh a table with refresh mv") } - // preserver preDatabase and set curDatabase + // preserve preDatabase and set curDatabase val preDatabase = spark.catalog.currentDatabase val curDatabase = catalogTable.properties.getOrElse(MV_QUERY_ORIGINAL_SQL_CUR_DB, "") if (curDatabase.isEmpty) { @@ -179,6 +182,7 @@ class OmniCacheExtensionAstBuilder(spark: SparkSession, delegate: ParserInterfac RewriteHelper.enableCachePlugin() throw e } finally { + // reset preDatabase spark.sessionState.catalogManager.setCurrentNamespace(Array(preDatabase)) } } @@ -221,8 +225,7 @@ class OmniCacheExtensionAstBuilder(spark: SparkSession, delegate: ParserInterfac case Seq(database, mv) => DropMaterializedViewCommand( TableIdentifier(mv, Some(database)), ifExists.isDefined, - purge = true - ) + purge = true) case _ => throw new AnalysisException( "The mv name is not valid: %s".format(identifier.mkString("."))) } @@ -297,6 +300,9 @@ class OmniCacheExtensionAstBuilder(spark: SparkSession, delegate: ParserInterfac visit(ctx.statement).asInstanceOf[LogicalPlan] } + /** + * alias tuple2 + */ type OmniCacheHeader = (Seq[String], Boolean) /** diff --git a/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/parser/OmniCacheExtensionSqlParser.scala b/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/parser/OmniCacheExtensionSqlParser.scala index 65077216a..aeab84232 100644 --- a/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/parser/OmniCacheExtensionSqlParser.scala +++ b/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/parser/OmniCacheExtensionSqlParser.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.parser import java.util.Locale -import org.antlr.v4.runtime.{CharStreams, CommonTokenStream} +import org.antlr.v4.runtime._ import org.antlr.v4.runtime.atn.PredictionMode import org.antlr.v4.runtime.misc.ParseCancellationException diff --git a/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/execution/command/OmniCacheCommand.scala b/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/execution/command/OmniCacheCommand.scala index 801fffbd2..c053b6252 100644 --- a/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/execution/command/OmniCacheCommand.scala +++ b/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/execution/command/OmniCacheCommand.scala @@ -54,6 +54,7 @@ case class OmniCacheCreateMvCommand( partitioning: Seq[String], query: LogicalPlan, outputColumnNames: Seq[String]) extends DataWritingCommand { + override def run(sparkSession: SparkSession, child: SparkPlan): Seq[Row] = { try { ViewMetadata.init(sparkSession) @@ -84,7 +85,7 @@ case class OmniCacheCreateMvCommand( throw new AnalysisException( s"Materialized View $tableIdentWithDB already exists. You need to drop it first") } else { - // Since the table already exists and the save mode is Ignore,we will just return. + // Since the table already exists and the save mode is Ignore, we will just return. return Seq.empty } } else { @@ -102,10 +103,11 @@ case class OmniCacheCreateMvCommand( storage = table.storage.copy(locationUri = tableLocation), // We will use the schema of resolved.relation as the schema of the table (instead of // the schema of df). It is important since the nullability may be changed by the relation - // provider (for example,see org.apache.spark.sql.parquet.DefaultSource). + // provider (for example, see org.apache.spark.sql.parquet.DefaultSource). schema = tableSchema) // Table location is already validated. No need to check it again during table creation. sessionState.catalog.createTable(newTable, ignoreIfExists = false, validateLocation = false) + result match { case _: HadoopFsRelation if table.partitionColumnNames.nonEmpty && sparkSession.sqlContext.conf.manageFilesourcePartitions => @@ -153,10 +155,19 @@ case class OmniCacheCreateMvCommand( } } +/** + * Drops a materialized view from the metastore and removes it if it is cached. + * + * The syntax of this command is: + * {{{ + * DROP MATERIALIZED VIEW [IF EXISTS] view_name; + * }}} + */ case class DropMaterializedViewCommand( tableName: TableIdentifier, ifExists: Boolean, purge: Boolean) extends RunnableCommand { + override def run(sparkSession: SparkSession): Seq[Row] = { ViewMetadata.init(sparkSession) val catalog = sparkSession.sessionState.catalog @@ -201,13 +212,18 @@ case class DropMaterializedViewCommand( } } +/** + * ShowMaterializedViewCommand RunnableCommand + * + */ case class ShowMaterializedViewCommand( databaseName: Option[String], tableIdentifierPattern: Option[String]) extends RunnableCommand { + // The result of SHOW MaterializedView has three basic columns: + // database, tableName and originalSql. override val output: Seq[Attribute] = { val tableExtendedInfo = Nil - AttributeReference("database", StringType, nullable = false)() :: AttributeReference("mvName", StringType, nullable = false)() :: AttributeReference("rewriteEnable", StringType, nullable = false)() :: @@ -254,6 +270,7 @@ case class ShowMaterializedViewCommand( case class AlterRewriteMaterializedViewCommand( tableName: TableIdentifier, enableRewrite: Boolean) extends RunnableCommand { + override def run(sparkSession: SparkSession): Seq[Row] = { ViewMetadata.init(sparkSession) val catalog = sparkSession.sessionState.catalog @@ -379,7 +396,6 @@ case class RefreshMaterializedViewCommand( } if (doInsertion) { - def refreshUpdatedPartitions(updatedPartitionPaths: Set[String]): Unit = { val updatedPartitions = updatedPartitionPaths.map(PartitioningUtils.parsePathFragment) if (partitionsTrackedByCatalog) { @@ -446,7 +462,6 @@ case class RefreshMaterializedViewCommand( if (catalogTable.nonEmpty) { CommandUtils.updateTableStats(sparkSession, catalogTable.get) } - } else { logInfo("Skipping insertion into a relation that already exists.") } diff --git a/omnicache/omnicache-spark-extension/pom.xml b/omnicache/omnicache-spark-extension/pom.xml index 1e068b0af..402e83aff 100644 --- a/omnicache/omnicache-spark-extension/pom.xml +++ b/omnicache/omnicache-spark-extension/pom.xml @@ -34,6 +34,7 @@ 3.1.2 1.4.11 8.29 + -- Gitee From 5fa0c701a2735570ab242a79899e39e805076d2e Mon Sep 17 00:00:00 2001 From: Xteen Date: Thu, 15 Sep 2022 20:41:46 +0800 Subject: [PATCH 3/7] add RewriteTime stat --- .../boostkit/spark/util/ExprSimplifier.scala | 2 +- .../boostkit/spark/util/ViewMetadata.scala | 42 +- .../rules/AbstractMaterializedViewRule.scala | 42 +- .../optimizer/rules/MVRewriteRule.scala | 30 +- .../rules/MaterializedViewJoinRule.scala | 1 - .../plugin/src/test/resources/tpcds_ddl.sql | 479 ++++++++++++++++++ .../catalyst/optimizer/rules/TpcdsSuite.scala | 95 ++++ 7 files changed, 661 insertions(+), 30 deletions(-) create mode 100644 omnicache/omnicache-spark-extension/plugin/src/test/resources/tpcds_ddl.sql create mode 100644 omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/optimizer/rules/TpcdsSuite.scala diff --git a/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/util/ExprSimplifier.scala b/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/util/ExprSimplifier.scala index ae0040823..5cb7d1925 100644 --- a/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/util/ExprSimplifier.scala +++ b/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/util/ExprSimplifier.scala @@ -482,7 +482,7 @@ case class ExprSimplifier(unknownAsFalse: Boolean, breaks3.break() } } else if (containsSql(ors, term)) { - // Simplification between a otherExpression and a orExpression. + // Simplification between a otherExpression and a OrExpression. terms.-=(orOp) breaks3.break() } diff --git a/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/util/ViewMetadata.scala b/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/util/ViewMetadata.scala index 07534fa38..a3ab16e76 100644 --- a/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/util/ViewMetadata.scala +++ b/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/util/ViewMetadata.scala @@ -24,7 +24,8 @@ import scala.collection.mutable import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.{CatalogTable, SessionCatalog} -import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias} +import org.apache.spark.sql.catalyst.optimizer.rules.RewriteTime +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, RepartitionByExpression, SubqueryAlias} object ViewMetadata extends RewriteHelper { @@ -73,11 +74,23 @@ object ViewMetadata extends RewriteHelper { // db.table val tableName = catalogTable.identifier.quotedString - val viewTablePlan = spark.table(tableName).queryExecution.analyzed match { - case SubqueryAlias(_, child) => child - case a@_ => a + val viewTablePlan = RewriteTime + .withTimeStat("viewTablePlan") { + spark.table(tableName).queryExecution.analyzed match { + case SubqueryAlias(_, child) => child + case a@_ => a + } + } + var viewQueryPlan = RewriteTime + .withTimeStat("viewQueryPlan") { + spark.sql(viewQuerySql).queryExecution.analyzed + } + viewQueryPlan = viewQueryPlan match { + case RepartitionByExpression(_, child, _) => + child + case _ => + viewQueryPlan } - val viewQueryPlan = spark.sql(viewQuerySql).queryExecution.analyzed // reset preDatabase spark.sessionState.catalogManager.setCurrentNamespace(Array(preDatabase)) @@ -85,7 +98,10 @@ object ViewMetadata extends RewriteHelper { val viewName = catalogTable.identifier.toString() // mappedViewQueryPlan and mappedViewContainsTables - val (mappedViewQueryPlan, mappedViewContainsTables) = extractTables(viewQueryPlan) + val (mappedViewQueryPlan, mappedViewContainsTables) = RewriteTime + .withTimeStat("extractTables") { + extractTables(viewQueryPlan) + } mappedViewContainsTables .foreach { mappedViewContainsTable => @@ -99,7 +115,10 @@ object ViewMetadata extends RewriteHelper { // match function is attributeReferenceEqualSimple, by name and data type // Attr of table cannot used, because same Attr in view query and view table, // it's table is different. - val mappedViewTablePlan = mapTablePlanAttrToQuery(viewTablePlan, mappedViewQueryPlan) + val mappedViewTablePlan = RewriteTime + .withTimeStat("mapTablePlanAttrToQuery") { + mapTablePlanAttrToQuery(viewTablePlan, mappedViewQueryPlan) + } viewToContainsTables.put(viewName, mappedViewContainsTables) viewToViewQueryPlan.putIfAbsent(viewName, mappedViewQueryPlan) @@ -149,12 +168,15 @@ object ViewMetadata extends RewriteHelper { def forceLoad(): Unit = this.synchronized { val catalog = spark.sessionState.catalog - // val db = OmniCachePluginConfig.getConf.OmniCacheDB // load from all db for (db <- catalog.listDatabases()) { - val tables = omniCacheFilter(catalog, db) - tables.foreach(tableData => saveViewMetadataToMap(tableData)) + val tables = RewriteTime.withTimeStat("loadTable") { + omniCacheFilter(catalog, db) + } + RewriteTime.withTimeStat("saveViewMetadataToMap") { + tables.foreach(tableData => saveViewMetadataToMap(tableData)) + } } } diff --git a/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/AbstractMaterializedViewRule.scala b/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/AbstractMaterializedViewRule.scala index 81c979c6f..15ff2d76a 100644 --- a/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/AbstractMaterializedViewRule.scala +++ b/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/AbstractMaterializedViewRule.scala @@ -49,7 +49,9 @@ abstract class AbstractMaterializedViewRule(sparkSession: SparkSession) if (ViewMetadata.status == ViewMetadata.STATUS_LOADING) { return finalPlan } - ViewMetadata.init(sparkSession) + RewriteTime.withTimeStat("viewMetadata") { + ViewMetadata.init(sparkSession) + } // 1.check query sql is match current rule if (ViewMetadata.isEmpty || !isValidPlan(plan)) { return finalPlan @@ -60,8 +62,10 @@ abstract class AbstractMaterializedViewRule(sparkSession: SparkSession) val (queryExpr, queryTables) = extractTables(finalPlan) // 3.use all tables to fetch views(may match) from ViewMetaData - val candidateViewPlans = getApplicableMaterializations(queryTables.map(t => t.tableName)) - .filter(x => !OmniCachePluginConfig.isMVInUpdate(x._2)) + val candidateViewPlans = RewriteTime.withTimeStat("getApplicableMaterializations") { + getApplicableMaterializations(queryTables.map(t => t.tableName)) + .filter(x => !OmniCachePluginConfig.isMVInUpdate(x._2)) + } if (candidateViewPlans.isEmpty) { return finalPlan } @@ -119,7 +123,9 @@ abstract class AbstractMaterializedViewRule(sparkSession: SparkSession) } // 4.5.extractPredictExpressions from viewQueryPlan and mappedQueryPlan - val queryPredictExpression = extractPredictExpressions(queryExpr, EMPTY_BIMAP) + val queryPredictExpression = RewriteTime.withTimeStat("extractPredictExpressions") { + extractPredictExpressions(queryExpr, EMPTY_BIMAP) + } val viewProjectList = extractTopProjectList(viewQueryExpr) val viewTableAttrs = viewTablePlan.output @@ -135,14 +141,18 @@ abstract class AbstractMaterializedViewRule(sparkSession: SparkSession) flatListMappings.foreach { queryToViewTableMapping => mappingLoop.breakable { val inverseTableMapping = queryToViewTableMapping.inverse() - val viewPredictExpression = extractPredictExpressions(viewQueryExpr, - inverseTableMapping) + val viewPredictExpression = RewriteTime.withTimeStat("extractPredictExpressions") { + extractPredictExpressions(viewQueryExpr, + inverseTableMapping) + } // 4.7.compute compensationPredicates between viewQueryPlan and queryPlan - var newViewTablePlan = computeCompensationPredicates(viewTablePlan, - queryPredictExpression, viewPredictExpression, inverseTableMapping, - viewPredictExpression._1.getEquivalenceClassesMap, - viewProjectList, viewTableAttrs) + var newViewTablePlan = RewriteTime.withTimeStat("computeCompensationPredicates") { + computeCompensationPredicates(viewTablePlan, + queryPredictExpression, viewPredictExpression, inverseTableMapping, + viewPredictExpression._1.getEquivalenceClassesMap, + viewProjectList, viewTableAttrs) + } // 4.8.compensationPredicates isEmpty, because view's row data cannot satisfy query if (newViewTablePlan.isEmpty) { mappingLoop.break() @@ -150,10 +160,12 @@ abstract class AbstractMaterializedViewRule(sparkSession: SparkSession) // 4.9.use viewTablePlan(join compensated), query project, // compensationPredicts to rewrite final plan - newViewTablePlan = rewriteView(newViewTablePlan.get, viewQueryExpr, - queryExpr, inverseTableMapping, - queryPredictExpression._1.getEquivalenceClassesMap, - viewProjectList, viewTableAttrs) + newViewTablePlan = RewriteTime.withTimeStat("rewriteView") { + rewriteView(newViewTablePlan.get, viewQueryExpr, + queryExpr, inverseTableMapping, + queryPredictExpression._1.getEquivalenceClassesMap, + viewProjectList, viewTableAttrs) + } if (newViewTablePlan.isEmpty) { mappingLoop.break() } @@ -453,7 +465,6 @@ abstract class AbstractMaterializedViewRule(sparkSession: SparkSession) None } - /** * split expression by or,then compute compensation * @@ -769,5 +780,4 @@ abstract class AbstractMaterializedViewRule(sparkSession: SparkSession) columnMapping: Map[ExpressionEqual, mutable.Set[ExpressionEqual]], viewProjectList: Seq[Expression], viewTableAttrs: Seq[Attribute]): Option[LogicalPlan] - } diff --git a/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/MVRewriteRule.scala b/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/MVRewriteRule.scala index 1cb701fdc..c7047c173 100644 --- a/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/MVRewriteRule.scala +++ b/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/MVRewriteRule.scala @@ -59,6 +59,7 @@ class MVRewriteRule(session: SparkSession) extends Rule[LogicalPlan] with Loggin def tryRewritePlan(plan: LogicalPlan): LogicalPlan = { val usingMvs = mutable.Set.empty[String] + RewriteTime.clear() val rewriteStartSecond = System.currentTimeMillis() val res = plan.transformDown { case p: Project => @@ -80,8 +81,10 @@ class MVRewriteRule(session: SparkSession) extends Rule[LogicalPlan] with Loggin case p => p } if (usingMvs.nonEmpty) { - if (!RewriteHelper.checkAttrsValid(res)) { - return plan + RewriteTime.withTimeStat("checkAttrsValid") { + if (!RewriteHelper.checkAttrsValid(res)) { + return plan + } } val sql = session.sparkContext.getLocalProperty(SparkContext.SPARK_JOB_DESCRIPTION) val mvs = usingMvs.mkString(";").replaceAll("`", "") @@ -92,6 +95,8 @@ class MVRewriteRule(session: SparkSession) extends Rule[LogicalPlan] with Loggin logDebug(log) session.sparkContext.listenerBus.post(SparkListenerMVRewriteSuccess(sql, mvs)) } + RewriteTime.statFromStartTime("total", rewriteStartSecond) + logDebug(RewriteTime.timeStat.toString()) res } } @@ -114,3 +119,24 @@ class MVRewriteSuccessListener( } } } + +object RewriteTime { + val timeStat: mutable.Map[String, Long] = mutable.HashMap[String, Long]() + + def statFromStartTime(key: String, startTime: Long): Unit = { + timeStat += (key -> (timeStat.getOrElse(key, 0L) + System.currentTimeMillis() - startTime)) + } + + def clear(): Unit = { + timeStat.clear() + } + + def withTimeStat[T](key: String)(f: => T): T = { + val startTime = System.currentTimeMillis() + try { + f + } finally { + statFromStartTime(key, startTime) + } + } +} \ No newline at end of file diff --git a/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/MaterializedViewJoinRule.scala b/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/MaterializedViewJoinRule.scala index 474972c3b..5c7c477dd 100644 --- a/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/MaterializedViewJoinRule.scala +++ b/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/MaterializedViewJoinRule.scala @@ -38,7 +38,6 @@ class MaterializedViewJoinRule(sparkSession: SparkSession) isValidLogicalPlan(logicalPlan) } - /** * queryTableInfo!=viewTableInfo , need do join compensate * diff --git a/omnicache/omnicache-spark-extension/plugin/src/test/resources/tpcds_ddl.sql b/omnicache/omnicache-spark-extension/plugin/src/test/resources/tpcds_ddl.sql new file mode 100644 index 000000000..ae18bd4b0 --- /dev/null +++ b/omnicache/omnicache-spark-extension/plugin/src/test/resources/tpcds_ddl.sql @@ -0,0 +1,479 @@ +create table if not exists store_sales( + ss_sold_date_sk bigint +, ss_sold_time_sk bigint +, ss_item_sk bigint +, ss_customer_sk bigint +, ss_cdemo_sk bigint +, ss_hdemo_sk bigint +, ss_addr_sk bigint +, ss_store_sk bigint +, ss_promo_sk bigint +, ss_ticket_number bigint +, ss_quantity int +, ss_wholesale_cost decimal(7,2) +, ss_list_price decimal(7,2) +, ss_sales_price decimal(7,2) +, ss_ext_discount_amt decimal(7,2) +, ss_ext_sales_price decimal(7,2) +, ss_ext_wholesale_cost decimal(7,2) +, ss_ext_list_price decimal(7,2) +, ss_ext_tax decimal(7,2) +, ss_coupon_amt decimal(7,2) +, ss_net_paid decimal(7,2) +, ss_net_paid_inc_tax decimal(7,2) +, ss_net_profit decimal(7,2) +); + +create table if not exists store_returns( + sr_returned_date_sk bigint +, sr_return_time_sk bigint +, sr_item_sk bigint +, sr_customer_sk bigint +, sr_cdemo_sk bigint +, sr_hdemo_sk bigint +, sr_addr_sk bigint +, sr_store_sk bigint +, sr_reason_sk bigint +, sr_ticket_number bigint +, sr_return_quantity int +, sr_return_amt decimal(7,2) +, sr_return_tax decimal(7,2) +, sr_return_amt_inc_tax decimal(7,2) +, sr_fee decimal(7,2) +, sr_return_ship_cost decimal(7,2) +, sr_refunded_cash decimal(7,2) +, sr_reversed_charge decimal(7,2) +, sr_store_credit decimal(7,2) +, sr_net_loss decimal(7,2) +); + +create table if not exists catalog_sales( + cs_sold_date_sk bigint +, cs_sold_time_sk bigint +, cs_ship_date_sk bigint +, cs_bill_customer_sk bigint +, cs_bill_cdemo_sk bigint +, cs_bill_hdemo_sk bigint +, cs_bill_addr_sk bigint +, cs_ship_customer_sk bigint +, cs_ship_cdemo_sk bigint +, cs_ship_hdemo_sk bigint +, cs_ship_addr_sk bigint +, cs_call_center_sk bigint +, cs_catalog_page_sk bigint +, cs_ship_mode_sk bigint +, cs_warehouse_sk bigint +, cs_item_sk bigint +, cs_promo_sk bigint +, cs_order_number bigint +, cs_quantity int +, cs_wholesale_cost decimal(7,2) +, cs_list_price decimal(7,2) +, cs_sales_price decimal(7,2) +, cs_ext_discount_amt decimal(7,2) +, cs_ext_sales_price decimal(7,2) +, cs_ext_wholesale_cost decimal(7,2) +, cs_ext_list_price decimal(7,2) +, cs_ext_tax decimal(7,2) +, cs_coupon_amt decimal(7,2) +, cs_ext_ship_cost decimal(7,2) +, cs_net_paid decimal(7,2) +, cs_net_paid_inc_tax decimal(7,2) +, cs_net_paid_inc_ship decimal(7,2) +, cs_net_paid_inc_ship_tax decimal(7,2) +, cs_net_profit decimal(7,2) +); + +create table if not exists catalog_returns( + cr_returned_date_sk bigint +, cr_returned_time_sk bigint +, cr_item_sk bigint +, cr_refunded_customer_sk bigint +, cr_refunded_cdemo_sk bigint +, cr_refunded_hdemo_sk bigint +, cr_refunded_addr_sk bigint +, cr_returning_customer_sk bigint +, cr_returning_cdemo_sk bigint +, cr_returning_hdemo_sk bigint +, cr_returning_addr_sk bigint +, cr_call_center_sk bigint +, cr_catalog_page_sk bigint +, cr_ship_mode_sk bigint +, cr_warehouse_sk bigint +, cr_reason_sk bigint +, cr_order_number bigint +, cr_return_quantity int +, cr_return_amount decimal(7,2) +, cr_return_tax decimal(7,2) +, cr_return_amt_inc_tax decimal(7,2) +, cr_fee decimal(7,2) +, cr_return_ship_cost decimal(7,2) +, cr_refunded_cash decimal(7,2) +, cr_reversed_charge decimal(7,2) +, cr_store_credit decimal(7,2) +, cr_net_loss decimal(7,2) +); + +create table if not exists web_sales( + ws_sold_date_sk bigint +, ws_sold_time_sk bigint +, ws_ship_date_sk bigint +, ws_item_sk bigint +, ws_bill_customer_sk bigint +, ws_bill_cdemo_sk bigint +, ws_bill_hdemo_sk bigint +, ws_bill_addr_sk bigint +, ws_ship_customer_sk bigint +, ws_ship_cdemo_sk bigint +, ws_ship_hdemo_sk bigint +, ws_ship_addr_sk bigint +, ws_web_page_sk bigint +, ws_web_site_sk bigint +, ws_ship_mode_sk bigint +, ws_warehouse_sk bigint +, ws_promo_sk bigint +, ws_order_number bigint +, ws_quantity int +, ws_wholesale_cost decimal(7,2) +, ws_list_price decimal(7,2) +, ws_sales_price decimal(7,2) +, ws_ext_discount_amt decimal(7,2) +, ws_ext_sales_price decimal(7,2) +, ws_ext_wholesale_cost decimal(7,2) +, ws_ext_list_price decimal(7,2) +, ws_ext_tax decimal(7,2) +, ws_coupon_amt decimal(7,2) +, ws_ext_ship_cost decimal(7,2) +, ws_net_paid decimal(7,2) +, ws_net_paid_inc_tax decimal(7,2) +, ws_net_paid_inc_ship decimal(7,2) +, ws_net_paid_inc_ship_tax decimal(7,2) +, ws_net_profit decimal(7,2) +); + +create table if not exists web_returns( + wr_returned_date_sk bigint +, wr_returned_time_sk bigint +, wr_item_sk bigint +, wr_refunded_customer_sk bigint +, wr_refunded_cdemo_sk bigint +, wr_refunded_hdemo_sk bigint +, wr_refunded_addr_sk bigint +, wr_returning_customer_sk bigint +, wr_returning_cdemo_sk bigint +, wr_returning_hdemo_sk bigint +, wr_returning_addr_sk bigint +, wr_web_page_sk bigint +, wr_reason_sk bigint +, wr_order_number bigint +, wr_return_quantity int +, wr_return_amt decimal(7,2) +, wr_return_tax decimal(7,2) +, wr_return_amt_inc_tax decimal(7,2) +, wr_fee decimal(7,2) +, wr_return_ship_cost decimal(7,2) +, wr_refunded_cash decimal(7,2) +, wr_reversed_charge decimal(7,2) +, wr_account_credit decimal(7,2) +, wr_net_loss decimal(7,2) +); + +create table if not exists inventory( + inv_date_sk bigint +, inv_item_sk bigint +, inv_warehouse_sk bigint +, inv_quantity_on_hand int +); +create table if not exists store( + s_store_sk bigint +, s_store_id char(16) +, s_rec_start_date date +, s_rec_end_date date +, s_closed_date_sk bigint +, s_store_name varchar(50) +, s_number_employees int +, s_floor_space int +, s_hours char(20) +, S_manager varchar(40) +, S_market_id int +, S_geography_class varchar(100) +, S_market_desc varchar(100) +, s_market_manager varchar(40) +, s_division_id int +, s_division_name varchar(50) +, s_company_id int +, s_company_name varchar(50) +, s_street_number varchar(10) +, s_street_name varchar(60) +, s_street_type char(15) +, s_suite_number char(10) +, s_city varchar(60) +, s_county varchar(30) +, s_state char(2) +, s_zip char(10) +, s_country varchar(20) +, s_gmt_offset decimal(5,2) +, s_tax_percentage decimal(5,2) +); +create table if not exists call_center( + cc_call_center_sk bigint +, cc_call_center_id char(16) +, cc_rec_start_date date +, cc_rec_end_date date +, cc_closed_date_sk bigint +, cc_open_date_sk bigint +, cc_name varchar(50) +, cc_class varchar(50) +, cc_employees int +, cc_sq_ft int +, cc_hours char(20) +, cc_manager varchar(40) +, cc_mkt_id int +, cc_mkt_class char(50) +, cc_mkt_desc varchar(100) +, cc_market_manager varchar(40) +, cc_division int +, cc_division_name varchar(50) +, cc_company int +, cc_company_name char(50) +, cc_street_number char(10) +, cc_street_name varchar(60) +, cc_street_type char(15) +, cc_suite_number char(10) +, cc_city varchar(60) +, cc_county varchar(30) +, cc_state char(2) +, cc_zip char(10) +, cc_country varchar(20) +, cc_gmt_offset decimal(5,2) +, cc_tax_percentage decimal(5,2) +); +create table if not exists catalog_page( + cp_catalog_page_sk bigint +, cp_catalog_page_id char(16) +, cp_start_date_sk bigint +, cp_end_date_sk bigint +, cp_department varchar(50) +, cp_catalog_number int +, cp_catalog_page_number int +, cp_description varchar(100) +, cp_type varchar(100) +); +create table if not exists web_site( + web_site_sk bigint +, web_site_id char(16) +, web_rec_start_date date +, web_rec_end_date date +, web_name varchar(50) +, web_open_date_sk bigint +, web_close_date_sk bigint +, web_class varchar(50) +, web_manager varchar(40) +, web_mkt_id int +, web_mkt_class varchar(50) +, web_mkt_desc varchar(100) +, web_market_manager varchar(40) +, web_company_id int +, web_company_name char(50) +, web_street_number char(10) +, web_street_name varchar(60) +, web_street_type char(15) +, web_suite_number char(10) +, web_city varchar(60) +, web_county varchar(30) +, web_state char(2) +, web_zip char(10) +, web_country varchar(20) +, web_gmt_offset decimal(5,2) +, web_tax_percentage decimal(5,2) +); +create table if not exists web_page( + wp_web_page_sk bigint +, wp_web_page_id char(16) +, wp_rec_start_date date +, wp_rec_end_date date +, wp_creation_date_sk bigint +, wp_access_date_sk bigint +, wp_autogen_flag char(1) +, wp_customer_sk bigint +, wp_url varchar(100) +, wp_type char(50) +, wp_char_count int +, wp_link_count int +, wp_image_count int +, wp_max_ad_count int +); +create table if not exists warehouse( + w_warehouse_sk bigint +, w_warehouse_id char(16) +, w_warehouse_name varchar(20) +, w_warehouse_sq_ft int +, w_street_number char(10) +, w_street_name varchar(60) +, w_street_type char(15) +, w_suite_number char(10) +, w_city varchar(60) +, w_county varchar(30) +, w_state char(2) +, w_zip char(10) +, w_country varchar(20) +, w_gmt_offset decimal(5,2) +); +create table if not exists customer( + c_customer_sk bigint +, c_customer_id char(16) +, c_current_cdemo_sk bigint +, c_current_hdemo_sk bigint +, c_current_addr_sk bigint +, c_first_shipto_date_sk bigint +, c_first_sales_date_sk bigint +, c_salutation char(10) +, c_first_name char(20) +, c_last_name char(30) +, c_preferred_cust_flag char(1) +, c_birth_day int +, c_birth_month int +, c_birth_year int +, c_birth_country varchar(20) +, c_login char(13) +, c_email_address char(50) +, c_last_review_date_sk bigint +); +create table if not exists customer_address( + ca_address_sk bigint +, ca_address_id char(16) +, ca_street_number char(10) +, ca_street_name varchar(60) +, ca_street_type char(15) +, ca_suite_number char(10) +, ca_city varchar(60) +, ca_county varchar(30) +, ca_state char(2) +, ca_zip char(10) +, ca_country varchar(20) +, ca_gmt_offset decimal(5,2) +, ca_location_type char(20) +); +create table if not exists customer_demographics( + cd_demo_sk bigint +, cd_gender char(1) +, cd_marital_status char(1) +, cd_education_status char(20) +, cd_purchase_estimate int +, cd_credit_rating char(10) +, cd_dep_count int +, cd_dep_employed_count int +, cd_dep_college_count int +); +create table if not exists date_dim( + d_date_sk bigint +, d_date_id char(16) +, d_date date +, d_month_seq int +, d_week_seq int +, d_quarter_seq int +, d_year int +, d_dow int +, d_moy int +, d_dom int +, d_qoy int +, d_fy_year int +, d_fy_quarter_seq int +, d_fy_week_seq int +, d_day_name char(9) +, d_quarter_name char(6) +, d_holiday char(1) +, d_weekend char(1) +, d_following_holiday char(1) +, d_first_dom int +, d_last_dom int +, d_same_day_ly int +, d_same_day_lq int +, d_current_day char(1) +, d_current_week char(1) +, d_current_month char(1) +, d_current_quarter char(1) +, d_current_year char(1) +); +create table if not exists household_demographics( + hd_demo_sk bigint +, hd_income_band_sk bigint +, hd_buy_potential char(15) +, hd_dep_count int +, hd_vehicle_count int +); +create table if not exists item( + i_item_sk bigint +, i_item_id char(16) +, i_rec_start_date date +, i_rec_end_date date +, i_item_desc varchar(200) +, i_current_price decimal(7,2) +, i_wholesale_cost decimal(7,2) +, i_brand_id int +, i_brand char(50) +, i_class_id int +, i_class char(50) +, i_category_id int +, i_category char(50) +, i_manufact_id int +, i_manufact char(50) +, i_size char(20) +, i_formulation char(20) +, i_color char(20) +, i_units char(10) +, i_container char(10) +, i_manager_id int +, i_product_name char(50) +); +create table if not exists income_band( + ib_income_band_sk bigint +, ib_lower_bound int +, ib_upper_bound int +); +create table if not exists promotion( + p_promo_sk bigint +, p_promo_id char(16) +, p_start_date_sk bigint +, p_end_date_sk bigint +, p_item_sk bigint +, p_cost decimal(15,2) +, p_response_target int +, p_promo_name char(50) +, p_channel_dmail char(1) +, p_channel_email char(1) +, p_channel_catalog char(1) +, p_channel_tv char(1) +, p_channel_radio char(1) +, p_channel_press char(1) +, p_channel_event char(1) +, p_channel_demo char(1) +, p_channel_details varchar(100) +, p_purpose char(15) +, p_discount_active char(1) +); +create table if not exists reason( + r_reason_sk bigint +, r_reason_id char(16) +, r_reason_desc char(100) +); +create table if not exists ship_mode( + sm_ship_mode_sk bigint +, sm_ship_mode_id char(16) +, sm_type char(30) +, sm_code char(10) +, sm_carrier char(20) +, sm_contract char(20) +); +create table if not exists time_dim( + t_time_sk bigint +, t_time_id char(16) +, t_time int +, t_hour int +, t_minute int +, t_second int +, t_am_pm char(2) +, t_shift char(20) +, t_sub_shift char(20) +, t_meal_time char(20) +); \ No newline at end of file diff --git a/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/optimizer/rules/TpcdsSuite.scala b/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/optimizer/rules/TpcdsSuite.scala new file mode 100644 index 000000000..42adf96cc --- /dev/null +++ b/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/optimizer/rules/TpcdsSuite.scala @@ -0,0 +1,95 @@ +/* + * 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.catalyst.optimizer.rules + +import org.apache.commons.io.IOUtils +import scala.collection.mutable + +import org.apache.spark.sql.catalyst.TableIdentifier + +class TpcdsSuite extends RewriteSuite { + + def createTable(): Unit = { + if (catalog.tableExists(TableIdentifier("store_sales"))) { + return + } + val fis = this.getClass.getResourceAsStream("/tpcds_ddl.sql") + val lines = IOUtils.readLines(fis, "UTF-8") + IOUtils.closeQuietly(fis) + + var sqls = Seq.empty[String] + val sql = mutable.StringBuilder.newBuilder + lines.forEach { line => + sql.append(line) + sql.append(" ") + if (line.contains(';')) { + sqls +:= sql.toString() + sql.clear() + } + } + sqls.foreach { sql => + spark.sql(sql) + } + } + + createTable() + + test("subQuery outReference") { + spark.sql("DROP MATERIALIZED VIEW IF EXISTS mv536") + spark.sql( + """ + |CREATE MATERIALIZED VIEW IF NOT EXISTS mv536 PARTITIONED BY (ws_sold_date_sk) AS + | SELECT + | web_sales.ws_ext_discount_amt, + | item.i_item_sk, + | web_sales.ws_sold_date_sk, + | web_sales.ws_item_sk, + | item.i_manufact_id + |FROM + | web_sales, + | item + |WHERE + | item.i_manufact_id = 350 + | AND web_sales.ws_item_sk = item.i_item_sk + |distribute by ws_sold_date_sk; + |""".stripMargin + ) + val sql = + """ + |SELECT sum(ws_ext_discount_amt) AS `Excess Discount Amount ` + |FROM web_sales, item, date_dim + |WHERE i_manufact_id = 350 + | AND i_item_sk = ws_item_sk + | AND d_date BETWEEN '2000-01-27' AND (cast('2000-01-27' AS DATE) + INTERVAL 90 days) + | AND d_date_sk = ws_sold_date_sk + | AND ws_ext_discount_amt > + | ( + | SELECT 1.3 * avg(ws_ext_discount_amt) + | FROM web_sales, date_dim + | WHERE ws_item_sk = i_item_sk + | AND d_date BETWEEN '2000-01-27' AND (cast('2000-01-27' AS DATE) + INTERVAL 90 days) + | AND d_date_sk = ws_sold_date_sk + | ) + |ORDER BY sum(ws_ext_discount_amt) + |LIMIT 100 + | + |""".stripMargin + compareNotRewriteAndRows(sql, noData = true) + spark.sql("DROP MATERIALIZED VIEW IF EXISTS mv536") + } +} -- Gitee From 945a2b062929ce095c34ca899077ca7a34adc6cb Mon Sep 17 00:00:00 2001 From: Xteen Date: Fri, 16 Sep 2022 11:31:13 +0800 Subject: [PATCH 4/7] add Suite --- omnicache/omnicache-spark-extension/build.sh | 4 +- .../application_1663257594501_0003.lz4 | Bin 0 -> 67545 bytes .../deploy/history/LogsParserSuite.scala | 45 ++++++++ .../optimizer/rules/MVRewriteRule.scala | 2 +- .../MaterializedViewAggregateRuleSuite.scala | 101 +++++++++++++++++- .../MaterializedViewFilterRuleSuite.scala | 98 ++++++++++++++++- .../rules/MaterializedViewJoinRuleSuite.scala | 6 +- .../optimizer/rules/RewriteSuite.scala | 1 - .../sql/catalyst/parser/SqlParserSuite.scala | 4 +- omnicache/omnicache-spark-extension/pom.xml | 1 + 10 files changed, 253 insertions(+), 9 deletions(-) create mode 100644 omnicache/omnicache-spark-extension/log-parser/src/test/resources/application_1663257594501_0003.lz4 create mode 100644 omnicache/omnicache-spark-extension/log-parser/src/test/scala/org/apache/spark/deploy/history/LogsParserSuite.scala diff --git a/omnicache/omnicache-spark-extension/build.sh b/omnicache/omnicache-spark-extension/build.sh index a319895de..6c2bb4744 100644 --- a/omnicache/omnicache-spark-extension/build.sh +++ b/omnicache/omnicache-spark-extension/build.sh @@ -1,3 +1,3 @@ #!/bin/bash -cpu_name=$(lscpu | grep Architecture | awk '{print $2}') -mvn clean package -Ddep.os.arch="-${cpu_name}" \ No newline at end of file +cpu_name="-"$(lscpu | grep Architecture | awk '{print $2}') +mvn clean package -Ddep.os.arch="${cpu_name}" \ No newline at end of file diff --git a/omnicache/omnicache-spark-extension/log-parser/src/test/resources/application_1663257594501_0003.lz4 b/omnicache/omnicache-spark-extension/log-parser/src/test/resources/application_1663257594501_0003.lz4 new file mode 100644 index 0000000000000000000000000000000000000000..f2b568905c4a786b9eec43dc8c5f05d1eee68e18 GIT binary patch literal 67545 zcmY(r2Y4LS^#^*+y?5%C*=J^<_jm^1H4{-y8O%8qe6k*%CxIgkpa$9kfftevpaJ;{-QY$%hnQ-t@hiQWG)@H`_jqbXv}VlMC?dqZDk1*_)FoR%-}8t!#@Ya}U5l9;~fg z7R`i{<92$&KM+nwQ-6em3Vr-qI}u5y>0T;#)>g*tcrrb)0i1on%RyNUO9J(cgWx+p zIXv8Dhf-?*#Uzl|!nD8uc#fyv7z|~`yeR5YIS+Cu1$h)U1+Bo&omZj!*x@Bw8A^mk z?DSs5LVA)S2>82~y#&&w2nM`V8A(U!mgqjZl9{9NtD;Ocl!%04$%IYc>r2v_&2O07 z*xW#uS{d3NjpyS2l_~tDu%V?fFt4GZrG;)G8mA=)#Z#5@gU!ti^-V3o`sVsTOA{n} z6Qgw33+b-c!Mxt#;VQcEPe4A19f|R1I+=*mX01s@LfPx#;pJ;r`nwau$#guFeVLte zY~}LMc*x(C+{g~O>yoK#T_!o4-4;sQbz351L94|IY^o0g>$ap@0Zfn$E=_|&;Gfl- z4uxa3zde~wCDTWPXLc)Z35Q}q@Qp9$yZrT5kp2r?$naPW<-9*-r?b&7nJ05L-)m*; zbTkx;?yw(a_a)OKRwxw;kJ{E$7`9fVCz1mu(5h4dy94X2(c2JM1$gZL@PmvZ%(4@q zp%{hC{K{-P_jh1^;BDYdy^D5bV^g3xI2l^EgwmOfTzH^0vo&Vf_)eDA&e}o`+S1n7 zy#b{DP%335p6B=ze8UG;_spLkoJY6Y++sCQq}E5_r3~FrBp0*ORy-N`iFb5%clMqN z4`p&g@o3gcO>{XwuI@P=n%@$r57f;7sS)6Hd?*~sQizUp z$5S!8m*YEZm~M77I&LzY3$T)^Qw5Dsi={Q%x^3iNIbux4`1{wp$ z!`?JKf<Zr)?po8|y=iINK*un+t8sNpprxhm5N@rb3&x`1^U$~HGWPh~rg^me!A*fcprIS# zDeSYiQ&xILfiAqAZqyphg|^vIs}uP~(br3R(>{qyivYK<-UubIld%B?InL(Bec)`P zbJ+DMb4-&sP}ottjz`0`l^M-tBgt(E>p0<0=Jv||hJa7OyPzO_zUqsAEq^J|D#t>( z1O-pp%8sV(P{a%$q53PEwlQpkP-c`4R%Dt`CQGfup;+c1ducdhWg0>j#ZSTxA0by- z^f@q16gDk`rbIH5h0TdvJYH*lhythl6a$PMgVLGnAP~giT5)(YxhcW;SWY?exf1k|)>; z`~czs)|aE@vTo&gmJC@auBn7S((%ZpquB{-ES|B30mgwnizUrH02=SJLb>c{4%pZ1 zAh*WsG+if|u~O-1A{3ey)!Muf?#pZyc* zd2`!M+J$V0o;+>#%E3f3F%|u``J?Qyj=+?e3T0`f&T{;V9dqO+tIgaPGIs>bc}E{< zwi>iP#|CQao0@hmIkNibCDn9`*?H1I*`n=D=CY|=)*8wU57WYAqC0NI2xabIW8)vd zwTy|WSY^ekoz8{4q=8oOR)Cij@w+@BEV_xV>M69%3jl3Pe#o!tNtI>4> z576=cCA=7>CCiw}baW(|IEV97+J-V&%FkI3@3n@rsij;8KeTpcQ=K(}|BmfS+at7c z>4_HApG%jb_?q(ys*{;fI~KDtp}S;w_4$}eT+jk5uZi%UDg(gr{k(H?ngiq;0vye3wVzQvJ+;I^`*?=jCnoZYFpc) z*-=yC>=9L5j&cwBqY1jsrwnyqpE$0vGL%=cRy>+`5#R=f+rgP)(r(^Hi!pwKwAzd( z6VW#{twHiP2M|PnyN&>STjTFBeY3ndnxuNLbstExwXijmZ=Ws!_@$(rEvZKe(attm zq1Z_B&rHELKy877_G8)Rfv&dtrsgL!aW@$Ik>eEk5Qgm(1-BWFg)(#=qNyv;_?669 z3fQT-u^M=0nLQlJ#j<}fJl);MHUXc(TdBy0s8?%qR)wrTBmYR#&ULX-}h{)*!&bdD)9T`()R(V0#9Sb#E(A^8Jlg<$JZ`j2xL9pqRpVw-2V zTFq#}3}S)V;SwsRgbQULCe(i$p$IKDC6#g7{FQAzbkH}o^>?gl+qAN6{i13gh#GXh z$eWh#2Pwodp%@uVm;j}rmhb$yPwXA%tP_m82 z9`~}h#Ld=FC}Y#!#$wSzWP_62g37}RpDx1`wJBqTHafk5=UjGxI)avYhd4e9xmak( zj%BQM0%piw>l0?Y#}t?cyanWlyn0nKV)v|s;6OH6;c=aao5RUi%nq;MVznk;31;=N zbreMg{0WD4Qh{x3k>!3moSD)8@Xk~+ns`9t&j_sumszxOA1KmsVEhGCwg_6Sm7&yx zZ*?{O)9Y8gOQ)%ZkLi`sR5?G{vbU$A=?U`_7NG2U8)Cc04{zndnF&l+C4W zH>*tBcS*}=L23~8%%T!NS;3C1@bYeIlk9BvcCB(GlNBL~$!I9EP2mYQ4x;l>)|QHe z!uPpb^Eq*jx{s zdqm7o>U;r{5I&TQqfj!g2M%9Lo2VF!#-BqKPXicXrk!4{$+A;M#BQ0b0e+QK*f$dk z+sZ*40Cly@FVN(tnHb}NmboprGx{~B2Uq7a)W|jNJ}a9Xvn!}v+S{6#@h_BdC8)oE z8PxYWluxCAH5?k@@TCNwb6&^rX$?jNE?0vYD{&H3jobG!ejURHbc_NdbXLG&v8y#- zNdE3(zj4s1Pmb)?e7OUX_PG#Dt{97c<@ANWQQ#}qm!yymr@VYGGcE=Ock!>aR5USS z#Ui1Ul}cu!+bt>^X=@}J$vE5S?^Qcp`Ee(j3J&F5pF{BqoUS|AK(E>?SmYiRRY;)a!`SQa2}0fHHuvdyGzFo z%{Wq4j9a;&(EDSll9DI-SJGHWortX8%3h)#kD8!}IYuYYG*Q~8Kt@0r=TAD8+S$;^ zpuB{N$!{Ez`j|(W`)?13FSDB%Qaj8 z&PAyFf_m$lklp8z+d!pcx`xw{j8o3OjbkrkNx^d^_+a_WfF;m))(H?UfjR;I(%^cR ze2t`>1s=<(`Lp()q=Vd~%{L>_w3(s27u`+==@MBKbF~pp-WH{ia$Cmi!qrwdl{2q* z@#Q!KWE}Qjz{b2urt*>H%J|A z^C5%JN9hN*XGg9GI^8rON2w_pad1J)e=TY~vJyqHz`?$uvl+^cp6upB5TK;~mczfZ zZW&b6W$y5;uA}2RTnC~7)lVcWgLdlg%qs*X;b@j-F7ph{KViP;5}!-*ZIV)|t6pEj zmeGvIcTs;wThB^suxHtW5;SYZeUgVS;&eJa(NnzQ5(z@6^#RV8#cK{WS%!YW-gH2@ z2X1!4yQtivE2ozPn*xE2Dqn#99b?-HJm!UnmF4(67xl=`Qrfpu{r#bYT`HiL@h?Ox zRY_~You7WK_-0RWa0wXCqck@j+CHBv6wRkH?S0Nq0Qf*tcRBdmx~mS95A@>TMGohW zT>cHnBP#C)=bZcqUmT!T|7Z$@TOBe@>OKx9F?dPBo2H5zCH^$D<)UWj^&*w>_HvY? z4Btoi3y5n9;ci2mkLnYQp@e-!?v?^i)7gT5h|b@sEbfCCn;uFeT=2C64tbr8aC104 zL510(?sRmit0EUY*ZVbV>lmnOAMCGNmFx>eBD8J$0H&h2yf|Z1`OZee!xW%mcRCrf zlF7LFjc(j0yHzT|t$I<(X8tNd0}n2D2haVe;a5r)fJ z)6t<^lamV$Crj=}n0y8k;~rjzaJjm9jQTh;^q5Z^FyMJvq(GdBVjuEX!FZSprv++~ z<3n6N1C*VTd?#046ElN>bJ4jTf{D0U=7a-EtKIl2 zpB2q~hRQy~si4MC-VZdusZ&7y2zZRG4h;=wGF8kl_~BV9w1cu8eajjcclzx%13C$v z8G2Y_e=v+2iUC%a!+j3pNRX%@UQAyUo19b%V0sae| z2@Ke&1J1WY(0y4P{>`^!ZDvOIopMu)l59Kxaq)rj@Q6?!H;4)#GQ$2DO2FrgJMbxOO@o zt(4(u@Xq~E!yf^j1YTQ|vLeyYNFw=br69AZ+%tBgj5j;ru5x2txr#Ih{2Ig>833C^a-0 z2OQ3%uH02pVJD8$!*+zG=Lv=c7i}}Qg|b4gNRN$(%7)=jC3sqvFXIF8&`31ALZg%| zY%~TLXUiRE=$`aR9*^D4@Ctx_7hy=&u$>57S|F1d4GO}Q;9|zEauhV|K|C7Z6HxvG zJ~N~_7BYUL#t(S{W-9j~!e$4f(|#++C5(OU@>t(P2cbjumT8{UH|2114RFR>XlN=B zd~2M*ot(g|wKOE&KUF>!@O-EE*rj`l@SmXmg!1`Q_&RhrxToI9VG#M*1+boZvS(}X zjl06$R^{?j=O&9$AG^YVnoGRR)U6;-Rq>_i@cJyAJ*(9sIP+;5+}Fpq({S~4rTj7| z5e~=9g2|4m#E8wr@BQ!^5Pzjj&2(ltlW-+lcZ?QMhf8ST9*SN5)| z?d==v?p@W^b0QO8yWs3H<>azHlkk{it{Tqfq841gz(v_?tW3z;QMr^s&Ouu-uM*VB zFkH2Ske(lXj4gqJx}DX9b@Kp!rOThn72(h*hrKwL4pPk|xWLQ*#c;-K+{gIC9M@Ed zFO)~j>_o~YgeXlVGNEhNW2?0znVjy%YpXq(!ddzgTr705n*f|;P+0)4&(Ut;9W+s` zJgWL6n!3ZIxx^SKyk;8vl=?ZzNENAAr?Tsr_E#rNt_o#R1VqK=>1a79JHT&QYi2YV z+bt2SA;GtT3lqOBf=ie?SE|f_5TP~e($PmN#Uc~uL0zqXF1UVTq49LmYNNjf=+mOZ z(QqqFpXj%TY39o=0a;?WOLdga6lY1idKwr^s+#7j|B!n}8)mZyR9FmnqD$Xg9;Bx7 z{HgG9HC*7rcb#myBECc1!1xIsb_P>EF2WmW&Qb95V%7|zlfzMx=oR3E29tn$k8zLF zNzE8+6(F?UZPDTEsk+W8!IxR?^;-jO@UE(6?)6SM1;n|4$I1LZlb}$@pQ}<4j!^k% z3C_F04`p71yIg!c-?2@|-^uhp2r7$3=7$m!L{7|4Zwp64uQOQ1@$90$Op2QJVy84& z^|WxrM7 z81N%!m&_2MNpQArNgxn!bt?Q7apDjW9Hu=yygMMqWc(9|PlR|A`Jf_?PG!A5oHE&6 zUShQQIDZ1A9o$&RLAi8FaqiM7CG^Kw@8W$K0>nmlsm4cMZFpdvJ=7Ih=YnAr?}Bkn zkswqY{x->_&-N-(x z!sV2tY3`>SlkpgTQw@GEayspo&BDFld=HJX3Rg!enT#EnF3uA2b9B@#@96sKo5M+J zEW!lcCeyE2Uh86n$pRcM-0y>#P*OJ&)H{7Nu!5_SFGZ;4u*%Km)jKkSXU$#@eZsax3g7WFu{L(q99S8KMzgn{>Fvbp#FS=5>zcLE^Qp zcZg$3pc6f@PbL|lX*iv(Rp3cpEaNroWx}Tj8@JQxWZFuvu4yHPBJeUR8ZNb72d0byGg**KPCbd!z$oy`v_zBee%h)HfF@^<73sCl=qhHybm|NI5 z(7v)rGB&uxo)H2g?crR^e8z1Y?E+uZHV(q4{6LRvI=55bBkw}qaM+T5{dYL_3rLAVOE_a*%_#(Z+CNfYRgm2jT|19EGWK+I1iCQCrL>aCRTx$6hc zM}<-|W%KsK$noop*XsucDDn38cOTKc>MmVgU#85`Agx8s9AgW)nW zM{}0$F+bw^{krR1@Nq8;uHYlh(LO4bo+(7a3 zY71Oe9iSbHmar3z#@b|N(^PNmI_J{SWZc$)U(EGs;2nQ%0kLZQH&eNb02`~qU@pLG zIqvW?5A364N`ob3sM`ChC5@37%L|ewpWoo9& z#}v4>GUH~AvkwZPxe}ssJAF(YH8P46Fh`@CdF2GrOInjIKAt3vmgmQj;jo331m$~I_l-1 z!}vkvFi>p#OLcfcb0&$wVyQ@1R!e+ZF?{F5V-;m?h5Tq~U1RerC^?ICyw3^8RboK( zntdhk;tWx&YilR-dX8lX8)w3e$TfzyOdhg`7KuiBJfaQs3nx?SEFML;$0e^9@D7)H zHFb*@&qYWu@hKNKFztmBBE05(4>EBauSxZGJo;lHexSkKl7b#%J(Z@f3gOGpukogkMwg%k4 zt@dXcZg=s6(3r`@Vk*R@^ktI-S>A<**kI4Vc(8V0pl7_ki-NqH4+Ccruur+$W}xrbDEywVqG zu?Y49AVe@rJBqV=yGJBYSrudl+#=iP0=L+Dztn{{@y@>x}slKnC zod}KtWmMXJoLLc#oDyWeQ~C(6i`X~|J_p#R8L629`#{xDd?DkqX(tl-k%$Z>w;!Af zTV+pdwu~2OO3P$mT>6J#?@ggg4KVDcp$Kcla^RCOKrPnsoW54n=Yevj+@S1_*h9Gy zy3lZPA(z9ggmbb#B0Eb0yMe7n{w+mttLoc+Xa=l7!c4U|7<1=IV%!hkO4V6AFbi(t z^6`~wH?UbW2U5fHy2mqdc?~;_i4jx-urx&y27)aqz)vkyk2F*hj-(J6Ge2jrgkveF zB|6M(!6M)_<=)UG6~1}5)yUWIHlj?;Z(G?N8oSUJB!u^~X6Iie=iPiLPmr0Zx20u- zU6{LyMYItL0mh~?i+bxq^%ammKzz;XF>jZbkUC+V%EeMiyjX~9yuM7g4pa{=MAo4b ztfItp_sAqMT*V7pgPBlfe6nNz6kg$kPg`gvkc%wtP`;~?`%!!+;MWQ}Muzi1$}rw2 zgfy4k10M5H6pI!Z7ds}wZ%g(P0v*qj!`YzMr1&k|H&JCp@6&ji6gn7uWEC2}uiQ&u#4SDP-V@ZzY zZFc0zMN9nYZIvA5VH#?1JPvU zm&6>=%=#&ptC0U@3dWquQy%eXsc|)9aH+?WDnz^>IG0Y_V}QQ~{*X{Nl(PbW-wUi( zdk7?sEh#9jUn}H!Za9k)bdU|Z#F+)`6M#P>c$`y{OX`|inj2Sv>uU$DD}(upH)N~s zH%jpd-~$e^7ziYwSrLwd^GSnZbuA@|}kdp)J%Dx3BVf7Zm}#(Tuy% z`CG~OMrRobj3BL@M4~Dou zV72P_Ah2Uq_I)Y5*+vnrpADj7ia(b+y@)L^M1MQhf%+b2eZYPxNz z30~>;hs~#avyg%k)#3aF@W5!u9oJv0NgoMBD@71`sC-N-8Ql~5VF5e2o1JRGwgS1e zhfhbiX&S%2n=}f8?bP*$Un9yE<%60q+F(i99s+{1)G}F_yBD{bM98fx@np8TVe(vp zlEp?;-eh|Tl8X>g8;gWrT?*5=auuS+oPEg6Do^@WcY8RK9_~}s%}_lW>T`>|z;EPy zuZ~5odaBxgE`--~<3qqc!bGy6X$^boh_S32OAj?2O~4B{u10a2ss}-MWgqD$>g%?Q z?q3K!3LQ+jqk#Qcz^4{8xIMuUO;(s=Lf|t?#Wv0-JF(tlTvg5WJ~x}n#ob!5`80+S zc9t?|WHy{GUaTb+#jYJnM+W!8CJXBTdL|=*gfX=^Yg1->|&{hBk zS8If^xH)WOe10n|oC0?j!5IKM75>Ibn5!F)PnV3n0oApJwT5${e}{EU91Miv!2#?}hy7bX~3VaCoyziH2}g2z^~-f}+Jf%RTv=VUn;P1aR_ zd>R)A%HfzJh{i}Kv)SM@pkDHPWWNWz-fz59%nw7R*H`NJb}?QT(B5VIQw5(h*rSr; zPm>%aLYWZof+kW^jI(BP*Jm=Mg{YstpF|aN8(SKh8z~(-_mEzID{ESeIg=dn{X#ha ztr6Op2YfQu^p_{WWj)?(&J8o5YG>U#$y3(?{J9S45C%Hu?lnP@S-V3gJ|_ zVM$js60<#6(s(h!Lqc4@@Vo+bDsRmz6;yb+JU$R_+{1wS~ zPLib?xVj6BXC(&Dl=l&h9ZFT}>rJ8r&1Z?Z(`ZW2y1@0Ba9^n?S6lezY{*H_Lzz{9 zD5$Rqyu8Sm&m{ghNy;wOed&aTR{{*LEHSSx(H?b5KY>_*VmC6~2{TnT&~NlOywF-g zLoY9r*qPurQ+IY2X3rAhZXMnQemaN7C2(dT9^xdKpt;;7UrI=`CVo`#Y#(3k@34lV z*&RNAuzpgz>}Bw_%!*iR+R7zb%RJHd41XwgMFErR14MFMIkP}&)g@yCJh!=g;gRoM6r@&$}N`-uOLsKG`x{tH#G<=Z5 zR_^>g!>5Aq#C&K%{KUD3(5w*s31?9+NVP9Cx_tEH@Lq4qPEbS8W@p;7>Gty$;-J$Q zoP)dp7A+6kb)fkf%;xMxz{M`Ol1p19U*^R|cpms6S(#vPt6Tn9@Hf}v4X8FEJnd{H ze8Kz=#~+GWwUckD(r@Pe(MaQRvN8lILOZd8iR|5uSAwD#Y)cq0N5t6c|N?6!&*OVa>*wO{3EEl7!);?CT8on zdIkFtVGd#tRAN()5Hu9_utAq+;!X)?I|Bi${&=@&Vdz?or-sI%f;xQ+mTOxK7!UcSZ%WIwObsSs7uBnCPwebB^@6eSFu7SG^&LIjULU={Q zp0*Zs844=l*M~jCtl=z08Qdb8`tQz)$iR3-pfeP{eNbE`;Z=GcjWW^Dhvn^r zG?>3?g6)ztlZh)N{1%;?6&~P@l1e5xel%OeKn*a-r+KVrJ)ZQxi^TbZmB-c;z}}kL znt{P(9sT_^r5uk1`GG=sP!CcgeN`7*UcsK?ux1ufXT}^|gJ%$0s@QxQGZq%qE!VXL zf^AlyVtvK%@G*|kP*!%kx^Mqrg}v zOWy78&64-XjoXvyb7XJ;KUd~IQK>}y19~zy1m;FFqq_@@YXsxh7C}3~H}k>ri1kQ~ zjac9$C1n%MalMjQ?A0IOY#4D(!R9mxvZLuCu{BDRjup0N>mtMm4wEKTdd6<)0NY2{ zP2xr`dO(~;5cdpt&CmBNGFWM$Mw-j%ol8lqXbsOU>f8XD*)2aK@nS}DKzz!53e~m4 z(DaR5@(O<>QC#aEF}KzC5fz^tyJs?NqvYaZGdR5N*2Wp!BDdv8Gx&z;I)%f9ML3_q z8iQqncv}%xap&dSC?fS7Y^d08rMCO(Gv>Gg zm{4dci-u&jq0zknv@^TaTPmO??2l$HSETMf*OLOW96#LPIQ1wrToB~&eiyG)@Kh%! z_UI&&y}TH95m_@8`knk*6nAxd%|aQj9q`2Om7xZa9=Lw96ju0gnl3i-f7ZK50%9`p zLxHro^oU_Wp1I!v0%hPZr|#-9Xf z3~g#_Ny|MqB-$ki%ij2trN%{E=3i*+3zR-|!o@CaGwvmlBtfxiZX5lq-V=L43Xq(2 zNfmZ^rKrv?qCRwnC-bh4#NVOJUrS-pl6u-TK3!Zc!DT{wawWeBu@4!cx|8eqdd>X+ zLKF+@I)Uj5eBFrCr>mdqti%8dT=sOMtBgTdVLfG4+d}Wnk_rWTtpE}==#h;@`y^<2E?FEIW$eqB@?g zue5qjEE2z&CN|b-_X9uE%VM(~rD9r3nIyrv8s>)C2b|yU)Q)F(4v2YNbaRH zyP2oyc+QD@y4VG|QnBbZI0R5Z(_J0Dw7&}J zMs-$%76>myDB*l(k#UF6pueWMbFeQqPkITB{R1d2bS)*(Z|vmFu%d&_viGMZ)e6LVB)!%@SPE~Lc_!l)$oF*w*h}J3_av!k4}|8QNixF->|{6PAmg$)J+)SG=y3%n0`i_mK>~GNiYfy_x3{ za8ZY6=+P46hY9VH={yt&&K6Sm4Sc`BOJPszjYh5;+O?90k`i{4=;7 zmH9hUjrr4@>>^b<+hChzW^n>Vq<0+Jc8_yZu1Lr4P_f+sU#Pz18BVxz)|BW^xDh1&kx`_ob;GDEKjLb&i1nMU zv&Idsr`+CfI)M=YvM1Dd;0y*267U3g8X+m=xZc3)iacXJMm;+=F=rRTpSXA<&cb2V zhRSi==u$u*D!>XC{|iWO=z?gLM+968aJv`&>V>2Ve_@2i8I69~SX&`D3rPQ@Nlt^? znE054U?5(fD*g%l&T95_HSd_yZ_>1FUR?+WU?s`ale>!l9O3480r%op75%rHK33n*PKoT%QLt^5f3B$N-&}{ zub)*E{=(-Uj)dHBWFtSNpWSJ3YR>-6_f;5hv11Cnv5Hgxv zwBRMq%Kh+!#dZ^sQR9y$m&^Wnf%j0}#FeNM+Jw?=c*4Ck#xOFOO3HY>HA-sq=$Nx& zVqk@n96)S-OrBW)BvL$#A=tq~(=4u_uX;kB|v@Yb*}J|e9p-g9A&?!gvOH-g&qRz4}8;x0`AB6dtQeQlmj^|!4b^Qi#MLi~3xMOk$p$pMl zX6j=;eu@iDC(W|~BMQ5q6sAs-%Ne^Gr1KRb;(Qaq1~@3Or!?$Tj7}e55hlF*KctDqUnnYaS7^0DE&>io(Yj`UFA&vNK;>L zCf-wY`r%Ed@+`V{%lIfWCT6*f(MmUS_;9CJnxPTm$2#*emyBbE_G1jP4x`VhA?SXq zXJZCCRuQS=3`(B#QvrUR0^h3QO@NvvzO@sdDwYYVCDqxvlcgp=QdsRRB1MMUhJ9Oe z$#m`~H(n2Ly52g!KG<@8r5pzKw1%JO5g+!l1liJpRIpOtEcg?Oh{?U7xS8IzzD`VN z+QdAzc14vHA-~0e5>Fu5&OZU^Tf^w!oS&u#t#r<-!DX|R6L@tdrYwS!l|HJ}naqM& zM4BwMw#Q?|#C__N)e*{+g1{>?)XB%Nl#5Lz+)PnVROKg;T~o~F%;j4f@Ryo=3G!V6 zPOoK=O89HABz7Y3sczxH&6!wU60=XPvNO-w(87#+>LsZap~nZLU%tA&FS5d(7`U*o zIP;wg-(b2dribE0{hS-`j{FN#tqZiRzf60L(y1e2JE7P)m^>o zR*@K^eN!(fD*C%gnL!icVE3}xPWm6Du~@l+L9=>C*01%Qoee8QrAU7}zkdslBVew|B61MR!k6ZQGh<-GiIfk-+2Q zAk(lgJnmLT99Ry9i^UaA*gb{MAdihyU=sBF(9E*BXLa>6Ntpon zwL_G2%R|T)yYK>%l_}yD4r5%gCL6<~fLI73JVfG7Wb*-KZE+7osGT-V&JedWu0r^v9Ew;=tW^NO4h{|9Xk zPJu(q`pH{mr}N%4Qd1xPMA8}^HkQ)hK|h+#TdEF=pF^ZmPUO{n1eWIgDtSKz1>FC~ zc=%t}Tqlh7WpqD;AZ*w|~;^g(-W{zN- zyx}V^D9L*hgI!?U432|5?{PIVhn-k?q?07^3R%2Pu<&ah|~QO7W7|tR^G!D zei6QAv1s0CNYJ#Iq0gt!*aBHIp7+uOP5|R+a34bQKBcU~)Oy~C6HPt{(h?ABL8LJ6 zeI(6J@&TyKSbNZ8x?nN=svml{WReNuTKO}0NY9Jz!j!#exy;n)L?%iPZ|0@U?*sIp zCvX;M0{x@WS@@+iVbKVaANKxdYaatl^G2s=;{We+ItHUZ`^?h^WJ1LCp|&wEU^^_i&ajH=W2eF%`jSL|>78}6*+GrI z$j~+EYWaFh9@c1>ZY-JkCG^q=VfztNqvXF#gyL0D5wI8-)GN?}v{C;W_u)DJ^;KoI zUCg|;%%K%;C4-bj^M_3-%t`uTCoUrrw|OIwIzJktik9ibj=WiqMhqITZ{&sMO?+RT zNXskp?2#SxjcRN)$%Y~0K(LAKF2m3HKg-*tP+&Fk*1RELGMtwehteDq%dX)s(SFRD z@js5_N$y#+6t~Wvm1$eq7XASDuOiplqGJO)XV1PH)H9J}B>%If#XjVf(H&P{pOyFP zJc`S@S9SKbfcrJ@Ml3;yKzcIjMtT0$Mje>))dIZ9eP)Nvy(UW-K}!z(TUS{QCdwE>5xzp>{pvkHUMzj~})gWl0ZjXLiYbq=2>J zrvZNk@E)%AFmVk3O8P&xWjHkwvdHXC=J7(9W{t^COkr8t6w;NL zPWstAvKNl^(Gk6y$@m&nOqd~_Cx?!ZiCoUb<75(cgW*;beico1NQJtFS4W0UR{R55 zo2geY{ww+9eE&C2M)zltALM;>ZnjU{!G}mXz9VXXOX3kMtGSTDAUl^XmRFf=VLr*yv${ zTC7HR%$ekgnMF^OH;fOd+6@A>L(2Tw%rzurhu)gk8!k{tU*t{8vMowr{+rkqCt36k_s?mI_oG~>}UdRo#5N2r^LCM4hPOS1a|o%`X0wn&Ve z<@UfPU3`Pi1_o^?K7sF)yuzQ*LW;!yy;C7imF`FPMRI^_75~=Y0cFq6f+aJV`cNVT z>i+*%vvfS@N6RVFJ<)B9v^lKf#{d>pKyqdvs+)A2G za+g|pvyc~7oA&mw-GOSUwE1U${?+u^q3vH`fJ{?sfW)&!CiSL%2fv>i9!c?zUg?6* zaMybC=NCvkFCB8MBkNghJEa$@W#%C1O>5i2d8cJMG%ym45Z*4o`{@mYYxfaonC$Os|DG{A ztf#4=ym2_5S|=%0)aL8KNFwt*yqMSMABHr^=leVZHdRQkxcbPb6wP+ z{-)gXY?d8)5cOtgB?K`shx1DT3*7v51}_8u3&KMEp;R(Kw#K|{EIaz7Gt|&nADlaP zje>~I62YHk#<)HU-d5P}wY-lYJa!GtibhEG@`$ECs`0IiC-{L=p0CpjB(&)+dUrxI*>%eRzHz`6N<}(b1Y1oq zI#3n3RTT?78rNy1=|(o^(lJ`D6Y(2~UcEqR;J{3KY+73-#W@$ ze2gsPAvsXTK|y7H5_$DSol_#bm#plY`8L|*~rRL6en26iG*cD>G1cDwY3`+MsZ=|K)ms3Uq99 zPAhO=8(9zYcBJ1F_$Iz*Kq&uqCw>OvJvsrD>?LIjW4{6<%jn099VeJBp zDC2HCO$^coWUR}J^il9NjDozH!$ag4;qo%RiK>PjzL{8{z1r_mmpjFE1GG?61WN-KiY8-%lT&yXQzF8py_r)?S_OPQI ztz^>p|eh`7phZhNJd1pMB{mnCkKofu$=tllGYlR*2?(Sm+?*3$HE^mxHqgFMkz8oaNuDQX@> zy46{nUFZ#1qmiz1Bk5HGU6J8;%Si61-ayW12ly5Ba*aKq%Ky;eIv3x_0@-kCF)X11 zFb!Cq6U#CwqEoZ&$^!qm(D=P;s&X3Oz3x7`^+fn|5W_+yNO52ikdBMAZr7-EADbzz z)A7*JUg-!PAa?pS8JkHT?(roTPT`;G(zz^8z&*v3Q-II$!buJ!ZsyJs@RX8g6O)|e z$yDcDzvK9%aKEI(Zq@mizB>Qff@X~y|Do45h0;HwBM?e=(s<-2HbM^23m$GLYo$?O zc&D#m`*D+jP(TKvmmFVlODo5N1>$viW!_|egyLy#2I(twp2FpaRs4Tgdk^rqs;g^w zpL5&vGo#+@xic!3Y|Ut@q>+o73&vVN*+9zsz5n|?KhI;PTur-WpS{;!Ywchz-IPJirk0do z@jxz}XJMx^x=53*%@-Xu6FS{A+wSPfVb_^0={7z~qFoL>IkY2-e&NX4xXUpU(u>g7 zWcww4A~}7#BucpLd-G+e{jQUj8Xf`RB&Q|_s3CMA*_$Q1bv`J|3nZJzZh4XwuS>Ma z^}quBy*f2%PX7v-oy=|Lle~?ac3D9LbjAOXr{pSfL`@5y%&|&y`$0)0frih*1!?1N zrc<4+PV{)zB)`h?cx%ov*J4$0+N^9gQ3ILl73n5fT5jg|PS}Y(KdXN3f~NCrroXFz z;Nr;rP&%ES%{*?$?OA-X*3#RB&UwCj*HR!NkX+1KmSx^fQdAZ1pqu(7+3)7!ah22G zDGDjWA9BcJkAPxA^%%eSkweom~x0jg)YkvO7shB{QxLvkEcs5W_31Gyd2$_ z>(wAogs)A?#f4gfV*h|_t4!c&NQ#Hra#_@AuwDOPGCH7v7OBH zoVjpt;JkE&Ysx)dHq8RkgsoN99tEH0#zI=|HVr$Kr)B9CMLLy%HzW_G8yxMJSwpvJ zjO+rx-_?n<6svAgzH8|F_JD6q@7I<(q+MglB6>S}KY%V&7b$BC_LG8WC>k41lm5%1 z{+3Ih@VWmdqX8#5gr$wsr)_}!CC zwt)<7u>;&Dw3*^-H?hO=e^AqH%V+;@V!B-?v~-0TpGf~MrMtYasD44IYWoY8IfcE* z>8Iv8AcsqK*uWmc<)3 z#d}Ayxht5-hKSq}a!Hkp(q-vN+idJ##31=8)$)nc^cSc4 z7LiL%_b)AzKO*`Av4(VXj>|~q2vOT))+br6Bv<_Fd{!~V<^Ng=z|QQ$N$LV(58G+` zBo|~S%2W0L#9d{|@EB}2#s+-#8i%RCohH^x+GMV-OH-$2ii~2nvLRhss--v0C*^)O z+I19k6%7ppUDADa`BG+oL`CAmW-!x1nLR|dds7;kYKemjJ`a{81=OS1;9*j#{F(oOK4=@;l;xQbe1=p%M*L$xL#ov=9_|iA}O~ z8@-KwUmEF~2k!D&AS!K!Cq-IXPgkb-O)L18@n!3e%xt)m`QiZ;#c0RFjTtFyX2dNC)s~p5!;y^MJaR-)>04`g?tsa zTqfZ>p#4_1Du;hiLYHfD2a{{jHq4N!rilBqxH5$T^9FnCD&jp$2YWZqwv?wWyxRXu zqduO6sD=D%{eF^eWTvic#4<|eFHY%1j+HCz7Pe4wyuzigCg}b?GZYFlwIV;d?;g zcdP6iiy5rE&rOIG@@Erih?K7;(hvR8p?rFp=~Y#iGWB_GKN%XBDLiCm=zuJg$r7>G3MJSE~&B0jN;3(j#c=nRWWW<6DqU)leuxx-u^Y*h!Aw7!RN}2^2+H?Pl4Zk6 zM9c&g;Cfl}sA+@kCG^h32q0{a+ccUhPJV;MgT)6$!JuI{ZTVW z%2O?Jz1Ikka;daS*LrmIJX!mJBF_fxnOSX4dq3khs~7rVFJqVmtovM%&dq15g|tLf zyjt=4fhNDQ-bAORx%|J%lD1|W;B_Qir7G>RG(g&~xb$ZeOneNRyftY-7yus1p)?y^ zZ=x4-tXoxbDvp_@tOfWtGd=4Q7g~57q@klvXVY1fl{`x95>5Icmz-wD1{Xz2q&>_q zdHG=>ZJUDc2r7r1BlF*w*rz48`5+gA8!9?PYnM;vFPn-((Htgzh5~4TmCbj6Nr(!F z0nN8^`anveQXGX~+2-WCB<+TLI8B-EH0XI(A*TR$*08#nTd6#lwj_zK@4Gpm?o{d1 zGWbwr@I4@7b6ha)2Rwu3$AoB8z|LX2{syoAI-#<91r zOLK0$0})_?A{DAn*en{e-k`ycXAoF;m%_{;QDQQwvx!}8lID=K+b@5_WIvN<$JmE# zQle*bnE$XO-G8{$+Q&sK&qVJS9WHjY!gfV@yM)*a70M_veqD8)p>}Z zk4H}{WN}ko?{b@Tk#v~Qq&J*F$5jQu=@e;p2CP8TPUJn%Uq0Yx=h^a;gR<#KBk{f! zI%~kyU1Ot(IeD4VR;&g>YpbBY=aIt?m6Eh3N^~w4UspKA z*Ukq`lbd?Wb;)TLO_~PLva8Kn#?tK)oDkk5 znosieJY|PBD_3-SB)Zp3xhCmU3q8-giNR+a%FhIw@1x%%X;#GLfG~e*-MM(Iw|jlt z(8zSl?YIUF_deW=oW*wCoJK%$;qQ z@@0woL6uL24&}n+a@?I?G_%Ag3$ZE0&<{9jYyQ8!SYmduUVSnr`}v9qOO`E zQVVH9MUzCwY$!P#dxYqc%<_;bz+p7qQN#Tz0p86{liZq^X3jK!VzN!62E$DDH_juD zgaHp4lw`xxFJ)_nZ~!;IU`y+w+3$fTCDTSbzbOlF9si2a^FjH1r?l53U67@It!hgw zhD*p>3V)=G9WcXyfZt|_1v7jPy6ET3w$lR2>cXb-IrTG}n_D`z=9saZ zAI&kl=(7v}@UuE*&TWN5M15OZOZ}2L9k7FFX#y8v=G?X=jrEOlTELmfHAuSyxI?Fx zFQ{KMyL`z>KOysx0-lDj`x=>$>imP0dM4Efac|K?<&p_#_e1)$_K8~QGhYI>41+_P zvP*4Wn1F>o<(7XrgT)G+z8r zW_j-<`oQaq{8XXeM&yh2;SFPPn2`40#49^EYsKNw4<&q75*h%{os}!EXW)VR7f|L(u1n9 zQBuElvQxBSGy{Qn55yh8k+J>{T%>TWO^2NmL;4nQGFKKosY*+2J5QLGW}4{~oyLI6 z6Fl>vy53%a)#w&ox?iz92}|fO-^Az~)qIjF-sQp;qY&(7-xW@Mk4p!SbVj{lb3I+@ z8dqVT7ap#_0)Uu%sw~%rokO*%bM=Ru-m{ytsI$gsH;*;d(qHgh82drGr?>xU?Gw|S zq0zSCp;5gu;%n#~)t3MXO!VlD9->!Ep1^@_o!`;6v|-`gCBuefGJuBT?RLF4&h3bs zqodt66jih(M}6FGfbYq_<+*}2#jKE~Y#acnbb~@qL)(H&Kws92vN7JTFNA|L_!@c} zn0pPx&Z#|0IQhP75^Z-Wg=9;R?HsUy7`%7rm|ZF=DohNXQ?qz@Jv=EIn6q~$B-`X1 z^!pdDgJbI2yOl8BrU#7p3a1|e0)J(|+5en1J(5pB7%ksun+dboR3P+M#0PpBiCvE_dGGs7{t0BoOtca^LL+DqBsPu$kVDtS7Yk6G;jFuuf|Ze) zjUO^;CELPQBRUN483P36tjKwQ^g26S_@kCX$G7|6U4)olU4#Q}69~`zvg{wDv z5<4HxKHMsYNq&vXw`Zw8CS?YfuLCp49N0MEKOL;X>=wgzM*ks4Frn|oq@T|cyU69; zJUWljV`Mo6ef!PThTXYx5oym#2TGSLEYsVX^w#-vck0Kz#m|u`Fqs}uO}~;eDy|@) z`q~WPiH6SdGq6FY%357Ze|+?;yoEZfx2t7(NMdJ8d^Jet3VkVwvm_xOQ(BX7=N|ct zi(8qsVq|nOW!{<`8yo?g0+vCDGQnC&CdyIc#pa>3Pu~G1@(7OAyOdSiw`OD&x!8KF zx5*Egyiys)&Rx1>mT%q^_IsIvnse(PrXQASy!9|D2EnCb0Spo9;r{c62CGP5;7!7? zfB1+oAFngKoA*@FdfWb`i#pnt)GwM@-&B<|)X9&?@UTb8=Zi(F{E7Z5|K{q-j*}Id zL#`nokAT#1(4wL)>L&ZMsUCE^WI+*82wAG`>}}c~~ph&uhneT+YOw zHNKfxwrm-T!KlwQl%dgwn$jZa@l6w~tkl1q_!LtQ&0s-Qc; z`WapaS42d6MZn2R}7y$^5s_CmdOi_6FK@Tl{TyI)G#x%}w3s z+gst(-x2}b`&~ww?x%ezn|8ylTQ?uolGE|YC+xaUxT*WgXkW^-+i;4nqjrPDr6VU> zK+bg;)f+%cf(1?zS=k+tUSHa7o^h!wGQzHp{LC8to zgnA1M?<{>VzAKYSP%~M@RK=mZjB#v%n6X|;^RWkQIWT1ja6Af72fjsXs!R5 zQ5{4mk~5soL=)IbK*TUSPtDf)jeZXo%{_tYvY3W0hOFrd$ z6XXn_wP@(#tzz%9Lg)I{j5<&U63<5h=m)<;>N|3sFX9h=!9)dfcVcqEP|{b%P7V0L zj(}D74Mnx^!=B1?{;ptqYeYVJ9~wY+g7V zUk^od-2d||*T&DFz+<{E_=O~fm~Q_SlfLxQj)~%zti-;QIzWvY9L?lq%qPfhSNxzC>5$Gh zE7Cqt$LpCmO_`#{M;4@IbgZiE=m5UsTf+ z-eQ`p(a+4{d~!ScJ~S;(g7q{ypaIo)Q*DlWOF*5{jpVI=#8;l)jB6u$wc_20Xn`VZ z)c0|kGmai+k*$y~maZ5l=7FKIK$`NlK|RDt2B+xFC`f9!cmh5_dLG~9^o%BAviJ-0 z)WN}e=#9zjY!d&)+z;bT!3IU-_k~uOyw1Msay#%_gOR7;VenhC{GvjKH2MP*_izt? zpS2DEI`2ATsrZ0RkuQ|Mm4fMC@D1|v1Eij$iTyN1GHZt1@m3gIsa`h2Oh3>lUv^hr zuIlp7d2QdufxCDK&tY^+wfLPnMgB2sN%cy9;t*#`9&}vg#!TaV8W2i0o+~+r*Jn9L z-csE$uja*}4fVZ1;%cUc@IAcg^qwYu;3!m~PB4{=40}mi5)93_VDyYB0JQD736jG> zpUPsh=8>!AU7$`wu{+o^)k?qNVj+2WhQ!S2EToBS$y474Pxg7Gb;LzDV zaHuCvMIdas1HVzcx5tM-1BCnxRJ-#`B2#r&Y~H#^En-Lq%3^#DV^^ASuIlI#@gJG# zMFx*j&+t~52Kf30$860Jf3@!<%|khkM+e71U4hrjNKelONtiD#85+L+8wN{1zwI9Z;= zes8S}_+zWGygUv1;g@n`tlphK{PA|{o$$)Je-=3uWz0Q|}$V%;c_??B4x7BSXWRdbh?$_A~^LTr7yum&Ni#2^VKh0alNXHPJ&} zTEJbto4VXX^LYvVC7Ye05!{`>&poh6WrLFRu7!Ufi$}6bOsxdqLOduHv%s`E#7Ddf zODdw7duFc3T$ER`=^dN6!Q^F6wDSY9RKyJX|I(a_&08k15-2mN$y#T4kFU(;iwcb3 z00^FUPGU<`cQDuCVKGTWp@=(^>bk%y`XY;7k=?P=^1RI9(R;e)nkbr1`@sq{yY+bs z@*^$vBO~!GxRPEpdtjw*4cQyugPDBREBdnv*lTj(K%%*h|I{o_S9N{?v4zAp*!cNe zsHQ3Gp)|3M724}fIRD{{Fl>WY%HYioVO9%w;pAS^6~>$nlRI3>yluU}u!ip~=5Dq( z;15{NJ>WGJGvbmSS&qJwk8KD_NqMJnRP= z>+_56YbE@{iuB=&BVsm{NW*D-p+d#7_yU2gic^s+qEMN2*qGBLE>b=4q#WAf8(!bL z*^ZsBGG1E9US~p6-SS<|R`le0dtiO8NOXZLULxbqrH6C)WuOf*@uX6!uitny-<^0! zDVOxT#hU=MB%Ky|B1ibK&LX-ECa;i`58PrA0FPz+lh@dNUN`nN`a?^G(n#=M0i-JcY=#`4m{CwWEEa;vhpQZ5keYDx51i}k+I z-;ww&zEi*yjwF5SnfwrEt2N!d*vH|D1T2Ln(V1|gA!Ti;SO802;42B( z6Tb&t0jtFca*6EWVUau&F1J2moqKi+lr!bFX zrz~EZG%Wzy_@XqCSu$DgUU&N-K7rNSM7``WXnDxT7HGnr?P*wpq9tbwl_}1R>lKln zSvcGU7FqYOuQcn5fWNhAK~sHO)8kw>orDt4On)oVBmOttVu$Q@^}KG@gMl~fA}SZk zMN^m_;E#9EdnGhW7Ox|BxJ#EW;?CeAqBEmi1b?diJ5kD?0cCI~3 zkeVhgU><%{WxtusZtvFeD*IDTEzx z3Uh}}VMY7!QW#NS`Ma2|@JECGS!ibUpUL#PwLnFI>b^>z7g8xU4Cx0jbH;ao5Qo66 z@Nktsm&C^n8QE~uA32C)+(cJEo3Jj7-xXQR3m3;X4)u@S$lTE$si+Z1Nq>igd^F~d z;*TFOa3g$laxmx%LKoLa9^0?^Fmex=MD(u1639!ADXG*OyJP&?G?XO1(7RM@yOQ>z z^Ou@UiPaYj@>h8?c3{;Ermq`kU&E5RiFEFjFe9xSDvScDq}3=BB72BO zBk3wj9QC6yFnkA+U>`tz+rI8_GM0;&RI za33yI3SD`81Q(t@O%-!^p*_Glx(uRV3r*Su!%LhW!y}a_0Rn9%@q6w$9p$FaPy!7~ z!ecI!>t%u`$8QK1w|97W3*-d7qfg2p!g~2L)R9W{o~0?YPi3pjplIz^OAer?XJ{c3 zBbRF8UCm<;aTqxb^d?_acO7T6#(%DfH`S6xgEBusLCO>Lzhd#gmS-gXv2y5}SwlvW zIms%{;l)OAF`IUb)6x@9BC#wocaB@gZ?cM)EZ#ay)Jh2M>^Y|4yI}T*`-Q&aWsXrm zj^4v|7^NS|!l2p9*J~{)D&#RmJgOA(muVkT@}v(w8{wHYgcmQ&tPLqko>?^FAUmu> zJza=eL7UZ!Q1dj`ec?;6Bvz@O;Lym>9mr#nsvcaoJ~YwPku=JHzslZDgn6tAd+buT ziM_xodyO7cKLA&Vd+w=;`#21vpT?P)GM8yy@4@Gg}%&won^fsf4+WLe< zuUSQnOANY7pbM%)*jO_rXqN)xl_K_qx)(|=M4QnqU#cbKl!6$tldXz0saX0cJ0E8d zY9x%h`m=>UufW-+WEcwr&H&$Q@F7v%)jbxkxQvuAp!Y?ZSZ^zqhaFuA8W{JM;2gYx z7nrD(JkrfH(;z5=L3BBl@JsB4@#J$Z@mBzReO&TEQZBoLt7M%%QKZiujAQVH?3Na# z)9qX|+KVN4CEU)IrE2$gHT2}fJ*=pqZ>Xo|MS(N~2insxyd~0?Wpl^cQE(3#%8}&8 z43k*N-1eVar0bbjno(qL2J0Km_%${v)$m2!bPSF|{=S|=nW&V(AAFa*?9Q}`%{?C5 zBqeOu2R0uhQG+CL4yq?;3;Gl5d%EvbN|3d)YcN6x382X)y(kIAR?<0diIHi%VsR#) zGmHC>Ex(s3=VXij$}Hq_9c^QSm<9EKN#uxP)?<~@({~xQj1LlxrIP5idZdF2-6GO` z+~MA0(@NxX(xjy%&dVwBtQ}hf*CHmKP&^=z)4>e-f+5)A@*L=Y!X#>N>K2+jNuOKg zdOAJG#3b3xlzet(nrO@PGMg&1e5!JY#}!Wqg(nK)k{t1>&C4Ik=o*L{qLf=L;*pFp z_8f2S!x{cd9^yAK6elG|Owj&8U^X!1>+3eaB4bWt2N;H$GKBCBck{6V?@L3;wY@m0 z#s!1FwotAjN|EMWYVAaB3P?GWCmymD+aYO~j-#NJ)U_>%`0kcO!r?L9Ej1fO)}Kt1 zpiODPK^yAn$P{M*!MKF;XPwe5nauAHeTwPI?b|KKM`gYN?W+6j6N57AQ`(lFxISfV z*rL6FxQ(Q<}>5~DU346W6dj`Sx3)Pf>#Xe>n3qf`P$jM3f3sH0HDAkLSHYbF}N z(}jw3$3)>^l-7~O>H>jK#b=aO(Ls7`aw~gly3Y{Z8O;n>&|xH*7^PV;2rNhGHh^sP z$2+ky`;u?-E?U@;a)Yg(IkUB?tu4erGQ`kP?^4KTA7oeKdo|DQm<7&aInkq(R{nb` z_LeWA;g;ssCD@5Ec02yKalz8IC77FVGecD-d1zN87^sY3!)dgpt#MA%!X{kOlPNH# zxotkFdl@aJp5{eM>gO(M1Hqc5w;6*(Te(6?wI)xwj{gG!zM~`ShxC z+q$7m@T$i|&ZM^h<7nc&e^bR4=p?t9QWFq?%0>(LDPUyaZN7nJp=<3OEWLotfTi7* zWZ7!a0S5h#X1?oRWU$*43sW+~`YiVghQJq$(~Ve{1En4~fO!dq9Qo2T;_$~oiIU}- zT2mQnrdwwk)1dI`!L?!7>g*$TX%TD=ms3k5jQ6+DQon(`e?T05yT39_m7Ty`Nx*_)R>{@FlS#_Qz|YFOyh=?%xIM>hUq4*mFfIxC{T% zibg2AsEAsMijW7NLVSRv7_lc=b4@H71lu31lm;k=TW3*45%OND4OR!{P=+hH1BpA4 z9tw;y@hO@wJwv}VGBk3>&ZLaJMMZ|Vq$sq2%(cNlcoAiqd=i6q+pkzN9>;Q6hBhQ= z4fG{1!?}Ax#EX{8;LJ~@#_7}bmI>t4VF#p7pT2sqTHb&b9S+$Nr@XHb`89gidg*_$ z4qex$_$Cme5h#(Uxina&&%^*(T0PjDnqH`{3b)tjtKbWoC}}7RR+ev;*{LGMQvyU9Jv6t5=0fMr2ycgs^mlp)x4-zEvsxdsp?8qM8UxIst(JgY(y2J53SLM8npoM_?xU~40ex!V|a=#y`Q8z7@1lG@ld64x4;) z#_bE2wI)(BpSCfdjV@qElnsWwrwwIk;}SjLpIqJ@^e6Arj+zXJdiCI}*5-ve4pLH2 z=<`6_(D#))J0?=C-W`O??t!<+rrN7DaL;t|Wjy_DY3w@O#F4CqKgf7y=WOlCD<-m^q&^dV| z`)ILY2CJ7Em)jJdK3Q)VfsCsg`RW`rI_KfF`drLF9nl-c;QHCxyLKJ^xax9;vnD`0 zA&P|5MK4Wv*5g#Ix!Vz(tT*lsVWnO&)H<~3PIz&W^jio0$P_`kFvp=>rtn)$yqJ`8 z=#Hk%=og~nr+LD53Y6>DtBVV(g5~;S6AUmb4_9sZ7b-QHK4*nh;c`8RAThF~uqslnyS3rM z>Z2`N_~|xH@&7Zo4zNUzGYK; z)P@0P8QjM7!m5L= zM*LNzpNosQq7HLC*uKKb9Oh|-;n=WM$)R|;J_mdH18cA~(SH^X>#I8dVyf@-cXpqK znL_5O&T{<&6Thg?>E8pn3e4Rx`4}Uq;#;@)7vO(w=eVv+jLb}w>oZZ{{+bbNxXXF8 zFbvnuvM`2v-4?+CLbuRROgj02!U&F&q35#GsUaL1DAPtAbTy8W;W&U4(Md1hIQ9Ge z{&Kwq?TNWYlhf1QJ`F|si%iZ6&g1qd0|aAXAEhtDZ>uajB168ST*mEf=-#D6a8Lw( z8dGkRTNWbCDV1XjRhRZU^g0?=r@+j+T{`&q&`_R<5t=?U`N} z$(3f~O{*>df=O(*;&_JtXhCAls15)tc_TTiG8w-?aUdi&b2yY|bl=4n*XWCe1|Lh0 z;u3PHjv=U3GQ^#d1kYmOP;w-T?5z5X^8_5olnMq`p!S-_4%8bg)VahJY0H@z^aViE@?o}L{i06<9DICkB$$XN@v6O5WfpQrJt8L=V z^*!<7ONd)18ta=Prkhj;a?TeuQ9-KuCFgrEzs;ncrO|4Wbf!kDO#C!Mo2{Zhz0#Dc zuQ2h46yD9~5k*|AvikONeb%3%J8Jai*2Z%EY?DdsF{NpbD)8s2#kMO<0ux)VFtu4$ za7#l@2HL0?ByK(1EQqB!y{~!YHD>zr1hLjUD}!qBXfRimch0% zoq+?9kx@{Ij?3%}%cVSHdoiYS!G4}BN&^!+C?g|sl97)}>MUXhO)h=q4EA%~dI)Fn z99;W1QjI&@g>)`i*K$GdfFgIQ zVkNGzq`E{O9n!nzri(6+GJT4Sn}afM(iH5VP3hQoWV=VfjV1->iLtfbJd_g0f?P6$Gb(`Gc~bSyEqNCZ8bgCyuvIJywv9b#jySD9M|cV`{;Q zYIq9ArWUaGi>bp&^8yA3BhMSe|NnslS?=9X%${m=>M-TUfZD>q{_0S5I8qrZ_rC9N z>Q@!gd{f3YCrQ)k*M`lCj!FjW33R&BuL{|@#aFHw8!R%$5ieQyx|yo1hk1WveAr;s z_7ny6_^3W>%ubIi0d+DpzP=_@xvnB}(TWM6GrN)}n}V$~XF|kTZ((+F&0JuThA6r) zKK!KRswB4HByB_UWx^5Kw%XgJ97uOv;4f+cKBp!7W)~D_S#Gfm^-k5J>U*4VC&pL+{?s6 z*PN3V3WuzS{k%rZ#B*WDfmSMMVm_NJ;!34EbOv4$IRoz&iYwv`%vTS-q0VuKx3k;x z+watz?b|i6DQHpFPs#DW6|5-ozr;mODa^s!{m$yECXxk)uK%?da{M7Q*GFcWvi+0I zvytbQ3M7osw3_8N1GXCC+szFvSb=1}bLg?+y@Sg!Qz0=@IsqJ-77Adn&HH|&v-Wey z5uvoAMZ|tYj)}xyGwYS1MQ*kjH0dBOB`r*R5(QvnUE)lyZ=A&b9%6f#Y^$cNc{G?V z{k;m&l-kTZVQ_kPq3{G?IoexQ%@5_$!v(y-Ob^GjcMDioP@70={PWQ|Z1Yh@c^ySJ z^!r=g0odux#(*3WNw-Yvoir!U>+jhbcCUK_KVlm>6Mvi9iP|1q(pM|Ai!+fj)d4H> z%M)%J85tWMZ60hyi@3%jKbVOO3$>+d`0DBwZzLM1e655Rkf$&HP^JQW{9(VLj!DEE z(1bKsga363DdZz+cGVe97E^3N1lH%KDT@dv{jmRR2OG7C^QKAsZ0Z>4>G3Bou$Ee9 zfz&!U)QrD}h22lA(lf9)CVn6k@DFcUm`krsQf`CuLqh;5!%GDLBw(*K0wjC>1aTB( z&Dtvl_g?8l260Y zSUMLNQ6V|fXEa4H)dnKsW9?4=jrnwLCEG9Y8FE@h9Y#d*iK3fLdK8{1BM{EEv^K9; zdCPR3J44%4%YzJ2HL??P={g$FEn7p5BDT01vu-OKJX#SSe%_JR@fo+?QpYbM*bJ-7 zR{^EC=3DO9EGF%X)f=p@cOA-Z#?It*XmyM}BhigqoPjJIir=+dJU}*a^$d;Oud=gU z0mRq8=YcXOec%+ME~CsF|8Ffj%OcVWq*~1JJKvQfPCLHsO6vB??n3Dlmw3->g88d* zg^Sw>lMLCJRc!l2DXWm#Xk-DHf!*u$gPHudX%sK4j>e+fn`v(|-JUHDFJLTMcvuD= zjiZWv&q@j`3W$ao_6|5-~3&-{{91-Xpv*c4)R_OaB$TwZn z8i&WupbJBx>R7BY91fjBS~?^?1qHANzkFfL5Id@DA+TZ$XiLF`)C z4n?bD(NH9a?}8WZ#@1I#?7PV|5;n6hCaWrM3Wj5q+fuI`!`7#Y?PIhG1fSWBL#!kg z_zSF6NQ}ToGP0SQ*cDNzIcmMh)(z1A$;H*x7~f(gR7`Hs4jaLZ)Ngq0jPz{AL=na; z7V(KOe|0{Z+!^{=-=iNa<6l)C*yf$LY~h?leE48FJ8D^v2hD5NWRl)I=ptTERmhJj zNCN#h9~PKVlAiWC!X)7T0V(k*Q1PwlP4ooGkxQL3(WC1gy|Y;J6F^AN^?EYt3&L7q zwBMGXKZEyPh8!)AYFpdd3P>7i*#c{?!5GP4FNkF@E+JQ zKn{S89|xrMZ}W$QUO^qkVK4WP6lrW(T1*#xbC{=47zfr(vyB6rOKLS8<=-C5a&Jo_ zzFyx$m|!-Dk1-mxhv*Z__ywo8A-_gSg2P&F>}Y*KL24zb8T4a3wKQcY(bLTyF`%^*y$Vfw%iD%%zWzkB;w}CT0)>%F zrV`k>GY=B3`_XR_Su9aW9iSHWY##^VINh5|0lY%t0koA%CD}sfd)vlReP8bascx=e z{!D}{d?moR2`mT%1Ts-9si#x-c$l^Fm&bb?Mly|n=auL-`o!cl3N^GP0Ggvs0O&H_ z4^D>H%>QHP{R@0OQu97jh%%_R2J%?$MF=)9V*Pbe)1}qNqwiHlyo(_i$9p|kb}4;+ zK#~L<=Jn&?-8jBovud07R_X-$>0J~*6)M7>v5_58hWBn`4|-G1e@0wf%L3t`*UjEO z5f}~Zr`&r;P6J{2L|puY<01fscQkJN7@&9C%g5v69pAwaJNaB-h{>I#?I2po5HJJZ z1kCi_xyVPj+rc`btHmS)T~=b zeS?Eco0rkPN=qo$F=e!f;j3Pk#8-^jNw>@a4fyzPt1g8(#7|99&V2vbI@!>dC8*5F70iL`o64` zk(zCVUL4vPH9}z1iF9igJujJW%wk_-E|L(}XC2$-jQzxhJMXdcA;7Y_LoRE2PSAnG z&q~i^Vnb&|G~lwZ@p#3EbLCJw=+~3sjRGQ zvq(1E71$e<(W+=wu*l-U#woSYNNh^h^5SUp6QHqnF^pLM+(faFE7+Ruj%=h{+{x`k zPc9IS{3zWK%Id7b6&0vH>J+i8ssui{sxrJGOU%g1u)P{plEhwXUFE08HX0g$G zMEMuY0v(ot1m>Rt$Lvqs?LA^@1m^94q$J$+3T!1PabOk+Y73>e?~3z#&{=e$+4J2 z5|&>gIZIeeK~Zu&nu*&oWY_~n#op9m3KajWkWklXkeo-i#IU2njhOB2LW^VgpSR zV;NrQbP|0T?b0R^D>GIglMo9tTBUxhW*Kde>C{}M}^%QcoLY;BG1G!DT7n{xw zu7n*yj~5?p(`+#m@xf4hT!HjTfUgVD^Z@^{*Jv6&Pj$cs<9`jvZhE_B)v92$yt=)7 zRWMRsX`;?>IYL1+jRK+ah(`AY%Y*Ristg3$+iwa%ZMb-6Re1=1U0uGH`eNll^!-mK zGX^HkX00hA3EWM7o9_<3qncV`!Duu##mQEMG3s*2!Av_{G@a;Y&UF3Dko-r+LnQ5V z)ftWWE|+W3t)UIao9@fPa0xQn?t>cb%~zdH3B||9QBbj%`h_C)S<>=@aeL0dk6rR- z+4es}@JpD_<#Jc=uxy|wgmQnHXm?FfKMvO!4KANc{K}zB4p#rwibSyqNMiR2*WMG& z!u2dF`q%^NjEUm6JRLfoc6a449^Ya?tV|PkWqL)ET;#4=&u$mLbhv4E<_m`Zv?28Z z`7<%{pw=}`E4UvvSgp)e`dihY6x`!DMmUEJP63Ex#$a|o3|>$DmPSAu!nN=*9w!or z=Tv2iK_TcIrx08f6W?VJ*hN_+e4hjJAvnzKuuwXP2dOSXahfXy72FRK^veaon&{J=ltK811EpW9B>ug;;Q(MyaBA# z;Uz9m(p}@s0nC7BO;&Z{G~uyf#-7X?EzX(s7W!lMek49AXfbhN9rN-`|Nc7n$h)wH zTp2-Xo5B^>MP?>Oyg(KY6QD(dKb&e_jsuK8v2CcSFsMfr#gysuP-aCnxq8xV75;Un zLQftc-JjSx<2cf2onF_2fc5KXwt5P{jOrKD>kL3~@+3Zu0^l6qk*)v>yD^>KoKbJ^ zjK8#`5XF@#M3JQ{0h|uI(&&OvlOG(`Mbv5_ii6*u4hhWXeDQ)BI9-C1Sy>GlB`(gh zsXYZS?kUI*{ZBhX)n?RxO6Zht~YZjKQ1eZcn;*Mn+4D8FoP!gcs?-%oJStJalGv1(h77m>P z(UL*>ebUbBs09G!XyOP5?;iVRmjO^3&G^%}_L1aTcP_M|I4&zBTBrRpMj20Gl(S}# z0a9Wlc)PUG04c!^yQ?k^cGx#inUFsbyET4p-zd_c9m+TyN7rx*L@hSk! zn{rYB=6;8QGyY88Yyg;NurOqnY?fp|m*!b%Y=$B)oe5Vv{=Wj( zh#W7&G8nC_bOX3_90x8B&^aDwTSgw)%GgI9xyp;G-1?OV2&SWWDeEg@+h>G<#e^^c z1e1G~@&Ml9Y(&$^Y^j$;GsN~8OT~s66U0@^YzD%}c29+^mVq&*msbY>G44aw$*$pz zCG7lkX5nHiU13|fcysv-5JfNL*JQR-hhm}Fq3JxaT-g)FhBWcUaw`(&*jvj{Jj;#6 zA~4TEzln->mUFtTh(DPp-d^s&OabZ9QkE%;?O@wZ%eI7~;z-onj02=Zrbh)uqlPuciYNOmk1t zW0mVBz)FTAh5Z=gS`IV&$>n@fK9I#VF%~UQPiG~;NO+;KZ}vk!4aVZV8VxRxJ|{FK zU5?U5`gy_9>Z+N*p(;Betj#U~5D32~crxk^w5BM@qv1u#jSr(Bth= zK0kpfo<5E$YChZ3E1gw;o+)oL@Dnjc6X}+DLgq@tFfgkvW_8XAZ72MvQ@i10Y~8d- zoJ`o~vUqVNuf$l>X{(NZq*#xF81?weEA8yV4w1JwGo$_e<$MvtGnQdg_+iXl_`T$K zbLCzTL8}ANVCd$?*bbC>)s>;jTN)?Ws^DkPnC1*^YRBy}flze_u5On%O1C8(w>D-( z)1ky~OlNN^unf>LRqd2U?PNuP8R?ykJYPX!s#TE&BeJ~-m|#eJXHZJk!lF{Vzv$RL zt}EjH#va;jM*+EB_{F8l9db#*)@KL*XQq98C(?xGh7`H_< z)givZY4Mm<{LGez)4+JP1kIDMI@2uvMxA(Q#|jp_YqtC$55T?n>Jo7T2lyznUuzY2 z*`4%WLOP#3;qc|Mm}$wBUT?v(ftFZxG*UIw0(tmwC=A#)6j^3X9aNAN1jd?RPH0J} zD(nb`B9$xCg_%vjQ$?LoP%kS(n5%M(t_W0BSA~$8g4gH4Mt4@=$f{;+508Ve5$!0n9_>SL4Nn!LjxyE9oqQJ!VM#@e3MAsyFzEJ#;J` z>HuJ|1{w_LwGwT$DB^|@%#3D1s^HX#4!gpE$|#7w|BAO6v}Z7l7sVICeI9sQd^-LX z?`!~VcthhXis9k6T?hkg>!sU`E(Fj#V7GdCsu6MY-vPHZlo?t>U#QQaJMQQp&oP^}q)VGE1!YM!oec1yOI^C9BtIb8w2q-XR7BqF zN*r1aW?2nXeq?knyDIz*FZ(?cn3wxH;VX{wzl@18&W~~fhWMw_$;~AbPfNX>925zxldM7?AI2s zUsuE0Gz9MVIRDi+Yik@w*{Vu|`u$xN>`00Q+dp)*1Y9pgg8f`E;JJTi!EQLtf}Lg$ zfCT%#BfNTWK@)|+fc>@ky9C&unNF07cT0C+9Ki89@_!=1))^ewd(FfD|4?9$$g2!} z(?4@yXE{KDo$((iu!psOp};=R|Cs{Y8Z`FX0d|-QQFGyop5Vj3&a_+_9_Pa@>2n!` z*mBh%#I{LQo&L_xxIu;uD3B@{IH3O=IxPP?AGY^R*KKXzGGY%h+Ct3FoA6l?Niy03 z^8cyrJm90M^7enut<%qx%uE{0WYX*OMv*|I_YNXLfFLSDsDcmy6=hdNuwelO3+mdi z_g7t4tZPSCU3)L9AQmiZ{?EC0CYglbzPtPW{d~;5b7wMS%02g-=l6Y{pZI7PwJLuG ziggRKaUXd7gas1D8L12u%Locz{u3=BFtHADK?e^zGQ-1SO%Q*L^RS}?Taq$ldHV6J zl1{{aFG(j#@-6|Or+O|WrJRUeL*?5tMC>CVVu#6&s(w}gv0ou&HN*dRrD9#=>MRwT ziXI@|jBBtT0IcEtNO-0ijuAVq5IAJXP&gY^}&Gk}+C?cM8&2!M0U{WEW>4Su#*?*5!keMfBp^ zXs!TC_A=SFif~G{Uxt#Mr(`JEt3b(~Ds-o0hia7UST3b;Om=}m{7kM)hS806zCqlo zk`{x21d@3M1@!D(P_lCj!d{J%eM)9mE3}<*Om;ES^_OQr*^2} zsDZMBaos+7Vm^2fjg}RTbCG)}y(q)Xu60VSMsi&~TXH03cDBaMPU^zUCNyR?sWG#n zA;Zj;1lk%YdEO)ZQxd)`P&O8jCruG>vwibi%_W5)Oo{&NEKA1w6K)a z5W}TjOJ7oy0zms*2B3YNN^g2CgNQFNi0BCI@Iv+%Vk%04@ro0}*+el@)ARY3^GR_G z&KnD@nc22eFfP(5TDUvc6Spt29!{h>{04o^psIzIr-?5LVvOBP`uf6i0npwGm|itM zDx05GIq&)oX5sm&R=wPBeqJh0ZIh)1Iy<|oKxb!<)!EsQ#?C%32?mXw{iDv#KB%#?TmU=ItKqW~ zIX=sVD|x|GXot?uJ`8rYT(z2MT{4ZbJrWXf;8|btA<(mPGxY3w!kPuOIPo2kR@2&< zZSNH=h(=;qMLk|f*LcZdyZGl~>C!^!FU7vdE2ZiNtOml-R9)aZo89kZ!X2fe#>x^0 z+t7*1=7peDrx*=*b8y*Djp))zvLHB5# zD)nO1bW08``@O?ahXP%*1>JE~HNY!Qpk!I0M#+9+o}4W=4WmjCrK5F7)>+}1Hy4mB zqI^%oMg9(dBL{3WNS2HY$m^-{csW~7oi)+edZau7l-)$hRTkq|fU;xh!^-M1$FR+A z_7q04VYDwBLLYX=h6UKgg0m))xESN41Q^yW4fjRY(}($wBKB9ImhF&}zQ`v0{?9ca z{G9;IYppxr8z0Mw**}ArJx685r6#4S%y3;ffY}>#U{-7{@rP4e3rSJe^3g;fvwd@r z*$_bHd>{BID}d9@IyF1E!oK*kF4Qa;Od9*(ayFD>vq;y1T&_X0s3gfiW|wQoY(;4X znRS^sGTT~U9@M~|u2eZA+iVn;8_8?p&JH1aSu1+j7LX5%g{ieed5(qJ%aX&$iVcNy zSEalvi^$sY*kwRu*V%bUHi8ZP?`(Qw&O-ErBuIxfpX?c}CC<9$5VWAPzt9NUR9Q(w>s?;*MumJkY0b#mKNQEHYWME0YL}ZiKigf^ z#%NhUp~{CE=B^Mbw}E`Z3I*@!dY^Q4iF)t1F`)iu)j!0Y?~8dg#W}N z^~q>UtUfj`*<7EFHiznC(K+eZeBfBA=469NZzl9yN~ob3cr*xeyvuMxXQ*e_6xwygziTM+Hkuy3fcx15cPn-cCP=_LYf z;H2$yVyh-%w=8>!7`F++B1&$g6EnPRM#auBx8pPo8vx#-D*Bu#tcHZWMq&@@5_WqJ z685W-cw_;aGq;6|Gq(@qDA-x``WXprth%Y!4RKAqu46g)ttz38*cHDuCL8D+S^CsL ze^7+Cs{G4aZf^iWK>zH3y22@fM{_{j#q=`74V++h`ZwEzOU-m~aWIyA55o2fnFqZ^ zehEnTma>N&jN`apYB+A6!E|Q>-B%*s%5mI9Iw$r=m8`0l+=X-yu?KGT`i9+-Pv~EJ zCUIAlHgXzlO`?2d%Q=B%84S!bQ0$K3(l>|7NfHKxD;9`_r53r6SzR3*E|OHZMh;63 zB=Y%Qjlxag7${ulfM^7FBkc8PcAuCheiRalMe=~)3cMt8Anuolec3VjWZ2OHhiy-7 zaAaBcxu$dmH<(q_t(S4T*vlk#M8ZH8v;?MYWvu?c}5c?z1bQmfWe&=*ZCVAl zr6*POQz~3VlZ%)7f2tSbW)hbfM>m1EmJ;P9jy{!5Snq-dS7@6B3^8X_r)aF|`Mk^xPBjFf&o~CCDv1JB-4#?cOcjn>o&qu9sxkCdB4GCY zD6vlsjW&Q%Eg%BAarlURd08VF92RjSk9b;cA`-kbSx(j7b}X)_6IDOeF;(OjN(mx3 z38@W?LoEZz_Lw*VojM5{Y48yE9(SO*&RHh$K8t(N z_c7Dxv^2&*pA`71G|kRWB|Mo<%?-8Gc<>A*;mC~wt3D==4YQC1GNGq#98YgLQ94K80>MsS>?_A3|xFBM&c~Yi76&uY@4Bk)E9}r66={pW;8p;#Uk-!XsJ~!wyui- zg9YAqGspjoGI45)fa9vmoS65 zj9g2_FK93tX+D|X4l)1YLTvHk4E=me1$XzpB=0w=s@X7rD5_aTuC2mf)Ia%l9-Uq< zp0$L1HApYYd_Imt){8qTGGBl<=M|X&um6c0U`x4uP(#=_H{WT()465@Gw8y-i#(B2 zO~4xCd!vrE0fSzQ5Csl{cc5+i$(WAB0*%6}Cc}V@#R|v?O}wr%&rc#4QBNb|HFn-k zbbHNE(!>l`HwBKx z3Mat{xzmW|3EYXk=1T{Wb^ksgx0+~#+#cakosh$)h-%tCVZV~8rtvj&3>iLpHElJu z`pJG8O?9oNT_wQFgVl9vu_%;^5meG{5ohaVv{!`vB3~(Yt)U&?rG{3m)zJ2nZqWKU zBAtgUg|kKXqJ!eS=obq6fuC2;mY{mJgxAHEkeTk}65=``NYBWYy_kqHe3?uiq4M{@ znPTDjG+&-f5k*D{^*Y!2nNu^m2BoX?8su3`Sh1*;1jFf*NJ%gqItg=`x5)YWDVd19 zMf|~ZxRt2*g76C03Ppa1`s1;A3M&;o(!&%JkN=^DAIJ&ZAJJLF>Mdfv6B*P0<%d>T3#&$`5Q6b<3QQ2u8CW9Az54>P1S{LI4V5Bagl5yd>SR!R4j*yo; zl>`u&r3EdNT~5aDNyar?Fqx$e4d}y-9jowC*>u`-#`&HThCrV1$Kc`COIFH`gi8No*0+ zjYxdHSx0|e!Xr84?;@J;B^N=bILV+rE*4!x>8qAFsvxgeIxxefcvNC39yP|KL5Tw? z8)%c+$@DlVS+}z}APX;KUCF||S+ejiAPcXAhW(C47EV5#EIhRfS?Kg{S2_cVY_BUC zS$JE|;Y1M|(86gwqlFjsgciQ+KozVyix%F^a?!%S+A?V2K%j*K{~fgO-$^8Ll#Tiw zt`OLI2Ul2M9XQ3#YBZ*hG?;_|E_*azYRw>px7$nB1UOT8VxAA&o*-??TcIO`6B&Z? zk3bae=^zSQa)`oWOC#E>EPK|{smxeyI$rpRB_qwub7**B#DSS07hndG$s8{XBeM!{ z;W(F!YuhjvS#QzcLTWx7F7!D!;tGcx`cAs=W5@5)g@x{-*j+|0u7WO%8HDz3bYYXT zlP(;t{x9gl1xKI@iJ9v#2L%W*OF?UNC^575x?{6ZQ8I8fCKo|$avc#t{K?%Dg6Q@9 z7YO3*l)R1JG%{^VbcgiG>CdmfMHtPMuoMUK)jEJAl%%|q<5&! zAJ&QxNpktahXu7#%9hALAYK4l>+cbWO`@=^L%vzqRlYe$g?C5<7uI)4u6#4f26G=~ z93SUFHi~BDn}<8(n;nAj!QY_}Z?@6vg;o&A97FurLw0Kz;)&337E0{hBWpNM{GNsb z2yvnYA@(HUyw?MOc!mxjzQh|^^avp4w|(QFt7PdHC;e0rWCPS2wCjd?)8rtZY3j|u zPW9$DT5V#7O2h%Nc(@sw&A7~6&k+`o?hmm3x@NPGYc?bnpvRVu)mbQxQGyOgw1SKwk{J@gwtmcz@|<&$?SAmBIxf3Ij-wk*IkoT@yo;j9+m z{Mexk7U8Z+JtY(i6YD&Q56abh>x7Fdq45XOBL%v>PO=hdZ@F|+9ot*y^CZuy!B$3g z01VMt4d=2NtYK{AY)SfO_819P2m_0`wBw6ng-Rc&@%!hzQ$%-H%WEn5mViF&r<4V; zX;x-c+LvvK5La)*eqv92)=N5|0f+l(&>nuRPTX0|UTVmf&ZOnhC^8zLzHA{L*;nrg zf6D`99E#8Y$!BFa90EItH*<~J@yA#wq{mL&D)-xaoU1<@^1>hjp5Yvr8(4+1!<`*b# z-WJU(YnwUFAdKU=CGS^|J0qmcuq@*doP!p(=EH1)%2tXXj~rulVe(%pNAJbED@Vr~ z|0CsSNk%z(&_HLHMJPv?TS$I7S2@~GJC&m=W8xAEL0|JuL9UN=9NPYm9BOma!RqqMqb-gL_e9)bv~*X`71nyz*~^ zz0oIv`^(v>G^Si)R(>V)G6N1+Q9O<*e)!l3TT7?nh@~nV{u&)7lF4srg#FQ4?~a`z z_!G%TWY){%cGfL$;9z)43=6dQCbBMq#alWN-%LtjnY9*L1gQr115S<`Z1%_e|6&c%k)KnxsKahRU(1De1*78)M(^h$kMA;1w- zS|-X{+-$RFsE1LDrUMp?Eg(#XQF%s*U;aTzC?fXS}$OFzV5)UI*P%MI8LZtr@)j=AMj1B=%Q# z5n1J6wuyDV=#L7+&S=Bt>B0=3Ft0!{Po(Eo88)j`$gYj&y~%WQljXFDhE^hN zX+UCRpIBYsn!!U5yzj@?>ZifN=|dg;O+kAgPT1|on27OcHg*(V|7BfaFl#OhrpIr7 zm;aLi`I{}vu~gTiH^VySRa*~{_0BOwAY*v+W`dL+yJ&COemWz<5h6swm4;(Tv0GU) zb3FEY+q>U+Ry)mB9X~N)~6wO)%*O*wXl8(k_7AV3fv_B(EMug^a zVxv4lQ@lel!H(xtJdQQ4<5@3{P&8c` zsxi31B-Z;eHeC^l6A0%VJYh2{##fj}P2SY9KcEwLVOT@|msyhyL>y$b_X!jEXyQyx zwL1D7V@2~PEFX9%U)9c&;Flu`@zk#+J3I3vC(nRmMZh>Zg9wQ!!ktzF=CJ#KlJ_OA zI^+*$)0?KEM+a9n0iMiRC=b)3?+hPJwO8GDl2A?L5LNC%3c{)x7)HrFn{qrMqh<(t zHsx@G@kU~tM;s}OO_*b_)gmkpl@WtIezENi9mfmDkYTuB8o;htyw1OV3VCH3iiQ3} z-Tl+XX+U0l;cPvY@;O^k>JkHb0@Y=L7(+(PXWkn$6l&6p4b#aZghOY6JTW$n*eQuk z`=D$5;@h@wI$<+^)8d7@Nkn$HN)vcy;&NIRsYXAk8FMe0cYzik83yhq$Jufk8?WU> z`si!;g6esZJiqcW>ZmR|Mnz1BzrR0RWj_i5lQEI)B>mLK?Zhd8BYT0$KM8uW;#*Vf zRY}QF^{1k(GZ5>DFvU9mYxZv8iY{_T79+zq2hF@<9}85qi^^95-QbY?bVwWtkyp$J zA-;AORqrCKni$!&f>k|@u$zRGCw-+z9wvq>#XWJJ9jQl2o^UMQ+&IoX<3di&@sv*< zCCP{*Hjg%f6$CUl0MOhEf|?h9L6Cb3U@REBT(AVO(LW>nq$J%z(ZtJ&j9Av2kn!;l-opu)MIT28VY$MyA6p&~ z%ahfAS8Pj&^o+}%SVY)oQY~u}h}auFsAiI|WJaqd!TO0X7U7bqxact+Lu(VUe*g+1 z!9>h0&aFs9zalk!w>A-Xi&cq)TeK&hP!zWqOr+eRH!tlL&1Hfo@{!DeKaXe$Bw`*> zc8ZmW*dDnu9CC?L*dRv*8`@nzEs=pDYm?oM5N6kHvON$>wBbQvL_3~{h!4@^q3kBH z5`q780^Vm4T3qC2@lMlD(eR=`(b@BA2Wf9*uX-gWpG^Gt8UW8WNUwW&@Sysp*NC3$ zJItA2O0=*vO>!E~C34sxH4C;&IB%z1Aq|PeXVxK|;zo&HX7SctZV@3dokytDhb0fUZ}L#zSB6qSIvbC-x~9|)a;cw8Fmwq*$|s(dyudyBC=w! zp3N}WSe|W@>2k{Y_cE|rZ|`6%RqI7Fo=gtn2I4c^>>bA?^kp6L|2~w&xcr~Kb(`UU z!t;V>wsAhMBR7N()!+5jlXMVG!cTZS0XIo#Qa@zFjGF>D-Eat(Vc4%VpstRf5jeUH z9)ApRw+q}&sf8?N^LXb1g^SN6lDlpfheyWpJl-Z8i!~9?Vu%?q)NUezFFKgm+Iu!n zY$3ELG7|Xwc~p53SsnQd2Bv)x^IQ&xv@t?`4 z+;2W|)nU0{-K(*875*8o5DWp!eulG}9FQ{G~o z1!HH98zuwpUfW}k?L2zO)^#Z5(Z1R794zidg!i*Jm5^d`N6tue9XW%KL)?MUWgyy# zO8`bheJI-5!S9~%u!xq>H8W?TRUMTa=r!c7i-)!g1aYk`lAd+>zKDLW?qDpv0j0fp z2(&hTvpo_9vpo_9vpo_9vprIEBDqH@Nu=foY>%|SmkLU~vC}sgPvTX| zdn9&=(cp_;VjRnV1o@9J|B>WBs`!rpTdSsh(O(oF-W?ZTV-`mnMddfMqwZoe%tu)l zQ#77Rp%R#F@qklLGfgd1_Ou|znwwQ%%{ayUvw?eBc%cmQNX=1|aYT(K&FN4G=BCNY z1V2wjy3L}~kB@at3f1sO#k{}ZUze-0iJ{*Mxsjn72Kj|q!ys=X84MCv3M*NLzbb-4 zQL7dky4jc$!I7}$vXF5tH}q61XCb{R+}_C4&&>^8Y3Xk|tHgb;HGw*3rhK)pk=sztvaOBq(z>)6IdZdHG7`U3s_?oPy zY>35V-9RnFum=<}jBE8?!O2F`zDP*|FX@YorA0OI)OMYWjC3U&d$1{%bBe^cEXZf12_H!F-b-kwZM%GEcss~Zn#Wm z@Wc&_mII5uAq;0DjrN)C(2Ba1x?)~>Cl#(VafEW2aeWq{>_bY|1a79%IHPYVpj5dx zY0(o#86mH`<*(?Y9qx^6%xKoy(W%+*oTqi?<*NM+=B*t(e-j4l|jw_)#6G)b;$+H9h}-1g_`bgS<$@k5067jUT0daPzQ$ewM@Z zd?T7O{wtu8&ZkN@pwgJ2_GMpNT9+dv0M{{9a0hPh9u&|cQh9h#Ko+U|k#?8zKk;0g zP0i5B%GOMpKuU_^J-WT1)ZcPi-u_7%@4!tJ3m!FCmNk!A4{gv^>ZT4iTalK zyhJ3Fm>WT^NS3nHlT6=7l4;PE(dLnvmwNpd>ow*6N+|a`CHCHtmHVH5Pr2vxDh&vaQ_p~ z%R>W2g)9pGQrGU? z(E%U>2LM&O!wa>3Cp)cz_~Ks}&F;vk3bHMad|hO?EDwMxeXmH|h^Cf;4DW!gf-UOB z@YsCx`m2z9(fdlFrB^KUML#cHN-p-x?^e=!FOF3DQomIe)g(uIpj6!~Vu=tG|9=o&oISrbpbAhP=^ zv1+)la=0(Oo<0WJCW(?&5A}MBQU&Cw#KL|LNHn*}JCeIoNwAxI$kBGj)uioP&KSsAI4+A(2~Yri{@zv1=Ivjp|Uw&?s1~jInH@xV_oE zCQHKR=CyiS`cSD$9Rncjt+PAo7@Z*OGZUR-KbYN>gqMC<}QWDbyBviO9|~dZ5iqpUpQr ztV-{od(ok==`pM(O$c|!xzwL+v6d0^2Kh-b3)26)W1@@DEtO&Q2`7<#em@*Czy)2UcG zeTivF1SydTFt&J0vGmR}m#VjT$v`LmMnB>%1M|~K9>aNU{<>ruxNSPhfnnrTf~<{| znhITYkQkjvH{5cPC?KdI8vYuzs?!`~Um;b~3I3F+FPfMG{3)Q}LnhU3_M*KIf?VZZ z60>Ch)T{GPWplm7_T(O!UY*Z=47l(ICEpX+)%mxu(UyhGSGZjnZ3#w0%ge@+#a{NH z(Szd#_PiNMn(LsxKX0y4hMLJKBHH?Sl)t;#ihx0y>`^~dY^zJhA^tOxn&lSXH_k>^ zD^$6@K$(g7w`g%5T&n6&y2MPF^fl*PARKH6a5aNRif4Jjj+hJ##afOCmdsjqF{UZt z9y3uOh-Sf*r7wunKp#DER|}-}XHr0_i4$?;?Gh(fHENaj=+LoMaP&M)d=*)NfE?t+ zpu&EBDjiOP@eZFs0>0=OM7^vdJL3lu*9P>l@n|xImO=1It|aP@-Pyt=uq3C^iUxq; zDm+Jc6|rW<^ixr)UX5Z3M)hlEg2vS%u?G(p0{+ocKy`AuH#4~FL@%c&oTTsXCV~Ga zxwij#95|9~vqkF%#LPH#uzN0(izL9RLSFL$uM7MbFV5|^l+N-9&_YHB?zML0l(p zI6S#~eKxuKCD9;>|BK-6X_^@#C$yV_5BHG9Zn522KZK-R3HI@Vc1LV?PvvasK$qBV zd)qNmDkrs@sDf^X=qa{4uWhfwF9VO}CU*ZK?6-7{?ADF!nKai6WGo2AX#&$T5^5NZ zqhCg)nJnoH@cw}io)Gk<(qqQ!dBCVUf#|Hlgtk)q)h;2g9UwVxO|ujqW=GWPWKCuI zLhi_VeO%bXJ-3h`CeB^lx>C)B#C4@91BvU%=e_7W+93{h6`lkGbf+46GWa-Yzs}IeB5ND8qm|Rf~LW*4g;GMWIs2Nqo(bnUMH;o`}R(ZF`(VJ`^dI^DRdY;fX^ z>!LdCr=7BsEi&n~6M4)k{8!qO#anv8`o^!v%jV@x5>co#B4zf%9#Uo_4z`@Bvf|fNDXr@wx<0nvKWmQU)>}9C#Ffz)khmvY(z|v}#4UpVFX525S?gz;xQruROEP^_9+K2`N-`3UUh5*?%JfNj zKkXjuzh9z)qengAJ>|O2VE;$i=~+SHZ2}l@GkC&Gs=qzX9Ks*-WuE7Mk4v0kr>~*^ zx9W>2svx{Whhhzi=^>}?Tx!S{bE%B4D9=*G^F?-n8u!Kgs1>dw!#%01ZR{vTvDK>* zmXhNN`J@(W!Oheor&UiX;o|E^C9Q&++lT}ZbZ~B@DxX0r;+Wm?e{nSy zif0PgD7Ih8N^};AgrkY;3bWYUCP7<6ekjuwF3c?|qRHKb`U%mE-5y%8_lu-1pbwLTvdQx34U<+l==VsVzmamaeW|Rzy~0=eSH_mlt`+D_=~bn&V#cj7xuIA*%Qzqq zjt+K2F$*pyi!E#mCdD^6RotvCk4GFR`-W4jP^vCxmzqLTuxO6Q0!BZA_{zyGM&`r( zvBJ!5GZ+MbT9=vxPdZc)jK*&_6v|ivTxtqjg4kM^wGy$=S#%wy$MIAs^c!Zymz#WQ zq#_I_U04!~d91e5gJVHoGF(xD9k}V{!EkoDiN6=aWx0>4s&=)-q!(XeaZO6vs!K`AE)#3(RFahA&FpO97@w{oo$c$? zkQQ?dX>mqF@_s3vDv9S4V$Xf2d;d5;4yp)XNTXTpp+ z(rhQ6mXjMR3{NE>)RAA~F8{~nw7uN&R6>iRUSW5PxuQH9N8MDBiKD)*&KJ5TQQ(RG zAhSD+&YDytV$MJL%Y>ANtT@K)BgU(Y_!KONyDMjl?J@Fro;b_NZ6WL%M=7@DzPQ!A z1UqT+zFqi7BXg$AHL37E!S5^<$J>2ma{sOdlDC7IXzI6u_J*qkN7Z*ho?Nypaac6< zkA~c6YNx-zm}4NBWXz4Gb_TmdQ$L-6e`IAs>M5FfaJYwP>hlI*YOvdsiKc!fnd=|< zGLi9*%t|@RW~Kg(XzG8@Kk}!%Bj-}v(_Q={Yn`mZrqT+NV2_f$7ILk*qYU*zD&rqn zJH)s2sy>4F>EMl-OzPh=kUSYXycp$}7mXp3`XG{WwMgnEM`y}Ur>8Z~$bgX<&qyq) zg?8+BW+JH<_2rS&^ZSzRBQ@8^Y1YGCBmMi39nla>BTq(z<1(StyOrO!jZ8keCKY|g zl<|#hwF)iWd?S|~t@}o1>jkG*sq(Lbsn6=yQ!q8^O-HhhY-)M}a6?YbDc3(TYw!{M zBM&ys))TC^_Bz|2X*d0(`TxK@BJ_>B0(Zyt9FTYOXqbw3_HX3<_H_FQ0<62*M;eDp zjhXmr%|615QH!Z?NZ`e&Kg3at>PWNxKu||Xjs^X!_6F3W?iI_Zo4MYI2{p}Q^YzER?UR5&5#<+E=y_Z}c(ll0F@ z$yCJNMbW2$(1Vqvs0dRZ^2Zi?s;QnnF^H#He_x5oBztqTluShu4>yw~u#Y72*%N~% zv-=12Vt=1y$eKNdH^S_}q@!K^QV~>Z$Q1>UML!arn~%e&1_}0A2}F^{2GIK~j}Bp% zdu85SaNqbwsG-}1X9@(QYwa6f=7}EPSAA=m)HoK)j_}7t;&TmH2TG&r*zHX=Hamjs z=%M0CmF(q+tRUuX10ryTM(mM)94)RR*0;t89~HA>#v47c56AF{BWPtQHl!jFwxs#Y z(RSQ8Wa|AWl3Gc;T#Mh}ev8kK;^hb1G2<(u;im@km&@mja$RQ5I6IxfOf$5a!rPvm zH4du9tP%f^XpdJMD`smBer36c>EaeGqMr=H)<&*%h)=_Z>cHC;B2&kU z(=6Dtk&o<>Zl6IF2-<5t(d6iUY@fB9{DI;~5WDq;n`T0Y&c|-LF$O@tjfj`h#b{*} zO-0Hl2ee0vN%1teW_}pNwqZ(&spmdx>G5fJxl^%QW{u~vZ2X&*KnwlsH6%bM-m@Ut z`otNi?C=Bl#2223&}tv{8-n5FD!hlUTN_x(z#?f&i+q_uJ#IA{yr%ttH+`px)h1a2rrn%GrFgR8EhH$VC)G`>qxp11+{~+*0(8tinQA-i#B3V{2 z3`dZurq@@qP4D6f9MW?`h7eDvlx)bf>qXl;KpethJ@1koJ?{<{XQeZq{EL62qkG?B zevqD<-yI}0M@IX<=zZ7S6LMJdJ4fn%dRXr}@*EWoZ-4h)S+1I<1ruu^S2miM>$6=u z;05|5as{*y2Bgj=cni8V!Hb{?UOQhNU(?Y94<*ya+%98ku1 zgxS?9?ZR*xYNr?ZYR9C!Z0q}Y#=zZQScp$3pgqe(O@jJj(9g+^ceH&VuXJVz=bSSh+r7$Klv-c52zL(4E_qL#VsP#i6X1Z9ihPHd(n;n2;Z zfL`oN1a_{VNNWTP&~W~Al!#ZvLXjl?-$c&mO~2xaaH2@aSI&c+{|5n;4Xsp+u%vLT zjSNU6;#~HRtU}j3IOD>+>122*5gp=H5mcJdZZIB$f5#U+lb~r{+CWnV-l{x3K#-RQ|qAwf7pGYv8I>>*2EpMH75hTAkRR+3ARSuz6 zjenXKg}z5sK0(9}W$X7uScU$1=NQcBRIr%(qNqLO&77qQqiGeIRCl=JW>Jzf99B(M zHg|Q$or2g6o<=g~NZCq9?xS~-;lHXX_m}BE47{qmN+Qb-t2Qh(XVR6P)#}RLMA&HP zO_bv-@?07C53-d!79_7^@@2LoJ>QZ~#_rHwJD7-UwTxpIo1*MOGjB`~!?v8+t^9VKve{%0TN!p|&&Ndt<{4PLqM-=1 z&6M9#isPk<-7+ANgncr=Z>_|40c+xDz_bn_@d(yQoQZ43g=RDtSc5N7IC7r$TajQq zY`18?fvVCH>WoI6>}(|A?w0Z3T6Q{mC0wY*&io}0J0gFjC27o{Ricvvl$qfH>^QdfdD zbxOf#RHIElLv^jY?%#{+{^cd(H0rcTe!olHd&9tz+`M;kwa~lDOWNgSbDI}7!zGp5 zE32y@K^`s0>6JCf*riw2L4OxP;wIy`7z#9tOk{Gy!sbQC=Li#|lC<$?ciek2?LnIO z+%TsB7qX?xwYFJgHud-E?3;BpkG|0xXT3`uW$)7ePwK?^Wg27MW8bW?5Fpd}G>#g>r2=* zrLwdfy{0`;c1fv2T1v>v`4ZIHv&*X34_-SCuzv=k>~k-_k4*t%_IGa?m+4;eYTfM~ z^=dNRU0%)Ic(qp(<}T6?o~<8T^Quq-Gg=PpEX)fyI*T~tzo#61r)0@ z#(OQ)m6InOC^Tpt?Y^kVcC@qZR>@?Ou(C4KW?`I3+FBbyMQOjAUy^nZE`mHy2BO@s zoHa;qDCB-6KmKkl`YcrG-3gYGotRN%aMek!CAfHVGi9TZK10SYMtk?UQM12g(Wf=` z{#Vz)K!lf$v8?@pu6|0(;eN_TrT4npDf?Qd`~7~+P5FS82qX$Oj9yk1gg>we0cxu0?f;Zv@9LY^SfH6hJXIW^z*if*YKZCMP@3M@69 zrpk0gH&yNk>ZZzusC`CVZ}aQ&*c4jsz79|2P%=(AufHB)81Vk(Ad4W`Oev|0g4GSjOFBQ#T`q?;-cpaxUr9?ewg;HJtQy_wNz za5dlP1oOVW6P@6h0P}{ zI(AXFCS}*F^w;8^@<9rF{l^NAy?;xBm2&guLFeFGBvxr97Sxa<*IR_&6lt@?RE7eZ zY7Ymk8}Uai!UvWtw^|yAT5bKwRtYX$GkeY$xO8dr>OpAE-LkZ*rMjV}&R^MI9thvg z#2+n<+@*rco#AY435|w8WaE1T2siCPr>z$X4v#dMZkquNYsiR}s%AE$aOVb9x!`Qp#pT(gou zw_}fhKOe1_wz-GnW@HJWKbA@l6{*|Hgyn8>W?%|N(UV*E6t$ww4yZ*_b~VWvn~wkf EA2ntTi~s-t literal 0 HcmV?d00001 diff --git a/omnicache/omnicache-spark-extension/log-parser/src/test/scala/org/apache/spark/deploy/history/LogsParserSuite.scala b/omnicache/omnicache-spark-extension/log-parser/src/test/scala/org/apache/spark/deploy/history/LogsParserSuite.scala new file mode 100644 index 000000000..9ed148c81 --- /dev/null +++ b/omnicache/omnicache-spark-extension/log-parser/src/test/scala/org/apache/spark/deploy/history/LogsParserSuite.scala @@ -0,0 +1,45 @@ +/* + * 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.deploy.history + +import java.io.FileInputStream + +import org.apache.commons.io.IOUtils +import org.json4s.DefaultFormats +import org.json4s.jackson.Json + +import org.apache.spark.SparkFunSuite + +class LogsParserSuite extends SparkFunSuite { + + test("parse") { + val path = this.getClass.getResource("/").getPath + val args: Array[String] = Array(path, "eventlog/parse" + , "application_1663257594501_0003.lz4") + ParseLog.main(args) + val fis = new FileInputStream("eventlog/parse/application_1663257594501_0003.lz4.json") + val lines = IOUtils.readLines(fis, "UTF-8") + IOUtils.closeQuietly(fis) + val json: Seq[Map[String, String]] = Json(DefaultFormats) + .read[Seq[Map[String, String]]](lines.get(0)) + assert(json.exists(map => + map.contains("materialized views") && + map("physical plan").contains(map("materialized views")) + )) + } +} diff --git a/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/MVRewriteRule.scala b/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/MVRewriteRule.scala index c7047c173..f2cff01d3 100644 --- a/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/MVRewriteRule.scala +++ b/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/MVRewriteRule.scala @@ -139,4 +139,4 @@ object RewriteTime { statFromStartTime(key, startTime) } } -} \ No newline at end of file +} diff --git a/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/optimizer/rules/MaterializedViewAggregateRuleSuite.scala b/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/optimizer/rules/MaterializedViewAggregateRuleSuite.scala index 23a783128..71f232d7a 100644 --- a/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/optimizer/rules/MaterializedViewAggregateRuleSuite.scala +++ b/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/optimizer/rules/MaterializedViewAggregateRuleSuite.scala @@ -162,7 +162,7 @@ class MaterializedViewAggregateRuleSuite extends RewriteSuite { } test("mv_agg3_3") { - // group by column(is empty) is subset of view,,additional agg on view column + // group by column(is empty) is subset of view,additional agg on view column val sql = """ |SELECT sum(c.integertype) as _sum, @@ -232,8 +232,107 @@ class MaterializedViewAggregateRuleSuite extends RewriteSuite { comparePlansAndRows(sql, "default", "mv_agg4", noData = false) } + test("mv_agg4_3") { + // group by column,agg is same to view, join more + val sql = + """ + |SELECT c.empid,c.deptno,c.locationid,sum(c.integertype) as _sum, + |max(c.longtype) as _max,min(c.floattype) as _min, + |count(c.doubletype) as _count,count(distinct c.datetype) as _count_distinct, + |avg(c.decimaltype) as _avg + |FROM column_type c JOIN emps e JOIN locations l + |ON c.empid=e.empid + |AND c.locationid=l.locationid + |AND c.empid=1 + |GROUP BY c.empid,c.deptno,c.locationid; + |""".stripMargin + comparePlansAndRows(sql, "default", "mv_agg4", noData = false) + } + test("mv_agg4_disable") { val sql = "ALTER MATERIALIZED VIEW mv_agg4 DISABLE REWRITE;" spark.sql(sql).show() } + + test("mv_agg5") { + spark.sql( + """ + |DROP MATERIALIZED VIEW IF EXISTS mv_agg5; + |""".stripMargin + ) + spark.sql( + """ + |CREATE MATERIALIZED VIEW IF NOT EXISTS mv_agg5 + |AS + |SELECT c.empid,c.deptno,c.locationid,sum(c.integertype) as _sum, + |max(c.longtype) as _max,min(c.floattype) as _min, + |count(c.doubletype) as _count,count(distinct c.datetype) as _count_distinct, + |avg(c.decimaltype) as _avg + |FROM column_type c JOIN emps e + |ON c.empid=e.empid + |AND c.empid=1 + |GROUP BY c.empid,c.deptno,c.locationid; + |""".stripMargin + ) + } + + test("mv_agg5_1") { + val sql = + """ + |SELECT c.empid,c.deptno,sum(c.integertype) as _sum, + |max(c.longtype) as _max,min(c.floattype) as _min, + |count(c.doubletype) as _count + |FROM column_type c JOIN emps e + |ON c.empid=e.empid + |AND c.empid=1 + |GROUP BY c.empid,c.deptno; + |""".stripMargin + comparePlansAndRows(sql, "default", "mv_agg5", noData = false) + } + + test("mv_agg5_2") { + val sql = + """ + |SELECT c.empid,c.deptno,count(distinct c.datetype) as _count_distinct + |FROM column_type c JOIN emps e + |ON c.empid=e.empid + |AND c.empid=1 + |GROUP BY c.empid,c.deptno; + |""".stripMargin + comparePlansAndRows(sql, "default", "mv_agg5", noData = false) + } + + test("mv_agg6") { + spark.sql( + """ + |DROP MATERIALIZED VIEW IF EXISTS mv_agg6; + |""".stripMargin + ) + spark.sql( + """ + |CREATE MATERIALIZED VIEW IF NOT EXISTS mv_agg6 + |AS + |SELECT c.empid,c.deptno,c.locationid,sum(distinct c.integertype) as _sum, + |max(distinct c.longtype) as _max,min(distinct c.floattype) as _min, + |count(distinct c.doubletype) as _count + |FROM column_type c + |GROUP BY c.empid,c.deptno,c.locationid; + |""".stripMargin + ) + + val sql = + """ + |SELECT c.empid,c.deptno,sum(distinct c.integertype) as _sum, + |max(distinct c.longtype) as _max,min(distinct c.floattype) as _min, + |count(distinct c.doubletype) as _count + |FROM column_type c + |GROUP BY c.empid,c.deptno; + |""".stripMargin + comparePlansAndRows(sql, "default", "mv_agg6", noData = false) + spark.sql( + """ + |DROP MATERIALIZED VIEW IF EXISTS mv_agg6; + |""".stripMargin + ) + } } diff --git a/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/optimizer/rules/MaterializedViewFilterRuleSuite.scala b/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/optimizer/rules/MaterializedViewFilterRuleSuite.scala index 473d3fde1..988b3180f 100644 --- a/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/optimizer/rules/MaterializedViewFilterRuleSuite.scala +++ b/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/optimizer/rules/MaterializedViewFilterRuleSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.optimizer.rules +import com.huawei.boostkit.spark.util.RewriteHelper + class MaterializedViewFilterRuleSuite extends RewriteSuite { test("mv_filter1") { @@ -273,7 +275,101 @@ class MaterializedViewFilterRuleSuite extends RewriteSuite { } test("mv_filter2_disable") { - val sql = "ALTER MATERIALIZED VIEW mv_filter1 DISABLE REWRITE;" + val sql = "ALTER MATERIALIZED VIEW mv_filter2 DISABLE REWRITE;" spark.sql(sql).show() } + + test("mv_filter_rand") { + spark.sql( + """ + |DROP MATERIALIZED VIEW IF EXISTS mv_filter_rand; + |""".stripMargin + ) + spark.sql( + """ + |CREATE MATERIALIZED VIEW IF NOT EXISTS mv_filter_rand + |AS + |SELECT * FROM COLUMN_TYPE WHERE doubletype=rand(1) + |""".stripMargin + ) + val sql = + """ + |SELECT * FROM COLUMN_TYPE WHERE doubletype=rand() + |""".stripMargin + compareNotRewriteAndRows(sql, noData = false) + + RewriteHelper.enableCachePlugin() + val sql2 = + """ + |SELECT * FROM COLUMN_TYPE WHERE doubletype=rand(1) + |""".stripMargin + comparePlansAndRows(sql2, "default", "mv_filter_rand", noData = true) + + spark.sql( + """ + |DROP MATERIALIZED VIEW IF EXISTS mv_filter_rand; + |""".stripMargin + ) + } + + test("mv_filter_if") { + spark.sql( + """ + |DROP MATERIALIZED VIEW IF EXISTS mv_filter_if; + |""".stripMargin + ) + spark.sql( + """ + |CREATE MATERIALIZED VIEW IF NOT EXISTS mv_filter_if + |AS + |SELECT e.empid,e.deptno,if(e.deptno<2,e.empid,e.deptno) as _if FROM emps e; + |""".stripMargin + ) + val sql = "SELECT if(e.deptno<3,e.empid,e.deptno) as _if FROM emps e" + comparePlansAndRows(sql, "default", "mv_if", noData = false) + spark.sql( + """ + |DROP MATERIALIZED VIEW IF EXISTS mv_filter_if; + |""".stripMargin + ) + } + + test("mv_filter3") { + // or + spark.sql( + """ + |DROP MATERIALIZED VIEW IF EXISTS mv_filter3; + |""".stripMargin + ) + + spark.sql( + """ + |CREATE MATERIALIZED VIEW IF NOT EXISTS mv_filter3 + |AS + |SELECT * FROM COLUMN_TYPE WHERE empid=3 + |OR bytetype>1 OR shorttype<5 OR integertype>=1 OR longtype<=5 + |OR floattype in(3.0) OR doubletype not in(2.0) + |OR datetype between DATE '2021-01-01' and DATE '2023-01-01' + |OR stringtype='stringtype3' + |OR timestamptype is not null OR decimaltype is null; + |""".stripMargin + ) + + val sql = + """ + |SELECT * FROM COLUMN_TYPE WHERE empid=3 + |OR bytetype>1 OR shorttype<5 OR integertype>=1 OR longtype<=5 + |OR floattype in(3.0) OR doubletype not in(2.0) + |OR datetype between DATE '2021-01-01' and DATE '2023-01-01' + |OR stringtype='stringtype3' + |OR timestamptype is not null OR decimaltype is null; + |""".stripMargin + comparePlansAndRows(sql, "default", "mv_filter3", noData = false) + spark.sql( + """ + |DROP MATERIALIZED VIEW IF EXISTS mv_filter3; + |""".stripMargin + ) + + } } diff --git a/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/optimizer/rules/MaterializedViewJoinRuleSuite.scala b/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/optimizer/rules/MaterializedViewJoinRuleSuite.scala index df7a1f255..da9e4faf2 100644 --- a/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/optimizer/rules/MaterializedViewJoinRuleSuite.scala +++ b/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/optimizer/rules/MaterializedViewJoinRuleSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.optimizer.rules +import com.huawei.boostkit.spark.util.RewriteHelper + class MaterializedViewJoinRuleSuite extends RewriteSuite { test("mv_join1") { @@ -150,13 +152,15 @@ class MaterializedViewJoinRuleSuite extends RewriteSuite { // view tables is same to query, equal tables val sql = """ - |SELECT e.*,c1.stringtype + |SELECT e.*,c2.stringtype |FROM emps e JOIN column_type c1 JOIN column_type c2 |ON e.deptno=c1.deptno AND e.deptno=c2.deptno |AND c1.deptno!=2 |AND c2.deptno=1; |""".stripMargin comparePlansAndRows(sql, "default", "mv_join2", noData = false) + RewriteHelper.enableCachePlugin() + comparePlansAndRows(sql, "default", "mv_join2", noData = false) } test("mv_join2_disable") { diff --git a/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/optimizer/rules/RewriteSuite.scala b/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/optimizer/rules/RewriteSuite.scala index 8ddc9dac1..3ba4b99a2 100644 --- a/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/optimizer/rules/RewriteSuite.scala +++ b/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/optimizer/rules/RewriteSuite.scala @@ -15,7 +15,6 @@ * limitations under the License. */ - package org.apache.spark.sql.catalyst.optimizer.rules import com.huawei.boostkit.spark.conf.OmniCachePluginConfig diff --git a/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlParserSuite.scala b/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlParserSuite.scala index 7b40f3111..fcf41c77d 100644 --- a/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlParserSuite.scala +++ b/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlParserSuite.scala @@ -568,10 +568,10 @@ class SqlParserSuite extends RewriteSuite { ).show() val sql1 = "SELECT * FROM column_type" comparePlansAndRows(sql1, "default", "mv_create1", noData = false) - + RewriteHelper.enableCachePlugin() spark.sql( """ - |ALTER MATERIALIZED VIEW default.mv_create1 ENABLE REWRITE + |ALTER MATERIALIZED VIEW default.mv_create1 DISABLE REWRITE |; |""".stripMargin ).show() diff --git a/omnicache/omnicache-spark-extension/pom.xml b/omnicache/omnicache-spark-extension/pom.xml index 402e83aff..287123740 100644 --- a/omnicache/omnicache-spark-extension/pom.xml +++ b/omnicache/omnicache-spark-extension/pom.xml @@ -12,6 +12,7 @@ plugin + log-parser BoostKit Spark MaterializedView Sql Engine Extension Parent Pom -- Gitee From c6b8e8c54e87bef23d56ddf5cd3fd8f008d76463 Mon Sep 17 00:00:00 2001 From: Xteen Date: Fri, 16 Sep 2022 14:54:59 +0800 Subject: [PATCH 5/7] add Suite --- .../spark/deploy/history/LogsParser.scala | 4 +-- .../deploy/history/LogsParserSuite.scala | 34 ++++++++++++++++++- .../boostkit/spark/util/RewriteHelper.scala | 2 +- .../MaterializedViewFilterRuleSuite.scala | 4 +-- .../sql/catalyst/parser/SqlParserSuite.scala | 1 + 5 files changed, 39 insertions(+), 6 deletions(-) diff --git a/omnicache/omnicache-spark-extension/log-parser/src/main/scala/org/apache/spark/deploy/history/LogsParser.scala b/omnicache/omnicache-spark-extension/log-parser/src/main/scala/org/apache/spark/deploy/history/LogsParser.scala index ce2d438a7..fa4469720 100644 --- a/omnicache/omnicache-spark-extension/log-parser/src/main/scala/org/apache/spark/deploy/history/LogsParser.scala +++ b/omnicache/omnicache-spark-extension/log-parser/src/main/scala/org/apache/spark/deploy/history/LogsParser.scala @@ -292,7 +292,7 @@ arg2: log file to be parsed, eg. application_1646816941391_0115.lz4 object ParseLog extends Logging { def main(args: Array[String]): Unit = { if (args == null || args.length != 3) { - logWarning("input params is invalid,such as below\n" + + throw new RuntimeException("input params is invalid,such as below\n" + "arg0: spark.eventLog.dir, eg. hdfs://server1:9000/spark2-history\n" + "arg1: output dir in hdfs, eg. hdfs://server1:9000/logParser\n" + "arg2: log file to be parsed, eg. application_1646816941391_0115.lz4\n") @@ -318,7 +318,7 @@ object ParseLog extends Logging { val appId = matcher.group logParser.parseAppHistoryLog(appId, logName) } else { - logWarning(logName + " is illegal") + throw new RuntimeException(logName + " is illegal") } } } diff --git a/omnicache/omnicache-spark-extension/log-parser/src/test/scala/org/apache/spark/deploy/history/LogsParserSuite.scala b/omnicache/omnicache-spark-extension/log-parser/src/test/scala/org/apache/spark/deploy/history/LogsParserSuite.scala index 9ed148c81..d686b0e37 100644 --- a/omnicache/omnicache-spark-extension/log-parser/src/test/scala/org/apache/spark/deploy/history/LogsParserSuite.scala +++ b/omnicache/omnicache-spark-extension/log-parser/src/test/scala/org/apache/spark/deploy/history/LogsParserSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy.history -import java.io.FileInputStream +import java.io.{FileInputStream, FileNotFoundException} import org.apache.commons.io.IOUtils import org.json4s.DefaultFormats @@ -42,4 +42,36 @@ class LogsParserSuite extends SparkFunSuite { map("physical plan").contains(map("materialized views")) )) } + + test("error_invalid_param") { + assertThrows[RuntimeException] { + val path = this.getClass.getResource("/").getPath + val args: Array[String] = Array(path, "eventlog/parse") + ParseLog.main(args) + } + assertThrows[RuntimeException] { + val path = this.getClass.getResource("/").getPath + val args: Array[String] = Array(path, "eventlog/parse" + , "application_1663257594501_0003.lz4", "1") + ParseLog.main(args) + } + } + + test("error_invalid_logname") { + assertThrows[RuntimeException] { + val path = this.getClass.getResource("/").getPath + val args: Array[String] = Array(path, "eventlog/parse" + , "xxx.lz4") + ParseLog.main(args) + } + } + + test("error_log_not_exist") { + assertThrows[FileNotFoundException] { + val path = this.getClass.getResource("/").getPath + val args: Array[String] = Array(path, "eventlog/parse" + , "application_1663257594501_00031.lz4") + ParseLog.main(args) + } + } } diff --git a/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/util/RewriteHelper.scala b/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/util/RewriteHelper.scala index 18f2db275..a3167bbc9 100644 --- a/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/util/RewriteHelper.scala +++ b/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/util/RewriteHelper.scala @@ -456,7 +456,7 @@ object RewriteHelper extends PredicateHelper { val input = plan.inputSet val missing = request -- input if (missing.nonEmpty) { - logWarning("checkAttrsValid failed for missing:%s".format(missing)) + logDebug("checkAttrsValid failed for missing:%s".format(missing)) return false } } diff --git a/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/optimizer/rules/MaterializedViewFilterRuleSuite.scala b/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/optimizer/rules/MaterializedViewFilterRuleSuite.scala index 988b3180f..0f9be71cc 100644 --- a/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/optimizer/rules/MaterializedViewFilterRuleSuite.scala +++ b/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/optimizer/rules/MaterializedViewFilterRuleSuite.scala @@ -296,7 +296,7 @@ class MaterializedViewFilterRuleSuite extends RewriteSuite { """ |SELECT * FROM COLUMN_TYPE WHERE doubletype=rand() |""".stripMargin - compareNotRewriteAndRows(sql, noData = false) + compareNotRewriteAndRows(sql, noData = true) RewriteHelper.enableCachePlugin() val sql2 = @@ -326,7 +326,7 @@ class MaterializedViewFilterRuleSuite extends RewriteSuite { |""".stripMargin ) val sql = "SELECT if(e.deptno<3,e.empid,e.deptno) as _if FROM emps e" - comparePlansAndRows(sql, "default", "mv_if", noData = false) + comparePlansAndRows(sql, "default", "mv_filter_if", noData = false) spark.sql( """ |DROP MATERIALIZED VIEW IF EXISTS mv_filter_if; diff --git a/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlParserSuite.scala b/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlParserSuite.scala index fcf41c77d..273197c91 100644 --- a/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlParserSuite.scala +++ b/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlParserSuite.scala @@ -560,6 +560,7 @@ class SqlParserSuite extends RewriteSuite { } test("mv_is_cached") { + spark.sql("DROP MATERIALIZED VIEW IF EXISTS mv_create2;") spark.sql( """ |ALTER MATERIALIZED VIEW default.mv_create1 ENABLE REWRITE -- Gitee From e55b4616dcab4ec0463e00247226fae282fba055 Mon Sep 17 00:00:00 2001 From: Xteen Date: Sat, 17 Sep 2022 10:06:11 +0800 Subject: [PATCH 6/7] add RewriteLogger --- .../spark/conf/OmniCachePluginConfig.scala | 6 +++ .../boostkit/spark/util/RewriteHelper.scala | 6 +-- .../boostkit/spark/util/RewriteLogger.scala | 38 +++++++++++++++++++ .../rules/AbstractMaterializedViewRule.scala | 2 +- .../optimizer/rules/MVRewriteRule.scala | 10 ++--- .../optimizer/rules/RewriteSuite.scala | 1 + 6 files changed, 54 insertions(+), 9 deletions(-) create mode 100644 omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/util/RewriteLogger.scala diff --git a/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/conf/OmniCachePluginConfig.scala b/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/conf/OmniCachePluginConfig.scala index 43214d6ae..77cacf067 100644 --- a/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/conf/OmniCachePluginConfig.scala +++ b/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/conf/OmniCachePluginConfig.scala @@ -17,6 +17,8 @@ package com.huawei.boostkit.spark.conf +import java.util.Locale + import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.CatalogTable @@ -50,6 +52,10 @@ class OmniCachePluginConfig(conf: SQLConf) { .getConfString("spark.sql.omnicache.default.datasource", "orc") val dataSourceSet: Set[String] = Set("orc", "parquet") + + def logLevel: String = conf + .getConfString("spark.sql.omnicache.logLevel", "DEBUG") + .toUpperCase(Locale.ROOT) } object OmniCachePluginConfig { diff --git a/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/util/RewriteHelper.scala b/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/util/RewriteHelper.scala index a3167bbc9..2ef67e536 100644 --- a/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/util/RewriteHelper.scala +++ b/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/util/RewriteHelper.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.internal.SQLConf -trait RewriteHelper extends PredicateHelper { +trait RewriteHelper extends PredicateHelper with RewriteLogger { val SESSION_CATALOG_NAME: String = "spark_catalog" @@ -265,7 +265,7 @@ trait RewriteHelper extends PredicateHelper { } } -object RewriteHelper extends PredicateHelper { +object RewriteHelper extends PredicateHelper with RewriteLogger { /** * Rewrite [[EqualTo]] and [[EqualNullSafe]] operator to keep order. The following cases will be * equivalent: @@ -456,7 +456,7 @@ object RewriteHelper extends PredicateHelper { val input = plan.inputSet val missing = request -- input if (missing.nonEmpty) { - logDebug("checkAttrsValid failed for missing:%s".format(missing)) + logBasedOnLevel("checkAttrsValid failed for missing:%s".format(missing)) return false } } diff --git a/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/util/RewriteLogger.scala b/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/util/RewriteLogger.scala new file mode 100644 index 000000000..047121904 --- /dev/null +++ b/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/util/RewriteLogger.scala @@ -0,0 +1,38 @@ +/* + * 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 com.huawei.boostkit.spark.util + +import com.huawei.boostkit.spark.conf.OmniCachePluginConfig + +import org.apache.spark.internal.Logging + +trait RewriteLogger extends Logging { + + private def logLevel: String = OmniCachePluginConfig.getConf.logLevel + + def logBasedOnLevel(f: => String): Unit = { + logLevel match { + case "TRACE" => logTrace(f) + case "DEBUG" => logDebug(f) + case "INFO" => logInfo(f) + case "WARN" => logWarning(f) + case "ERROR" => logError(f) + case _ => logTrace(f) + } + } +} \ No newline at end of file diff --git a/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/AbstractMaterializedViewRule.scala b/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/AbstractMaterializedViewRule.scala index 15ff2d76a..7d398c2cf 100644 --- a/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/AbstractMaterializedViewRule.scala +++ b/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/AbstractMaterializedViewRule.scala @@ -699,7 +699,7 @@ abstract class AbstractMaterializedViewRule(sparkSession: SparkSession) expr.foreach { case attr: AttributeReference => if (!viewTableAttrsSet.contains(attr.exprId)) { - logDebug(s"attr:%s cannot found in view:%s" + logBasedOnLevel(s"attr:%s cannot found in view:%s" .format(attr, OmniCachePluginConfig.getConf.curMatchMV)) return None } diff --git a/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/MVRewriteRule.scala b/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/MVRewriteRule.scala index f2cff01d3..ba0c17f22 100644 --- a/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/MVRewriteRule.scala +++ b/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/MVRewriteRule.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.optimizer.rules import com.fasterxml.jackson.annotation.JsonIgnore import com.huawei.boostkit.spark.conf.OmniCachePluginConfig -import com.huawei.boostkit.spark.util.RewriteHelper +import com.huawei.boostkit.spark.util.{RewriteHelper, RewriteLogger} import scala.collection.mutable import org.apache.spark.SparkContext @@ -33,8 +33,8 @@ import org.apache.spark.sql.execution.command.OmniCacheCreateMvCommand import org.apache.spark.status.ElementTrackingStore import org.apache.spark.util.kvstore.KVIndex -class MVRewriteRule(session: SparkSession) extends Rule[LogicalPlan] with Logging { - val omniCacheConf: OmniCachePluginConfig = OmniCachePluginConfig.getSessionConf +class MVRewriteRule(session: SparkSession) extends Rule[LogicalPlan] with RewriteLogger { + val omniCacheConf: OmniCachePluginConfig = OmniCachePluginConfig.getConf val joinRule = new MaterializedViewJoinRule(session) val aggregateRule = new MaterializedViewAggregateRule(session) @@ -92,11 +92,11 @@ class MVRewriteRule(session: SparkSession) extends Rule[LogicalPlan] with Loggin val log = ("logicalPlan MVRewrite success," + "using materialized view:[%s],cost %s milliseconds,original sql:%s") .format(mvs, costSecond, sql) - logDebug(log) + logBasedOnLevel(log) session.sparkContext.listenerBus.post(SparkListenerMVRewriteSuccess(sql, mvs)) } RewriteTime.statFromStartTime("total", rewriteStartSecond) - logDebug(RewriteTime.timeStat.toString()) + logBasedOnLevel(RewriteTime.timeStat.toString()) res } } diff --git a/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/optimizer/rules/RewriteSuite.scala b/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/optimizer/rules/RewriteSuite.scala index 3ba4b99a2..e913b59ac 100644 --- a/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/optimizer/rules/RewriteSuite.scala +++ b/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/optimizer/rules/RewriteSuite.scala @@ -43,6 +43,7 @@ class RewriteSuite extends SparkFunSuite with PredicateHelper { .config("hive.exec.dynamic.partition.mode", "nonstrict") .config("spark.ui.port", "4050") // .config("spark.sql.planChangeLog.level","WARN") + .config("spark.sql.omnicache.logLevel", "WARN") .enableHiveSupport() .getOrCreate() spark.sparkContext.setLogLevel("WARN") -- Gitee From dae6d9199a37f5f1a5cb73be589ab8c59b4d3c7b Mon Sep 17 00:00:00 2001 From: Xteen Date: Thu, 15 Sep 2022 14:51:09 +0800 Subject: [PATCH 7/7] modify style add RewriteLogger add Suite add RewriteTime stat add some comment and adjust code style add log parser --- omnicache/omnicache-spark-extension/build.sh | 3 +- .../log-parser/pom.xml | 240 +++++++++ .../spark/deploy/history/LogsParser.scala | 324 ++++++++++++ .../application_1663257594501_0003.lz4 | Bin 0 -> 67545 bytes .../deploy/history/LogsParserSuite.scala | 77 +++ .../omnicache-spark-extension/plugin/pom.xml | 1 + .../spark/conf/OmniCachePluginConfig.scala | 14 + .../boostkit/spark/util/ExprSimplifier.scala | 7 +- .../boostkit/spark/util/RewriteHelper.scala | 59 ++- .../boostkit/spark/util/RewriteLogger.scala | 38 ++ .../boostkit/spark/util/ViewMetadata.scala | 80 +-- .../rules/AbstractMaterializedViewRule.scala | 145 ++++-- .../optimizer/rules/MVRewriteRule.scala | 70 ++- .../rules/MaterializedViewAggregateRule.scala | 5 +- .../rules/MaterializedViewJoinRule.scala | 18 +- .../parser/OmniCacheExtensionAstBuilder.scala | 20 +- .../parser/OmniCacheExtensionSqlParser.scala | 2 +- .../execution/command/OmniCacheCommand.scala | 25 +- .../plugin/src/test/resources/tpcds_ddl.sql | 479 ++++++++++++++++++ .../MaterializedViewAggregateRuleSuite.scala | 106 +++- .../MaterializedViewFilterRuleSuite.scala | 97 +++- .../rules/MaterializedViewJoinRuleSuite.scala | 6 +- .../optimizer/rules/RewriteSuite.scala | 2 +- .../catalyst/optimizer/rules/TpcdsSuite.scala | 95 ++++ .../sql/catalyst/parser/SqlParserSuite.scala | 5 +- omnicache/omnicache-spark-extension/pom.xml | 2 + 26 files changed, 1777 insertions(+), 143 deletions(-) create mode 100644 omnicache/omnicache-spark-extension/log-parser/pom.xml create mode 100644 omnicache/omnicache-spark-extension/log-parser/src/main/scala/org/apache/spark/deploy/history/LogsParser.scala create mode 100644 omnicache/omnicache-spark-extension/log-parser/src/test/resources/application_1663257594501_0003.lz4 create mode 100644 omnicache/omnicache-spark-extension/log-parser/src/test/scala/org/apache/spark/deploy/history/LogsParserSuite.scala create mode 100644 omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/util/RewriteLogger.scala create mode 100644 omnicache/omnicache-spark-extension/plugin/src/test/resources/tpcds_ddl.sql create mode 100644 omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/optimizer/rules/TpcdsSuite.scala diff --git a/omnicache/omnicache-spark-extension/build.sh b/omnicache/omnicache-spark-extension/build.sh index ad436cddd..6c2bb4744 100644 --- a/omnicache/omnicache-spark-extension/build.sh +++ b/omnicache/omnicache-spark-extension/build.sh @@ -1,2 +1,3 @@ #!/bin/bash -mvn clean package \ No newline at end of file +cpu_name="-"$(lscpu | grep Architecture | awk '{print $2}') +mvn clean package -Ddep.os.arch="${cpu_name}" \ No newline at end of file diff --git a/omnicache/omnicache-spark-extension/log-parser/pom.xml b/omnicache/omnicache-spark-extension/log-parser/pom.xml new file mode 100644 index 000000000..75f93ca23 --- /dev/null +++ b/omnicache/omnicache-spark-extension/log-parser/pom.xml @@ -0,0 +1,240 @@ + + + + + com.huawei.kunpeng + boostkit-omnicache-spark-parent + 3.1.1-1.0.0 + + 4.0.0 + + boostkit-omnicache-logparser-spark + jar + 3.1.1-1.0.0 + + log-parser + + + 14.0.1 + + + + + com.huawei.kunpeng + boostkit-omnicache-spark + 3.1.1-1.0.0 + + + com.google.guava + guava + ${guava.version} + + + org.apache.spark + spark-sql_${scala.binary.version} + + + org.apache.spark + spark-catalyst_${scala.binary.version} + + + org.apache.spark + spark-hive_${scala.binary.version} + + + + + org.apache.spark + spark-core_${scala.binary.version} + test-jar + test + + + org.apache.hadoop + hadoop-client + + + org.apache.curator + curator-recipes + + + + + junit + junit + test + + + org.scalatest + scalatest_${scala.binary.version} + test + + + org.scala-lang + scala-library + + + + + ${artifactId}-${version}${dep.os.arch} + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + org.antlr + antlr4-maven-plugin + + + + antlr4 + + + + + true + src/main/antlr4 + true + + + + org.apache.maven.plugins + maven-compiler-plugin + + ${java.version} + ${java.version} + + + + compile + + compile + + + + + + net.alchim31.maven + scala-maven-plugin + + ${scala.recompile.mode} + + + + scala-compile-first + process-resources + + add-source + compile + + + + scala-test-compile + process-test-resources + + testCompile + + + + + + org.scalastyle + scalastyle-maven-plugin + + false + true + true + false + ${project.basedir}/src/main/scala + ${project.basedir}/src/test/scala + ${user.dir}/scalastyle-config.xml + ${project.basedir}/target/scalastyle-output.xml + ${project.build.sourceEncoding} + ${project.reporting.outputEncoding} + + + + + check + + + + + + org.apache.maven.plugins + maven-checkstyle-plugin + + false + true + + ${project.basedir}/src/main/java + ${project.basedir}/src/main/scala + + + ${project.basedir}/src/test/java + ${project.basedir}/src/test/scala + + dev/checkstyle.xml + ${project.basedir}/target/checkstyle-output.xml + ${project.build.sourceEncoding} + ${project.reporting.outputEncoding} + + + + com.puppycrawl.tools + checkstyle + ${puppycrawl.checkstyle.version} + + + + + + check + + + + + + + + org.scalatest + scalatest-maven-plugin + + false + + ${project.build.directory}/surefire-reports + . + TestSuite.txt + + + + test + + test + + + + + + org.scoverage + scoverage-maven-plugin + + + test + test + + report + + + + + true + true + ${project.build.sourceEncoding} + true + + + + + \ No newline at end of file diff --git a/omnicache/omnicache-spark-extension/log-parser/src/main/scala/org/apache/spark/deploy/history/LogsParser.scala b/omnicache/omnicache-spark-extension/log-parser/src/main/scala/org/apache/spark/deploy/history/LogsParser.scala new file mode 100644 index 000000000..fa4469720 --- /dev/null +++ b/omnicache/omnicache-spark-extension/log-parser/src/main/scala/org/apache/spark/deploy/history/LogsParser.scala @@ -0,0 +1,324 @@ +/* + * 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.deploy.history + +import java.util.ServiceLoader +import java.util.regex.Pattern + +import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} +import org.json4s.DefaultFormats +import org.json4s.jackson.Json +import scala.collection.JavaConverters.iterableAsScalaIterableConverter +import scala.collection.mutable +import scala.util.control.Breaks + +import org.apache.spark.{JobExecutionStatus, SparkConf} +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.Status.ASYNC_TRACKING_ENABLED +import org.apache.spark.scheduler.ReplayListenerBus +import org.apache.spark.scheduler.ReplayListenerBus.{ReplayEventsFilter, SELECT_ALL_FILTER} +import org.apache.spark.sql.catalyst.optimizer.rules._ +import org.apache.spark.sql.execution.ui._ +import org.apache.spark.status._ +import org.apache.spark.util.Utils +import org.apache.spark.util.kvstore.{InMemoryStore, KVStore} + +class LogsParser(conf: SparkConf, eventLogDir: String, outPutDir: String) extends Logging { + + private val LINE_SEPARATOR = "\n" + private val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) + // Visible for testing + private[history] val fs: FileSystem = new Path(eventLogDir).getFileSystem(hadoopConf) + + /** + * parseAppHistoryLog + * + * @param appId appId + * @param fileName fileName + */ + def parseAppHistoryLog(appId: String, fileName: String): Unit = { + val inMemoryStore = new InMemoryStore() + val reader = EventLogFileReader(fs, new Path(eventLogDir, appId)) + rebuildAppStore(inMemoryStore, reader.get) + + val sqlStatusStore = new SQLAppStatusStore(inMemoryStore) + val mvRewriteSuccessInfo = getMVRewriteSuccessInfo(inMemoryStore) + + // create OutputDir + if (!fs.exists(new Path(outPutDir))) { + fs.mkdirs(new Path(outPutDir)) + } + + // continue for curLoop + val curLoop = new Breaks + + var jsons = Seq.empty[Map[String, String]] + sqlStatusStore.executionsList().foreach { execution => + curLoop.breakable { + // skip unNormal execution + val isRunning = execution.completionTime.isEmpty || + execution.jobs.exists { case (_, status) => status == JobExecutionStatus.RUNNING } + val isFailed = execution + .jobs.exists { case (_, status) => status == JobExecutionStatus.FAILED } + if (isRunning || isFailed) { + curLoop.break() + } + + val uiData = execution + val executionId = uiData.executionId + val planDesc = uiData.physicalPlanDescription + val query = uiData.description + var mvs = mvRewriteSuccessInfo.getOrElse(query, "") + if (mvs.nonEmpty) { + mvs.split(";").foreach { mv => + if (!planDesc.contains(mv)) { + mvs = "" + } + } + } + + // write dot + val graph: SparkPlanGraph = sqlStatusStore.planGraph(executionId) + sqlStatusStore.planGraph(executionId) + val metrics = sqlStatusStore.executionMetrics(executionId) + val node = getNodeInfo(graph) + + val jsonMap = Map( + "executionId" -> executionId.toString, + "original query" -> query, + "materialized views" -> mvs, + "physical plan" -> planDesc, + "dot metrics" -> graph.makeDotFile(metrics), + "node metrics" -> node) + jsons :+= jsonMap + } + } + // write json file into hdfs + val jsonFile: String = Json(DefaultFormats).write(jsons) + writeFile(fs, outPutDir + "/" + fileName + ".json", jsonFile) + } + + /** + * getMVRewriteSuccessInfo + * + * @param store KVStore + * @return {sql:mvs} + */ + def getMVRewriteSuccessInfo(store: KVStore): mutable.Map[String, String] = { + val infos = mutable.Map.empty[String, String] + try { + // The ApplicationInfo may not be available when Spark is starting up. + Utils.tryWithResource( + store.view(classOf[SparkListenerMVRewriteSuccess]) + .closeableIterator() + ) { it => + while (it.hasNext) { + val info = it.next() + infos += (info.sql -> info.usingMvs) + } + } + } catch { + case e: NoSuchElementException => + logWarning("getMVRewriteSuccessInfo is failed for ", e) + } + infos + } + + /** + * getNodeInfo from graph + * + * @param graph SparkPlanGraph + * @return NodeInfo + */ + def getNodeInfo(graph: SparkPlanGraph): String = { + // write node + val tmpContext = new StringBuilder + tmpContext.append("[PlanMetric]") + nextLine(tmpContext) + graph.allNodes.foreach { node => + tmpContext.append(s"id:${node.id} name:${node.name} desc:${node.desc}") + nextLine(tmpContext) + node.metrics.foreach { metric => + metric.toString + tmpContext.append("SQLPlanMetric(") + tmpContext.append(metric.name) + tmpContext.append(",") + if (metric.metricType == "timing") { + tmpContext.append(s"${metric.accumulatorId * 1000000} ns, ") + } else if (metric.metricType == "nsTiming") { + tmpContext.append(s"${metric.accumulatorId} ns, ") + } else { + tmpContext.append(s"${metric.accumulatorId}, ") + } + tmpContext.append(metric.metricType) + tmpContext.append(")") + nextLine(tmpContext) + } + nextLine(tmpContext) + nextLine(tmpContext) + nextLine(tmpContext) + } + + graph.edges.foreach { edges => + tmpContext.append(edges.makeDotEdge) + nextLine(tmpContext) + } + + tmpContext.append("[SubGraph]") + nextLine(tmpContext) + graph.allNodes.foreach { + case cluster: SparkPlanGraphCluster => + tmpContext.append(s"cluster ${cluster.id} : ") + for (i <- cluster.nodes.indices) { + tmpContext.append(s"${cluster.nodes(i).id} ") + } + nextLine(tmpContext) + case node => + } + nextLine(tmpContext) + tmpContext.toString() + } + + def nextLine(context: StringBuilder): Unit = { + context.append(LINE_SEPARATOR) + } + + /** + * rebuildAppStore + * + * @param store KVStore + * @param reader EventLogFileReader + */ + private[spark] def rebuildAppStore(store: KVStore, reader: EventLogFileReader): Unit = { + // Disable async updates, since they cause higher memory usage, and it's ok to take longer + // to parse the event logs in the SHS. + val replayConf = conf.clone().set(ASYNC_TRACKING_ENABLED, false) + val trackingStore = new ElementTrackingStore(store, replayConf) + val replayBus = new ReplayListenerBus() + val listener = new AppStatusListener(trackingStore, replayConf, false) + replayBus.addListener(listener) + replayBus.addListener(new MVRewriteSuccessListener(trackingStore)) + + for { + plugin <- loadPlugins() + listener <- plugin.createListeners(conf, trackingStore) + } replayBus.addListener(listener) + + try { + val eventLogFiles = reader.listEventLogFiles + + // parse event log + parseAppEventLogs(eventLogFiles, replayBus, !reader.completed) + trackingStore.close(false) + } catch { + case e: Exception => + Utils.tryLogNonFatalError { + trackingStore.close() + } + throw e + } + } + + /** + * loadPlugins + * + * @return Plugins + */ + private def loadPlugins(): Iterable[AppHistoryServerPlugin] = { + val plugins = ServiceLoader.load(classOf[AppHistoryServerPlugin], + Utils.getContextOrSparkClassLoader).asScala + plugins + } + + /** + * parseAppEventLogs + * + * @param logFiles Seq[FileStatus] + * @param replayBus ReplayListenerBus + * @param maybeTruncated Boolean + * @param eventsFilter ReplayEventsFilter + */ + private def parseAppEventLogs(logFiles: Seq[FileStatus], + replayBus: ReplayListenerBus, + maybeTruncated: Boolean, + eventsFilter: ReplayEventsFilter = SELECT_ALL_FILTER): Unit = { + // stop replaying next log files if ReplayListenerBus indicates some error or halt + var continueReplay = true + logFiles.foreach { file => + if (continueReplay) { + Utils.tryWithResource(EventLogFileReader.openEventLog(file.getPath, fs)) { in => + continueReplay = replayBus.replay(in, file.getPath.toString, + maybeTruncated = maybeTruncated, eventsFilter = eventsFilter) + } + } + } + } + + /** + * write parsed logInfo to logPath + * + * @param fs FileSystem + * @param logPath logPath + * @param context logInfo + */ + private def writeFile(fs: FileSystem, logPath: String, context: String): Unit = { + val os = fs.create(new Path(logPath)) + os.write(context.getBytes()) + os.close() + } +} + +/* +arg0: spark.eventLog.dir, eg. hdfs://server1:9000/spark2-history +arg1: output dir in hdfs, eg. hdfs://server1:9000/logParser +arg2: log file to be parsed, eg. application_1646816941391_0115.lz4 + */ +object ParseLog extends Logging { + def main(args: Array[String]): Unit = { + if (args == null || args.length != 3) { + throw new RuntimeException("input params is invalid,such as below\n" + + "arg0: spark.eventLog.dir, eg. hdfs://server1:9000/spark2-history\n" + + "arg1: output dir in hdfs, eg. hdfs://server1:9000/logParser\n" + + "arg2: log file to be parsed, eg. application_1646816941391_0115.lz4\n") + } + val sparkEventLogDir = args(0) + val outputDir = args(1) + val logName = args(2) + + val conf = new SparkConf + // spark.eventLog.dir + conf.set("spark.eventLog.dir", sparkEventLogDir) + // spark.eventLog.compress + conf.set("spark.eventLog.compress", "true") + // fs.hdfs.impl + conf.set("fs.hdfs.impl", "org.apache.hadoop.hdfs.DistributedFileSystem") + val logParser = new LogsParser(conf, sparkEventLogDir, outputDir) + + // file pattern + val regex = "application_[0-9]+._[0-9]+.lz4" + val pattern = Pattern.compile(regex) + val matcher = pattern.matcher(logName) + if (matcher.find) { + val appId = matcher.group + logParser.parseAppHistoryLog(appId, logName) + } else { + throw new RuntimeException(logName + " is illegal") + } + } +} diff --git a/omnicache/omnicache-spark-extension/log-parser/src/test/resources/application_1663257594501_0003.lz4 b/omnicache/omnicache-spark-extension/log-parser/src/test/resources/application_1663257594501_0003.lz4 new file mode 100644 index 0000000000000000000000000000000000000000..f2b568905c4a786b9eec43dc8c5f05d1eee68e18 GIT binary patch literal 67545 zcmY(r2Y4LS^#^*+y?5%C*=J^<_jm^1H4{-y8O%8qe6k*%CxIgkpa$9kfftevpaJ;{-QY$%hnQ-t@hiQWG)@H`_jqbXv}VlMC?dqZDk1*_)FoR%-}8t!#@Ya}U5l9;~fg z7R`i{<92$&KM+nwQ-6em3Vr-qI}u5y>0T;#)>g*tcrrb)0i1on%RyNUO9J(cgWx+p zIXv8Dhf-?*#Uzl|!nD8uc#fyv7z|~`yeR5YIS+Cu1$h)U1+Bo&omZj!*x@Bw8A^mk z?DSs5LVA)S2>82~y#&&w2nM`V8A(U!mgqjZl9{9NtD;Ocl!%04$%IYc>r2v_&2O07 z*xW#uS{d3NjpyS2l_~tDu%V?fFt4GZrG;)G8mA=)#Z#5@gU!ti^-V3o`sVsTOA{n} z6Qgw33+b-c!Mxt#;VQcEPe4A19f|R1I+=*mX01s@LfPx#;pJ;r`nwau$#guFeVLte zY~}LMc*x(C+{g~O>yoK#T_!o4-4;sQbz351L94|IY^o0g>$ap@0Zfn$E=_|&;Gfl- z4uxa3zde~wCDTWPXLc)Z35Q}q@Qp9$yZrT5kp2r?$naPW<-9*-r?b&7nJ05L-)m*; zbTkx;?yw(a_a)OKRwxw;kJ{E$7`9fVCz1mu(5h4dy94X2(c2JM1$gZL@PmvZ%(4@q zp%{hC{K{-P_jh1^;BDYdy^D5bV^g3xI2l^EgwmOfTzH^0vo&Vf_)eDA&e}o`+S1n7 zy#b{DP%335p6B=ze8UG;_spLkoJY6Y++sCQq}E5_r3~FrBp0*ORy-N`iFb5%clMqN z4`p&g@o3gcO>{XwuI@P=n%@$r57f;7sS)6Hd?*~sQizUp z$5S!8m*YEZm~M77I&LzY3$T)^Qw5Dsi={Q%x^3iNIbux4`1{wp$ z!`?JKf<Zr)?po8|y=iINK*un+t8sNpprxhm5N@rb3&x`1^U$~HGWPh~rg^me!A*fcprIS# zDeSYiQ&xILfiAqAZqyphg|^vIs}uP~(br3R(>{qyivYK<-UubIld%B?InL(Bec)`P zbJ+DMb4-&sP}ottjz`0`l^M-tBgt(E>p0<0=Jv||hJa7OyPzO_zUqsAEq^J|D#t>( z1O-pp%8sV(P{a%$q53PEwlQpkP-c`4R%Dt`CQGfup;+c1ducdhWg0>j#ZSTxA0by- z^f@q16gDk`rbIH5h0TdvJYH*lhythl6a$PMgVLGnAP~giT5)(YxhcW;SWY?exf1k|)>; z`~czs)|aE@vTo&gmJC@auBn7S((%ZpquB{-ES|B30mgwnizUrH02=SJLb>c{4%pZ1 zAh*WsG+if|u~O-1A{3ey)!Muf?#pZyc* zd2`!M+J$V0o;+>#%E3f3F%|u``J?Qyj=+?e3T0`f&T{;V9dqO+tIgaPGIs>bc}E{< zwi>iP#|CQao0@hmIkNibCDn9`*?H1I*`n=D=CY|=)*8wU57WYAqC0NI2xabIW8)vd zwTy|WSY^ekoz8{4q=8oOR)Cij@w+@BEV_xV>M69%3jl3Pe#o!tNtI>4> z576=cCA=7>CCiw}baW(|IEV97+J-V&%FkI3@3n@rsij;8KeTpcQ=K(}|BmfS+at7c z>4_HApG%jb_?q(ys*{;fI~KDtp}S;w_4$}eT+jk5uZi%UDg(gr{k(H?ngiq;0vye3wVzQvJ+;I^`*?=jCnoZYFpc) z*-=yC>=9L5j&cwBqY1jsrwnyqpE$0vGL%=cRy>+`5#R=f+rgP)(r(^Hi!pwKwAzd( z6VW#{twHiP2M|PnyN&>STjTFBeY3ndnxuNLbstExwXijmZ=Ws!_@$(rEvZKe(attm zq1Z_B&rHELKy877_G8)Rfv&dtrsgL!aW@$Ik>eEk5Qgm(1-BWFg)(#=qNyv;_?669 z3fQT-u^M=0nLQlJ#j<}fJl);MHUXc(TdBy0s8?%qR)wrTBmYR#&ULX-}h{)*!&bdD)9T`()R(V0#9Sb#E(A^8Jlg<$JZ`j2xL9pqRpVw-2V zTFq#}3}S)V;SwsRgbQULCe(i$p$IKDC6#g7{FQAzbkH}o^>?gl+qAN6{i13gh#GXh z$eWh#2Pwodp%@uVm;j}rmhb$yPwXA%tP_m82 z9`~}h#Ld=FC}Y#!#$wSzWP_62g37}RpDx1`wJBqTHafk5=UjGxI)avYhd4e9xmak( zj%BQM0%piw>l0?Y#}t?cyanWlyn0nKV)v|s;6OH6;c=aao5RUi%nq;MVznk;31;=N zbreMg{0WD4Qh{x3k>!3moSD)8@Xk~+ns`9t&j_sumszxOA1KmsVEhGCwg_6Sm7&yx zZ*?{O)9Y8gOQ)%ZkLi`sR5?G{vbU$A=?U`_7NG2U8)Cc04{zndnF&l+C4W zH>*tBcS*}=L23~8%%T!NS;3C1@bYeIlk9BvcCB(GlNBL~$!I9EP2mYQ4x;l>)|QHe z!uPpb^Eq*jx{s zdqm7o>U;r{5I&TQqfj!g2M%9Lo2VF!#-BqKPXicXrk!4{$+A;M#BQ0b0e+QK*f$dk z+sZ*40Cly@FVN(tnHb}NmboprGx{~B2Uq7a)W|jNJ}a9Xvn!}v+S{6#@h_BdC8)oE z8PxYWluxCAH5?k@@TCNwb6&^rX$?jNE?0vYD{&H3jobG!ejURHbc_NdbXLG&v8y#- zNdE3(zj4s1Pmb)?e7OUX_PG#Dt{97c<@ANWQQ#}qm!yymr@VYGGcE=Ock!>aR5USS z#Ui1Ul}cu!+bt>^X=@}J$vE5S?^Qcp`Ee(j3J&F5pF{BqoUS|AK(E>?SmYiRRY;)a!`SQa2}0fHHuvdyGzFo z%{Wq4j9a;&(EDSll9DI-SJGHWortX8%3h)#kD8!}IYuYYG*Q~8Kt@0r=TAD8+S$;^ zpuB{N$!{Ez`j|(W`)?13FSDB%Qaj8 z&PAyFf_m$lklp8z+d!pcx`xw{j8o3OjbkrkNx^d^_+a_WfF;m))(H?UfjR;I(%^cR ze2t`>1s=<(`Lp()q=Vd~%{L>_w3(s27u`+==@MBKbF~pp-WH{ia$Cmi!qrwdl{2q* z@#Q!KWE}Qjz{b2urt*>H%J|A z^C5%JN9hN*XGg9GI^8rON2w_pad1J)e=TY~vJyqHz`?$uvl+^cp6upB5TK;~mczfZ zZW&b6W$y5;uA}2RTnC~7)lVcWgLdlg%qs*X;b@j-F7ph{KViP;5}!-*ZIV)|t6pEj zmeGvIcTs;wThB^suxHtW5;SYZeUgVS;&eJa(NnzQ5(z@6^#RV8#cK{WS%!YW-gH2@ z2X1!4yQtivE2ozPn*xE2Dqn#99b?-HJm!UnmF4(67xl=`Qrfpu{r#bYT`HiL@h?Ox zRY_~You7WK_-0RWa0wXCqck@j+CHBv6wRkH?S0Nq0Qf*tcRBdmx~mS95A@>TMGohW zT>cHnBP#C)=bZcqUmT!T|7Z$@TOBe@>OKx9F?dPBo2H5zCH^$D<)UWj^&*w>_HvY? z4Btoi3y5n9;ci2mkLnYQp@e-!?v?^i)7gT5h|b@sEbfCCn;uFeT=2C64tbr8aC104 zL510(?sRmit0EUY*ZVbV>lmnOAMCGNmFx>eBD8J$0H&h2yf|Z1`OZee!xW%mcRCrf zlF7LFjc(j0yHzT|t$I<(X8tNd0}n2D2haVe;a5r)fJ z)6t<^lamV$Crj=}n0y8k;~rjzaJjm9jQTh;^q5Z^FyMJvq(GdBVjuEX!FZSprv++~ z<3n6N1C*VTd?#046ElN>bJ4jTf{D0U=7a-EtKIl2 zpB2q~hRQy~si4MC-VZdusZ&7y2zZRG4h;=wGF8kl_~BV9w1cu8eajjcclzx%13C$v z8G2Y_e=v+2iUC%a!+j3pNRX%@UQAyUo19b%V0sae| z2@Ke&1J1WY(0y4P{>`^!ZDvOIopMu)l59Kxaq)rj@Q6?!H;4)#GQ$2DO2FrgJMbxOO@o zt(4(u@Xq~E!yf^j1YTQ|vLeyYNFw=br69AZ+%tBgj5j;ru5x2txr#Ih{2Ig>833C^a-0 z2OQ3%uH02pVJD8$!*+zG=Lv=c7i}}Qg|b4gNRN$(%7)=jC3sqvFXIF8&`31ALZg%| zY%~TLXUiRE=$`aR9*^D4@Ctx_7hy=&u$>57S|F1d4GO}Q;9|zEauhV|K|C7Z6HxvG zJ~N~_7BYUL#t(S{W-9j~!e$4f(|#++C5(OU@>t(P2cbjumT8{UH|2114RFR>XlN=B zd~2M*ot(g|wKOE&KUF>!@O-EE*rj`l@SmXmg!1`Q_&RhrxToI9VG#M*1+boZvS(}X zjl06$R^{?j=O&9$AG^YVnoGRR)U6;-Rq>_i@cJyAJ*(9sIP+;5+}Fpq({S~4rTj7| z5e~=9g2|4m#E8wr@BQ!^5Pzjj&2(ltlW-+lcZ?QMhf8ST9*SN5)| z?d==v?p@W^b0QO8yWs3H<>azHlkk{it{Tqfq841gz(v_?tW3z;QMr^s&Ouu-uM*VB zFkH2Ske(lXj4gqJx}DX9b@Kp!rOThn72(h*hrKwL4pPk|xWLQ*#c;-K+{gIC9M@Ed zFO)~j>_o~YgeXlVGNEhNW2?0znVjy%YpXq(!ddzgTr705n*f|;P+0)4&(Ut;9W+s` zJgWL6n!3ZIxx^SKyk;8vl=?ZzNENAAr?Tsr_E#rNt_o#R1VqK=>1a79JHT&QYi2YV z+bt2SA;GtT3lqOBf=ie?SE|f_5TP~e($PmN#Uc~uL0zqXF1UVTq49LmYNNjf=+mOZ z(QqqFpXj%TY39o=0a;?WOLdga6lY1idKwr^s+#7j|B!n}8)mZyR9FmnqD$Xg9;Bx7 z{HgG9HC*7rcb#myBECc1!1xIsb_P>EF2WmW&Qb95V%7|zlfzMx=oR3E29tn$k8zLF zNzE8+6(F?UZPDTEsk+W8!IxR?^;-jO@UE(6?)6SM1;n|4$I1LZlb}$@pQ}<4j!^k% z3C_F04`p71yIg!c-?2@|-^uhp2r7$3=7$m!L{7|4Zwp64uQOQ1@$90$Op2QJVy84& z^|WxrM7 z81N%!m&_2MNpQArNgxn!bt?Q7apDjW9Hu=yygMMqWc(9|PlR|A`Jf_?PG!A5oHE&6 zUShQQIDZ1A9o$&RLAi8FaqiM7CG^Kw@8W$K0>nmlsm4cMZFpdvJ=7Ih=YnAr?}Bkn zkswqY{x->_&-N-(x z!sV2tY3`>SlkpgTQw@GEayspo&BDFld=HJX3Rg!enT#EnF3uA2b9B@#@96sKo5M+J zEW!lcCeyE2Uh86n$pRcM-0y>#P*OJ&)H{7Nu!5_SFGZ;4u*%Km)jKkSXU$#@eZsax3g7WFu{L(q99S8KMzgn{>Fvbp#FS=5>zcLE^Qp zcZg$3pc6f@PbL|lX*iv(Rp3cpEaNroWx}Tj8@JQxWZFuvu4yHPBJeUR8ZNb72d0byGg**KPCbd!z$oy`v_zBee%h)HfF@^<73sCl=qhHybm|NI5 z(7v)rGB&uxo)H2g?crR^e8z1Y?E+uZHV(q4{6LRvI=55bBkw}qaM+T5{dYL_3rLAVOE_a*%_#(Z+CNfYRgm2jT|19EGWK+I1iCQCrL>aCRTx$6hc zM}<-|W%KsK$noop*XsucDDn38cOTKc>MmVgU#85`Agx8s9AgW)nW zM{}0$F+bw^{krR1@Nq8;uHYlh(LO4bo+(7a3 zY71Oe9iSbHmar3z#@b|N(^PNmI_J{SWZc$)U(EGs;2nQ%0kLZQH&eNb02`~qU@pLG zIqvW?5A364N`ob3sM`ChC5@37%L|ewpWoo9& z#}v4>GUH~AvkwZPxe}ssJAF(YH8P46Fh`@CdF2GrOInjIKAt3vmgmQj;jo331m$~I_l-1 z!}vkvFi>p#OLcfcb0&$wVyQ@1R!e+ZF?{F5V-;m?h5Tq~U1RerC^?ICyw3^8RboK( zntdhk;tWx&YilR-dX8lX8)w3e$TfzyOdhg`7KuiBJfaQs3nx?SEFML;$0e^9@D7)H zHFb*@&qYWu@hKNKFztmBBE05(4>EBauSxZGJo;lHexSkKl7b#%J(Z@f3gOGpukogkMwg%k4 zt@dXcZg=s6(3r`@Vk*R@^ktI-S>A<**kI4Vc(8V0pl7_ki-NqH4+Ccruur+$W}xrbDEywVqG zu?Y49AVe@rJBqV=yGJBYSrudl+#=iP0=L+Dztn{{@y@>x}slKnC zod}KtWmMXJoLLc#oDyWeQ~C(6i`X~|J_p#R8L629`#{xDd?DkqX(tl-k%$Z>w;!Af zTV+pdwu~2OO3P$mT>6J#?@ggg4KVDcp$Kcla^RCOKrPnsoW54n=Yevj+@S1_*h9Gy zy3lZPA(z9ggmbb#B0Eb0yMe7n{w+mttLoc+Xa=l7!c4U|7<1=IV%!hkO4V6AFbi(t z^6`~wH?UbW2U5fHy2mqdc?~;_i4jx-urx&y27)aqz)vkyk2F*hj-(J6Ge2jrgkveF zB|6M(!6M)_<=)UG6~1}5)yUWIHlj?;Z(G?N8oSUJB!u^~X6Iie=iPiLPmr0Zx20u- zU6{LyMYItL0mh~?i+bxq^%ammKzz;XF>jZbkUC+V%EeMiyjX~9yuM7g4pa{=MAo4b ztfItp_sAqMT*V7pgPBlfe6nNz6kg$kPg`gvkc%wtP`;~?`%!!+;MWQ}Muzi1$}rw2 zgfy4k10M5H6pI!Z7ds}wZ%g(P0v*qj!`YzMr1&k|H&JCp@6&ji6gn7uWEC2}uiQ&u#4SDP-V@ZzY zZFc0zMN9nYZIvA5VH#?1JPvU zm&6>=%=#&ptC0U@3dWquQy%eXsc|)9aH+?WDnz^>IG0Y_V}QQ~{*X{Nl(PbW-wUi( zdk7?sEh#9jUn}H!Za9k)bdU|Z#F+)`6M#P>c$`y{OX`|inj2Sv>uU$DD}(upH)N~s zH%jpd-~$e^7ziYwSrLwd^GSnZbuA@|}kdp)J%Dx3BVf7Zm}#(Tuy% z`CG~OMrRobj3BL@M4~Dou zV72P_Ah2Uq_I)Y5*+vnrpADj7ia(b+y@)L^M1MQhf%+b2eZYPxNz z30~>;hs~#avyg%k)#3aF@W5!u9oJv0NgoMBD@71`sC-N-8Ql~5VF5e2o1JRGwgS1e zhfhbiX&S%2n=}f8?bP*$Un9yE<%60q+F(i99s+{1)G}F_yBD{bM98fx@np8TVe(vp zlEp?;-eh|Tl8X>g8;gWrT?*5=auuS+oPEg6Do^@WcY8RK9_~}s%}_lW>T`>|z;EPy zuZ~5odaBxgE`--~<3qqc!bGy6X$^boh_S32OAj?2O~4B{u10a2ss}-MWgqD$>g%?Q z?q3K!3LQ+jqk#Qcz^4{8xIMuUO;(s=Lf|t?#Wv0-JF(tlTvg5WJ~x}n#ob!5`80+S zc9t?|WHy{GUaTb+#jYJnM+W!8CJXBTdL|=*gfX=^Yg1->|&{hBk zS8If^xH)WOe10n|oC0?j!5IKM75>Ibn5!F)PnV3n0oApJwT5${e}{EU91Miv!2#?}hy7bX~3VaCoyziH2}g2z^~-f}+Jf%RTv=VUn;P1aR_ zd>R)A%HfzJh{i}Kv)SM@pkDHPWWNWz-fz59%nw7R*H`NJb}?QT(B5VIQw5(h*rSr; zPm>%aLYWZof+kW^jI(BP*Jm=Mg{YstpF|aN8(SKh8z~(-_mEzID{ESeIg=dn{X#ha ztr6Op2YfQu^p_{WWj)?(&J8o5YG>U#$y3(?{J9S45C%Hu?lnP@S-V3gJ|_ zVM$js60<#6(s(h!Lqc4@@Vo+bDsRmz6;yb+JU$R_+{1wS~ zPLib?xVj6BXC(&Dl=l&h9ZFT}>rJ8r&1Z?Z(`ZW2y1@0Ba9^n?S6lezY{*H_Lzz{9 zD5$Rqyu8Sm&m{ghNy;wOed&aTR{{*LEHSSx(H?b5KY>_*VmC6~2{TnT&~NlOywF-g zLoY9r*qPurQ+IY2X3rAhZXMnQemaN7C2(dT9^xdKpt;;7UrI=`CVo`#Y#(3k@34lV z*&RNAuzpgz>}Bw_%!*iR+R7zb%RJHd41XwgMFErR14MFMIkP}&)g@yCJh!=g;gRoM6r@&$}N`-uOLsKG`x{tH#G<=Z5 zR_^>g!>5Aq#C&K%{KUD3(5w*s31?9+NVP9Cx_tEH@Lq4qPEbS8W@p;7>Gty$;-J$Q zoP)dp7A+6kb)fkf%;xMxz{M`Ol1p19U*^R|cpms6S(#vPt6Tn9@Hf}v4X8FEJnd{H ze8Kz=#~+GWwUckD(r@Pe(MaQRvN8lILOZd8iR|5uSAwD#Y)cq0N5t6c|N?6!&*OVa>*wO{3EEl7!);?CT8on zdIkFtVGd#tRAN()5Hu9_utAq+;!X)?I|Bi${&=@&Vdz?or-sI%f;xQ+mTOxK7!UcSZ%WIwObsSs7uBnCPwebB^@6eSFu7SG^&LIjULU={Q zp0*Zs844=l*M~jCtl=z08Qdb8`tQz)$iR3-pfeP{eNbE`;Z=GcjWW^Dhvn^r zG?>3?g6)ztlZh)N{1%;?6&~P@l1e5xel%OeKn*a-r+KVrJ)ZQxi^TbZmB-c;z}}kL znt{P(9sT_^r5uk1`GG=sP!CcgeN`7*UcsK?ux1ufXT}^|gJ%$0s@QxQGZq%qE!VXL zf^AlyVtvK%@G*|kP*!%kx^Mqrg}v zOWy78&64-XjoXvyb7XJ;KUd~IQK>}y19~zy1m;FFqq_@@YXsxh7C}3~H}k>ri1kQ~ zjac9$C1n%MalMjQ?A0IOY#4D(!R9mxvZLuCu{BDRjup0N>mtMm4wEKTdd6<)0NY2{ zP2xr`dO(~;5cdpt&CmBNGFWM$Mw-j%ol8lqXbsOU>f8XD*)2aK@nS}DKzz!53e~m4 z(DaR5@(O<>QC#aEF}KzC5fz^tyJs?NqvYaZGdR5N*2Wp!BDdv8Gx&z;I)%f9ML3_q z8iQqncv}%xap&dSC?fS7Y^d08rMCO(Gv>Gg zm{4dci-u&jq0zknv@^TaTPmO??2l$HSETMf*OLOW96#LPIQ1wrToB~&eiyG)@Kh%! z_UI&&y}TH95m_@8`knk*6nAxd%|aQj9q`2Om7xZa9=Lw96ju0gnl3i-f7ZK50%9`p zLxHro^oU_Wp1I!v0%hPZr|#-9Xf z3~g#_Ny|MqB-$ki%ij2trN%{E=3i*+3zR-|!o@CaGwvmlBtfxiZX5lq-V=L43Xq(2 zNfmZ^rKrv?qCRwnC-bh4#NVOJUrS-pl6u-TK3!Zc!DT{wawWeBu@4!cx|8eqdd>X+ zLKF+@I)Uj5eBFrCr>mdqti%8dT=sOMtBgTdVLfG4+d}Wnk_rWTtpE}==#h;@`y^<2E?FEIW$eqB@?g zue5qjEE2z&CN|b-_X9uE%VM(~rD9r3nIyrv8s>)C2b|yU)Q)F(4v2YNbaRH zyP2oyc+QD@y4VG|QnBbZI0R5Z(_J0Dw7&}J zMs-$%76>myDB*l(k#UF6pueWMbFeQqPkITB{R1d2bS)*(Z|vmFu%d&_viGMZ)e6LVB)!%@SPE~Lc_!l)$oF*w*h}J3_av!k4}|8QNixF->|{6PAmg$)J+)SG=y3%n0`i_mK>~GNiYfy_x3{ za8ZY6=+P46hY9VH={yt&&K6Sm4Sc`BOJPszjYh5;+O?90k`i{4=;7 zmH9hUjrr4@>>^b<+hChzW^n>Vq<0+Jc8_yZu1Lr4P_f+sU#Pz18BVxz)|BW^xDh1&kx`_ob;GDEKjLb&i1nMU zv&Idsr`+CfI)M=YvM1Dd;0y*267U3g8X+m=xZc3)iacXJMm;+=F=rRTpSXA<&cb2V zhRSi==u$u*D!>XC{|iWO=z?gLM+968aJv`&>V>2Ve_@2i8I69~SX&`D3rPQ@Nlt^? znE054U?5(fD*g%l&T95_HSd_yZ_>1FUR?+WU?s`ale>!l9O3480r%op75%rHK33n*PKoT%QLt^5f3B$N-&}{ zub)*E{=(-Uj)dHBWFtSNpWSJ3YR>-6_f;5hv11Cnv5Hgxv zwBRMq%Kh+!#dZ^sQR9y$m&^Wnf%j0}#FeNM+Jw?=c*4Ck#xOFOO3HY>HA-sq=$Nx& zVqk@n96)S-OrBW)BvL$#A=tq~(=4u_uX;kB|v@Yb*}J|e9p-g9A&?!gvOH-g&qRz4}8;x0`AB6dtQeQlmj^|!4b^Qi#MLi~3xMOk$p$pMl zX6j=;eu@iDC(W|~BMQ5q6sAs-%Ne^Gr1KRb;(Qaq1~@3Or!?$Tj7}e55hlF*KctDqUnnYaS7^0DE&>io(Yj`UFA&vNK;>L zCf-wY`r%Ed@+`V{%lIfWCT6*f(MmUS_;9CJnxPTm$2#*emyBbE_G1jP4x`VhA?SXq zXJZCCRuQS=3`(B#QvrUR0^h3QO@NvvzO@sdDwYYVCDqxvlcgp=QdsRRB1MMUhJ9Oe z$#m`~H(n2Ly52g!KG<@8r5pzKw1%JO5g+!l1liJpRIpOtEcg?Oh{?U7xS8IzzD`VN z+QdAzc14vHA-~0e5>Fu5&OZU^Tf^w!oS&u#t#r<-!DX|R6L@tdrYwS!l|HJ}naqM& zM4BwMw#Q?|#C__N)e*{+g1{>?)XB%Nl#5Lz+)PnVROKg;T~o~F%;j4f@Ryo=3G!V6 zPOoK=O89HABz7Y3sczxH&6!wU60=XPvNO-w(87#+>LsZap~nZLU%tA&FS5d(7`U*o zIP;wg-(b2dribE0{hS-`j{FN#tqZiRzf60L(y1e2JE7P)m^>o zR*@K^eN!(fD*C%gnL!icVE3}xPWm6Du~@l+L9=>C*01%Qoee8QrAU7}zkdslBVew|B61MR!k6ZQGh<-GiIfk-+2Q zAk(lgJnmLT99Ry9i^UaA*gb{MAdihyU=sBF(9E*BXLa>6Ntpon zwL_G2%R|T)yYK>%l_}yD4r5%gCL6<~fLI73JVfG7Wb*-KZE+7osGT-V&JedWu0r^v9Ew;=tW^NO4h{|9Xk zPJu(q`pH{mr}N%4Qd1xPMA8}^HkQ)hK|h+#TdEF=pF^ZmPUO{n1eWIgDtSKz1>FC~ zc=%t}Tqlh7WpqD;AZ*w|~;^g(-W{zN- zyx}V^D9L*hgI!?U432|5?{PIVhn-k?q?07^3R%2Pu<&ah|~QO7W7|tR^G!D zei6QAv1s0CNYJ#Iq0gt!*aBHIp7+uOP5|R+a34bQKBcU~)Oy~C6HPt{(h?ABL8LJ6 zeI(6J@&TyKSbNZ8x?nN=svml{WReNuTKO}0NY9Jz!j!#exy;n)L?%iPZ|0@U?*sIp zCvX;M0{x@WS@@+iVbKVaANKxdYaatl^G2s=;{We+ItHUZ`^?h^WJ1LCp|&wEU^^_i&ajH=W2eF%`jSL|>78}6*+GrI z$j~+EYWaFh9@c1>ZY-JkCG^q=VfztNqvXF#gyL0D5wI8-)GN?}v{C;W_u)DJ^;KoI zUCg|;%%K%;C4-bj^M_3-%t`uTCoUrrw|OIwIzJktik9ibj=WiqMhqITZ{&sMO?+RT zNXskp?2#SxjcRN)$%Y~0K(LAKF2m3HKg-*tP+&Fk*1RELGMtwehteDq%dX)s(SFRD z@js5_N$y#+6t~Wvm1$eq7XASDuOiplqGJO)XV1PH)H9J}B>%If#XjVf(H&P{pOyFP zJc`S@S9SKbfcrJ@Ml3;yKzcIjMtT0$Mje>))dIZ9eP)Nvy(UW-K}!z(TUS{QCdwE>5xzp>{pvkHUMzj~})gWl0ZjXLiYbq=2>J zrvZNk@E)%AFmVk3O8P&xWjHkwvdHXC=J7(9W{t^COkr8t6w;NL zPWstAvKNl^(Gk6y$@m&nOqd~_Cx?!ZiCoUb<75(cgW*;beico1NQJtFS4W0UR{R55 zo2geY{ww+9eE&C2M)zltALM;>ZnjU{!G}mXz9VXXOX3kMtGSTDAUl^XmRFf=VLr*yv${ zTC7HR%$ekgnMF^OH;fOd+6@A>L(2Tw%rzurhu)gk8!k{tU*t{8vMowr{+rkqCt36k_s?mI_oG~>}UdRo#5N2r^LCM4hPOS1a|o%`X0wn&Ve z<@UfPU3`Pi1_o^?K7sF)yuzQ*LW;!yy;C7imF`FPMRI^_75~=Y0cFq6f+aJV`cNVT z>i+*%vvfS@N6RVFJ<)B9v^lKf#{d>pKyqdvs+)A2G za+g|pvyc~7oA&mw-GOSUwE1U${?+u^q3vH`fJ{?sfW)&!CiSL%2fv>i9!c?zUg?6* zaMybC=NCvkFCB8MBkNghJEa$@W#%C1O>5i2d8cJMG%ym45Z*4o`{@mYYxfaonC$Os|DG{A ztf#4=ym2_5S|=%0)aL8KNFwt*yqMSMABHr^=leVZHdRQkxcbPb6wP+ z{-)gXY?d8)5cOtgB?K`shx1DT3*7v51}_8u3&KMEp;R(Kw#K|{EIaz7Gt|&nADlaP zje>~I62YHk#<)HU-d5P}wY-lYJa!GtibhEG@`$ECs`0IiC-{L=p0CpjB(&)+dUrxI*>%eRzHz`6N<}(b1Y1oq zI#3n3RTT?78rNy1=|(o^(lJ`D6Y(2~UcEqR;J{3KY+73-#W@$ ze2gsPAvsXTK|y7H5_$DSol_#bm#plY`8L|*~rRL6en26iG*cD>G1cDwY3`+MsZ=|K)ms3Uq99 zPAhO=8(9zYcBJ1F_$Iz*Kq&uqCw>OvJvsrD>?LIjW4{6<%jn099VeJBp zDC2HCO$^coWUR}J^il9NjDozH!$ag4;qo%RiK>PjzL{8{z1r_mmpjFE1GG?61WN-KiY8-%lT&yXQzF8py_r)?S_OPQI ztz^>p|eh`7phZhNJd1pMB{mnCkKofu$=tllGYlR*2?(Sm+?*3$HE^mxHqgFMkz8oaNuDQX@> zy46{nUFZ#1qmiz1Bk5HGU6J8;%Si61-ayW12ly5Ba*aKq%Ky;eIv3x_0@-kCF)X11 zFb!Cq6U#CwqEoZ&$^!qm(D=P;s&X3Oz3x7`^+fn|5W_+yNO52ikdBMAZr7-EADbzz z)A7*JUg-!PAa?pS8JkHT?(roTPT`;G(zz^8z&*v3Q-II$!buJ!ZsyJs@RX8g6O)|e z$yDcDzvK9%aKEI(Zq@mizB>Qff@X~y|Do45h0;HwBM?e=(s<-2HbM^23m$GLYo$?O zc&D#m`*D+jP(TKvmmFVlODo5N1>$viW!_|egyLy#2I(twp2FpaRs4Tgdk^rqs;g^w zpL5&vGo#+@xic!3Y|Ut@q>+o73&vVN*+9zsz5n|?KhI;PTur-WpS{;!Ywchz-IPJirk0do z@jxz}XJMx^x=53*%@-Xu6FS{A+wSPfVb_^0={7z~qFoL>IkY2-e&NX4xXUpU(u>g7 zWcww4A~}7#BucpLd-G+e{jQUj8Xf`RB&Q|_s3CMA*_$Q1bv`J|3nZJzZh4XwuS>Ma z^}quBy*f2%PX7v-oy=|Lle~?ac3D9LbjAOXr{pSfL`@5y%&|&y`$0)0frih*1!?1N zrc<4+PV{)zB)`h?cx%ov*J4$0+N^9gQ3ILl73n5fT5jg|PS}Y(KdXN3f~NCrroXFz z;Nr;rP&%ES%{*?$?OA-X*3#RB&UwCj*HR!NkX+1KmSx^fQdAZ1pqu(7+3)7!ah22G zDGDjWA9BcJkAPxA^%%eSkweom~x0jg)YkvO7shB{QxLvkEcs5W_31Gyd2$_ z>(wAogs)A?#f4gfV*h|_t4!c&NQ#Hra#_@AuwDOPGCH7v7OBH zoVjpt;JkE&Ysx)dHq8RkgsoN99tEH0#zI=|HVr$Kr)B9CMLLy%HzW_G8yxMJSwpvJ zjO+rx-_?n<6svAgzH8|F_JD6q@7I<(q+MglB6>S}KY%V&7b$BC_LG8WC>k41lm5%1 z{+3Ih@VWmdqX8#5gr$wsr)_}!CC zwt)<7u>;&Dw3*^-H?hO=e^AqH%V+;@V!B-?v~-0TpGf~MrMtYasD44IYWoY8IfcE* z>8Iv8AcsqK*uWmc<)3 z#d}Ayxht5-hKSq}a!Hkp(q-vN+idJ##31=8)$)nc^cSc4 z7LiL%_b)AzKO*`Av4(VXj>|~q2vOT))+br6Bv<_Fd{!~V<^Ng=z|QQ$N$LV(58G+` zBo|~S%2W0L#9d{|@EB}2#s+-#8i%RCohH^x+GMV-OH-$2ii~2nvLRhss--v0C*^)O z+I19k6%7ppUDADa`BG+oL`CAmW-!x1nLR|dds7;kYKemjJ`a{81=OS1;9*j#{F(oOK4=@;l;xQbe1=p%M*L$xL#ov=9_|iA}O~ z8@-KwUmEF~2k!D&AS!K!Cq-IXPgkb-O)L18@n!3e%xt)m`QiZ;#c0RFjTtFyX2dNC)s~p5!;y^MJaR-)>04`g?tsa zTqfZ>p#4_1Du;hiLYHfD2a{{jHq4N!rilBqxH5$T^9FnCD&jp$2YWZqwv?wWyxRXu zqduO6sD=D%{eF^eWTvic#4<|eFHY%1j+HCz7Pe4wyuzigCg}b?GZYFlwIV;d?;g zcdP6iiy5rE&rOIG@@Erih?K7;(hvR8p?rFp=~Y#iGWB_GKN%XBDLiCm=zuJg$r7>G3MJSE~&B0jN;3(j#c=nRWWW<6DqU)leuxx-u^Y*h!Aw7!RN}2^2+H?Pl4Zk6 zM9c&g;Cfl}sA+@kCG^h32q0{a+ccUhPJV;MgT)6$!JuI{ZTVW z%2O?Jz1Ikka;daS*LrmIJX!mJBF_fxnOSX4dq3khs~7rVFJqVmtovM%&dq15g|tLf zyjt=4fhNDQ-bAORx%|J%lD1|W;B_Qir7G>RG(g&~xb$ZeOneNRyftY-7yus1p)?y^ zZ=x4-tXoxbDvp_@tOfWtGd=4Q7g~57q@klvXVY1fl{`x95>5Icmz-wD1{Xz2q&>_q zdHG=>ZJUDc2r7r1BlF*w*rz48`5+gA8!9?PYnM;vFPn-((Htgzh5~4TmCbj6Nr(!F z0nN8^`anveQXGX~+2-WCB<+TLI8B-EH0XI(A*TR$*08#nTd6#lwj_zK@4Gpm?o{d1 zGWbwr@I4@7b6ha)2Rwu3$AoB8z|LX2{syoAI-#<91r zOLK0$0})_?A{DAn*en{e-k`ycXAoF;m%_{;QDQQwvx!}8lID=K+b@5_WIvN<$JmE# zQle*bnE$XO-G8{$+Q&sK&qVJS9WHjY!gfV@yM)*a70M_veqD8)p>}Z zk4H}{WN}ko?{b@Tk#v~Qq&J*F$5jQu=@e;p2CP8TPUJn%Uq0Yx=h^a;gR<#KBk{f! zI%~kyU1Ot(IeD4VR;&g>YpbBY=aIt?m6Eh3N^~w4UspKA z*Ukq`lbd?Wb;)TLO_~PLva8Kn#?tK)oDkk5 znosieJY|PBD_3-SB)Zp3xhCmU3q8-giNR+a%FhIw@1x%%X;#GLfG~e*-MM(Iw|jlt z(8zSl?YIUF_deW=oW*wCoJK%$;qQ z@@0woL6uL24&}n+a@?I?G_%Ag3$ZE0&<{9jYyQ8!SYmduUVSnr`}v9qOO`E zQVVH9MUzCwY$!P#dxYqc%<_;bz+p7qQN#Tz0p86{liZq^X3jK!VzN!62E$DDH_juD zgaHp4lw`xxFJ)_nZ~!;IU`y+w+3$fTCDTSbzbOlF9si2a^FjH1r?l53U67@It!hgw zhD*p>3V)=G9WcXyfZt|_1v7jPy6ET3w$lR2>cXb-IrTG}n_D`z=9saZ zAI&kl=(7v}@UuE*&TWN5M15OZOZ}2L9k7FFX#y8v=G?X=jrEOlTELmfHAuSyxI?Fx zFQ{KMyL`z>KOysx0-lDj`x=>$>imP0dM4Efac|K?<&p_#_e1)$_K8~QGhYI>41+_P zvP*4Wn1F>o<(7XrgT)G+z8r zW_j-<`oQaq{8XXeM&yh2;SFPPn2`40#49^EYsKNw4<&q75*h%{os}!EXW)VR7f|L(u1n9 zQBuElvQxBSGy{Qn55yh8k+J>{T%>TWO^2NmL;4nQGFKKosY*+2J5QLGW}4{~oyLI6 z6Fl>vy53%a)#w&ox?iz92}|fO-^Az~)qIjF-sQp;qY&(7-xW@Mk4p!SbVj{lb3I+@ z8dqVT7ap#_0)Uu%sw~%rokO*%bM=Ru-m{ytsI$gsH;*;d(qHgh82drGr?>xU?Gw|S zq0zSCp;5gu;%n#~)t3MXO!VlD9->!Ep1^@_o!`;6v|-`gCBuefGJuBT?RLF4&h3bs zqodt66jih(M}6FGfbYq_<+*}2#jKE~Y#acnbb~@qL)(H&Kws92vN7JTFNA|L_!@c} zn0pPx&Z#|0IQhP75^Z-Wg=9;R?HsUy7`%7rm|ZF=DohNXQ?qz@Jv=EIn6q~$B-`X1 z^!pdDgJbI2yOl8BrU#7p3a1|e0)J(|+5en1J(5pB7%ksun+dboR3P+M#0PpBiCvE_dGGs7{t0BoOtca^LL+DqBsPu$kVDtS7Yk6G;jFuuf|Ze) zjUO^;CELPQBRUN483P36tjKwQ^g26S_@kCX$G7|6U4)olU4#Q}69~`zvg{wDv z5<4HxKHMsYNq&vXw`Zw8CS?YfuLCp49N0MEKOL;X>=wgzM*ks4Frn|oq@T|cyU69; zJUWljV`Mo6ef!PThTXYx5oym#2TGSLEYsVX^w#-vck0Kz#m|u`Fqs}uO}~;eDy|@) z`q~WPiH6SdGq6FY%357Ze|+?;yoEZfx2t7(NMdJ8d^Jet3VkVwvm_xOQ(BX7=N|ct zi(8qsVq|nOW!{<`8yo?g0+vCDGQnC&CdyIc#pa>3Pu~G1@(7OAyOdSiw`OD&x!8KF zx5*Egyiys)&Rx1>mT%q^_IsIvnse(PrXQASy!9|D2EnCb0Spo9;r{c62CGP5;7!7? zfB1+oAFngKoA*@FdfWb`i#pnt)GwM@-&B<|)X9&?@UTb8=Zi(F{E7Z5|K{q-j*}Id zL#`nokAT#1(4wL)>L&ZMsUCE^WI+*82wAG`>}}c~~ph&uhneT+YOw zHNKfxwrm-T!KlwQl%dgwn$jZa@l6w~tkl1q_!LtQ&0s-Qc; z`WapaS42d6MZn2R}7y$^5s_CmdOi_6FK@Tl{TyI)G#x%}w3s z+gst(-x2}b`&~ww?x%ezn|8ylTQ?uolGE|YC+xaUxT*WgXkW^-+i;4nqjrPDr6VU> zK+bg;)f+%cf(1?zS=k+tUSHa7o^h!wGQzHp{LC8to zgnA1M?<{>VzAKYSP%~M@RK=mZjB#v%n6X|;^RWkQIWT1ja6Af72fjsXs!R5 zQ5{4mk~5soL=)IbK*TUSPtDf)jeZXo%{_tYvY3W0hOFrd$ z6XXn_wP@(#tzz%9Lg)I{j5<&U63<5h=m)<;>N|3sFX9h=!9)dfcVcqEP|{b%P7V0L zj(}D74Mnx^!=B1?{;ptqYeYVJ9~wY+g7V zUk^od-2d||*T&DFz+<{E_=O~fm~Q_SlfLxQj)~%zti-;QIzWvY9L?lq%qPfhSNxzC>5$Gh zE7Cqt$LpCmO_`#{M;4@IbgZiE=m5UsTf+ z-eQ`p(a+4{d~!ScJ~S;(g7q{ypaIo)Q*DlWOF*5{jpVI=#8;l)jB6u$wc_20Xn`VZ z)c0|kGmai+k*$y~maZ5l=7FKIK$`NlK|RDt2B+xFC`f9!cmh5_dLG~9^o%BAviJ-0 z)WN}e=#9zjY!d&)+z;bT!3IU-_k~uOyw1Msay#%_gOR7;VenhC{GvjKH2MP*_izt? zpS2DEI`2ATsrZ0RkuQ|Mm4fMC@D1|v1Eij$iTyN1GHZt1@m3gIsa`h2Oh3>lUv^hr zuIlp7d2QdufxCDK&tY^+wfLPnMgB2sN%cy9;t*#`9&}vg#!TaV8W2i0o+~+r*Jn9L z-csE$uja*}4fVZ1;%cUc@IAcg^qwYu;3!m~PB4{=40}mi5)93_VDyYB0JQD736jG> zpUPsh=8>!AU7$`wu{+o^)k?qNVj+2WhQ!S2EToBS$y474Pxg7Gb;LzDV zaHuCvMIdas1HVzcx5tM-1BCnxRJ-#`B2#r&Y~H#^En-Lq%3^#DV^^ASuIlI#@gJG# zMFx*j&+t~52Kf30$860Jf3@!<%|khkM+e71U4hrjNKelONtiD#85+L+8wN{1zwI9Z;= zes8S}_+zWGygUv1;g@n`tlphK{PA|{o$$)Je-=3uWz0Q|}$V%;c_??B4x7BSXWRdbh?$_A~^LTr7yum&Ni#2^VKh0alNXHPJ&} zTEJbto4VXX^LYvVC7Ye05!{`>&poh6WrLFRu7!Ufi$}6bOsxdqLOduHv%s`E#7Ddf zODdw7duFc3T$ER`=^dN6!Q^F6wDSY9RKyJX|I(a_&08k15-2mN$y#T4kFU(;iwcb3 z00^FUPGU<`cQDuCVKGTWp@=(^>bk%y`XY;7k=?P=^1RI9(R;e)nkbr1`@sq{yY+bs z@*^$vBO~!GxRPEpdtjw*4cQyugPDBREBdnv*lTj(K%%*h|I{o_S9N{?v4zAp*!cNe zsHQ3Gp)|3M724}fIRD{{Fl>WY%HYioVO9%w;pAS^6~>$nlRI3>yluU}u!ip~=5Dq( z;15{NJ>WGJGvbmSS&qJwk8KD_NqMJnRP= z>+_56YbE@{iuB=&BVsm{NW*D-p+d#7_yU2gic^s+qEMN2*qGBLE>b=4q#WAf8(!bL z*^ZsBGG1E9US~p6-SS<|R`le0dtiO8NOXZLULxbqrH6C)WuOf*@uX6!uitny-<^0! zDVOxT#hU=MB%Ky|B1ibK&LX-ECa;i`58PrA0FPz+lh@dNUN`nN`a?^G(n#=M0i-JcY=#`4m{CwWEEa;vhpQZ5keYDx51i}k+I z-;ww&zEi*yjwF5SnfwrEt2N!d*vH|D1T2Ln(V1|gA!Ti;SO802;42B( z6Tb&t0jtFca*6EWVUau&F1J2moqKi+lr!bFX zrz~EZG%Wzy_@XqCSu$DgUU&N-K7rNSM7``WXnDxT7HGnr?P*wpq9tbwl_}1R>lKln zSvcGU7FqYOuQcn5fWNhAK~sHO)8kw>orDt4On)oVBmOttVu$Q@^}KG@gMl~fA}SZk zMN^m_;E#9EdnGhW7Ox|BxJ#EW;?CeAqBEmi1b?diJ5kD?0cCI~3 zkeVhgU><%{WxtusZtvFeD*IDTEzx z3Uh}}VMY7!QW#NS`Ma2|@JECGS!ibUpUL#PwLnFI>b^>z7g8xU4Cx0jbH;ao5Qo66 z@Nktsm&C^n8QE~uA32C)+(cJEo3Jj7-xXQR3m3;X4)u@S$lTE$si+Z1Nq>igd^F~d z;*TFOa3g$laxmx%LKoLa9^0?^Fmex=MD(u1639!ADXG*OyJP&?G?XO1(7RM@yOQ>z z^Ou@UiPaYj@>h8?c3{;Ermq`kU&E5RiFEFjFe9xSDvScDq}3=BB72BO zBk3wj9QC6yFnkA+U>`tz+rI8_GM0;&RI za33yI3SD`81Q(t@O%-!^p*_Glx(uRV3r*Su!%LhW!y}a_0Rn9%@q6w$9p$FaPy!7~ z!ecI!>t%u`$8QK1w|97W3*-d7qfg2p!g~2L)R9W{o~0?YPi3pjplIz^OAer?XJ{c3 zBbRF8UCm<;aTqxb^d?_acO7T6#(%DfH`S6xgEBusLCO>Lzhd#gmS-gXv2y5}SwlvW zIms%{;l)OAF`IUb)6x@9BC#wocaB@gZ?cM)EZ#ay)Jh2M>^Y|4yI}T*`-Q&aWsXrm zj^4v|7^NS|!l2p9*J~{)D&#RmJgOA(muVkT@}v(w8{wHYgcmQ&tPLqko>?^FAUmu> zJza=eL7UZ!Q1dj`ec?;6Bvz@O;Lym>9mr#nsvcaoJ~YwPku=JHzslZDgn6tAd+buT ziM_xodyO7cKLA&Vd+w=;`#21vpT?P)GM8yy@4@Gg}%&won^fsf4+WLe< zuUSQnOANY7pbM%)*jO_rXqN)xl_K_qx)(|=M4QnqU#cbKl!6$tldXz0saX0cJ0E8d zY9x%h`m=>UufW-+WEcwr&H&$Q@F7v%)jbxkxQvuAp!Y?ZSZ^zqhaFuA8W{JM;2gYx z7nrD(JkrfH(;z5=L3BBl@JsB4@#J$Z@mBzReO&TEQZBoLt7M%%QKZiujAQVH?3Na# z)9qX|+KVN4CEU)IrE2$gHT2}fJ*=pqZ>Xo|MS(N~2insxyd~0?Wpl^cQE(3#%8}&8 z43k*N-1eVar0bbjno(qL2J0Km_%${v)$m2!bPSF|{=S|=nW&V(AAFa*?9Q}`%{?C5 zBqeOu2R0uhQG+CL4yq?;3;Gl5d%EvbN|3d)YcN6x382X)y(kIAR?<0diIHi%VsR#) zGmHC>Ex(s3=VXij$}Hq_9c^QSm<9EKN#uxP)?<~@({~xQj1LlxrIP5idZdF2-6GO` z+~MA0(@NxX(xjy%&dVwBtQ}hf*CHmKP&^=z)4>e-f+5)A@*L=Y!X#>N>K2+jNuOKg zdOAJG#3b3xlzet(nrO@PGMg&1e5!JY#}!Wqg(nK)k{t1>&C4Ik=o*L{qLf=L;*pFp z_8f2S!x{cd9^yAK6elG|Owj&8U^X!1>+3eaB4bWt2N;H$GKBCBck{6V?@L3;wY@m0 z#s!1FwotAjN|EMWYVAaB3P?GWCmymD+aYO~j-#NJ)U_>%`0kcO!r?L9Ej1fO)}Kt1 zpiODPK^yAn$P{M*!MKF;XPwe5nauAHeTwPI?b|KKM`gYN?W+6j6N57AQ`(lFxISfV z*rL6FxQ(Q<}>5~DU346W6dj`Sx3)Pf>#Xe>n3qf`P$jM3f3sH0HDAkLSHYbF}N z(}jw3$3)>^l-7~O>H>jK#b=aO(Ls7`aw~gly3Y{Z8O;n>&|xH*7^PV;2rNhGHh^sP z$2+ky`;u?-E?U@;a)Yg(IkUB?tu4erGQ`kP?^4KTA7oeKdo|DQm<7&aInkq(R{nb` z_LeWA;g;ssCD@5Ec02yKalz8IC77FVGecD-d1zN87^sY3!)dgpt#MA%!X{kOlPNH# zxotkFdl@aJp5{eM>gO(M1Hqc5w;6*(Te(6?wI)xwj{gG!zM~`ShxC z+q$7m@T$i|&ZM^h<7nc&e^bR4=p?t9QWFq?%0>(LDPUyaZN7nJp=<3OEWLotfTi7* zWZ7!a0S5h#X1?oRWU$*43sW+~`YiVghQJq$(~Ve{1En4~fO!dq9Qo2T;_$~oiIU}- zT2mQnrdwwk)1dI`!L?!7>g*$TX%TD=ms3k5jQ6+DQon(`e?T05yT39_m7Ty`Nx*_)R>{@FlS#_Qz|YFOyh=?%xIM>hUq4*mFfIxC{T% zibg2AsEAsMijW7NLVSRv7_lc=b4@H71lu31lm;k=TW3*45%OND4OR!{P=+hH1BpA4 z9tw;y@hO@wJwv}VGBk3>&ZLaJMMZ|Vq$sq2%(cNlcoAiqd=i6q+pkzN9>;Q6hBhQ= z4fG{1!?}Ax#EX{8;LJ~@#_7}bmI>t4VF#p7pT2sqTHb&b9S+$Nr@XHb`89gidg*_$ z4qex$_$Cme5h#(Uxina&&%^*(T0PjDnqH`{3b)tjtKbWoC}}7RR+ev;*{LGMQvyU9Jv6t5=0fMr2ycgs^mlp)x4-zEvsxdsp?8qM8UxIst(JgY(y2J53SLM8npoM_?xU~40ex!V|a=#y`Q8z7@1lG@ld64x4;) z#_bE2wI)(BpSCfdjV@qElnsWwrwwIk;}SjLpIqJ@^e6Arj+zXJdiCI}*5-ve4pLH2 z=<`6_(D#))J0?=C-W`O??t!<+rrN7DaL;t|Wjy_DY3w@O#F4CqKgf7y=WOlCD<-m^q&^dV| z`)ILY2CJ7Em)jJdK3Q)VfsCsg`RW`rI_KfF`drLF9nl-c;QHCxyLKJ^xax9;vnD`0 zA&P|5MK4Wv*5g#Ix!Vz(tT*lsVWnO&)H<~3PIz&W^jio0$P_`kFvp=>rtn)$yqJ`8 z=#Hk%=og~nr+LD53Y6>DtBVV(g5~;S6AUmb4_9sZ7b-QHK4*nh;c`8RAThF~uqslnyS3rM z>Z2`N_~|xH@&7Zo4zNUzGYK; z)P@0P8QjM7!m5L= zM*LNzpNosQq7HLC*uKKb9Oh|-;n=WM$)R|;J_mdH18cA~(SH^X>#I8dVyf@-cXpqK znL_5O&T{<&6Thg?>E8pn3e4Rx`4}Uq;#;@)7vO(w=eVv+jLb}w>oZZ{{+bbNxXXF8 zFbvnuvM`2v-4?+CLbuRROgj02!U&F&q35#GsUaL1DAPtAbTy8W;W&U4(Md1hIQ9Ge z{&Kwq?TNWYlhf1QJ`F|si%iZ6&g1qd0|aAXAEhtDZ>uajB168ST*mEf=-#D6a8Lw( z8dGkRTNWbCDV1XjRhRZU^g0?=r@+j+T{`&q&`_R<5t=?U`N} z$(3f~O{*>df=O(*;&_JtXhCAls15)tc_TTiG8w-?aUdi&b2yY|bl=4n*XWCe1|Lh0 z;u3PHjv=U3GQ^#d1kYmOP;w-T?5z5X^8_5olnMq`p!S-_4%8bg)VahJY0H@z^aViE@?o}L{i06<9DICkB$$XN@v6O5WfpQrJt8L=V z^*!<7ONd)18ta=Prkhj;a?TeuQ9-KuCFgrEzs;ncrO|4Wbf!kDO#C!Mo2{Zhz0#Dc zuQ2h46yD9~5k*|AvikONeb%3%J8Jai*2Z%EY?DdsF{NpbD)8s2#kMO<0ux)VFtu4$ za7#l@2HL0?ByK(1EQqB!y{~!YHD>zr1hLjUD}!qBXfRimch0% zoq+?9kx@{Ij?3%}%cVSHdoiYS!G4}BN&^!+C?g|sl97)}>MUXhO)h=q4EA%~dI)Fn z99;W1QjI&@g>)`i*K$GdfFgIQ zVkNGzq`E{O9n!nzri(6+GJT4Sn}afM(iH5VP3hQoWV=VfjV1->iLtfbJd_g0f?P6$Gb(`Gc~bSyEqNCZ8bgCyuvIJywv9b#jySD9M|cV`{;Q zYIq9ArWUaGi>bp&^8yA3BhMSe|NnslS?=9X%${m=>M-TUfZD>q{_0S5I8qrZ_rC9N z>Q@!gd{f3YCrQ)k*M`lCj!FjW33R&BuL{|@#aFHw8!R%$5ieQyx|yo1hk1WveAr;s z_7ny6_^3W>%ubIi0d+DpzP=_@xvnB}(TWM6GrN)}n}V$~XF|kTZ((+F&0JuThA6r) zKK!KRswB4HByB_UWx^5Kw%XgJ97uOv;4f+cKBp!7W)~D_S#Gfm^-k5J>U*4VC&pL+{?s6 z*PN3V3WuzS{k%rZ#B*WDfmSMMVm_NJ;!34EbOv4$IRoz&iYwv`%vTS-q0VuKx3k;x z+watz?b|i6DQHpFPs#DW6|5-ozr;mODa^s!{m$yECXxk)uK%?da{M7Q*GFcWvi+0I zvytbQ3M7osw3_8N1GXCC+szFvSb=1}bLg?+y@Sg!Qz0=@IsqJ-77Adn&HH|&v-Wey z5uvoAMZ|tYj)}xyGwYS1MQ*kjH0dBOB`r*R5(QvnUE)lyZ=A&b9%6f#Y^$cNc{G?V z{k;m&l-kTZVQ_kPq3{G?IoexQ%@5_$!v(y-Ob^GjcMDioP@70={PWQ|Z1Yh@c^ySJ z^!r=g0odux#(*3WNw-Yvoir!U>+jhbcCUK_KVlm>6Mvi9iP|1q(pM|Ai!+fj)d4H> z%M)%J85tWMZ60hyi@3%jKbVOO3$>+d`0DBwZzLM1e655Rkf$&HP^JQW{9(VLj!DEE z(1bKsga363DdZz+cGVe97E^3N1lH%KDT@dv{jmRR2OG7C^QKAsZ0Z>4>G3Bou$Ee9 zfz&!U)QrD}h22lA(lf9)CVn6k@DFcUm`krsQf`CuLqh;5!%GDLBw(*K0wjC>1aTB( z&Dtvl_g?8l260Y zSUMLNQ6V|fXEa4H)dnKsW9?4=jrnwLCEG9Y8FE@h9Y#d*iK3fLdK8{1BM{EEv^K9; zdCPR3J44%4%YzJ2HL??P={g$FEn7p5BDT01vu-OKJX#SSe%_JR@fo+?QpYbM*bJ-7 zR{^EC=3DO9EGF%X)f=p@cOA-Z#?It*XmyM}BhigqoPjJIir=+dJU}*a^$d;Oud=gU z0mRq8=YcXOec%+ME~CsF|8Ffj%OcVWq*~1JJKvQfPCLHsO6vB??n3Dlmw3->g88d* zg^Sw>lMLCJRc!l2DXWm#Xk-DHf!*u$gPHudX%sK4j>e+fn`v(|-JUHDFJLTMcvuD= zjiZWv&q@j`3W$ao_6|5-~3&-{{91-Xpv*c4)R_OaB$TwZn z8i&WupbJBx>R7BY91fjBS~?^?1qHANzkFfL5Id@DA+TZ$XiLF`)C z4n?bD(NH9a?}8WZ#@1I#?7PV|5;n6hCaWrM3Wj5q+fuI`!`7#Y?PIhG1fSWBL#!kg z_zSF6NQ}ToGP0SQ*cDNzIcmMh)(z1A$;H*x7~f(gR7`Hs4jaLZ)Ngq0jPz{AL=na; z7V(KOe|0{Z+!^{=-=iNa<6l)C*yf$LY~h?leE48FJ8D^v2hD5NWRl)I=ptTERmhJj zNCN#h9~PKVlAiWC!X)7T0V(k*Q1PwlP4ooGkxQL3(WC1gy|Y;J6F^AN^?EYt3&L7q zwBMGXKZEyPh8!)AYFpdd3P>7i*#c{?!5GP4FNkF@E+JQ zKn{S89|xrMZ}W$QUO^qkVK4WP6lrW(T1*#xbC{=47zfr(vyB6rOKLS8<=-C5a&Jo_ zzFyx$m|!-Dk1-mxhv*Z__ywo8A-_gSg2P&F>}Y*KL24zb8T4a3wKQcY(bLTyF`%^*y$Vfw%iD%%zWzkB;w}CT0)>%F zrV`k>GY=B3`_XR_Su9aW9iSHWY##^VINh5|0lY%t0koA%CD}sfd)vlReP8bascx=e z{!D}{d?moR2`mT%1Ts-9si#x-c$l^Fm&bb?Mly|n=auL-`o!cl3N^GP0Ggvs0O&H_ z4^D>H%>QHP{R@0OQu97jh%%_R2J%?$MF=)9V*Pbe)1}qNqwiHlyo(_i$9p|kb}4;+ zK#~L<=Jn&?-8jBovud07R_X-$>0J~*6)M7>v5_58hWBn`4|-G1e@0wf%L3t`*UjEO z5f}~Zr`&r;P6J{2L|puY<01fscQkJN7@&9C%g5v69pAwaJNaB-h{>I#?I2po5HJJZ z1kCi_xyVPj+rc`btHmS)T~=b zeS?Eco0rkPN=qo$F=e!f;j3Pk#8-^jNw>@a4fyzPt1g8(#7|99&V2vbI@!>dC8*5F70iL`o64` zk(zCVUL4vPH9}z1iF9igJujJW%wk_-E|L(}XC2$-jQzxhJMXdcA;7Y_LoRE2PSAnG z&q~i^Vnb&|G~lwZ@p#3EbLCJw=+~3sjRGQ zvq(1E71$e<(W+=wu*l-U#woSYNNh^h^5SUp6QHqnF^pLM+(faFE7+Ruj%=h{+{x`k zPc9IS{3zWK%Id7b6&0vH>J+i8ssui{sxrJGOU%g1u)P{plEhwXUFE08HX0g$G zMEMuY0v(ot1m>Rt$Lvqs?LA^@1m^94q$J$+3T!1PabOk+Y73>e?~3z#&{=e$+4J2 z5|&>gIZIeeK~Zu&nu*&oWY_~n#op9m3KajWkWklXkeo-i#IU2njhOB2LW^VgpSR zV;NrQbP|0T?b0R^D>GIglMo9tTBUxhW*Kde>C{}M}^%QcoLY;BG1G!DT7n{xw zu7n*yj~5?p(`+#m@xf4hT!HjTfUgVD^Z@^{*Jv6&Pj$cs<9`jvZhE_B)v92$yt=)7 zRWMRsX`;?>IYL1+jRK+ah(`AY%Y*Ristg3$+iwa%ZMb-6Re1=1U0uGH`eNll^!-mK zGX^HkX00hA3EWM7o9_<3qncV`!Duu##mQEMG3s*2!Av_{G@a;Y&UF3Dko-r+LnQ5V z)ftWWE|+W3t)UIao9@fPa0xQn?t>cb%~zdH3B||9QBbj%`h_C)S<>=@aeL0dk6rR- z+4es}@JpD_<#Jc=uxy|wgmQnHXm?FfKMvO!4KANc{K}zB4p#rwibSyqNMiR2*WMG& z!u2dF`q%^NjEUm6JRLfoc6a449^Ya?tV|PkWqL)ET;#4=&u$mLbhv4E<_m`Zv?28Z z`7<%{pw=}`E4UvvSgp)e`dihY6x`!DMmUEJP63Ex#$a|o3|>$DmPSAu!nN=*9w!or z=Tv2iK_TcIrx08f6W?VJ*hN_+e4hjJAvnzKuuwXP2dOSXahfXy72FRK^veaon&{J=ltK811EpW9B>ug;;Q(MyaBA# z;Uz9m(p}@s0nC7BO;&Z{G~uyf#-7X?EzX(s7W!lMek49AXfbhN9rN-`|Nc7n$h)wH zTp2-Xo5B^>MP?>Oyg(KY6QD(dKb&e_jsuK8v2CcSFsMfr#gysuP-aCnxq8xV75;Un zLQftc-JjSx<2cf2onF_2fc5KXwt5P{jOrKD>kL3~@+3Zu0^l6qk*)v>yD^>KoKbJ^ zjK8#`5XF@#M3JQ{0h|uI(&&OvlOG(`Mbv5_ii6*u4hhWXeDQ)BI9-C1Sy>GlB`(gh zsXYZS?kUI*{ZBhX)n?RxO6Zht~YZjKQ1eZcn;*Mn+4D8FoP!gcs?-%oJStJalGv1(h77m>P z(UL*>ebUbBs09G!XyOP5?;iVRmjO^3&G^%}_L1aTcP_M|I4&zBTBrRpMj20Gl(S}# z0a9Wlc)PUG04c!^yQ?k^cGx#inUFsbyET4p-zd_c9m+TyN7rx*L@hSk! zn{rYB=6;8QGyY88Yyg;NurOqnY?fp|m*!b%Y=$B)oe5Vv{=Wj( zh#W7&G8nC_bOX3_90x8B&^aDwTSgw)%GgI9xyp;G-1?OV2&SWWDeEg@+h>G<#e^^c z1e1G~@&Ml9Y(&$^Y^j$;GsN~8OT~s66U0@^YzD%}c29+^mVq&*msbY>G44aw$*$pz zCG7lkX5nHiU13|fcysv-5JfNL*JQR-hhm}Fq3JxaT-g)FhBWcUaw`(&*jvj{Jj;#6 zA~4TEzln->mUFtTh(DPp-d^s&OabZ9QkE%;?O@wZ%eI7~;z-onj02=Zrbh)uqlPuciYNOmk1t zW0mVBz)FTAh5Z=gS`IV&$>n@fK9I#VF%~UQPiG~;NO+;KZ}vk!4aVZV8VxRxJ|{FK zU5?U5`gy_9>Z+N*p(;Betj#U~5D32~crxk^w5BM@qv1u#jSr(Bth= zK0kpfo<5E$YChZ3E1gw;o+)oL@Dnjc6X}+DLgq@tFfgkvW_8XAZ72MvQ@i10Y~8d- zoJ`o~vUqVNuf$l>X{(NZq*#xF81?weEA8yV4w1JwGo$_e<$MvtGnQdg_+iXl_`T$K zbLCzTL8}ANVCd$?*bbC>)s>;jTN)?Ws^DkPnC1*^YRBy}flze_u5On%O1C8(w>D-( z)1ky~OlNN^unf>LRqd2U?PNuP8R?ykJYPX!s#TE&BeJ~-m|#eJXHZJk!lF{Vzv$RL zt}EjH#va;jM*+EB_{F8l9db#*)@KL*XQq98C(?xGh7`H_< z)givZY4Mm<{LGez)4+JP1kIDMI@2uvMxA(Q#|jp_YqtC$55T?n>Jo7T2lyznUuzY2 z*`4%WLOP#3;qc|Mm}$wBUT?v(ftFZxG*UIw0(tmwC=A#)6j^3X9aNAN1jd?RPH0J} zD(nb`B9$xCg_%vjQ$?LoP%kS(n5%M(t_W0BSA~$8g4gH4Mt4@=$f{;+508Ve5$!0n9_>SL4Nn!LjxyE9oqQJ!VM#@e3MAsyFzEJ#;J` z>HuJ|1{w_LwGwT$DB^|@%#3D1s^HX#4!gpE$|#7w|BAO6v}Z7l7sVICeI9sQd^-LX z?`!~VcthhXis9k6T?hkg>!sU`E(Fj#V7GdCsu6MY-vPHZlo?t>U#QQaJMQQp&oP^}q)VGE1!YM!oec1yOI^C9BtIb8w2q-XR7BqF zN*r1aW?2nXeq?knyDIz*FZ(?cn3wxH;VX{wzl@18&W~~fhWMw_$;~AbPfNX>925zxldM7?AI2s zUsuE0Gz9MVIRDi+Yik@w*{Vu|`u$xN>`00Q+dp)*1Y9pgg8f`E;JJTi!EQLtf}Lg$ zfCT%#BfNTWK@)|+fc>@ky9C&unNF07cT0C+9Ki89@_!=1))^ewd(FfD|4?9$$g2!} z(?4@yXE{KDo$((iu!psOp};=R|Cs{Y8Z`FX0d|-QQFGyop5Vj3&a_+_9_Pa@>2n!` z*mBh%#I{LQo&L_xxIu;uD3B@{IH3O=IxPP?AGY^R*KKXzGGY%h+Ct3FoA6l?Niy03 z^8cyrJm90M^7enut<%qx%uE{0WYX*OMv*|I_YNXLfFLSDsDcmy6=hdNuwelO3+mdi z_g7t4tZPSCU3)L9AQmiZ{?EC0CYglbzPtPW{d~;5b7wMS%02g-=l6Y{pZI7PwJLuG ziggRKaUXd7gas1D8L12u%Locz{u3=BFtHADK?e^zGQ-1SO%Q*L^RS}?Taq$ldHV6J zl1{{aFG(j#@-6|Or+O|WrJRUeL*?5tMC>CVVu#6&s(w}gv0ou&HN*dRrD9#=>MRwT ziXI@|jBBtT0IcEtNO-0ijuAVq5IAJXP&gY^}&Gk}+C?cM8&2!M0U{WEW>4Su#*?*5!keMfBp^ zXs!TC_A=SFif~G{Uxt#Mr(`JEt3b(~Ds-o0hia7UST3b;Om=}m{7kM)hS806zCqlo zk`{x21d@3M1@!D(P_lCj!d{J%eM)9mE3}<*Om;ES^_OQr*^2} zsDZMBaos+7Vm^2fjg}RTbCG)}y(q)Xu60VSMsi&~TXH03cDBaMPU^zUCNyR?sWG#n zA;Zj;1lk%YdEO)ZQxd)`P&O8jCruG>vwibi%_W5)Oo{&NEKA1w6K)a z5W}TjOJ7oy0zms*2B3YNN^g2CgNQFNi0BCI@Iv+%Vk%04@ro0}*+el@)ARY3^GR_G z&KnD@nc22eFfP(5TDUvc6Spt29!{h>{04o^psIzIr-?5LVvOBP`uf6i0npwGm|itM zDx05GIq&)oX5sm&R=wPBeqJh0ZIh)1Iy<|oKxb!<)!EsQ#?C%32?mXw{iDv#KB%#?TmU=ItKqW~ zIX=sVD|x|GXot?uJ`8rYT(z2MT{4ZbJrWXf;8|btA<(mPGxY3w!kPuOIPo2kR@2&< zZSNH=h(=;qMLk|f*LcZdyZGl~>C!^!FU7vdE2ZiNtOml-R9)aZo89kZ!X2fe#>x^0 z+t7*1=7peDrx*=*b8y*Djp))zvLHB5# zD)nO1bW08``@O?ahXP%*1>JE~HNY!Qpk!I0M#+9+o}4W=4WmjCrK5F7)>+}1Hy4mB zqI^%oMg9(dBL{3WNS2HY$m^-{csW~7oi)+edZau7l-)$hRTkq|fU;xh!^-M1$FR+A z_7q04VYDwBLLYX=h6UKgg0m))xESN41Q^yW4fjRY(}($wBKB9ImhF&}zQ`v0{?9ca z{G9;IYppxr8z0Mw**}ArJx685r6#4S%y3;ffY}>#U{-7{@rP4e3rSJe^3g;fvwd@r z*$_bHd>{BID}d9@IyF1E!oK*kF4Qa;Od9*(ayFD>vq;y1T&_X0s3gfiW|wQoY(;4X znRS^sGTT~U9@M~|u2eZA+iVn;8_8?p&JH1aSu1+j7LX5%g{ieed5(qJ%aX&$iVcNy zSEalvi^$sY*kwRu*V%bUHi8ZP?`(Qw&O-ErBuIxfpX?c}CC<9$5VWAPzt9NUR9Q(w>s?;*MumJkY0b#mKNQEHYWME0YL}ZiKigf^ z#%NhUp~{CE=B^Mbw}E`Z3I*@!dY^Q4iF)t1F`)iu)j!0Y?~8dg#W}N z^~q>UtUfj`*<7EFHiznC(K+eZeBfBA=469NZzl9yN~ob3cr*xeyvuMxXQ*e_6xwygziTM+Hkuy3fcx15cPn-cCP=_LYf z;H2$yVyh-%w=8>!7`F++B1&$g6EnPRM#auBx8pPo8vx#-D*Bu#tcHZWMq&@@5_WqJ z685W-cw_;aGq;6|Gq(@qDA-x``WXprth%Y!4RKAqu46g)ttz38*cHDuCL8D+S^CsL ze^7+Cs{G4aZf^iWK>zH3y22@fM{_{j#q=`74V++h`ZwEzOU-m~aWIyA55o2fnFqZ^ zehEnTma>N&jN`apYB+A6!E|Q>-B%*s%5mI9Iw$r=m8`0l+=X-yu?KGT`i9+-Pv~EJ zCUIAlHgXzlO`?2d%Q=B%84S!bQ0$K3(l>|7NfHKxD;9`_r53r6SzR3*E|OHZMh;63 zB=Y%Qjlxag7${ulfM^7FBkc8PcAuCheiRalMe=~)3cMt8Anuolec3VjWZ2OHhiy-7 zaAaBcxu$dmH<(q_t(S4T*vlk#M8ZH8v;?MYWvu?c}5c?z1bQmfWe&=*ZCVAl zr6*POQz~3VlZ%)7f2tSbW)hbfM>m1EmJ;P9jy{!5Snq-dS7@6B3^8X_r)aF|`Mk^xPBjFf&o~CCDv1JB-4#?cOcjn>o&qu9sxkCdB4GCY zD6vlsjW&Q%Eg%BAarlURd08VF92RjSk9b;cA`-kbSx(j7b}X)_6IDOeF;(OjN(mx3 z38@W?LoEZz_Lw*VojM5{Y48yE9(SO*&RHh$K8t(N z_c7Dxv^2&*pA`71G|kRWB|Mo<%?-8Gc<>A*;mC~wt3D==4YQC1GNGq#98YgLQ94K80>MsS>?_A3|xFBM&c~Yi76&uY@4Bk)E9}r66={pW;8p;#Uk-!XsJ~!wyui- zg9YAqGspjoGI45)fa9vmoS65 zj9g2_FK93tX+D|X4l)1YLTvHk4E=me1$XzpB=0w=s@X7rD5_aTuC2mf)Ia%l9-Uq< zp0$L1HApYYd_Imt){8qTGGBl<=M|X&um6c0U`x4uP(#=_H{WT()465@Gw8y-i#(B2 zO~4xCd!vrE0fSzQ5Csl{cc5+i$(WAB0*%6}Cc}V@#R|v?O}wr%&rc#4QBNb|HFn-k zbbHNE(!>l`HwBKx z3Mat{xzmW|3EYXk=1T{Wb^ksgx0+~#+#cakosh$)h-%tCVZV~8rtvj&3>iLpHElJu z`pJG8O?9oNT_wQFgVl9vu_%;^5meG{5ohaVv{!`vB3~(Yt)U&?rG{3m)zJ2nZqWKU zBAtgUg|kKXqJ!eS=obq6fuC2;mY{mJgxAHEkeTk}65=``NYBWYy_kqHe3?uiq4M{@ znPTDjG+&-f5k*D{^*Y!2nNu^m2BoX?8su3`Sh1*;1jFf*NJ%gqItg=`x5)YWDVd19 zMf|~ZxRt2*g76C03Ppa1`s1;A3M&;o(!&%JkN=^DAIJ&ZAJJLF>Mdfv6B*P0<%d>T3#&$`5Q6b<3QQ2u8CW9Az54>P1S{LI4V5Bagl5yd>SR!R4j*yo; zl>`u&r3EdNT~5aDNyar?Fqx$e4d}y-9jowC*>u`-#`&HThCrV1$Kc`COIFH`gi8No*0+ zjYxdHSx0|e!Xr84?;@J;B^N=bILV+rE*4!x>8qAFsvxgeIxxefcvNC39yP|KL5Tw? z8)%c+$@DlVS+}z}APX;KUCF||S+ejiAPcXAhW(C47EV5#EIhRfS?Kg{S2_cVY_BUC zS$JE|;Y1M|(86gwqlFjsgciQ+KozVyix%F^a?!%S+A?V2K%j*K{~fgO-$^8Ll#Tiw zt`OLI2Ul2M9XQ3#YBZ*hG?;_|E_*azYRw>px7$nB1UOT8VxAA&o*-??TcIO`6B&Z? zk3bae=^zSQa)`oWOC#E>EPK|{smxeyI$rpRB_qwub7**B#DSS07hndG$s8{XBeM!{ z;W(F!YuhjvS#QzcLTWx7F7!D!;tGcx`cAs=W5@5)g@x{-*j+|0u7WO%8HDz3bYYXT zlP(;t{x9gl1xKI@iJ9v#2L%W*OF?UNC^575x?{6ZQ8I8fCKo|$avc#t{K?%Dg6Q@9 z7YO3*l)R1JG%{^VbcgiG>CdmfMHtPMuoMUK)jEJAl%%|q<5&! zAJ&QxNpktahXu7#%9hALAYK4l>+cbWO`@=^L%vzqRlYe$g?C5<7uI)4u6#4f26G=~ z93SUFHi~BDn}<8(n;nAj!QY_}Z?@6vg;o&A97FurLw0Kz;)&337E0{hBWpNM{GNsb z2yvnYA@(HUyw?MOc!mxjzQh|^^avp4w|(QFt7PdHC;e0rWCPS2wCjd?)8rtZY3j|u zPW9$DT5V#7O2h%Nc(@sw&A7~6&k+`o?hmm3x@NPGYc?bnpvRVu)mbQxQGyOgw1SKwk{J@gwtmcz@|<&$?SAmBIxf3Ij-wk*IkoT@yo;j9+m z{Mexk7U8Z+JtY(i6YD&Q56abh>x7Fdq45XOBL%v>PO=hdZ@F|+9ot*y^CZuy!B$3g z01VMt4d=2NtYK{AY)SfO_819P2m_0`wBw6ng-Rc&@%!hzQ$%-H%WEn5mViF&r<4V; zX;x-c+LvvK5La)*eqv92)=N5|0f+l(&>nuRPTX0|UTVmf&ZOnhC^8zLzHA{L*;nrg zf6D`99E#8Y$!BFa90EItH*<~J@yA#wq{mL&D)-xaoU1<@^1>hjp5Yvr8(4+1!<`*b# z-WJU(YnwUFAdKU=CGS^|J0qmcuq@*doP!p(=EH1)%2tXXj~rulVe(%pNAJbED@Vr~ z|0CsSNk%z(&_HLHMJPv?TS$I7S2@~GJC&m=W8xAEL0|JuL9UN=9NPYm9BOma!RqqMqb-gL_e9)bv~*X`71nyz*~^ zz0oIv`^(v>G^Si)R(>V)G6N1+Q9O<*e)!l3TT7?nh@~nV{u&)7lF4srg#FQ4?~a`z z_!G%TWY){%cGfL$;9z)43=6dQCbBMq#alWN-%LtjnY9*L1gQr115S<`Z1%_e|6&c%k)KnxsKahRU(1De1*78)M(^h$kMA;1w- zS|-X{+-$RFsE1LDrUMp?Eg(#XQF%s*U;aTzC?fXS}$OFzV5)UI*P%MI8LZtr@)j=AMj1B=%Q# z5n1J6wuyDV=#L7+&S=Bt>B0=3Ft0!{Po(Eo88)j`$gYj&y~%WQljXFDhE^hN zX+UCRpIBYsn!!U5yzj@?>ZifN=|dg;O+kAgPT1|on27OcHg*(V|7BfaFl#OhrpIr7 zm;aLi`I{}vu~gTiH^VySRa*~{_0BOwAY*v+W`dL+yJ&COemWz<5h6swm4;(Tv0GU) zb3FEY+q>U+Ry)mB9X~N)~6wO)%*O*wXl8(k_7AV3fv_B(EMug^a zVxv4lQ@lel!H(xtJdQQ4<5@3{P&8c` zsxi31B-Z;eHeC^l6A0%VJYh2{##fj}P2SY9KcEwLVOT@|msyhyL>y$b_X!jEXyQyx zwL1D7V@2~PEFX9%U)9c&;Flu`@zk#+J3I3vC(nRmMZh>Zg9wQ!!ktzF=CJ#KlJ_OA zI^+*$)0?KEM+a9n0iMiRC=b)3?+hPJwO8GDl2A?L5LNC%3c{)x7)HrFn{qrMqh<(t zHsx@G@kU~tM;s}OO_*b_)gmkpl@WtIezENi9mfmDkYTuB8o;htyw1OV3VCH3iiQ3} z-Tl+XX+U0l;cPvY@;O^k>JkHb0@Y=L7(+(PXWkn$6l&6p4b#aZghOY6JTW$n*eQuk z`=D$5;@h@wI$<+^)8d7@Nkn$HN)vcy;&NIRsYXAk8FMe0cYzik83yhq$Jufk8?WU> z`si!;g6esZJiqcW>ZmR|Mnz1BzrR0RWj_i5lQEI)B>mLK?Zhd8BYT0$KM8uW;#*Vf zRY}QF^{1k(GZ5>DFvU9mYxZv8iY{_T79+zq2hF@<9}85qi^^95-QbY?bVwWtkyp$J zA-;AORqrCKni$!&f>k|@u$zRGCw-+z9wvq>#XWJJ9jQl2o^UMQ+&IoX<3di&@sv*< zCCP{*Hjg%f6$CUl0MOhEf|?h9L6Cb3U@REBT(AVO(LW>nq$J%z(ZtJ&j9Av2kn!;l-opu)MIT28VY$MyA6p&~ z%ahfAS8Pj&^o+}%SVY)oQY~u}h}auFsAiI|WJaqd!TO0X7U7bqxact+Lu(VUe*g+1 z!9>h0&aFs9zalk!w>A-Xi&cq)TeK&hP!zWqOr+eRH!tlL&1Hfo@{!DeKaXe$Bw`*> zc8ZmW*dDnu9CC?L*dRv*8`@nzEs=pDYm?oM5N6kHvON$>wBbQvL_3~{h!4@^q3kBH z5`q780^Vm4T3qC2@lMlD(eR=`(b@BA2Wf9*uX-gWpG^Gt8UW8WNUwW&@Sysp*NC3$ zJItA2O0=*vO>!E~C34sxH4C;&IB%z1Aq|PeXVxK|;zo&HX7SctZV@3dokytDhb0fUZ}L#zSB6qSIvbC-x~9|)a;cw8Fmwq*$|s(dyudyBC=w! zp3N}WSe|W@>2k{Y_cE|rZ|`6%RqI7Fo=gtn2I4c^>>bA?^kp6L|2~w&xcr~Kb(`UU z!t;V>wsAhMBR7N()!+5jlXMVG!cTZS0XIo#Qa@zFjGF>D-Eat(Vc4%VpstRf5jeUH z9)ApRw+q}&sf8?N^LXb1g^SN6lDlpfheyWpJl-Z8i!~9?Vu%?q)NUezFFKgm+Iu!n zY$3ELG7|Xwc~p53SsnQd2Bv)x^IQ&xv@t?`4 z+;2W|)nU0{-K(*875*8o5DWp!eulG}9FQ{G~o z1!HH98zuwpUfW}k?L2zO)^#Z5(Z1R794zidg!i*Jm5^d`N6tue9XW%KL)?MUWgyy# zO8`bheJI-5!S9~%u!xq>H8W?TRUMTa=r!c7i-)!g1aYk`lAd+>zKDLW?qDpv0j0fp z2(&hTvpo_9vpo_9vpo_9vprIEBDqH@Nu=foY>%|SmkLU~vC}sgPvTX| zdn9&=(cp_;VjRnV1o@9J|B>WBs`!rpTdSsh(O(oF-W?ZTV-`mnMddfMqwZoe%tu)l zQ#77Rp%R#F@qklLGfgd1_Ou|znwwQ%%{ayUvw?eBc%cmQNX=1|aYT(K&FN4G=BCNY z1V2wjy3L}~kB@at3f1sO#k{}ZUze-0iJ{*Mxsjn72Kj|q!ys=X84MCv3M*NLzbb-4 zQL7dky4jc$!I7}$vXF5tH}q61XCb{R+}_C4&&>^8Y3Xk|tHgb;HGw*3rhK)pk=sztvaOBq(z>)6IdZdHG7`U3s_?oPy zY>35V-9RnFum=<}jBE8?!O2F`zDP*|FX@YorA0OI)OMYWjC3U&d$1{%bBe^cEXZf12_H!F-b-kwZM%GEcss~Zn#Wm z@Wc&_mII5uAq;0DjrN)C(2Ba1x?)~>Cl#(VafEW2aeWq{>_bY|1a79%IHPYVpj5dx zY0(o#86mH`<*(?Y9qx^6%xKoy(W%+*oTqi?<*NM+=B*t(e-j4l|jw_)#6G)b;$+H9h}-1g_`bgS<$@k5067jUT0daPzQ$ewM@Z zd?T7O{wtu8&ZkN@pwgJ2_GMpNT9+dv0M{{9a0hPh9u&|cQh9h#Ko+U|k#?8zKk;0g zP0i5B%GOMpKuU_^J-WT1)ZcPi-u_7%@4!tJ3m!FCmNk!A4{gv^>ZT4iTalK zyhJ3Fm>WT^NS3nHlT6=7l4;PE(dLnvmwNpd>ow*6N+|a`CHCHtmHVH5Pr2vxDh&vaQ_p~ z%R>W2g)9pGQrGU? z(E%U>2LM&O!wa>3Cp)cz_~Ks}&F;vk3bHMad|hO?EDwMxeXmH|h^Cf;4DW!gf-UOB z@YsCx`m2z9(fdlFrB^KUML#cHN-p-x?^e=!FOF3DQomIe)g(uIpj6!~Vu=tG|9=o&oISrbpbAhP=^ zv1+)la=0(Oo<0WJCW(?&5A}MBQU&Cw#KL|LNHn*}JCeIoNwAxI$kBGj)uioP&KSsAI4+A(2~Yri{@zv1=Ivjp|Uw&?s1~jInH@xV_oE zCQHKR=CyiS`cSD$9Rncjt+PAo7@Z*OGZUR-KbYN>gqMC<}QWDbyBviO9|~dZ5iqpUpQr ztV-{od(ok==`pM(O$c|!xzwL+v6d0^2Kh-b3)26)W1@@DEtO&Q2`7<#em@*Czy)2UcG zeTivF1SydTFt&J0vGmR}m#VjT$v`LmMnB>%1M|~K9>aNU{<>ruxNSPhfnnrTf~<{| znhITYkQkjvH{5cPC?KdI8vYuzs?!`~Um;b~3I3F+FPfMG{3)Q}LnhU3_M*KIf?VZZ z60>Ch)T{GPWplm7_T(O!UY*Z=47l(ICEpX+)%mxu(UyhGSGZjnZ3#w0%ge@+#a{NH z(Szd#_PiNMn(LsxKX0y4hMLJKBHH?Sl)t;#ihx0y>`^~dY^zJhA^tOxn&lSXH_k>^ zD^$6@K$(g7w`g%5T&n6&y2MPF^fl*PARKH6a5aNRif4Jjj+hJ##afOCmdsjqF{UZt z9y3uOh-Sf*r7wunKp#DER|}-}XHr0_i4$?;?Gh(fHENaj=+LoMaP&M)d=*)NfE?t+ zpu&EBDjiOP@eZFs0>0=OM7^vdJL3lu*9P>l@n|xImO=1It|aP@-Pyt=uq3C^iUxq; zDm+Jc6|rW<^ixr)UX5Z3M)hlEg2vS%u?G(p0{+ocKy`AuH#4~FL@%c&oTTsXCV~Ga zxwij#95|9~vqkF%#LPH#uzN0(izL9RLSFL$uM7MbFV5|^l+N-9&_YHB?zML0l(p zI6S#~eKxuKCD9;>|BK-6X_^@#C$yV_5BHG9Zn522KZK-R3HI@Vc1LV?PvvasK$qBV zd)qNmDkrs@sDf^X=qa{4uWhfwF9VO}CU*ZK?6-7{?ADF!nKai6WGo2AX#&$T5^5NZ zqhCg)nJnoH@cw}io)Gk<(qqQ!dBCVUf#|Hlgtk)q)h;2g9UwVxO|ujqW=GWPWKCuI zLhi_VeO%bXJ-3h`CeB^lx>C)B#C4@91BvU%=e_7W+93{h6`lkGbf+46GWa-Yzs}IeB5ND8qm|Rf~LW*4g;GMWIs2Nqo(bnUMH;o`}R(ZF`(VJ`^dI^DRdY;fX^ z>!LdCr=7BsEi&n~6M4)k{8!qO#anv8`o^!v%jV@x5>co#B4zf%9#Uo_4z`@Bvf|fNDXr@wx<0nvKWmQU)>}9C#Ffz)khmvY(z|v}#4UpVFX525S?gz;xQruROEP^_9+K2`N-`3UUh5*?%JfNj zKkXjuzh9z)qengAJ>|O2VE;$i=~+SHZ2}l@GkC&Gs=qzX9Ks*-WuE7Mk4v0kr>~*^ zx9W>2svx{Whhhzi=^>}?Tx!S{bE%B4D9=*G^F?-n8u!Kgs1>dw!#%01ZR{vTvDK>* zmXhNN`J@(W!Oheor&UiX;o|E^C9Q&++lT}ZbZ~B@DxX0r;+Wm?e{nSy zif0PgD7Ih8N^};AgrkY;3bWYUCP7<6ekjuwF3c?|qRHKb`U%mE-5y%8_lu-1pbwLTvdQx34U<+l==VsVzmamaeW|Rzy~0=eSH_mlt`+D_=~bn&V#cj7xuIA*%Qzqq zjt+K2F$*pyi!E#mCdD^6RotvCk4GFR`-W4jP^vCxmzqLTuxO6Q0!BZA_{zyGM&`r( zvBJ!5GZ+MbT9=vxPdZc)jK*&_6v|ivTxtqjg4kM^wGy$=S#%wy$MIAs^c!Zymz#WQ zq#_I_U04!~d91e5gJVHoGF(xD9k}V{!EkoDiN6=aWx0>4s&=)-q!(XeaZO6vs!K`AE)#3(RFahA&FpO97@w{oo$c$? zkQQ?dX>mqF@_s3vDv9S4V$Xf2d;d5;4yp)XNTXTpp+ z(rhQ6mXjMR3{NE>)RAA~F8{~nw7uN&R6>iRUSW5PxuQH9N8MDBiKD)*&KJ5TQQ(RG zAhSD+&YDytV$MJL%Y>ANtT@K)BgU(Y_!KONyDMjl?J@Fro;b_NZ6WL%M=7@DzPQ!A z1UqT+zFqi7BXg$AHL37E!S5^<$J>2ma{sOdlDC7IXzI6u_J*qkN7Z*ho?Nypaac6< zkA~c6YNx-zm}4NBWXz4Gb_TmdQ$L-6e`IAs>M5FfaJYwP>hlI*YOvdsiKc!fnd=|< zGLi9*%t|@RW~Kg(XzG8@Kk}!%Bj-}v(_Q={Yn`mZrqT+NV2_f$7ILk*qYU*zD&rqn zJH)s2sy>4F>EMl-OzPh=kUSYXycp$}7mXp3`XG{WwMgnEM`y}Ur>8Z~$bgX<&qyq) zg?8+BW+JH<_2rS&^ZSzRBQ@8^Y1YGCBmMi39nla>BTq(z<1(StyOrO!jZ8keCKY|g zl<|#hwF)iWd?S|~t@}o1>jkG*sq(Lbsn6=yQ!q8^O-HhhY-)M}a6?YbDc3(TYw!{M zBM&ys))TC^_Bz|2X*d0(`TxK@BJ_>B0(Zyt9FTYOXqbw3_HX3<_H_FQ0<62*M;eDp zjhXmr%|615QH!Z?NZ`e&Kg3at>PWNxKu||Xjs^X!_6F3W?iI_Zo4MYI2{p}Q^YzER?UR5&5#<+E=y_Z}c(ll0F@ z$yCJNMbW2$(1Vqvs0dRZ^2Zi?s;QnnF^H#He_x5oBztqTluShu4>yw~u#Y72*%N~% zv-=12Vt=1y$eKNdH^S_}q@!K^QV~>Z$Q1>UML!arn~%e&1_}0A2}F^{2GIK~j}Bp% zdu85SaNqbwsG-}1X9@(QYwa6f=7}EPSAA=m)HoK)j_}7t;&TmH2TG&r*zHX=Hamjs z=%M0CmF(q+tRUuX10ryTM(mM)94)RR*0;t89~HA>#v47c56AF{BWPtQHl!jFwxs#Y z(RSQ8Wa|AWl3Gc;T#Mh}ev8kK;^hb1G2<(u;im@km&@mja$RQ5I6IxfOf$5a!rPvm zH4du9tP%f^XpdJMD`smBer36c>EaeGqMr=H)<&*%h)=_Z>cHC;B2&kU z(=6Dtk&o<>Zl6IF2-<5t(d6iUY@fB9{DI;~5WDq;n`T0Y&c|-LF$O@tjfj`h#b{*} zO-0Hl2ee0vN%1teW_}pNwqZ(&spmdx>G5fJxl^%QW{u~vZ2X&*KnwlsH6%bM-m@Ut z`otNi?C=Bl#2223&}tv{8-n5FD!hlUTN_x(z#?f&i+q_uJ#IA{yr%ttH+`px)h1a2rrn%GrFgR8EhH$VC)G`>qxp11+{~+*0(8tinQA-i#B3V{2 z3`dZurq@@qP4D6f9MW?`h7eDvlx)bf>qXl;KpethJ@1koJ?{<{XQeZq{EL62qkG?B zevqD<-yI}0M@IX<=zZ7S6LMJdJ4fn%dRXr}@*EWoZ-4h)S+1I<1ruu^S2miM>$6=u z;05|5as{*y2Bgj=cni8V!Hb{?UOQhNU(?Y94<*ya+%98ku1 zgxS?9?ZR*xYNr?ZYR9C!Z0q}Y#=zZQScp$3pgqe(O@jJj(9g+^ceH&VuXJVz=bSSh+r7$Klv-c52zL(4E_qL#VsP#i6X1Z9ihPHd(n;n2;Z zfL`oN1a_{VNNWTP&~W~Al!#ZvLXjl?-$c&mO~2xaaH2@aSI&c+{|5n;4Xsp+u%vLT zjSNU6;#~HRtU}j3IOD>+>122*5gp=H5mcJdZZIB$f5#U+lb~r{+CWnV-l{x3K#-RQ|qAwf7pGYv8I>>*2EpMH75hTAkRR+3ARSuz6 zjenXKg}z5sK0(9}W$X7uScU$1=NQcBRIr%(qNqLO&77qQqiGeIRCl=JW>Jzf99B(M zHg|Q$or2g6o<=g~NZCq9?xS~-;lHXX_m}BE47{qmN+Qb-t2Qh(XVR6P)#}RLMA&HP zO_bv-@?07C53-d!79_7^@@2LoJ>QZ~#_rHwJD7-UwTxpIo1*MOGjB`~!?v8+t^9VKve{%0TN!p|&&Ndt<{4PLqM-=1 z&6M9#isPk<-7+ANgncr=Z>_|40c+xDz_bn_@d(yQoQZ43g=RDtSc5N7IC7r$TajQq zY`18?fvVCH>WoI6>}(|A?w0Z3T6Q{mC0wY*&io}0J0gFjC27o{Ricvvl$qfH>^QdfdD zbxOf#RHIElLv^jY?%#{+{^cd(H0rcTe!olHd&9tz+`M;kwa~lDOWNgSbDI}7!zGp5 zE32y@K^`s0>6JCf*riw2L4OxP;wIy`7z#9tOk{Gy!sbQC=Li#|lC<$?ciek2?LnIO z+%TsB7qX?xwYFJgHud-E?3;BpkG|0xXT3`uW$)7ePwK?^Wg27MW8bW?5Fpd}G>#g>r2=* zrLwdfy{0`;c1fv2T1v>v`4ZIHv&*X34_-SCuzv=k>~k-_k4*t%_IGa?m+4;eYTfM~ z^=dNRU0%)Ic(qp(<}T6?o~<8T^Quq-Gg=PpEX)fyI*T~tzo#61r)0@ z#(OQ)m6InOC^Tpt?Y^kVcC@qZR>@?Ou(C4KW?`I3+FBbyMQOjAUy^nZE`mHy2BO@s zoHa;qDCB-6KmKkl`YcrG-3gYGotRN%aMek!CAfHVGi9TZK10SYMtk?UQM12g(Wf=` z{#Vz)K!lf$v8?@pu6|0(;eN_TrT4npDf?Qd`~7~+P5FS82qX$Oj9yk1gg>we0cxu0?f;Zv@9LY^SfH6hJXIW^z*if*YKZCMP@3M@69 zrpk0gH&yNk>ZZzusC`CVZ}aQ&*c4jsz79|2P%=(AufHB)81Vk(Ad4W`Oev|0g4GSjOFBQ#T`q?;-cpaxUr9?ewg;HJtQy_wNz za5dlP1oOVW6P@6h0P}{ zI(AXFCS}*F^w;8^@<9rF{l^NAy?;xBm2&guLFeFGBvxr97Sxa<*IR_&6lt@?RE7eZ zY7Ymk8}Uai!UvWtw^|yAT5bKwRtYX$GkeY$xO8dr>OpAE-LkZ*rMjV}&R^MI9thvg z#2+n<+@*rco#AY435|w8WaE1T2siCPr>z$X4v#dMZkquNYsiR}s%AE$aOVb9x!`Qp#pT(gou zw_}fhKOe1_wz-GnW@HJWKbA@l6{*|Hgyn8>W?%|N(UV*E6t$ww4yZ*_b~VWvn~wkf EA2ntTi~s-t literal 0 HcmV?d00001 diff --git a/omnicache/omnicache-spark-extension/log-parser/src/test/scala/org/apache/spark/deploy/history/LogsParserSuite.scala b/omnicache/omnicache-spark-extension/log-parser/src/test/scala/org/apache/spark/deploy/history/LogsParserSuite.scala new file mode 100644 index 000000000..d686b0e37 --- /dev/null +++ b/omnicache/omnicache-spark-extension/log-parser/src/test/scala/org/apache/spark/deploy/history/LogsParserSuite.scala @@ -0,0 +1,77 @@ +/* + * 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.deploy.history + +import java.io.{FileInputStream, FileNotFoundException} + +import org.apache.commons.io.IOUtils +import org.json4s.DefaultFormats +import org.json4s.jackson.Json + +import org.apache.spark.SparkFunSuite + +class LogsParserSuite extends SparkFunSuite { + + test("parse") { + val path = this.getClass.getResource("/").getPath + val args: Array[String] = Array(path, "eventlog/parse" + , "application_1663257594501_0003.lz4") + ParseLog.main(args) + val fis = new FileInputStream("eventlog/parse/application_1663257594501_0003.lz4.json") + val lines = IOUtils.readLines(fis, "UTF-8") + IOUtils.closeQuietly(fis) + val json: Seq[Map[String, String]] = Json(DefaultFormats) + .read[Seq[Map[String, String]]](lines.get(0)) + assert(json.exists(map => + map.contains("materialized views") && + map("physical plan").contains(map("materialized views")) + )) + } + + test("error_invalid_param") { + assertThrows[RuntimeException] { + val path = this.getClass.getResource("/").getPath + val args: Array[String] = Array(path, "eventlog/parse") + ParseLog.main(args) + } + assertThrows[RuntimeException] { + val path = this.getClass.getResource("/").getPath + val args: Array[String] = Array(path, "eventlog/parse" + , "application_1663257594501_0003.lz4", "1") + ParseLog.main(args) + } + } + + test("error_invalid_logname") { + assertThrows[RuntimeException] { + val path = this.getClass.getResource("/").getPath + val args: Array[String] = Array(path, "eventlog/parse" + , "xxx.lz4") + ParseLog.main(args) + } + } + + test("error_log_not_exist") { + assertThrows[FileNotFoundException] { + val path = this.getClass.getResource("/").getPath + val args: Array[String] = Array(path, "eventlog/parse" + , "application_1663257594501_00031.lz4") + ParseLog.main(args) + } + } +} diff --git a/omnicache/omnicache-spark-extension/plugin/pom.xml b/omnicache/omnicache-spark-extension/plugin/pom.xml index 3879b778d..721879bdd 100644 --- a/omnicache/omnicache-spark-extension/plugin/pom.xml +++ b/omnicache/omnicache-spark-extension/plugin/pom.xml @@ -73,6 +73,7 @@ + ${artifactId}-${version}${dep.os.arch} target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes diff --git a/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/conf/OmniCachePluginConfig.scala b/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/conf/OmniCachePluginConfig.scala index d9fc9b317..77cacf067 100644 --- a/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/conf/OmniCachePluginConfig.scala +++ b/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/conf/OmniCachePluginConfig.scala @@ -17,9 +17,13 @@ package com.huawei.boostkit.spark.conf +import java.util.Locale + import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.CatalogTable +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.internal.SQLConf class OmniCachePluginConfig(conf: SQLConf) { @@ -48,6 +52,10 @@ class OmniCachePluginConfig(conf: SQLConf) { .getConfString("spark.sql.omnicache.default.datasource", "orc") val dataSourceSet: Set[String] = Set("orc", "parquet") + + def logLevel: String = conf + .getConfString("spark.sql.omnicache.logLevel", "DEBUG") + .toUpperCase(Locale.ROOT) } object OmniCachePluginConfig { @@ -86,4 +94,10 @@ object OmniCachePluginConfig { val catalogTable = spark.sessionState.catalog.getTableMetadata(mv) !catalogTable.properties.getOrElse(MV_UPDATE_REWRITE_ENABLED, "true").toBoolean } + + def isMVInUpdate(viewTablePlan: LogicalPlan): Boolean = { + val logicalRelation = viewTablePlan.asInstanceOf[LogicalRelation] + !logicalRelation.catalogTable.get + .properties.getOrElse(MV_UPDATE_REWRITE_ENABLED, "true").toBoolean + } } diff --git a/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/util/ExprSimplifier.scala b/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/util/ExprSimplifier.scala index 13b1a607d..5cb7d1925 100644 --- a/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/util/ExprSimplifier.scala +++ b/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/util/ExprSimplifier.scala @@ -30,7 +30,6 @@ import org.apache.spark.sql.catalyst.optimizer._ import org.apache.spark.sql.catalyst.plans.logical.{Filter, Join, LogicalPlan, OneRowRelation} import org.apache.spark.sql.types.{BooleanType, DataType, NullType} - case class ExprSimplifier(unknownAsFalse: Boolean, pulledUpPredicates: Set[Expression]) { @@ -476,14 +475,14 @@ case class ExprSimplifier(unknownAsFalse: Boolean, for (term <- terms) { // Excluding self-simplification if (!term.eq(orOp)) { - // Simplification between a orExpression and a orExpression. + // Simplification between a OrExpression and a OrExpression. if (term.isInstanceOf[Or]) { if (containsAllSql(ors, decomposeDisjunctions(term).toSet)) { terms.-=(orOp) breaks3.break() } } else if (containsSql(ors, term)) { - // Simplification between a otherExpression and a orExpression. + // Simplification between a otherExpression and a OrExpression. terms.-=(orOp) breaks3.break() } @@ -615,7 +614,7 @@ case class ExprSimplifier(unknownAsFalse: Boolean, case c@Not(c1@EqualNullSafe(_, Literal.TrueLiteral)) => Not(c1.copy(left = child2)) case c => - throw new RuntimeException("unSupport type is predict simplify :%s".format(c)) + throw new RuntimeException("unSupport type is predicate simplify :%s".format(c)) } return condition2 } diff --git a/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/util/RewriteHelper.scala b/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/util/RewriteHelper.scala index 38b7d40a3..2ef67e536 100644 --- a/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/util/RewriteHelper.scala +++ b/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/util/RewriteHelper.scala @@ -28,14 +28,14 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.internal.SQLConf -class RewriteHelper extends PredicateHelper { +trait RewriteHelper extends PredicateHelper with RewriteLogger { val SESSION_CATALOG_NAME: String = "spark_catalog" - val EMPTY_BITMAP: HashBiMap[String, String] = HashBiMap.create[String, String]() + val EMPTY_BIMAP: HashBiMap[String, String] = HashBiMap.create[String, String]() val EMPTY_MAP: Map[ExpressionEqual, mutable.Set[ExpressionEqual]] = Map[ExpressionEqual, mutable.Set[ExpressionEqual]]() - val EMPTY_MULTIMAP: Multimap[Int, Int] = ArrayListMultimap.create[Int, Int] + val EMPTY_MULTIMAP: Multimap[Int, Int] = ArrayListMultimap.create[Int, Int]() def mergeConjunctiveExpressions(e: Seq[Expression]): Expression = { if (e.isEmpty) { @@ -81,8 +81,9 @@ class RewriteHelper extends PredicateHelper { for ((project, column) <- topProjectList.zip(viewTablePlan.output)) { project match { // only map attr - case _@Alias(attr@AttributeReference(_, _, _, _), _) => - exprIdToQualifier += (column.exprId -> attr) + case a@Alias(attr@AttributeReference(_, _, _, _), _) => + exprIdToQualifier += (column.exprId -> + attr.copy(name = a.name)(exprId = attr.exprId, qualifier = attr.qualifier)) case a@AttributeReference(_, _, _, _) => exprIdToQualifier += (column.exprId -> a) // skip function @@ -204,11 +205,11 @@ class RewriteHelper extends PredicateHelper { (mappedQuery, mappedTables) } - def swapTableColumnReferences[T <: Iterable[Expression]](expression: T, + def swapTableColumnReferences[T <: Iterable[Expression]](expressions: T, tableMapping: BiMap[String, String], columnMapping: Map[ExpressionEqual, mutable.Set[ExpressionEqual]]): T = { - var result: T = expression + var result: T = expressions if (!tableMapping.isEmpty) { result = result.map { expr => expr.transform { @@ -243,28 +244,28 @@ class RewriteHelper extends PredicateHelper { result } - def swapColumnTableReferences[T <: Iterable[Expression]](expression: T, + def swapColumnTableReferences[T <: Iterable[Expression]](expressions: T, tableMapping: BiMap[String, String], columnMapping: Map[ExpressionEqual, mutable.Set[ExpressionEqual]]): T = { - var result = swapTableColumnReferences(expression, EMPTY_BITMAP, columnMapping) + var result = swapTableColumnReferences(expressions, EMPTY_BIMAP, columnMapping) result = swapTableColumnReferences(result, tableMapping, EMPTY_MAP) result } - def swapTableReferences[T <: Iterable[Expression]](expression: T, + def swapTableReferences[T <: Iterable[Expression]](expressions: T, tableMapping: BiMap[String, String]): T = { - swapTableColumnReferences(expression, tableMapping, EMPTY_MAP) + swapTableColumnReferences(expressions, tableMapping, EMPTY_MAP) } - def swapColumnReferences[T <: Iterable[Expression]](expression: T, + def swapColumnReferences[T <: Iterable[Expression]](expressions: T, columnMapping: Map[ExpressionEqual, mutable.Set[ExpressionEqual]]): T = { - swapTableColumnReferences(expression, EMPTY_BITMAP, columnMapping) + swapTableColumnReferences(expressions, EMPTY_BIMAP, columnMapping) } } -object RewriteHelper extends PredicateHelper { +object RewriteHelper extends PredicateHelper with RewriteLogger { /** * Rewrite [[EqualTo]] and [[EqualNullSafe]] operator to keep order. The following cases will be * equivalent: @@ -431,6 +432,36 @@ object RewriteHelper extends PredicateHelper { def disableCachePlugin(): Unit = { SQLConf.get.setConfString("spark.sql.omnicache.enable", "false") } + + def checkAttrsValid(logicalPlan: LogicalPlan): Boolean = { + logicalPlan.foreachUp { + case _: LeafNode => + case plan => + val attributeSets = plan.expressions.map { expression => + AttributeSet.fromAttributeSets( + expression.collect { + case s: SubqueryExpression => + var res = s.references + s.plan.transformAllExpressions { + case e@OuterReference(ar) => + res ++= AttributeSet(ar.references) + e + case e => e + } + res + case e => e.references + }) + } + val request = AttributeSet.fromAttributeSets(attributeSets) + val input = plan.inputSet + val missing = request -- input + if (missing.nonEmpty) { + logBasedOnLevel("checkAttrsValid failed for missing:%s".format(missing)) + return false + } + } + true + } } case class ExpressionEqual(expression: Expression) { diff --git a/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/util/RewriteLogger.scala b/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/util/RewriteLogger.scala new file mode 100644 index 000000000..f2b63f5ca --- /dev/null +++ b/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/util/RewriteLogger.scala @@ -0,0 +1,38 @@ +/* + * 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 com.huawei.boostkit.spark.util + +import com.huawei.boostkit.spark.conf.OmniCachePluginConfig + +import org.apache.spark.internal.Logging + +trait RewriteLogger extends Logging { + + private def logLevel: String = OmniCachePluginConfig.getConf.logLevel + + def logBasedOnLevel(f: => String): Unit = { + logLevel match { + case "TRACE" => logTrace(f) + case "DEBUG" => logDebug(f) + case "INFO" => logInfo(f) + case "WARN" => logWarning(f) + case "ERROR" => logError(f) + case _ => logTrace(f) + } + } +} diff --git a/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/util/ViewMetadata.scala b/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/util/ViewMetadata.scala index 807418400..a3ab16e76 100644 --- a/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/util/ViewMetadata.scala +++ b/omnicache/omnicache-spark-extension/plugin/src/main/scala/com/huawei/boostkit/spark/util/ViewMetadata.scala @@ -17,15 +17,15 @@ package com.huawei.boostkit.spark.util -import com.google.common.collect.Lists import com.huawei.boostkit.spark.conf.OmniCachePluginConfig._ import java.util.concurrent.ConcurrentHashMap -import org.apache.calcite.util.graph._ +import scala.collection.mutable import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.{CatalogTable, SessionCatalog} -import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias} +import org.apache.spark.sql.catalyst.optimizer.rules.RewriteTime +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, RepartitionByExpression, SubqueryAlias} object ViewMetadata extends RewriteHelper { @@ -35,9 +35,7 @@ object ViewMetadata extends RewriteHelper { val viewToContainsTables = new ConcurrentHashMap[String, Set[TableEqual]]() - val usesGraph: DirectedGraph[String, DefaultEdge] = DefaultDirectedGraph.create() - - var frozenGraph: Graphs.FrozenGraph[String, DefaultEdge] = Graphs.makeImmutable(usesGraph) + val tableToViews = new ConcurrentHashMap[String, mutable.Set[String]]() var spark: SparkSession = _ @@ -52,10 +50,6 @@ object ViewMetadata extends RewriteHelper { status = STATUS_LOADING } - def usesGraphTopologicalOrderIterator: java.lang.Iterable[String] = { - TopologicalOrderIterator.of[String, DefaultEdge](usesGraph) - } - def saveViewMetadataToMap(catalogTable: CatalogTable): Unit = this.synchronized { // if QUERY_REWRITE_ENABLED is false, doesn't load ViewMetadata if (!catalogTable.properties.getOrElse(MV_REWRITE_ENABLED, "false").toBoolean) { @@ -80,40 +74,58 @@ object ViewMetadata extends RewriteHelper { // db.table val tableName = catalogTable.identifier.quotedString - val viewTablePlan = spark.table(tableName).queryExecution.analyzed match { - case SubqueryAlias(_, child) => child - case a@_ => a + val viewTablePlan = RewriteTime + .withTimeStat("viewTablePlan") { + spark.table(tableName).queryExecution.analyzed match { + case SubqueryAlias(_, child) => child + case a@_ => a + } + } + var viewQueryPlan = RewriteTime + .withTimeStat("viewQueryPlan") { + spark.sql(viewQuerySql).queryExecution.analyzed + } + viewQueryPlan = viewQueryPlan match { + case RepartitionByExpression(_, child, _) => + child + case _ => + viewQueryPlan } - val viewQueryPlan = spark.sql(viewQuerySql).queryExecution.analyzed // reset preDatabase spark.sessionState.catalogManager.setCurrentNamespace(Array(preDatabase)) // spark_catalog.db.table val viewName = catalogTable.identifier.toString() - // mappedViewQueryPlan and mappedViewContainsTable - val (mappedViewQueryPlan, mappedViewContainsTables) = extractTables(viewQueryPlan) + // mappedViewQueryPlan and mappedViewContainsTables + val (mappedViewQueryPlan, mappedViewContainsTables) = RewriteTime + .withTimeStat("extractTables") { + extractTables(viewQueryPlan) + } - usesGraph.addVertex(viewName) mappedViewContainsTables .foreach { mappedViewContainsTable => val name = mappedViewContainsTable.tableName - usesGraph.addVertex(name) - usesGraph.addEdge(name, viewName) + val views = tableToViews.getOrDefault(name, mutable.Set.empty) + views += viewName + tableToViews.put(name, views) } // extract view query project's Attr and replace view table's Attr by query project's Attr // match function is attributeReferenceEqualSimple, by name and data type // Attr of table cannot used, because same Attr in view query and view table, // it's table is different. - val mappedViewTablePlan = mapTablePlanAttrToQuery(viewTablePlan, mappedViewQueryPlan) + val mappedViewTablePlan = RewriteTime + .withTimeStat("mapTablePlanAttrToQuery") { + mapTablePlanAttrToQuery(viewTablePlan, mappedViewQueryPlan) + } viewToContainsTables.put(viewName, mappedViewContainsTables) viewToViewQueryPlan.putIfAbsent(viewName, mappedViewQueryPlan) viewToTablePlan.putIfAbsent(viewName, mappedViewTablePlan) } catch { case e: Throwable => - logDebug(s"Failed to saveViewMetadataToMap. errmsg: ${e.getMessage}") + logDebug(s"Failed to saveViewMetadataToMap,errmsg: ${e.getMessage}") // reset preDatabase spark.sessionState.catalogManager.setCurrentNamespace(Array(preDatabase)) } @@ -129,21 +141,19 @@ object ViewMetadata extends RewriteHelper { def addCatalogTableToCache(table: CatalogTable): Unit = this.synchronized { saveViewMetadataToMap(table) - rebuildGraph() - } - - def rebuildGraph(): Unit = { - frozenGraph = Graphs.makeImmutable(usesGraph) } def removeMVCache(tableName: TableIdentifier): Unit = this.synchronized { val viewName = tableName.toString() - usesGraph.removeAllVertices(Lists.newArrayList(viewName)) viewToContainsTables.remove(viewName) viewToViewQueryPlan.remove(viewName) viewToTablePlan.remove(viewName) - viewToContainsTables.remove(viewName) - rebuildGraph() + tableToViews.forEach { (key, value) => + if (value.contains(viewName)) { + value -= viewName + tableToViews.put(key, value) + } + } } def init(sparkSession: SparkSession): Unit = { @@ -158,14 +168,16 @@ object ViewMetadata extends RewriteHelper { def forceLoad(): Unit = this.synchronized { val catalog = spark.sessionState.catalog - // val db = OmniCachePluginConfig.getConf.OmniCacheDB // load from all db for (db <- catalog.listDatabases()) { - val tables = omniCacheFilter(catalog, db) - tables.foreach(tableData => saveViewMetadataToMap(tableData)) + val tables = RewriteTime.withTimeStat("loadTable") { + omniCacheFilter(catalog, db) + } + RewriteTime.withTimeStat("saveViewMetadataToMap") { + tables.foreach(tableData => saveViewMetadataToMap(tableData)) + } } - rebuildGraph() } def omniCacheFilter(catalog: SessionCatalog, @@ -176,7 +188,7 @@ object ViewMetadata extends RewriteHelper { tableData.properties.contains(MV_QUERY_ORIGINAL_SQL) } } catch { - // if db exists a table hive materialized view, will throw annalysis exception + // if db exists a table hive materialized view, will throw analysis exception case e: Throwable => logDebug(s"Failed to listTables in $mvDataBase, errmsg: ${e.getMessage}") Seq.empty[CatalogTable] diff --git a/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/AbstractMaterializedViewRule.scala b/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/AbstractMaterializedViewRule.scala index df26d192f..7d398c2cf 100644 --- a/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/AbstractMaterializedViewRule.scala +++ b/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/AbstractMaterializedViewRule.scala @@ -49,7 +49,9 @@ abstract class AbstractMaterializedViewRule(sparkSession: SparkSession) if (ViewMetadata.status == ViewMetadata.STATUS_LOADING) { return finalPlan } - ViewMetadata.init(sparkSession) + RewriteTime.withTimeStat("viewMetadata") { + ViewMetadata.init(sparkSession) + } // 1.check query sql is match current rule if (ViewMetadata.isEmpty || !isValidPlan(plan)) { return finalPlan @@ -60,8 +62,10 @@ abstract class AbstractMaterializedViewRule(sparkSession: SparkSession) val (queryExpr, queryTables) = extractTables(finalPlan) // 3.use all tables to fetch views(may match) from ViewMetaData - val candidateViewPlans = getApplicableMaterializations(queryTables.map(t => t.tableName)) - .filter(x => !OmniCachePluginConfig.isMVInUpdate(sparkSession, x._1)) + val candidateViewPlans = RewriteTime.withTimeStat("getApplicableMaterializations") { + getApplicableMaterializations(queryTables.map(t => t.tableName)) + .filter(x => !OmniCachePluginConfig.isMVInUpdate(x._2)) + } if (candidateViewPlans.isEmpty) { return finalPlan } @@ -112,16 +116,21 @@ abstract class AbstractMaterializedViewRule(sparkSession: SparkSession) val (newViewTablePlan, newViewQueryPlan, newTopViewProject) = newViewPlans.get viewTablePlan = newViewTablePlan viewQueryPlan = newViewQueryPlan + if (newTopViewProject.isEmpty) { + viewQueryExpr = newViewQueryPlan + } topViewProject = newTopViewProject } // 4.5.extractPredictExpressions from viewQueryPlan and mappedQueryPlan - val queryPredictExpression = extractPredictExpressions(queryExpr, EMPTY_BITMAP) + val queryPredictExpression = RewriteTime.withTimeStat("extractPredictExpressions") { + extractPredictExpressions(queryExpr, EMPTY_BIMAP) + } val viewProjectList = extractTopProjectList(viewQueryExpr) val viewTableAttrs = viewTablePlan.output - // 4.6.if a table emps used >=2 times in s sql (query and view) + // 4.6.if a table emps used >=2 times in a sql (query and view) // we should try the combination,switch the seq // view:SELECT V1.locationid,V2.empname FROM emps V1 JOIN emps V2 // ON V1.deptno='1' AND V2.deptno='2' AND V1.empname = V2.empname; @@ -132,25 +141,31 @@ abstract class AbstractMaterializedViewRule(sparkSession: SparkSession) flatListMappings.foreach { queryToViewTableMapping => mappingLoop.breakable { val inverseTableMapping = queryToViewTableMapping.inverse() - val viewPredictExpression = extractPredictExpressions(viewQueryExpr, - inverseTableMapping) + val viewPredictExpression = RewriteTime.withTimeStat("extractPredictExpressions") { + extractPredictExpressions(viewQueryExpr, + inverseTableMapping) + } // 4.7.compute compensationPredicates between viewQueryPlan and queryPlan - var newViewTablePlan = computeCompensationPredicates(viewTablePlan, - queryPredictExpression, viewPredictExpression, inverseTableMapping, - viewPredictExpression._1.getEquivalenceClassesMap, - viewProjectList, viewTableAttrs) - // 4.8.compensationPredicates isEmpty, because view's row data cannot satify query + var newViewTablePlan = RewriteTime.withTimeStat("computeCompensationPredicates") { + computeCompensationPredicates(viewTablePlan, + queryPredictExpression, viewPredictExpression, inverseTableMapping, + viewPredictExpression._1.getEquivalenceClassesMap, + viewProjectList, viewTableAttrs) + } + // 4.8.compensationPredicates isEmpty, because view's row data cannot satisfy query if (newViewTablePlan.isEmpty) { mappingLoop.break() } // 4.9.use viewTablePlan(join compensated), query project, // compensationPredicts to rewrite final plan - newViewTablePlan = rewriteView(newViewTablePlan.get, viewQueryExpr, - queryExpr, inverseTableMapping, - queryPredictExpression._1.getEquivalenceClassesMap, - viewProjectList, viewTableAttrs) + newViewTablePlan = RewriteTime.withTimeStat("rewriteView") { + rewriteView(newViewTablePlan.get, viewQueryExpr, + queryExpr, inverseTableMapping, + queryPredictExpression._1.getEquivalenceClassesMap, + viewProjectList, viewTableAttrs) + } if (newViewTablePlan.isEmpty) { mappingLoop.break() } @@ -165,13 +180,19 @@ abstract class AbstractMaterializedViewRule(sparkSession: SparkSession) } /** - * cehck plan if match current rule + * check plan if match current rule * * @param logicalPlan LogicalPlan * @return true:matched ; false:unMatched */ def isValidPlan(logicalPlan: LogicalPlan): Boolean + /** + * basic check for all rule + * + * @param logicalPlan LogicalPlan + * @return true:matched ; false:unMatched + */ def isValidLogicalPlan(logicalPlan: LogicalPlan): Boolean = { logicalPlan.foreach { case _: LogicalRelation => @@ -199,19 +220,19 @@ abstract class AbstractMaterializedViewRule(sparkSession: SparkSession) LogicalPlan, LogicalPlan)] = { // viewName, viewTablePlan, viewQueryPlan var viewPlans = Seq.empty[(String, LogicalPlan, LogicalPlan)] + val viewNames = mutable.Set.empty[String] // 1.topological iterate graph - ViewMetadata.usesGraphTopologicalOrderIterator.forEach { viewName => - // 2.check this node is mv - if (ViewMetadata.viewToTablePlan.containsKey(viewName) - // 3.iterate mv used tables and check edge of (table,mv) in graph - && usesTable(viewName, ViewMetadata - .viewToContainsTables.get(viewName), ViewMetadata.frozenGraph)) { - // 4.add plan info - val viewQueryPlan = ViewMetadata.viewToViewQueryPlan.get(viewName) - val viewTablePlan = ViewMetadata.viewToTablePlan.get(viewName) - viewPlans +:= (viewName, viewTablePlan, viewQueryPlan) + tableNames.foreach { tableName => + if (ViewMetadata.tableToViews.containsKey(tableName)) { + viewNames ++= ViewMetadata.tableToViews.get(tableName) } } + viewNames.foreach { viewName => + // 4.add plan info + val viewQueryPlan = ViewMetadata.viewToViewQueryPlan.get(viewName) + val viewTablePlan = ViewMetadata.viewToTablePlan.get(viewName) + viewPlans +:= (viewName, viewTablePlan, viewQueryPlan) + } viewPlans } @@ -239,7 +260,7 @@ abstract class AbstractMaterializedViewRule(sparkSession: SparkSession) * @param viewTablePlan viewTablePlan * @param viewQueryPlan viewQueryPlan * @param topViewProject topViewProject - * @param needTables needTables + * @param needTables need join compensate tables * @return join compensated viewTablePlan */ def compensateViewPartial(viewTablePlan: LogicalPlan, @@ -248,6 +269,14 @@ abstract class AbstractMaterializedViewRule(sparkSession: SparkSession) needTables: Set[TableEqual]): Option[(LogicalPlan, LogicalPlan, Option[Project])] = None + /** + * We map every table in the query to a table with the same qualified + * name (all query tables are contained in the view, thus this is equivalent + * to mapping every table in the query to a view table). + * + * @param queryTables queryTables + * @return + */ def generateTableMappings(queryTables: Set[TableEqual]): Seq[BiMap[String, String]] = { val multiMapTables: Multimap[String, String] = ArrayListMultimap.create() for (t1 <- queryTables) { @@ -269,7 +298,7 @@ abstract class AbstractMaterializedViewRule(sparkSession: SparkSession) } // continue } else { - // Multiple reference: flatten + // Multiple references: flatten val newResult: ImmutableList.Builder[BiMap[String, String]] = ImmutableList.builder() t2s.forEach { target => result.forEach { m => @@ -284,6 +313,7 @@ abstract class AbstractMaterializedViewRule(sparkSession: SparkSession) } } JavaConverters.asScalaIteratorConverter(result.iterator()).asScala.toSeq + .sortWith((map1, map2) => map1.toString < map2.toString) } /** @@ -346,10 +376,17 @@ abstract class AbstractMaterializedViewRule(sparkSession: SparkSession) Some(compensationPredicate) } + /** + * extractPossibleMapping {queryEquivalenceClasses:[contained viewEquivalenceClasses]} + * + * @param queryEquivalenceClasses queryEquivalenceClasses + * @param viewEquivalenceClasses viewEquivalenceClasses + * @return {queryEquivalenceClasses:[contained viewEquivalenceClasses]} + */ def extractPossibleMapping(queryEquivalenceClasses: List[mutable.Set[ExpressionEqual]], viewEquivalenceClasses: List[mutable.Set[ExpressionEqual]]): Option[Multimap[Int, Int]] = { // extractPossibleMapping {queryEquivalenceClasses:[contained viewEquivalenceClasses]} - // query:c1=c2=c3=c4 view:c1=c2 ,c3=c4 + // query:c1=c2=c3=c4 view:c1=c2 , c3=c4 val mapping = ArrayListMultimap.create[Int, Int]() val breakLoop = new Breaks @@ -381,6 +418,12 @@ abstract class AbstractMaterializedViewRule(sparkSession: SparkSession) Some(mapping) } + /** + * + * @param queryExpression queryExpression + * @param viewExpression viewExpression + * @return compensate Expression + */ def splitFilter(queryExpression: Expression, viewExpression: Expression): Option[Expression] = { // 1.canonicalize expression,main for reorder val queryExpression2 = RewriteHelper.canonicalize(ExprSimplifier.simplify(queryExpression)) @@ -422,7 +465,6 @@ abstract class AbstractMaterializedViewRule(sparkSession: SparkSession) None } - /** * split expression by or,then compute compensation * @@ -434,7 +476,7 @@ abstract class AbstractMaterializedViewRule(sparkSession: SparkSession) val queries = ExprOptUtil.disjunctions(queryExpression) val views = ExprOptUtil.disjunctions(viewExpression) - // 1.compare difference which queries residue + // 1.compute difference which queries residue val difference = queries.map(ExpressionEqual) -- views.map(ExpressionEqual) // 2.1.queries equal to views,just return true @@ -533,9 +575,8 @@ abstract class AbstractMaterializedViewRule(sparkSession: SparkSession) true } - /** - * compute compensationPredicts between viewQueryPlan and mappedQueryPlan + * compute compensationPredicates between viewQueryPlan and mappedQueryPlan * * @param viewTablePlan viewTablePlan * @param queryPredict queryPredict @@ -594,18 +635,6 @@ abstract class AbstractMaterializedViewRule(sparkSession: SparkSession) if (columnsEquiPredictsResult.isEmpty) { return None } - val viewTableAttrSet = viewTableAttrs.map(ExpressionEqual).toSet - columnsEquiPredictsResult.get.foreach { expr => - expr.foreach { - case attr: AttributeReference => - if (!viewTableAttrSet.contains(ExpressionEqual(attr))) { - logDebug(s"attr:%s cannot found in view:%s" - .format(attr, OmniCachePluginConfig.getConf.curMatchMV)) - return None - } - case _ => - } - } // 5.rewrite rangeCompensation,residualCompensation by viewTableAttrs val otherPredictsResult = rewriteExpressions(Seq(compensationRangePredicts.get, @@ -627,10 +656,10 @@ abstract class AbstractMaterializedViewRule(sparkSession: SparkSession) * @param swapTableColumn true:swapTableColumn;false:swapColumnTable * @param tableMapping tableMapping * @param columnMapping columnMapping - * @param viewProjectList viewProjectList + * @param viewProjectList viewProjectList/viewAggExpression * @param viewTableAttrs viewTableAttrs * @tparam T T <: Iterable[Expression] - * @return rewriteExprs + * @return rewritedExprs */ def rewriteExpressions[T <: Iterable[Expression]]( exprsToRewrite: T, swapTableColumn: Boolean, @@ -642,7 +671,7 @@ abstract class AbstractMaterializedViewRule(sparkSession: SparkSession) val swapProjectList = if (swapTableColumn) { swapTableColumnReferences(viewProjectList, tableMapping, columnMapping) } else { - swapColumnTableReferences(viewProjectList, tableMapping, columnMapping) + swapTableColumnReferences(viewProjectList, tableMapping, columnMapping) } val swapTableAttrs = swapTableReferences(viewTableAttrs, tableMapping) @@ -665,12 +694,12 @@ abstract class AbstractMaterializedViewRule(sparkSession: SparkSession) }.asInstanceOf[T] // 4.iterate result and dfs check every AttributeReference in ViewTableAttributeReference - val viewTableAttrSet = swapTableAttrs.map(ExpressionEqual).toSet + val viewTableAttrsSet = swapTableAttrs.map(_.exprId).toSet result.foreach { expr => expr.foreach { case attr: AttributeReference => - if (!viewTableAttrSet.contains(ExpressionEqual(attr))) { - logDebug(s"attr:%s cannot found in view:%s" + if (!viewTableAttrsSet.contains(attr.exprId)) { + logBasedOnLevel(s"attr:%s cannot found in view:%s" .format(attr, OmniCachePluginConfig.getConf.curMatchMV)) return None } @@ -680,7 +709,6 @@ abstract class AbstractMaterializedViewRule(sparkSession: SparkSession) Some(result) } - /** * if the rewrite expression exprId != origin expression exprId, * replace by Alias(rewrite expression,origin.name)(exprId=origin.exprId) @@ -706,6 +734,16 @@ abstract class AbstractMaterializedViewRule(sparkSession: SparkSession) /** * replace and alias expression or attr by viewTableAttr + * + * @param exprsToRewrite exprsToRewrite + * @param swapTableColumn true:swapTableColumn;false:swapColumnTable + * @param tableMapping tableMapping + * @param columnMapping columnMapping + * @param viewProjectList viewProjectList/viewAggExpression + * @param viewTableAttrs viewTableAttrs + * @param originExpressions originExpressions + * @tparam T T <: Iterable[Expression] + * @return rewrited and alias expression */ def rewriteAndAliasExpressions[T <: Iterable[Expression]]( exprsToRewrite: T, swapTableColumn: Boolean, @@ -742,5 +780,4 @@ abstract class AbstractMaterializedViewRule(sparkSession: SparkSession) columnMapping: Map[ExpressionEqual, mutable.Set[ExpressionEqual]], viewProjectList: Seq[Expression], viewTableAttrs: Seq[Attribute]): Option[LogicalPlan] - } diff --git a/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/MVRewriteRule.scala b/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/MVRewriteRule.scala index 98989c460..ba0c17f22 100644 --- a/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/MVRewriteRule.scala +++ b/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/MVRewriteRule.scala @@ -17,19 +17,23 @@ package org.apache.spark.sql.catalyst.optimizer.rules +import com.fasterxml.jackson.annotation.JsonIgnore import com.huawei.boostkit.spark.conf.OmniCachePluginConfig -import com.huawei.boostkit.spark.util.RewriteHelper +import com.huawei.boostkit.spark.util.{RewriteHelper, RewriteLogger} import scala.collection.mutable import org.apache.spark.SparkContext +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.internal.Logging -import org.apache.spark.scheduler.SparkListenerEvent +import org.apache.spark.scheduler.{SparkListener, SparkListenerEvent} import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LogicalPlan, Project} +import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.command.OmniCacheCreateMvCommand +import org.apache.spark.status.ElementTrackingStore +import org.apache.spark.util.kvstore.KVIndex -class MVRewriteRule(session: SparkSession) extends Rule[LogicalPlan] with Logging { +class MVRewriteRule(session: SparkSession) extends Rule[LogicalPlan] with RewriteLogger { val omniCacheConf: OmniCachePluginConfig = OmniCachePluginConfig.getConf val joinRule = new MaterializedViewJoinRule(session) @@ -55,6 +59,8 @@ class MVRewriteRule(session: SparkSession) extends Rule[LogicalPlan] with Loggin def tryRewritePlan(plan: LogicalPlan): LogicalPlan = { val usingMvs = mutable.Set.empty[String] + RewriteTime.clear() + val rewriteStartSecond = System.currentTimeMillis() val res = plan.transformDown { case p: Project => joinRule.perform(Some(p), p.child, usingMvs) @@ -66,23 +72,71 @@ class MVRewriteRule(session: SparkSession) extends Rule[LogicalPlan] with Loggin RewriteHelper.extractAllAttrsFromExpression(a.aggregateExpressions).toSeq, a.child) val rewritedChild = joinRule.perform(Some(child), child.child, usingMvs) if (rewritedChild != child) { - rewritedPlan = a.copy(child = rewritedChild) + val projectChild = rewritedChild.asInstanceOf[Project] + rewritedPlan = a.copy(child = Project( + projectChild.projectList ++ projectChild.child.output, projectChild.child)) } } rewritedPlan case p => p } if (usingMvs.nonEmpty) { + RewriteTime.withTimeStat("checkAttrsValid") { + if (!RewriteHelper.checkAttrsValid(res)) { + return plan + } + } val sql = session.sparkContext.getLocalProperty(SparkContext.SPARK_JOB_DESCRIPTION) val mvs = usingMvs.mkString(";").replaceAll("`", "") - val log = "logicalPlan MVRewrite success,using materialized view:[%s],original sql:%s" - .format(mvs, sql) - logDebug(log) + val costSecond = (System.currentTimeMillis() - rewriteStartSecond).toString + val log = ("logicalPlan MVRewrite success," + + "using materialized view:[%s],cost %s milliseconds,original sql:%s") + .format(mvs, costSecond, sql) + logBasedOnLevel(log) session.sparkContext.listenerBus.post(SparkListenerMVRewriteSuccess(sql, mvs)) } + RewriteTime.statFromStartTime("total", rewriteStartSecond) + logBasedOnLevel(RewriteTime.timeStat.toString()) res } } +@DeveloperApi case class SparkListenerMVRewriteSuccess(sql: String, usingMvs: String) extends SparkListenerEvent { + @JsonIgnore + @KVIndex + def id: String = (System.currentTimeMillis() + "%s%s".format(sql, usingMvs).hashCode).toString +} + +class MVRewriteSuccessListener( + kvStore: ElementTrackingStore) extends SparkListener with Logging { + + override def onOtherEvent(event: SparkListenerEvent): Unit = { + event match { + case _: SparkListenerMVRewriteSuccess => + kvStore.write(event) + case _ => + } + } +} + +object RewriteTime { + val timeStat: mutable.Map[String, Long] = mutable.HashMap[String, Long]() + + def statFromStartTime(key: String, startTime: Long): Unit = { + timeStat += (key -> (timeStat.getOrElse(key, 0L) + System.currentTimeMillis() - startTime)) + } + + def clear(): Unit = { + timeStat.clear() + } + + def withTimeStat[T](key: String)(f: => T): T = { + val startTime = System.currentTimeMillis() + try { + f + } finally { + statFromStartTime(key, startTime) + } + } } diff --git a/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/MaterializedViewAggregateRule.scala b/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/MaterializedViewAggregateRule.scala index fe82c4744..6dda86891 100644 --- a/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/MaterializedViewAggregateRule.scala +++ b/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/MaterializedViewAggregateRule.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ class MaterializedViewAggregateRule(sparkSession: SparkSession) extends AbstractMaterializedViewRule(sparkSession: SparkSession) { /** - * cehck plan if match current rule + * check plan if match current rule * * @param logicalPlan LogicalPlan * @return true:matched ; false:unMatched @@ -43,7 +43,6 @@ class MaterializedViewAggregateRule(sparkSession: SparkSession) logicalPlan.children.forall(isValidLogicalPlan) } - /** * queryTableInfo!=viewTableInfo , need do join compensate * @@ -208,7 +207,6 @@ class MaterializedViewAggregateRule(sparkSession: SparkSession) // 4.rewrite and alias queryAggExpressions // if the rewrite expression exprId != origin expression exprId, // replace by Alias(rewrite expression,origin.name)(exprId=origin.exprId) - val rewritedQueryAggExpressions = rewriteAndAliasExpressions(newQueryAggExpressions, swapTableColumn = true, tableMapping, columnMapping, viewProjectList, viewTableAttrs, queryAgg.aggregateExpressions) @@ -240,5 +238,4 @@ class MaterializedViewAggregateRule(sparkSession: SparkSession) qualifier = qualifier, explicitMetadata = alias.explicitMetadata, nonInheritableMetadataKeys = alias.nonInheritableMetadataKeys) } - } diff --git a/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/MaterializedViewJoinRule.scala b/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/MaterializedViewJoinRule.scala index 15717fa16..5c7c477dd 100644 --- a/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/MaterializedViewJoinRule.scala +++ b/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/optimizer/rules/MaterializedViewJoinRule.scala @@ -22,14 +22,14 @@ import com.huawei.boostkit.spark.util.{ExpressionEqual, TableEqual} import scala.collection.mutable import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, NamedExpression} +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.Inner -import org.apache.spark.sql.catalyst.plans.logical.{Join, JoinHint, LogicalPlan, Project} +import org.apache.spark.sql.catalyst.plans.logical._ class MaterializedViewJoinRule(sparkSession: SparkSession) extends AbstractMaterializedViewRule(sparkSession: SparkSession) { /** - * cehck plan if match current rule + * check plan if match current rule * * @param logicalPlan LogicalPlan * @return true:matched ; false:unMatched @@ -38,7 +38,6 @@ class MaterializedViewJoinRule(sparkSession: SparkSession) isValidLogicalPlan(logicalPlan) } - /** * queryTableInfo!=viewTableInfo , need do join compensate * @@ -66,11 +65,20 @@ class MaterializedViewJoinRule(sparkSession: SparkSession) topViewProject.get } + var projectList: Seq[NamedExpression] = newViewQueryPlan match { + case p: Project => + p.projectList + case _ => + newViewQueryPlan.output + } + needTables.foreach { needTable => newViewQueryPlan = Join(newViewQueryPlan, needTable.logicalPlan, Inner, None, JoinHint.NONE) + projectList ++= needTable.logicalPlan.output } - Some(newViewTablePlan, viewQueryPlan, None) + newViewQueryPlan = Project(projectList, newViewQueryPlan) + Some(newViewTablePlan, newViewQueryPlan, None) } /** diff --git a/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/parser/OmniCacheExtensionAstBuilder.scala b/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/parser/OmniCacheExtensionAstBuilder.scala index a41680244..414afa602 100644 --- a/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/parser/OmniCacheExtensionAstBuilder.scala +++ b/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/parser/OmniCacheExtensionAstBuilder.scala @@ -35,6 +35,11 @@ import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.execution.datasources._ +/** + * Parse extended AST to LogicalPlan + * + * @param delegate Spark default ParserInterface + */ class OmniCacheExtensionAstBuilder(spark: SparkSession, delegate: ParserInterface) extends OmniCacheSqlExtensionsBaseVisitor[AnyRef] with SQLConfHelper with Logging { @@ -61,8 +66,7 @@ class OmniCacheExtensionAstBuilder(spark: SparkSession, delegate: ParserInterfac case Seq(mv) => (None, mv) case Seq(database, mv) => (Some(database), mv) case _ => throw new AnalysisException( - "The mv name is not valid: %s".format(identifier.mkString(".")) - ) + "The mv name is not valid: %s".format(identifier.mkString("."))) } try { @@ -102,8 +106,7 @@ class OmniCacheExtensionAstBuilder(spark: SparkSession, delegate: ParserInterfac case Seq(mv) => (None, mv) case Seq(database, mv) => (Some(database), mv) case _ => throw new AnalysisException( - "The mv name is not valid: %s".format(tableIdent.mkString(".")) - ) + "The mv name is not valid: %s".format(tableIdent.mkString("."))) } val tableIdentifier = TableIdentifier(name, databaseName) @@ -119,7 +122,7 @@ class OmniCacheExtensionAstBuilder(spark: SparkSession, delegate: ParserInterfac throw new RuntimeException("cannot refresh a table with refresh mv") } - // preserver preDatabase and set curDatabase + // preserve preDatabase and set curDatabase val preDatabase = spark.catalog.currentDatabase val curDatabase = catalogTable.properties.getOrElse(MV_QUERY_ORIGINAL_SQL_CUR_DB, "") if (curDatabase.isEmpty) { @@ -179,6 +182,7 @@ class OmniCacheExtensionAstBuilder(spark: SparkSession, delegate: ParserInterfac RewriteHelper.enableCachePlugin() throw e } finally { + // reset preDatabase spark.sessionState.catalogManager.setCurrentNamespace(Array(preDatabase)) } } @@ -221,8 +225,7 @@ class OmniCacheExtensionAstBuilder(spark: SparkSession, delegate: ParserInterfac case Seq(database, mv) => DropMaterializedViewCommand( TableIdentifier(mv, Some(database)), ifExists.isDefined, - purge = true - ) + purge = true) case _ => throw new AnalysisException( "The mv name is not valid: %s".format(identifier.mkString("."))) } @@ -297,6 +300,9 @@ class OmniCacheExtensionAstBuilder(spark: SparkSession, delegate: ParserInterfac visit(ctx.statement).asInstanceOf[LogicalPlan] } + /** + * alias tuple2 + */ type OmniCacheHeader = (Seq[String], Boolean) /** diff --git a/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/parser/OmniCacheExtensionSqlParser.scala b/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/parser/OmniCacheExtensionSqlParser.scala index 65077216a..aeab84232 100644 --- a/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/parser/OmniCacheExtensionSqlParser.scala +++ b/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/catalyst/parser/OmniCacheExtensionSqlParser.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.parser import java.util.Locale -import org.antlr.v4.runtime.{CharStreams, CommonTokenStream} +import org.antlr.v4.runtime._ import org.antlr.v4.runtime.atn.PredictionMode import org.antlr.v4.runtime.misc.ParseCancellationException diff --git a/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/execution/command/OmniCacheCommand.scala b/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/execution/command/OmniCacheCommand.scala index 801fffbd2..c053b6252 100644 --- a/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/execution/command/OmniCacheCommand.scala +++ b/omnicache/omnicache-spark-extension/plugin/src/main/scala/org/apache/spark/sql/execution/command/OmniCacheCommand.scala @@ -54,6 +54,7 @@ case class OmniCacheCreateMvCommand( partitioning: Seq[String], query: LogicalPlan, outputColumnNames: Seq[String]) extends DataWritingCommand { + override def run(sparkSession: SparkSession, child: SparkPlan): Seq[Row] = { try { ViewMetadata.init(sparkSession) @@ -84,7 +85,7 @@ case class OmniCacheCreateMvCommand( throw new AnalysisException( s"Materialized View $tableIdentWithDB already exists. You need to drop it first") } else { - // Since the table already exists and the save mode is Ignore,we will just return. + // Since the table already exists and the save mode is Ignore, we will just return. return Seq.empty } } else { @@ -102,10 +103,11 @@ case class OmniCacheCreateMvCommand( storage = table.storage.copy(locationUri = tableLocation), // We will use the schema of resolved.relation as the schema of the table (instead of // the schema of df). It is important since the nullability may be changed by the relation - // provider (for example,see org.apache.spark.sql.parquet.DefaultSource). + // provider (for example, see org.apache.spark.sql.parquet.DefaultSource). schema = tableSchema) // Table location is already validated. No need to check it again during table creation. sessionState.catalog.createTable(newTable, ignoreIfExists = false, validateLocation = false) + result match { case _: HadoopFsRelation if table.partitionColumnNames.nonEmpty && sparkSession.sqlContext.conf.manageFilesourcePartitions => @@ -153,10 +155,19 @@ case class OmniCacheCreateMvCommand( } } +/** + * Drops a materialized view from the metastore and removes it if it is cached. + * + * The syntax of this command is: + * {{{ + * DROP MATERIALIZED VIEW [IF EXISTS] view_name; + * }}} + */ case class DropMaterializedViewCommand( tableName: TableIdentifier, ifExists: Boolean, purge: Boolean) extends RunnableCommand { + override def run(sparkSession: SparkSession): Seq[Row] = { ViewMetadata.init(sparkSession) val catalog = sparkSession.sessionState.catalog @@ -201,13 +212,18 @@ case class DropMaterializedViewCommand( } } +/** + * ShowMaterializedViewCommand RunnableCommand + * + */ case class ShowMaterializedViewCommand( databaseName: Option[String], tableIdentifierPattern: Option[String]) extends RunnableCommand { + // The result of SHOW MaterializedView has three basic columns: + // database, tableName and originalSql. override val output: Seq[Attribute] = { val tableExtendedInfo = Nil - AttributeReference("database", StringType, nullable = false)() :: AttributeReference("mvName", StringType, nullable = false)() :: AttributeReference("rewriteEnable", StringType, nullable = false)() :: @@ -254,6 +270,7 @@ case class ShowMaterializedViewCommand( case class AlterRewriteMaterializedViewCommand( tableName: TableIdentifier, enableRewrite: Boolean) extends RunnableCommand { + override def run(sparkSession: SparkSession): Seq[Row] = { ViewMetadata.init(sparkSession) val catalog = sparkSession.sessionState.catalog @@ -379,7 +396,6 @@ case class RefreshMaterializedViewCommand( } if (doInsertion) { - def refreshUpdatedPartitions(updatedPartitionPaths: Set[String]): Unit = { val updatedPartitions = updatedPartitionPaths.map(PartitioningUtils.parsePathFragment) if (partitionsTrackedByCatalog) { @@ -446,7 +462,6 @@ case class RefreshMaterializedViewCommand( if (catalogTable.nonEmpty) { CommandUtils.updateTableStats(sparkSession, catalogTable.get) } - } else { logInfo("Skipping insertion into a relation that already exists.") } diff --git a/omnicache/omnicache-spark-extension/plugin/src/test/resources/tpcds_ddl.sql b/omnicache/omnicache-spark-extension/plugin/src/test/resources/tpcds_ddl.sql new file mode 100644 index 000000000..ae18bd4b0 --- /dev/null +++ b/omnicache/omnicache-spark-extension/plugin/src/test/resources/tpcds_ddl.sql @@ -0,0 +1,479 @@ +create table if not exists store_sales( + ss_sold_date_sk bigint +, ss_sold_time_sk bigint +, ss_item_sk bigint +, ss_customer_sk bigint +, ss_cdemo_sk bigint +, ss_hdemo_sk bigint +, ss_addr_sk bigint +, ss_store_sk bigint +, ss_promo_sk bigint +, ss_ticket_number bigint +, ss_quantity int +, ss_wholesale_cost decimal(7,2) +, ss_list_price decimal(7,2) +, ss_sales_price decimal(7,2) +, ss_ext_discount_amt decimal(7,2) +, ss_ext_sales_price decimal(7,2) +, ss_ext_wholesale_cost decimal(7,2) +, ss_ext_list_price decimal(7,2) +, ss_ext_tax decimal(7,2) +, ss_coupon_amt decimal(7,2) +, ss_net_paid decimal(7,2) +, ss_net_paid_inc_tax decimal(7,2) +, ss_net_profit decimal(7,2) +); + +create table if not exists store_returns( + sr_returned_date_sk bigint +, sr_return_time_sk bigint +, sr_item_sk bigint +, sr_customer_sk bigint +, sr_cdemo_sk bigint +, sr_hdemo_sk bigint +, sr_addr_sk bigint +, sr_store_sk bigint +, sr_reason_sk bigint +, sr_ticket_number bigint +, sr_return_quantity int +, sr_return_amt decimal(7,2) +, sr_return_tax decimal(7,2) +, sr_return_amt_inc_tax decimal(7,2) +, sr_fee decimal(7,2) +, sr_return_ship_cost decimal(7,2) +, sr_refunded_cash decimal(7,2) +, sr_reversed_charge decimal(7,2) +, sr_store_credit decimal(7,2) +, sr_net_loss decimal(7,2) +); + +create table if not exists catalog_sales( + cs_sold_date_sk bigint +, cs_sold_time_sk bigint +, cs_ship_date_sk bigint +, cs_bill_customer_sk bigint +, cs_bill_cdemo_sk bigint +, cs_bill_hdemo_sk bigint +, cs_bill_addr_sk bigint +, cs_ship_customer_sk bigint +, cs_ship_cdemo_sk bigint +, cs_ship_hdemo_sk bigint +, cs_ship_addr_sk bigint +, cs_call_center_sk bigint +, cs_catalog_page_sk bigint +, cs_ship_mode_sk bigint +, cs_warehouse_sk bigint +, cs_item_sk bigint +, cs_promo_sk bigint +, cs_order_number bigint +, cs_quantity int +, cs_wholesale_cost decimal(7,2) +, cs_list_price decimal(7,2) +, cs_sales_price decimal(7,2) +, cs_ext_discount_amt decimal(7,2) +, cs_ext_sales_price decimal(7,2) +, cs_ext_wholesale_cost decimal(7,2) +, cs_ext_list_price decimal(7,2) +, cs_ext_tax decimal(7,2) +, cs_coupon_amt decimal(7,2) +, cs_ext_ship_cost decimal(7,2) +, cs_net_paid decimal(7,2) +, cs_net_paid_inc_tax decimal(7,2) +, cs_net_paid_inc_ship decimal(7,2) +, cs_net_paid_inc_ship_tax decimal(7,2) +, cs_net_profit decimal(7,2) +); + +create table if not exists catalog_returns( + cr_returned_date_sk bigint +, cr_returned_time_sk bigint +, cr_item_sk bigint +, cr_refunded_customer_sk bigint +, cr_refunded_cdemo_sk bigint +, cr_refunded_hdemo_sk bigint +, cr_refunded_addr_sk bigint +, cr_returning_customer_sk bigint +, cr_returning_cdemo_sk bigint +, cr_returning_hdemo_sk bigint +, cr_returning_addr_sk bigint +, cr_call_center_sk bigint +, cr_catalog_page_sk bigint +, cr_ship_mode_sk bigint +, cr_warehouse_sk bigint +, cr_reason_sk bigint +, cr_order_number bigint +, cr_return_quantity int +, cr_return_amount decimal(7,2) +, cr_return_tax decimal(7,2) +, cr_return_amt_inc_tax decimal(7,2) +, cr_fee decimal(7,2) +, cr_return_ship_cost decimal(7,2) +, cr_refunded_cash decimal(7,2) +, cr_reversed_charge decimal(7,2) +, cr_store_credit decimal(7,2) +, cr_net_loss decimal(7,2) +); + +create table if not exists web_sales( + ws_sold_date_sk bigint +, ws_sold_time_sk bigint +, ws_ship_date_sk bigint +, ws_item_sk bigint +, ws_bill_customer_sk bigint +, ws_bill_cdemo_sk bigint +, ws_bill_hdemo_sk bigint +, ws_bill_addr_sk bigint +, ws_ship_customer_sk bigint +, ws_ship_cdemo_sk bigint +, ws_ship_hdemo_sk bigint +, ws_ship_addr_sk bigint +, ws_web_page_sk bigint +, ws_web_site_sk bigint +, ws_ship_mode_sk bigint +, ws_warehouse_sk bigint +, ws_promo_sk bigint +, ws_order_number bigint +, ws_quantity int +, ws_wholesale_cost decimal(7,2) +, ws_list_price decimal(7,2) +, ws_sales_price decimal(7,2) +, ws_ext_discount_amt decimal(7,2) +, ws_ext_sales_price decimal(7,2) +, ws_ext_wholesale_cost decimal(7,2) +, ws_ext_list_price decimal(7,2) +, ws_ext_tax decimal(7,2) +, ws_coupon_amt decimal(7,2) +, ws_ext_ship_cost decimal(7,2) +, ws_net_paid decimal(7,2) +, ws_net_paid_inc_tax decimal(7,2) +, ws_net_paid_inc_ship decimal(7,2) +, ws_net_paid_inc_ship_tax decimal(7,2) +, ws_net_profit decimal(7,2) +); + +create table if not exists web_returns( + wr_returned_date_sk bigint +, wr_returned_time_sk bigint +, wr_item_sk bigint +, wr_refunded_customer_sk bigint +, wr_refunded_cdemo_sk bigint +, wr_refunded_hdemo_sk bigint +, wr_refunded_addr_sk bigint +, wr_returning_customer_sk bigint +, wr_returning_cdemo_sk bigint +, wr_returning_hdemo_sk bigint +, wr_returning_addr_sk bigint +, wr_web_page_sk bigint +, wr_reason_sk bigint +, wr_order_number bigint +, wr_return_quantity int +, wr_return_amt decimal(7,2) +, wr_return_tax decimal(7,2) +, wr_return_amt_inc_tax decimal(7,2) +, wr_fee decimal(7,2) +, wr_return_ship_cost decimal(7,2) +, wr_refunded_cash decimal(7,2) +, wr_reversed_charge decimal(7,2) +, wr_account_credit decimal(7,2) +, wr_net_loss decimal(7,2) +); + +create table if not exists inventory( + inv_date_sk bigint +, inv_item_sk bigint +, inv_warehouse_sk bigint +, inv_quantity_on_hand int +); +create table if not exists store( + s_store_sk bigint +, s_store_id char(16) +, s_rec_start_date date +, s_rec_end_date date +, s_closed_date_sk bigint +, s_store_name varchar(50) +, s_number_employees int +, s_floor_space int +, s_hours char(20) +, S_manager varchar(40) +, S_market_id int +, S_geography_class varchar(100) +, S_market_desc varchar(100) +, s_market_manager varchar(40) +, s_division_id int +, s_division_name varchar(50) +, s_company_id int +, s_company_name varchar(50) +, s_street_number varchar(10) +, s_street_name varchar(60) +, s_street_type char(15) +, s_suite_number char(10) +, s_city varchar(60) +, s_county varchar(30) +, s_state char(2) +, s_zip char(10) +, s_country varchar(20) +, s_gmt_offset decimal(5,2) +, s_tax_percentage decimal(5,2) +); +create table if not exists call_center( + cc_call_center_sk bigint +, cc_call_center_id char(16) +, cc_rec_start_date date +, cc_rec_end_date date +, cc_closed_date_sk bigint +, cc_open_date_sk bigint +, cc_name varchar(50) +, cc_class varchar(50) +, cc_employees int +, cc_sq_ft int +, cc_hours char(20) +, cc_manager varchar(40) +, cc_mkt_id int +, cc_mkt_class char(50) +, cc_mkt_desc varchar(100) +, cc_market_manager varchar(40) +, cc_division int +, cc_division_name varchar(50) +, cc_company int +, cc_company_name char(50) +, cc_street_number char(10) +, cc_street_name varchar(60) +, cc_street_type char(15) +, cc_suite_number char(10) +, cc_city varchar(60) +, cc_county varchar(30) +, cc_state char(2) +, cc_zip char(10) +, cc_country varchar(20) +, cc_gmt_offset decimal(5,2) +, cc_tax_percentage decimal(5,2) +); +create table if not exists catalog_page( + cp_catalog_page_sk bigint +, cp_catalog_page_id char(16) +, cp_start_date_sk bigint +, cp_end_date_sk bigint +, cp_department varchar(50) +, cp_catalog_number int +, cp_catalog_page_number int +, cp_description varchar(100) +, cp_type varchar(100) +); +create table if not exists web_site( + web_site_sk bigint +, web_site_id char(16) +, web_rec_start_date date +, web_rec_end_date date +, web_name varchar(50) +, web_open_date_sk bigint +, web_close_date_sk bigint +, web_class varchar(50) +, web_manager varchar(40) +, web_mkt_id int +, web_mkt_class varchar(50) +, web_mkt_desc varchar(100) +, web_market_manager varchar(40) +, web_company_id int +, web_company_name char(50) +, web_street_number char(10) +, web_street_name varchar(60) +, web_street_type char(15) +, web_suite_number char(10) +, web_city varchar(60) +, web_county varchar(30) +, web_state char(2) +, web_zip char(10) +, web_country varchar(20) +, web_gmt_offset decimal(5,2) +, web_tax_percentage decimal(5,2) +); +create table if not exists web_page( + wp_web_page_sk bigint +, wp_web_page_id char(16) +, wp_rec_start_date date +, wp_rec_end_date date +, wp_creation_date_sk bigint +, wp_access_date_sk bigint +, wp_autogen_flag char(1) +, wp_customer_sk bigint +, wp_url varchar(100) +, wp_type char(50) +, wp_char_count int +, wp_link_count int +, wp_image_count int +, wp_max_ad_count int +); +create table if not exists warehouse( + w_warehouse_sk bigint +, w_warehouse_id char(16) +, w_warehouse_name varchar(20) +, w_warehouse_sq_ft int +, w_street_number char(10) +, w_street_name varchar(60) +, w_street_type char(15) +, w_suite_number char(10) +, w_city varchar(60) +, w_county varchar(30) +, w_state char(2) +, w_zip char(10) +, w_country varchar(20) +, w_gmt_offset decimal(5,2) +); +create table if not exists customer( + c_customer_sk bigint +, c_customer_id char(16) +, c_current_cdemo_sk bigint +, c_current_hdemo_sk bigint +, c_current_addr_sk bigint +, c_first_shipto_date_sk bigint +, c_first_sales_date_sk bigint +, c_salutation char(10) +, c_first_name char(20) +, c_last_name char(30) +, c_preferred_cust_flag char(1) +, c_birth_day int +, c_birth_month int +, c_birth_year int +, c_birth_country varchar(20) +, c_login char(13) +, c_email_address char(50) +, c_last_review_date_sk bigint +); +create table if not exists customer_address( + ca_address_sk bigint +, ca_address_id char(16) +, ca_street_number char(10) +, ca_street_name varchar(60) +, ca_street_type char(15) +, ca_suite_number char(10) +, ca_city varchar(60) +, ca_county varchar(30) +, ca_state char(2) +, ca_zip char(10) +, ca_country varchar(20) +, ca_gmt_offset decimal(5,2) +, ca_location_type char(20) +); +create table if not exists customer_demographics( + cd_demo_sk bigint +, cd_gender char(1) +, cd_marital_status char(1) +, cd_education_status char(20) +, cd_purchase_estimate int +, cd_credit_rating char(10) +, cd_dep_count int +, cd_dep_employed_count int +, cd_dep_college_count int +); +create table if not exists date_dim( + d_date_sk bigint +, d_date_id char(16) +, d_date date +, d_month_seq int +, d_week_seq int +, d_quarter_seq int +, d_year int +, d_dow int +, d_moy int +, d_dom int +, d_qoy int +, d_fy_year int +, d_fy_quarter_seq int +, d_fy_week_seq int +, d_day_name char(9) +, d_quarter_name char(6) +, d_holiday char(1) +, d_weekend char(1) +, d_following_holiday char(1) +, d_first_dom int +, d_last_dom int +, d_same_day_ly int +, d_same_day_lq int +, d_current_day char(1) +, d_current_week char(1) +, d_current_month char(1) +, d_current_quarter char(1) +, d_current_year char(1) +); +create table if not exists household_demographics( + hd_demo_sk bigint +, hd_income_band_sk bigint +, hd_buy_potential char(15) +, hd_dep_count int +, hd_vehicle_count int +); +create table if not exists item( + i_item_sk bigint +, i_item_id char(16) +, i_rec_start_date date +, i_rec_end_date date +, i_item_desc varchar(200) +, i_current_price decimal(7,2) +, i_wholesale_cost decimal(7,2) +, i_brand_id int +, i_brand char(50) +, i_class_id int +, i_class char(50) +, i_category_id int +, i_category char(50) +, i_manufact_id int +, i_manufact char(50) +, i_size char(20) +, i_formulation char(20) +, i_color char(20) +, i_units char(10) +, i_container char(10) +, i_manager_id int +, i_product_name char(50) +); +create table if not exists income_band( + ib_income_band_sk bigint +, ib_lower_bound int +, ib_upper_bound int +); +create table if not exists promotion( + p_promo_sk bigint +, p_promo_id char(16) +, p_start_date_sk bigint +, p_end_date_sk bigint +, p_item_sk bigint +, p_cost decimal(15,2) +, p_response_target int +, p_promo_name char(50) +, p_channel_dmail char(1) +, p_channel_email char(1) +, p_channel_catalog char(1) +, p_channel_tv char(1) +, p_channel_radio char(1) +, p_channel_press char(1) +, p_channel_event char(1) +, p_channel_demo char(1) +, p_channel_details varchar(100) +, p_purpose char(15) +, p_discount_active char(1) +); +create table if not exists reason( + r_reason_sk bigint +, r_reason_id char(16) +, r_reason_desc char(100) +); +create table if not exists ship_mode( + sm_ship_mode_sk bigint +, sm_ship_mode_id char(16) +, sm_type char(30) +, sm_code char(10) +, sm_carrier char(20) +, sm_contract char(20) +); +create table if not exists time_dim( + t_time_sk bigint +, t_time_id char(16) +, t_time int +, t_hour int +, t_minute int +, t_second int +, t_am_pm char(2) +, t_shift char(20) +, t_sub_shift char(20) +, t_meal_time char(20) +); \ No newline at end of file diff --git a/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/optimizer/rules/MaterializedViewAggregateRuleSuite.scala b/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/optimizer/rules/MaterializedViewAggregateRuleSuite.scala index 23a783128..a173eba2f 100644 --- a/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/optimizer/rules/MaterializedViewAggregateRuleSuite.scala +++ b/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/optimizer/rules/MaterializedViewAggregateRuleSuite.scala @@ -162,7 +162,7 @@ class MaterializedViewAggregateRuleSuite extends RewriteSuite { } test("mv_agg3_3") { - // group by column(is empty) is subset of view,,additional agg on view column + // group by column(is empty) is subset of view,additional agg on view column val sql = """ |SELECT sum(c.integertype) as _sum, @@ -232,8 +232,112 @@ class MaterializedViewAggregateRuleSuite extends RewriteSuite { comparePlansAndRows(sql, "default", "mv_agg4", noData = false) } + test("mv_agg4_3") { + // group by column,agg is same to view, join more + val sql = + """ + |SELECT c.empid,c.deptno,c.locationid,sum(c.integertype) as _sum, + |max(c.longtype) as _max,min(c.floattype) as _min, + |count(c.doubletype) as _count,count(distinct c.datetype) as _count_distinct, + |avg(c.decimaltype) as _avg + |FROM column_type c JOIN emps e JOIN locations l + |ON c.empid=e.empid + |AND c.locationid=l.locationid + |AND c.empid=1 + |GROUP BY c.empid,c.deptno,c.locationid; + |""".stripMargin + comparePlansAndRows(sql, "default", "mv_agg4", noData = false) + } + test("mv_agg4_disable") { val sql = "ALTER MATERIALIZED VIEW mv_agg4 DISABLE REWRITE;" spark.sql(sql).show() } + + test("mv_agg5") { + spark.sql( + """ + |DROP MATERIALIZED VIEW IF EXISTS mv_agg5; + |""".stripMargin + ) + spark.sql( + """ + |CREATE MATERIALIZED VIEW IF NOT EXISTS mv_agg5 + |AS + |SELECT c.empid,c.deptno,c.locationid,sum(c.integertype) as _sum, + |max(c.longtype) as _max,min(c.floattype) as _min, + |count(c.doubletype) as _count,count(distinct c.datetype) as _count_distinct, + |avg(c.decimaltype) as _avg + |FROM column_type c JOIN emps e + |ON c.empid=e.empid + |AND c.empid=1 + |GROUP BY c.empid,c.deptno,c.locationid; + |""".stripMargin + ) + } + + test("mv_agg5_1") { + val sql = + """ + |SELECT c.empid,c.deptno,sum(c.integertype) as _sum, + |max(c.longtype) as _max,min(c.floattype) as _min, + |count(c.doubletype) as _count + |FROM column_type c JOIN emps e + |ON c.empid=e.empid + |AND c.empid=1 + |GROUP BY c.empid,c.deptno; + |""".stripMargin + comparePlansAndRows(sql, "default", "mv_agg5", noData = false) + } + + test("mv_agg5_2") { + val sql = + """ + |SELECT c.empid,c.deptno,count(distinct c.datetype) as _count_distinct + |FROM column_type c JOIN emps e + |ON c.empid=e.empid + |AND c.empid=1 + |GROUP BY c.empid,c.deptno; + |""".stripMargin + comparePlansAndRows(sql, "default", "mv_agg5", noData = false) + spark.sql( + """ + |DROP MATERIALIZED VIEW IF EXISTS mv_agg5; + |""".stripMargin + ) + } + + test("mv_agg6") { + spark.sql( + """ + |DROP MATERIALIZED VIEW IF EXISTS mv_agg6; + |""".stripMargin + ) + spark.sql( + """ + |CREATE MATERIALIZED VIEW IF NOT EXISTS mv_agg6 + |AS + |SELECT c.empid,c.deptno,c.locationid,sum(distinct c.integertype) as _sum, + |max(distinct c.longtype) as _max,min(distinct c.floattype) as _min, + |count(distinct c.doubletype) as _count + |FROM column_type c + |GROUP BY c.empid,c.deptno,c.locationid; + |""".stripMargin + ) + + val sql = + """ + |SELECT c.empid,c.deptno,sum(distinct c.integertype) as _sum, + |max(distinct c.longtype) as _max,min(distinct c.floattype) as _min, + |count(distinct c.doubletype) as _count + |FROM column_type c + |GROUP BY c.empid,c.deptno; + |""".stripMargin + comparePlansAndRows(sql, "default", "mv_agg6", noData = false) + spark.sql( + """ + |DROP MATERIALIZED VIEW IF EXISTS mv_agg6; + |""".stripMargin + ) + } } diff --git a/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/optimizer/rules/MaterializedViewFilterRuleSuite.scala b/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/optimizer/rules/MaterializedViewFilterRuleSuite.scala index 473d3fde1..f34e2b74e 100644 --- a/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/optimizer/rules/MaterializedViewFilterRuleSuite.scala +++ b/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/optimizer/rules/MaterializedViewFilterRuleSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.optimizer.rules +import com.huawei.boostkit.spark.util.RewriteHelper + class MaterializedViewFilterRuleSuite extends RewriteSuite { test("mv_filter1") { @@ -273,7 +275,100 @@ class MaterializedViewFilterRuleSuite extends RewriteSuite { } test("mv_filter2_disable") { - val sql = "ALTER MATERIALIZED VIEW mv_filter1 DISABLE REWRITE;" + val sql = "ALTER MATERIALIZED VIEW mv_filter2 DISABLE REWRITE;" spark.sql(sql).show() } + + test("mv_filter_rand") { + spark.sql( + """ + |DROP MATERIALIZED VIEW IF EXISTS mv_filter_rand; + |""".stripMargin + ) + spark.sql( + """ + |CREATE MATERIALIZED VIEW IF NOT EXISTS mv_filter_rand + |AS + |SELECT * FROM COLUMN_TYPE WHERE doubletype=rand(1) + |""".stripMargin + ) + val sql = + """ + |SELECT * FROM COLUMN_TYPE WHERE doubletype=rand() + |""".stripMargin + compareNotRewriteAndRows(sql, noData = true) + + RewriteHelper.enableCachePlugin() + val sql2 = + """ + |SELECT * FROM COLUMN_TYPE WHERE doubletype=rand(1) + |""".stripMargin + comparePlansAndRows(sql2, "default", "mv_filter_rand", noData = true) + + spark.sql( + """ + |DROP MATERIALIZED VIEW IF EXISTS mv_filter_rand; + |""".stripMargin + ) + } + + test("mv_filter_if") { + spark.sql( + """ + |DROP MATERIALIZED VIEW IF EXISTS mv_filter_if; + |""".stripMargin + ) + spark.sql( + """ + |CREATE MATERIALIZED VIEW IF NOT EXISTS mv_filter_if + |AS + |SELECT e.empid,e.deptno,if(e.deptno<2,e.empid,e.deptno) as _if FROM emps e; + |""".stripMargin + ) + val sql = "SELECT if(e.deptno<3,e.empid,e.deptno) as _if FROM emps e" + comparePlansAndRows(sql, "default", "mv_filter_if", noData = false) + spark.sql( + """ + |DROP MATERIALIZED VIEW IF EXISTS mv_filter_if; + |""".stripMargin + ) + } + + test("mv_filter3") { + // or + spark.sql( + """ + |DROP MATERIALIZED VIEW IF EXISTS mv_filter3; + |""".stripMargin + ) + + spark.sql( + """ + |CREATE MATERIALIZED VIEW IF NOT EXISTS mv_filter3 + |AS + |SELECT * FROM COLUMN_TYPE WHERE empid=3 + |OR bytetype>1 OR shorttype<5 OR integertype>=1 OR longtype<=5 + |OR floattype in(3.0) OR doubletype not in(2.0) + |OR datetype between DATE '2021-01-01' and DATE '2023-01-01' + |OR stringtype='stringtype3' + |OR timestamptype is not null OR decimaltype is null; + |""".stripMargin + ) + + val sql = + """ + |SELECT * FROM COLUMN_TYPE WHERE empid=3 + |OR bytetype>1 OR shorttype<5 OR integertype>=1 OR longtype<=5 + |OR floattype in(3.0) OR doubletype not in(2.0) + |OR datetype between DATE '2021-01-01' and DATE '2023-01-01' + |OR stringtype='stringtype3' + |OR timestamptype is not null OR decimaltype is null; + |""".stripMargin + comparePlansAndRows(sql, "default", "mv_filter3", noData = false) + spark.sql( + """ + |DROP MATERIALIZED VIEW IF EXISTS mv_filter3; + |""".stripMargin + ) + } } diff --git a/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/optimizer/rules/MaterializedViewJoinRuleSuite.scala b/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/optimizer/rules/MaterializedViewJoinRuleSuite.scala index df7a1f255..da9e4faf2 100644 --- a/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/optimizer/rules/MaterializedViewJoinRuleSuite.scala +++ b/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/optimizer/rules/MaterializedViewJoinRuleSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.optimizer.rules +import com.huawei.boostkit.spark.util.RewriteHelper + class MaterializedViewJoinRuleSuite extends RewriteSuite { test("mv_join1") { @@ -150,13 +152,15 @@ class MaterializedViewJoinRuleSuite extends RewriteSuite { // view tables is same to query, equal tables val sql = """ - |SELECT e.*,c1.stringtype + |SELECT e.*,c2.stringtype |FROM emps e JOIN column_type c1 JOIN column_type c2 |ON e.deptno=c1.deptno AND e.deptno=c2.deptno |AND c1.deptno!=2 |AND c2.deptno=1; |""".stripMargin comparePlansAndRows(sql, "default", "mv_join2", noData = false) + RewriteHelper.enableCachePlugin() + comparePlansAndRows(sql, "default", "mv_join2", noData = false) } test("mv_join2_disable") { diff --git a/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/optimizer/rules/RewriteSuite.scala b/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/optimizer/rules/RewriteSuite.scala index 8ddc9dac1..e913b59ac 100644 --- a/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/optimizer/rules/RewriteSuite.scala +++ b/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/optimizer/rules/RewriteSuite.scala @@ -15,7 +15,6 @@ * limitations under the License. */ - package org.apache.spark.sql.catalyst.optimizer.rules import com.huawei.boostkit.spark.conf.OmniCachePluginConfig @@ -44,6 +43,7 @@ class RewriteSuite extends SparkFunSuite with PredicateHelper { .config("hive.exec.dynamic.partition.mode", "nonstrict") .config("spark.ui.port", "4050") // .config("spark.sql.planChangeLog.level","WARN") + .config("spark.sql.omnicache.logLevel", "WARN") .enableHiveSupport() .getOrCreate() spark.sparkContext.setLogLevel("WARN") diff --git a/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/optimizer/rules/TpcdsSuite.scala b/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/optimizer/rules/TpcdsSuite.scala new file mode 100644 index 000000000..42adf96cc --- /dev/null +++ b/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/optimizer/rules/TpcdsSuite.scala @@ -0,0 +1,95 @@ +/* + * 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.catalyst.optimizer.rules + +import org.apache.commons.io.IOUtils +import scala.collection.mutable + +import org.apache.spark.sql.catalyst.TableIdentifier + +class TpcdsSuite extends RewriteSuite { + + def createTable(): Unit = { + if (catalog.tableExists(TableIdentifier("store_sales"))) { + return + } + val fis = this.getClass.getResourceAsStream("/tpcds_ddl.sql") + val lines = IOUtils.readLines(fis, "UTF-8") + IOUtils.closeQuietly(fis) + + var sqls = Seq.empty[String] + val sql = mutable.StringBuilder.newBuilder + lines.forEach { line => + sql.append(line) + sql.append(" ") + if (line.contains(';')) { + sqls +:= sql.toString() + sql.clear() + } + } + sqls.foreach { sql => + spark.sql(sql) + } + } + + createTable() + + test("subQuery outReference") { + spark.sql("DROP MATERIALIZED VIEW IF EXISTS mv536") + spark.sql( + """ + |CREATE MATERIALIZED VIEW IF NOT EXISTS mv536 PARTITIONED BY (ws_sold_date_sk) AS + | SELECT + | web_sales.ws_ext_discount_amt, + | item.i_item_sk, + | web_sales.ws_sold_date_sk, + | web_sales.ws_item_sk, + | item.i_manufact_id + |FROM + | web_sales, + | item + |WHERE + | item.i_manufact_id = 350 + | AND web_sales.ws_item_sk = item.i_item_sk + |distribute by ws_sold_date_sk; + |""".stripMargin + ) + val sql = + """ + |SELECT sum(ws_ext_discount_amt) AS `Excess Discount Amount ` + |FROM web_sales, item, date_dim + |WHERE i_manufact_id = 350 + | AND i_item_sk = ws_item_sk + | AND d_date BETWEEN '2000-01-27' AND (cast('2000-01-27' AS DATE) + INTERVAL 90 days) + | AND d_date_sk = ws_sold_date_sk + | AND ws_ext_discount_amt > + | ( + | SELECT 1.3 * avg(ws_ext_discount_amt) + | FROM web_sales, date_dim + | WHERE ws_item_sk = i_item_sk + | AND d_date BETWEEN '2000-01-27' AND (cast('2000-01-27' AS DATE) + INTERVAL 90 days) + | AND d_date_sk = ws_sold_date_sk + | ) + |ORDER BY sum(ws_ext_discount_amt) + |LIMIT 100 + | + |""".stripMargin + compareNotRewriteAndRows(sql, noData = true) + spark.sql("DROP MATERIALIZED VIEW IF EXISTS mv536") + } +} diff --git a/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlParserSuite.scala b/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlParserSuite.scala index 7b40f3111..273197c91 100644 --- a/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlParserSuite.scala +++ b/omnicache/omnicache-spark-extension/plugin/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlParserSuite.scala @@ -560,6 +560,7 @@ class SqlParserSuite extends RewriteSuite { } test("mv_is_cached") { + spark.sql("DROP MATERIALIZED VIEW IF EXISTS mv_create2;") spark.sql( """ |ALTER MATERIALIZED VIEW default.mv_create1 ENABLE REWRITE @@ -568,10 +569,10 @@ class SqlParserSuite extends RewriteSuite { ).show() val sql1 = "SELECT * FROM column_type" comparePlansAndRows(sql1, "default", "mv_create1", noData = false) - + RewriteHelper.enableCachePlugin() spark.sql( """ - |ALTER MATERIALIZED VIEW default.mv_create1 ENABLE REWRITE + |ALTER MATERIALIZED VIEW default.mv_create1 DISABLE REWRITE |; |""".stripMargin ).show() diff --git a/omnicache/omnicache-spark-extension/pom.xml b/omnicache/omnicache-spark-extension/pom.xml index 1e068b0af..287123740 100644 --- a/omnicache/omnicache-spark-extension/pom.xml +++ b/omnicache/omnicache-spark-extension/pom.xml @@ -12,6 +12,7 @@ plugin + log-parser BoostKit Spark MaterializedView Sql Engine Extension Parent Pom @@ -34,6 +35,7 @@ 3.1.2 1.4.11 8.29 + -- Gitee