Skip to content

Commit

Permalink
Invalidating the cached query plans if set command contains aqp prope…
Browse files Browse the repository at this point in the history
…rties (#1307)
  • Loading branch information
ahshahid authored May 9, 2019
1 parent 97d2007 commit e0f3f58
Show file tree
Hide file tree
Showing 2 changed files with 33 additions and 1 deletion.
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,7 @@ import org.apache.spark.sql.streaming.StreamPlanProvider
import org.apache.spark.sql.types._
import org.apache.spark.sql.{SnappyParserConsts => Consts}
import org.apache.spark.streaming._
import org.apache.spark.sql.execution.InvalidateCachedPlans

abstract class SnappyDDLParser(session: SparkSession)
extends SnappyBaseParser(session) {
Expand Down Expand Up @@ -702,7 +703,11 @@ abstract class SnappyDDLParser(session: SparkSession)
if (separatorIndex >= 0) {
val key = rest.substring(0, separatorIndex).trim
val value = rest.substring(separatorIndex + 1).trim
SetCommand(Some(key -> Option(value)))
if (key.startsWith("spark.sql.aqp.")) {
new SetCommand(Some(key -> Option(value))) with InvalidateCachedPlans
} else {
SetCommand(Some(key -> Option(value)))
}
} else if (rest.nonEmpty) {
SetCommand(Some(rest.trim -> None))
} else {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,27 @@
/*
* Copyright (c) 2018 SnappyData, Inc. All rights reserved.
*
* Licensed 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. See accompanying
* LICENSE file.
*/
package org.apache.spark.sql.execution

import org.apache.spark.sql.{Row, SnappySession, SparkSession}
import org.apache.spark.sql.execution.command.RunnableCommand

trait InvalidateCachedPlans extends RunnableCommand{
abstract override def run(sparkSession: SparkSession): Seq[Row] = {
sparkSession.asInstanceOf[SnappySession].clearPlanCache()
super.run(sparkSession)
}
}

0 comments on commit e0f3f58

Please sign in to comment.