This is an automated email from the ASF dual-hosted git repository. wenchen pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git
The following commit(s) were added to refs/heads/master by this push: new b8dd84b [SPARK-27011][SQL] reset command fails with cache b8dd84b is described below commit b8dd84b9e4690efa9773f73787e19a4d60257e5e Author: Ajith <ajith2...@gmail.com> AuthorDate: Tue Mar 12 11:02:09 2019 +0800 [SPARK-27011][SQL] reset command fails with cache ## What changes were proposed in this pull request? When cache is enabled ( i.e once cache table command is executed), any following sql will trigger CacheManager#lookupCachedData which will create a copy of the tree node, which inturn calls TreeNode#makeCopy. Here the problem is it will try to create a copy instance. But as ResetCommand is a case object this will fail ## How was this patch tested? Added UT to reproduce the issue Closes #23918 from ajithme/reset. Authored-by: Ajith <ajith2...@gmail.com> Signed-off-by: Wenchen Fan <wenc...@databricks.com> --- .../catalyst/plans/logical/IgnoreCachedData.scala | 23 ++++++++++++++++++++++ .../apache/spark/sql/execution/CacheManager.scala | 3 ++- .../spark/sql/execution/command/SetCommand.scala | 3 ++- .../scala/org/apache/spark/sql/SQLQuerySuite.scala | 10 ++++++++++ 4 files changed, 37 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/IgnoreCachedData.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/IgnoreCachedData.scala new file mode 100644 index 0000000..85958cb --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/IgnoreCachedData.scala @@ -0,0 +1,23 @@ +/* + * 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.plans.logical + +/** + * A [[LogicalPlan]] operator that does not use the cached results stored in CacheManager + */ +trait IgnoreCachedData extends LogicalPlan {} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala index f7a78ea..2815a28 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala @@ -27,7 +27,7 @@ import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.spark.internal.Logging import org.apache.spark.sql.{Dataset, SparkSession} import org.apache.spark.sql.catalyst.expressions.SubqueryExpression -import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, ResolvedHint} +import org.apache.spark.sql.catalyst.plans.logical.{IgnoreCachedData, LogicalPlan, ResolvedHint} import org.apache.spark.sql.execution.columnar.InMemoryRelation import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} import org.apache.spark.storage.StorageLevel @@ -239,6 +239,7 @@ class CacheManager extends Logging { /** Replaces segments of the given logical plan with cached versions where possible. */ def useCachedData(plan: LogicalPlan): LogicalPlan = { val newPlan = plan transformDown { + case command: IgnoreCachedData => command // Do not lookup the cache by hint node. Hint node is special, we should ignore it when // canonicalizing plans, so that plans which are same except hint can hit the same cache. // However, we also want to keep the hint info after cache lookup. Here we skip the hint diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala index 3c900be..ca25dc5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution.command import org.apache.spark.internal.Logging import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.logical.IgnoreCachedData import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION import org.apache.spark.sql.types.{StringType, StructField, StructType} @@ -161,7 +162,7 @@ object SetCommand { * reset; * }}} */ -case object ResetCommand extends RunnableCommand with Logging { +case object ResetCommand extends RunnableCommand with IgnoreCachedData with Logging { override def run(sparkSession: SparkSession): Seq[Row] = { sparkSession.sessionState.conf.clear() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index b8c4d73..3d374a1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -3003,6 +3003,16 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } } } + + test("reset command should not fail with cache") { + withTable("tbl") { + val provider = spark.sessionState.conf.defaultDataSourceName + sql(s"CREATE TABLE tbl(i INT, j STRING) USING $provider") + sql("reset") + sql("cache table tbl") + sql("reset") + } + } } case class Foo(bar: Option[String]) --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org