You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kyuubi.apache.org by ya...@apache.org on 2022/09/06 05:51:51 UTC

[incubator-kyuubi] branch master updated: [KYUUBI #3414] Tidy up plan only related modes and styles

This is an automated email from the ASF dual-hosted git repository.

yao pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-kyuubi.git


The following commit(s) were added to refs/heads/master by this push:
     new 331f1a3bd [KYUUBI #3414] Tidy up plan only related modes and styles
331f1a3bd is described below

commit 331f1a3bdf553a50330f5856091ecb50ac6604b6
Author: yikf <yi...@gmail.com>
AuthorDate: Tue Sep 6 13:51:42 2022 +0800

    [KYUUBI #3414] Tidy up plan only related modes and styles
    
    ### _Why are the changes needed?_
    
    Fix https://github.com/apache/incubator-kyuubi/issues/3414
    
    Currently, Kyuubi supports output of planOnly in different modes in different styles, As more and more modes and styles are supported, defining enumerations in kyuubiConf is a bit messy and not easy to extend, so this pr aims to tidy up these
    
    ### _How was this patch tested?_
    - [ ] Add some test cases that check the changes thoroughly including negative and positive cases if possible
    
    - [ ] Add screenshots for manual tests if appropriate
    
    - [ ] [Run test](https://kyuubi.apache.org/docs/latest/develop_tools/testing.html#running-tests) locally before make a pull request
    
    Closes #3415 from Yikf/tidy-up.
    
    Closes #3414
    
    e81142e5 [yikf] Tidy up plan only related modes and styles
    
    Authored-by: yikf <yi...@gmail.com>
    Signed-off-by: Kent Yao <ya...@apache.org>
---
 docs/deployment/settings.md                        |   4 +-
 .../flink/operation/FlinkSQLOperationManager.scala |  11 +-
 .../engine/flink/operation/PlanOnlyStatement.scala |  16 +--
 .../flink/operation/FlinkOperationSuite.scala      |   5 +-
 .../flink/operation/PlanOnlyOperationSuite.scala   |  17 ++-
 .../engine/spark/operation/PlanOnlyStatement.scala |  52 ++++-----
 .../spark/operation/SparkSQLOperationManager.scala |  15 +--
 .../org/apache/kyuubi/config/KyuubiConf.scala      |  54 +++------
 .../org/apache/kyuubi/operation/PlanOnlyMode.scala | 122 +++++++++++++++++++++
 .../apache/kyuubi/operation/SparkQueryTests.scala  |   2 +-
 .../kyuubi/operation/PlanOnlyOperationSuite.scala  |  49 ++++-----
 11 files changed, 219 insertions(+), 128 deletions(-)

diff --git a/docs/deployment/settings.md b/docs/deployment/settings.md
index 5a0b6ea23..744647013 100644
--- a/docs/deployment/settings.md
+++ b/docs/deployment/settings.md
@@ -406,8 +406,8 @@ kyuubi.operation.interrupt.on.cancel|true|When true, all running tasks will be i
 kyuubi.operation.language|SQL|Choose a programing language for the following inputs <ul><li>SQL: (Default) Run all following statements as SQL queries.</li> <li>SCALA: Run all following input a scala codes</li></ul>|string|1.5.0
 kyuubi.operation.log.dir.root|server_operation_logs|Root directory for query operation log at server-side.|string|1.4.0
 kyuubi.operation.plan.only.excludes|ResetCommand,SetCommand,SetNamespaceCommand,UseStatement,SetCatalogAndNamespace|Comma-separated list of query plan names, in the form of simple class names, i.e, for `set abc=xyz`, the value will be `SetCommand`. For those auxiliary plans, such as `switch databases`, `set properties`, or `create temporary view` e.t.c, which are used for setup evaluating environments for analyzing actual queries, we can use this config to exclude them and let them take  [...]
-kyuubi.operation.plan.only.mode|NONE|Configures the statement performed mode, The value can be 'parse', 'analyze', 'optimize', 'optimize_with_stats', 'physical', 'execution', or 'none', when it is 'none', indicate to the statement will be fully executed, otherwise only way without executing the query. different engines currently support different modes, the Spark engine supports all modes, and the Flink engine supports 'parse', 'physical', and 'execution', other engines do not support pl [...]
-kyuubi.operation.plan.only.output.style|PLAIN|Configures the planOnly output style, The value can be 'plain' and 'json', default value is 'plain', this configuration supports only the output styles of the Spark engine|string|1.7.0
+kyuubi.operation.plan.only.mode|none|Configures the statement performed mode, The value can be 'parse', 'analyze', 'optimize', 'optimize_with_stats', 'physical', 'execution', or 'none', when it is 'none', indicate to the statement will be fully executed, otherwise only way without executing the query. different engines currently support different modes, the Spark engine supports all modes, and the Flink engine supports 'parse', 'physical', and 'execution', other engines do not support pl [...]
+kyuubi.operation.plan.only.output.style|plain|Configures the planOnly output style, The value can be 'plain' and 'json', default value is 'plain', this configuration supports only the output styles of the Spark engine|string|1.7.0
 kyuubi.operation.progress.enabled|false|Whether to enable the operation progress. When true, the operation progress will be returned in `GetOperationStatus`.|boolean|1.6.0
 kyuubi.operation.query.timeout|&lt;undefined&gt;|Timeout for query executions at server-side, take affect with client-side timeout(`java.sql.Statement.setQueryTimeout`) together, a running query will be cancelled automatically if timeout. It's off by default, which means only client-side take fully control whether the query should timeout or not. If set, client-side timeout capped at this point. To cancel the queries right away without waiting task to finish, consider enabling kyuubi.ope [...]
 kyuubi.operation.result.max.rows|0|Max rows of Spark query results. Rows that exceeds the limit would be ignored. By setting this value to 0 to disable the max rows limit.|int|1.6.0
diff --git a/externals/kyuubi-flink-sql-engine/src/main/scala/org/apache/kyuubi/engine/flink/operation/FlinkSQLOperationManager.scala b/externals/kyuubi-flink-sql-engine/src/main/scala/org/apache/kyuubi/engine/flink/operation/FlinkSQLOperationManager.scala
index 20ad4b434..a42075abc 100644
--- a/externals/kyuubi-flink-sql-engine/src/main/scala/org/apache/kyuubi/engine/flink/operation/FlinkSQLOperationManager.scala
+++ b/externals/kyuubi-flink-sql-engine/src/main/scala/org/apache/kyuubi/engine/flink/operation/FlinkSQLOperationManager.scala
@@ -23,10 +23,9 @@ import scala.collection.JavaConverters._
 
 import org.apache.kyuubi.KyuubiSQLException
 import org.apache.kyuubi.config.KyuubiConf._
-import org.apache.kyuubi.config.KyuubiConf.OperationModes._
 import org.apache.kyuubi.engine.flink.result.Constants
 import org.apache.kyuubi.engine.flink.session.FlinkSessionImpl
-import org.apache.kyuubi.operation.{Operation, OperationManager}
+import org.apache.kyuubi.operation.{NoneMode, Operation, OperationManager, PlanOnlyMode}
 import org.apache.kyuubi.session.Session
 
 class FlinkSQLOperationManager extends OperationManager("FlinkSQLOperationManager") {
@@ -53,16 +52,18 @@ class FlinkSQLOperationManager extends OperationManager("FlinkSQLOperationManage
         return catalogDatabaseOperation
       }
     }
-    val mode = OperationModes(flinkSession.sessionContext.getConfigMap.getOrDefault(
+
+    val mode = PlanOnlyMode.fromString(flinkSession.sessionContext.getConfigMap.getOrDefault(
       OPERATION_PLAN_ONLY_MODE.key,
       operationModeDefault))
-    flinkSession.sessionContext.set(OPERATION_PLAN_ONLY_MODE.key, mode.toString)
+
+    flinkSession.sessionContext.set(OPERATION_PLAN_ONLY_MODE.key, mode.name)
     val resultMaxRows =
       flinkSession.normalizedConf.getOrElse(
         ENGINE_FLINK_MAX_ROWS.key,
         resultMaxRowsDefault.toString).toInt
     val op = mode match {
-      case NONE =>
+      case NoneMode =>
         // FLINK-24427 seals calcite classes which required to access in async mode, considering
         // there is no much benefit in async mode, here we just ignore `runAsync` and always run
         // statement in sync mode as a workaround
diff --git a/externals/kyuubi-flink-sql-engine/src/main/scala/org/apache/kyuubi/engine/flink/operation/PlanOnlyStatement.scala b/externals/kyuubi-flink-sql-engine/src/main/scala/org/apache/kyuubi/engine/flink/operation/PlanOnlyStatement.scala
index 73cd49f47..5a7b8c10f 100644
--- a/externals/kyuubi-flink-sql-engine/src/main/scala/org/apache/kyuubi/engine/flink/operation/PlanOnlyStatement.scala
+++ b/externals/kyuubi-flink-sql-engine/src/main/scala/org/apache/kyuubi/engine/flink/operation/PlanOnlyStatement.scala
@@ -20,9 +20,9 @@ package org.apache.kyuubi.engine.flink.operation
 import org.apache.flink.table.api.TableEnvironment
 import org.apache.flink.table.operations.command._
 
-import org.apache.kyuubi.KyuubiSQLException
-import org.apache.kyuubi.config.KyuubiConf.OperationModes._
 import org.apache.kyuubi.engine.flink.result.ResultSetUtil
+import org.apache.kyuubi.operation.{ExecutionMode, ParseMode, PhysicalMode, PlanOnlyMode, UnknownMode}
+import org.apache.kyuubi.operation.PlanOnlyMode.{notSupportedModeError, unknownModeError}
 import org.apache.kyuubi.operation.log.OperationLog
 import org.apache.kyuubi.session.Session
 
@@ -32,7 +32,7 @@ import org.apache.kyuubi.session.Session
 class PlanOnlyStatement(
     session: Session,
     override val statement: String,
-    mode: OperationMode) extends FlinkOperation(session) {
+    mode: PlanOnlyMode) extends FlinkOperation(session) {
 
   private val operationLog: OperationLog = OperationLog.createOperationLog(session, getHandle)
   private val lineSeparator: String = System.lineSeparator()
@@ -64,13 +64,13 @@ class PlanOnlyStatement(
     val explainPlans =
       tableEnv.explainSql(statement).split(s"$lineSeparator$lineSeparator")
     val operationPlan = mode match {
-      case PARSE => explainPlans(0).split(s"== Abstract Syntax Tree ==$lineSeparator")(1)
-      case PHYSICAL =>
+      case ParseMode => explainPlans(0).split(s"== Abstract Syntax Tree ==$lineSeparator")(1)
+      case PhysicalMode =>
         explainPlans(1).split(s"== Optimized Physical Plan ==$lineSeparator")(1)
-      case EXECUTION =>
+      case ExecutionMode =>
         explainPlans(2).split(s"== Optimized Execution Plan ==$lineSeparator")(1)
-      case _ =>
-        throw KyuubiSQLException(s"The operation mode $mode doesn't support in Flink SQL engine.")
+      case UnknownMode => throw unknownModeError(mode)
+      case _ => throw notSupportedModeError(mode, "Flink SQL")
     }
     resultSet =
       ResultSetUtil.stringListToResultSet(
diff --git a/externals/kyuubi-flink-sql-engine/src/test/scala/org/apache/kyuubi/engine/flink/operation/FlinkOperationSuite.scala b/externals/kyuubi-flink-sql-engine/src/test/scala/org/apache/kyuubi/engine/flink/operation/FlinkOperationSuite.scala
index c402d5cd5..2c2485b94 100644
--- a/externals/kyuubi-flink-sql-engine/src/test/scala/org/apache/kyuubi/engine/flink/operation/FlinkOperationSuite.scala
+++ b/externals/kyuubi-flink-sql-engine/src/test/scala/org/apache/kyuubi/engine/flink/operation/FlinkOperationSuite.scala
@@ -29,18 +29,17 @@ import org.scalatest.concurrent.PatienceConfiguration.Timeout
 import org.scalatest.time.SpanSugar._
 
 import org.apache.kyuubi.config.KyuubiConf._
-import org.apache.kyuubi.config.KyuubiConf.OperationModes.NONE
 import org.apache.kyuubi.engine.flink.FlinkEngineUtils._
 import org.apache.kyuubi.engine.flink.WithFlinkSQLEngine
 import org.apache.kyuubi.engine.flink.result.Constants
 import org.apache.kyuubi.engine.flink.util.TestUserClassLoaderJar
-import org.apache.kyuubi.operation.HiveJDBCTestHelper
+import org.apache.kyuubi.operation.{HiveJDBCTestHelper, NoneMode}
 import org.apache.kyuubi.operation.meta.ResultSetSchemaConstant._
 import org.apache.kyuubi.service.ServiceState._
 
 class FlinkOperationSuite extends WithFlinkSQLEngine with HiveJDBCTestHelper {
   override def withKyuubiConf: Map[String, String] =
-    Map(OPERATION_PLAN_ONLY_MODE.key -> NONE.toString)
+    Map(OPERATION_PLAN_ONLY_MODE.key -> NoneMode.name)
 
   override protected def jdbcUrl: String =
     s"jdbc:hive2://${engine.frontendServices.head.connectionUrl}/;"
diff --git a/externals/kyuubi-flink-sql-engine/src/test/scala/org/apache/kyuubi/engine/flink/operation/PlanOnlyOperationSuite.scala b/externals/kyuubi-flink-sql-engine/src/test/scala/org/apache/kyuubi/engine/flink/operation/PlanOnlyOperationSuite.scala
index 6a4d90de3..1194f3582 100644
--- a/externals/kyuubi-flink-sql-engine/src/test/scala/org/apache/kyuubi/engine/flink/operation/PlanOnlyOperationSuite.scala
+++ b/externals/kyuubi-flink-sql-engine/src/test/scala/org/apache/kyuubi/engine/flink/operation/PlanOnlyOperationSuite.scala
@@ -20,16 +20,15 @@ package org.apache.kyuubi.engine.flink.operation
 import java.sql.Statement
 
 import org.apache.kyuubi.config.KyuubiConf
-import org.apache.kyuubi.config.KyuubiConf.OperationModes._
 import org.apache.kyuubi.engine.flink.WithFlinkSQLEngine
-import org.apache.kyuubi.operation.HiveJDBCTestHelper
+import org.apache.kyuubi.operation.{AnalyzeMode, ExecutionMode, HiveJDBCTestHelper, ParseMode, PhysicalMode}
 
 class PlanOnlyOperationSuite extends WithFlinkSQLEngine with HiveJDBCTestHelper {
 
   override def withKyuubiConf: Map[String, String] =
     Map(
       KyuubiConf.ENGINE_SHARE_LEVEL.key -> "user",
-      KyuubiConf.OPERATION_PLAN_ONLY_MODE.key -> PARSE.toString,
+      KyuubiConf.OPERATION_PLAN_ONLY_MODE.key -> ParseMode.name,
       KyuubiConf.ENGINE_SHARE_LEVEL_SUBDOMAIN.key -> "plan-only")
 
   override protected def jdbcUrl: String =
@@ -42,26 +41,26 @@ class PlanOnlyOperationSuite extends WithFlinkSQLEngine with HiveJDBCTestHelper
   }
 
   test("Plan only operation with session conf") {
-    withSessionConf()(Map(KyuubiConf.OPERATION_PLAN_ONLY_MODE.key -> ANALYZE.toString))(Map.empty) {
+    withSessionConf()(Map(KyuubiConf.OPERATION_PLAN_ONLY_MODE.key -> AnalyzeMode.name))(Map.empty) {
       withJdbcStatement() { statement =>
         val exceptionMsg = intercept[Exception](statement.executeQuery("select 1")).getMessage
         assert(exceptionMsg.contains(
-          s"The operation mode $ANALYZE doesn't support in Flink SQL engine."))
+          s"The operation mode ${AnalyzeMode.name} doesn't support in Flink SQL engine."))
       }
     }
   }
 
   test("Plan only operation with set command") {
-    withSessionConf()(Map(KyuubiConf.OPERATION_PLAN_ONLY_MODE.key -> ANALYZE.toString))(Map.empty) {
+    withSessionConf()(Map(KyuubiConf.OPERATION_PLAN_ONLY_MODE.key -> AnalyzeMode.name))(Map.empty) {
       withJdbcStatement() { statement =>
-        statement.execute(s"set ${KyuubiConf.OPERATION_PLAN_ONLY_MODE.key}=$PARSE")
+        statement.execute(s"set ${KyuubiConf.OPERATION_PLAN_ONLY_MODE.key}=${ParseMode.name}")
         testPlanOnlyStatementWithParseMode(statement)
       }
     }
   }
 
   test("Plan only operation with PHYSICAL mode") {
-    withSessionConf()(Map(KyuubiConf.OPERATION_PLAN_ONLY_MODE.key -> PHYSICAL.toString))(
+    withSessionConf()(Map(KyuubiConf.OPERATION_PLAN_ONLY_MODE.key -> PhysicalMode.name))(
       Map.empty) {
       withJdbcStatement() { statement =>
         val operationPlan = getOperationPlanWithStatement(statement)
@@ -72,7 +71,7 @@ class PlanOnlyOperationSuite extends WithFlinkSQLEngine with HiveJDBCTestHelper
   }
 
   test("Plan only operation with EXECUTION mode") {
-    withSessionConf()(Map(KyuubiConf.OPERATION_PLAN_ONLY_MODE.key -> EXECUTION.toString))(
+    withSessionConf()(Map(KyuubiConf.OPERATION_PLAN_ONLY_MODE.key -> ExecutionMode.name))(
       Map.empty) {
       withJdbcStatement() { statement =>
         val operationPlan = getOperationPlanWithStatement(statement)
diff --git a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/operation/PlanOnlyStatement.scala b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/operation/PlanOnlyStatement.scala
index 24151e7c0..486e6bb63 100644
--- a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/operation/PlanOnlyStatement.scala
+++ b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/operation/PlanOnlyStatement.scala
@@ -23,10 +23,10 @@ import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.types.StructType
 
 import org.apache.kyuubi.KyuubiSQLException
-import org.apache.kyuubi.config.KyuubiConf.{OPERATION_PLAN_ONLY_EXCLUDES, OPERATION_PLAN_ONLY_OUT_STYLE, PlanOnlyStyles}
-import org.apache.kyuubi.config.KyuubiConf.OperationModes._
-import org.apache.kyuubi.config.KyuubiConf.PlanOnlyStyles.{JSON, PLAIN}
-import org.apache.kyuubi.operation.{ArrayFetchIterator, IterableFetchIterator}
+import org.apache.kyuubi.config.KyuubiConf.{OPERATION_PLAN_ONLY_EXCLUDES, OPERATION_PLAN_ONLY_OUT_STYLE}
+import org.apache.kyuubi.operation.{AnalyzeMode, ArrayFetchIterator, ExecutionMode, IterableFetchIterator, JsonStyle, OptimizeMode, OptimizeWithStatsMode, ParseMode, PhysicalMode, PlainStyle, PlanOnlyMode, PlanOnlyStyle, UnknownMode, UnknownStyle}
+import org.apache.kyuubi.operation.PlanOnlyMode.{notSupportedModeError, unknownModeError}
+import org.apache.kyuubi.operation.PlanOnlyStyle.{notSupportedStyleError, unknownStyleError}
 import org.apache.kyuubi.operation.log.OperationLog
 import org.apache.kyuubi.session.Session
 
@@ -36,7 +36,7 @@ import org.apache.kyuubi.session.Session
 class PlanOnlyStatement(
     session: Session,
     override val statement: String,
-    mode: OperationMode)
+    mode: PlanOnlyMode)
   extends SparkOperation(session) {
 
   private val operationLog: OperationLog = OperationLog.createOperationLog(session, getHandle)
@@ -45,10 +45,10 @@ class PlanOnlyStatement(
       .getOrElse(session.sessionManager.getConf.get(OPERATION_PLAN_ONLY_EXCLUDES))
   }
 
-  private val style = PlanOnlyStyles(spark.conf.get(
+  private val style = PlanOnlyStyle.fromString(spark.conf.get(
     OPERATION_PLAN_ONLY_OUT_STYLE.key,
     session.sessionManager.getConf.get(OPERATION_PLAN_ONLY_OUT_STYLE)))
-  spark.conf.set(OPERATION_PLAN_ONLY_OUT_STYLE.key, style.toString)
+  spark.conf.set(OPERATION_PLAN_ONLY_OUT_STYLE.key, style.name)
 
   override def getOperationLog: Option[OperationLog] = Option(operationLog)
 
@@ -64,16 +64,17 @@ class PlanOnlyStatement(
     try {
       SQLConf.withExistingConf(spark.sessionState.conf) {
         val parsed = spark.sessionState.sqlParser.parsePlan(statement)
+
         parsed match {
           case cmd if planExcludes.contains(cmd.getClass.getSimpleName) =>
             result = spark.sql(statement)
             iter = new ArrayFetchIterator(result.collect())
 
           case plan => style match {
-              case PLAIN => explainWithPlainStyle(plan)
-              case JSON => explainWithJsonStyle(plan)
-              case _ => throw KyuubiSQLException(s"The plan only style $style" +
-                  " doesn't support in Spark SQL engine.")
+              case PlainStyle => explainWithPlainStyle(plan)
+              case JsonStyle => explainWithJsonStyle(plan)
+              case UnknownStyle => unknownStyleError(style)
+              case other => throw notSupportedStyleError(other, "Spark SQL")
             }
         }
       }
@@ -84,18 +85,18 @@ class PlanOnlyStatement(
 
   private def explainWithPlainStyle(plan: LogicalPlan): Unit = {
     mode match {
-      case PARSE =>
+      case ParseMode =>
         iter = new IterableFetchIterator(Seq(Row(plan.toString())))
-      case ANALYZE =>
+      case AnalyzeMode =>
         val analyzed = spark.sessionState.analyzer.execute(plan)
         spark.sessionState.analyzer.checkAnalysis(analyzed)
         iter = new IterableFetchIterator(Seq(Row(analyzed.toString())))
-      case OPTIMIZE =>
+      case OptimizeMode =>
         val analyzed = spark.sessionState.analyzer.execute(plan)
         spark.sessionState.analyzer.checkAnalysis(analyzed)
         val optimized = spark.sessionState.optimizer.execute(analyzed)
         iter = new IterableFetchIterator(Seq(Row(optimized.toString())))
-      case OPTIMIZE_WITH_STATS =>
+      case OptimizeWithStatsMode =>
         val analyzed = spark.sessionState.analyzer.execute(plan)
         spark.sessionState.analyzer.checkAnalysis(analyzed)
         val optimized = spark.sessionState.optimizer.execute(analyzed)
@@ -105,41 +106,40 @@ class PlanOnlyStatement(
           addSuffix = true,
           SQLConf.get.maxToStringFields,
           printOperatorId = false))))
-      case PHYSICAL =>
+      case PhysicalMode =>
         val physical = spark.sql(statement).queryExecution.sparkPlan
         iter = new IterableFetchIterator(Seq(Row(physical.toString())))
-      case EXECUTION =>
+      case ExecutionMode =>
         val executed = spark.sql(statement).queryExecution.executedPlan
         iter = new IterableFetchIterator(Seq(Row(executed.toString())))
-      case _ =>
-        throw KyuubiSQLException(s"The operation mode $mode" +
-          " doesn't support in Spark SQL engine.")
+      case UnknownMode => throw unknownModeError(mode)
+      case _ => throw notSupportedModeError(mode, "Spark SQL")
     }
   }
 
   private def explainWithJsonStyle(plan: LogicalPlan): Unit = {
     mode match {
-      case PARSE =>
+      case ParseMode =>
         iter = new IterableFetchIterator(Seq(Row(plan.toJSON)))
-      case ANALYZE =>
+      case AnalyzeMode =>
         val analyzed = spark.sessionState.analyzer.execute(plan)
         spark.sessionState.analyzer.checkAnalysis(analyzed)
         iter = new IterableFetchIterator(Seq(Row(analyzed.toJSON)))
-      case OPTIMIZE | OPTIMIZE_WITH_STATS =>
+      case OptimizeMode | OptimizeWithStatsMode =>
         val analyzed = spark.sessionState.analyzer.execute(plan)
         spark.sessionState.analyzer.checkAnalysis(analyzed)
         val optimized = spark.sessionState.optimizer.execute(analyzed)
         iter = new IterableFetchIterator(Seq(Row(optimized.toJSON)))
-      case PHYSICAL =>
+      case PhysicalMode =>
         val physical = spark.sql(statement).queryExecution.sparkPlan
         iter = new IterableFetchIterator(Seq(Row(physical.toJSON)))
-      case EXECUTION =>
+      case ExecutionMode =>
         val executed = spark.sql(statement).queryExecution.executedPlan
         iter = new IterableFetchIterator(Seq(Row(executed.toJSON)))
+      case UnknownMode => throw unknownModeError(mode)
       case _ =>
         throw KyuubiSQLException(s"The operation mode $mode" +
           " doesn't support in Spark SQL engine.")
     }
   }
-
 }
diff --git a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/operation/SparkSQLOperationManager.scala b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/operation/SparkSQLOperationManager.scala
index 13b8bbc51..7a3f25eaa 100644
--- a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/operation/SparkSQLOperationManager.scala
+++ b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/operation/SparkSQLOperationManager.scala
@@ -23,18 +23,17 @@ import scala.collection.JavaConverters._
 
 import org.apache.kyuubi.KyuubiSQLException
 import org.apache.kyuubi.config.KyuubiConf._
-import org.apache.kyuubi.config.KyuubiConf.OperationModes._
 import org.apache.kyuubi.engine.spark.repl.KyuubiSparkILoop
 import org.apache.kyuubi.engine.spark.session.SparkSessionImpl
 import org.apache.kyuubi.engine.spark.shim.SparkCatalogShim
-import org.apache.kyuubi.operation.{Operation, OperationManager}
+import org.apache.kyuubi.operation.{NoneMode, Operation, OperationManager, PlanOnlyMode}
 import org.apache.kyuubi.session.{Session, SessionHandle}
 
 class SparkSQLOperationManager private (name: String) extends OperationManager(name) {
 
   def this() = this(classOf[SparkSQLOperationManager].getSimpleName)
 
-  private lazy val operationModeDefault = getConf.get(OPERATION_PLAN_ONLY_MODE)
+  private lazy val planOnlyModeDefault = getConf.get(OPERATION_PLAN_ONLY_MODE)
   private lazy val operationIncrementalCollectDefault = getConf.get(OPERATION_INCREMENTAL_COLLECT)
   private lazy val operationLanguageDefault = getConf.get(OPERATION_LANGUAGE)
   private lazy val operationConvertCatalogDatabaseDefault =
@@ -67,11 +66,13 @@ class SparkSQLOperationManager private (name: String) extends OperationManager(n
     val operation =
       lang match {
         case OperationLanguages.SQL =>
-          val mode =
-            OperationModes(spark.conf.get(OPERATION_PLAN_ONLY_MODE.key, operationModeDefault))
-          spark.conf.set(OPERATION_PLAN_ONLY_MODE.key, mode.toString)
+          val mode = PlanOnlyMode.fromString(spark.conf.get(
+            OPERATION_PLAN_ONLY_MODE.key,
+            planOnlyModeDefault))
+
+          spark.conf.set(OPERATION_PLAN_ONLY_MODE.key, mode.name)
           mode match {
-            case NONE =>
+            case NoneMode =>
               val incrementalCollect = spark.conf.getOption(OPERATION_INCREMENTAL_COLLECT.key)
                 .map(_.toBoolean).getOrElse(operationIncrementalCollectDefault)
               new ExecuteStatement(session, statement, runAsync, queryTimeout, incrementalCollect)
diff --git a/kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiConf.scala b/kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiConf.scala
index 5628dfef2..7ad4d1856 100644
--- a/kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiConf.scala
+++ b/kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiConf.scala
@@ -29,6 +29,7 @@ import scala.util.matching.Regex
 import org.apache.kyuubi.{Logging, Utils}
 import org.apache.kyuubi.config.KyuubiConf._
 import org.apache.kyuubi.engine.{EngineType, ShareLevel}
+import org.apache.kyuubi.operation.{NoneMode, PlainStyle}
 import org.apache.kyuubi.service.authentication.{AuthTypes, SaslQOP}
 
 case class KyuubiConf(loadSysDefault: Boolean = true) extends Logging {
@@ -1637,26 +1638,6 @@ object KyuubiConf {
       .stringConf
       .createOptional
 
-  object OperationModes extends Enumeration with Logging {
-    type OperationMode = Value
-    val PARSE, ANALYZE, OPTIMIZE, OPTIMIZE_WITH_STATS, PHYSICAL, EXECUTION, NONE, UNKNOWN = Value
-
-    def apply(mode: String): OperationMode = {
-      mode.toUpperCase(Locale.ROOT) match {
-        case "PARSE" => PARSE
-        case "ANALYZE" => ANALYZE
-        case "OPTIMIZE" => OPTIMIZE
-        case "OPTIMIZE_WITH_STATS" => OPTIMIZE_WITH_STATS
-        case "PHYSICAL" => PHYSICAL
-        case "EXECUTION" => EXECUTION
-        case "NONE" => NONE
-        case other =>
-          warn(s"Unsupported operation mode: $mode, using UNKNOWN instead")
-          UNKNOWN
-      }
-    }
-  }
-
   val OPERATION_PLAN_ONLY_MODE: ConfigEntry[String] =
     buildConf("kyuubi.operation.plan.only.mode")
       .doc("Configures the statement performed mode, The value can be 'parse', 'analyze', " +
@@ -1669,26 +1650,19 @@ object KyuubiConf {
       .stringConf
       .transform(_.toUpperCase(Locale.ROOT))
       .checkValue(
-        mode => OperationModes.values.map(_.toString).contains(mode),
+        mode =>
+          Set(
+            "PARSE",
+            "ANALYZE",
+            "OPTIMIZE",
+            "OPTIMIZE_WITH_STATS",
+            "PHYSICAL",
+            "EXECUTION",
+            "NONE").contains(mode),
         "Invalid value for 'kyuubi.operation.plan.only.mode'. Valid values are" +
-          "'parse', 'analyze', 'optimize', 'optimize_with_stats', 'physical', 'execution' " +
-          "and 'none'.")
-      .createWithDefault(OperationModes.NONE.toString)
-
-  object PlanOnlyStyles extends Enumeration with Logging {
-    type PlanOnlyStyle = Value
-    val PLAIN, JSON, UNKNOWN = Value
-
-    def apply(mode: String): PlanOnlyStyle = {
-      mode.toUpperCase(Locale.ROOT) match {
-        case "PLAIN" => PLAIN
-        case "JSON" => JSON
-        case other =>
-          warn(s"Unsupported plan only style: $mode, using UNKNOWN instead")
-          UNKNOWN
-      }
-    }
-  }
+          "'parse', 'analyze', 'optimize', 'optimize_with_stats', 'physical', 'execution' and " +
+          "'none'.")
+      .createWithDefault(NoneMode.name)
 
   val OPERATION_PLAN_ONLY_OUT_STYLE: ConfigEntry[String] =
     buildConf("kyuubi.operation.plan.only.output.style")
@@ -1701,7 +1675,7 @@ object KyuubiConf {
         mode => Set("PLAIN", "JSON").contains(mode),
         "Invalid value for 'kyuubi.operation.plan.only.output.style'. Valid values are " +
           "'plain', 'json'.")
-      .createWithDefault(PlanOnlyStyles.PLAIN.toString)
+      .createWithDefault(PlainStyle.name)
 
   val OPERATION_PLAN_ONLY_EXCLUDES: ConfigEntry[Seq[String]] =
     buildConf("kyuubi.operation.plan.only.excludes")
diff --git a/kyuubi-common/src/main/scala/org/apache/kyuubi/operation/PlanOnlyMode.scala b/kyuubi-common/src/main/scala/org/apache/kyuubi/operation/PlanOnlyMode.scala
new file mode 100644
index 000000000..3e170f05f
--- /dev/null
+++ b/kyuubi-common/src/main/scala/org/apache/kyuubi/operation/PlanOnlyMode.scala
@@ -0,0 +1,122 @@
+/*
+ * 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.kyuubi.operation
+
+import java.util.Locale
+
+import org.apache.kyuubi.KyuubiException
+
+sealed trait PlanOnlyMode {
+
+  /**
+   * String name of the plan only mode.
+   */
+  def name: String
+}
+
+case object ParseMode extends PlanOnlyMode { val name = "parse" }
+
+case object AnalyzeMode extends PlanOnlyMode { val name = "analyze" }
+
+case object OptimizeMode extends PlanOnlyMode { val name = "optimize" }
+
+case object OptimizeWithStatsMode extends PlanOnlyMode { val name = "optimize_with_stats" }
+
+case object PhysicalMode extends PlanOnlyMode { val name = "physical" }
+
+case object ExecutionMode extends PlanOnlyMode { val name = "execution" }
+
+case object NoneMode extends PlanOnlyMode { val name = "none" }
+
+case object UnknownMode extends PlanOnlyMode {
+  var name = "unknown"
+
+  def mode(mode: String): UnknownMode.type = {
+    name = mode
+    this
+  }
+}
+
+object PlanOnlyMode {
+
+  /**
+   * Returns the plan only mode from the given string.
+   */
+  def fromString(mode: String): PlanOnlyMode = mode.toLowerCase(Locale.ROOT) match {
+    case ParseMode.name => ParseMode
+    case AnalyzeMode.name => AnalyzeMode
+    case OptimizeMode.name => OptimizeMode
+    case OptimizeWithStatsMode.name => OptimizeWithStatsMode
+    case PhysicalMode.name => PhysicalMode
+    case ExecutionMode.name => ExecutionMode
+    case NoneMode.name => NoneMode
+    case other => UnknownMode.mode(other)
+  }
+
+  def unknownModeError(mode: PlanOnlyMode): KyuubiException = {
+    new KyuubiException(s"Unknown planOnly mode: ${mode.name}. Accepted " +
+      "planOnly modes are 'parse', 'analyze', 'optimize', 'optimize_with_stats', 'physical', " +
+      "execution, none.")
+  }
+
+  def notSupportedModeError(mode: PlanOnlyMode, engine: String): KyuubiException = {
+    new KyuubiException(s"The operation mode ${mode.name} doesn't support in $engine engine.")
+  }
+}
+
+sealed trait PlanOnlyStyle {
+
+  /**
+   * String name of the plan only style.
+   */
+  def name: String
+}
+
+case object PlainStyle extends PlanOnlyStyle { val name = "plain" }
+
+case object JsonStyle extends PlanOnlyStyle { val name = "json" }
+
+case object UnknownStyle extends PlanOnlyStyle {
+  var name = "unknown"
+
+  def style(style: String): UnknownStyle.type = {
+    name = style
+    this
+  }
+}
+
+object PlanOnlyStyle {
+
+  /**
+   * Returns the plan only style from the given string.
+   */
+  def fromString(style: String): PlanOnlyStyle = style.toLowerCase(Locale.ROOT) match {
+    case PlainStyle.name => PlainStyle
+    case JsonStyle.name => JsonStyle
+    case other => UnknownStyle.style(other)
+  }
+
+  def unknownStyleError(style: PlanOnlyStyle): KyuubiException = {
+    new KyuubiException(s"Unknown planOnly style: ${style.name}. Accepted " +
+      "planOnly styles are 'plain', 'json'.")
+  }
+
+  def notSupportedStyleError(style: PlanOnlyStyle, engine: String): KyuubiException = {
+    new KyuubiException(s"The plan only style ${style.name} doesn't support in $engine engine.")
+  }
+}
diff --git a/kyuubi-common/src/test/scala/org/apache/kyuubi/operation/SparkQueryTests.scala b/kyuubi-common/src/test/scala/org/apache/kyuubi/operation/SparkQueryTests.scala
index 8c9f120c7..f0407cb52 100644
--- a/kyuubi-common/src/test/scala/org/apache/kyuubi/operation/SparkQueryTests.scala
+++ b/kyuubi-common/src/test/scala/org/apache/kyuubi/operation/SparkQueryTests.scala
@@ -480,7 +480,7 @@ trait SparkQueryTests extends HiveJDBCTestHelper {
         assert(set.getString("plan") startsWith "Create")
         val set0 = statement.executeQuery(setkey)
         assert(set0.next())
-        assert(set0.getString(2) === "OPTIMIZE")
+        assert(set0.getString(2) === "optimize")
         val e1 = intercept[SQLException](statement.executeQuery(dql))
         assert(e1.getMessage.contains("Table or view not found"))
         statement.execute("SET kyuubi.operation.plan.only.mode=analyze")
diff --git a/kyuubi-server/src/test/scala/org/apache/kyuubi/operation/PlanOnlyOperationSuite.scala b/kyuubi-server/src/test/scala/org/apache/kyuubi/operation/PlanOnlyOperationSuite.scala
index 07055efc8..6a37e823d 100644
--- a/kyuubi-server/src/test/scala/org/apache/kyuubi/operation/PlanOnlyOperationSuite.scala
+++ b/kyuubi-server/src/test/scala/org/apache/kyuubi/operation/PlanOnlyOperationSuite.scala
@@ -17,19 +17,18 @@
 
 package org.apache.kyuubi.operation
 
-import java.sql.{SQLException, Statement}
+import java.sql.Statement
 
 import org.apache.kyuubi.WithKyuubiServer
 import org.apache.kyuubi.config.KyuubiConf
-import org.apache.kyuubi.config.KyuubiConf.OperationModes._
-import org.apache.kyuubi.config.KyuubiConf.PlanOnlyStyles.{JSON, PLAIN}
+import org.apache.kyuubi.jdbc.hive.KyuubiSQLException
 
 class PlanOnlyOperationSuite extends WithKyuubiServer with HiveJDBCTestHelper {
 
   override protected val conf: KyuubiConf = {
     KyuubiConf()
       .set(KyuubiConf.ENGINE_SHARE_LEVEL, "user")
-      .set(KyuubiConf.OPERATION_PLAN_ONLY_MODE, OPTIMIZE.toString)
+      .set(KyuubiConf.OPERATION_PLAN_ONLY_MODE, OptimizeMode.name)
       .set(KyuubiConf.ENGINE_SHARE_LEVEL_SUBDOMAIN.key, "plan-only")
   }
 
@@ -43,7 +42,7 @@ class PlanOnlyOperationSuite extends WithKyuubiServer with HiveJDBCTestHelper {
   }
 
   test("KYUUBI #1059: Plan only operation with session conf") {
-    withSessionConf()(Map(KyuubiConf.OPERATION_PLAN_ONLY_MODE.key -> ANALYZE.toString))(Map.empty) {
+    withSessionConf()(Map(KyuubiConf.OPERATION_PLAN_ONLY_MODE.key -> AnalyzeMode.name))(Map.empty) {
       withJdbcStatement() { statement =>
         val operationPlan = getOperationPlanWithStatement(statement)
         assert(operationPlan.startsWith("Project") && operationPlan.contains("Filter"))
@@ -52,9 +51,9 @@ class PlanOnlyOperationSuite extends WithKyuubiServer with HiveJDBCTestHelper {
   }
 
   test("KYUUBI #1059: Plan only operation with set command") {
-    withSessionConf()(Map(KyuubiConf.OPERATION_PLAN_ONLY_MODE.key -> ANALYZE.toString))(Map.empty) {
+    withSessionConf()(Map(KyuubiConf.OPERATION_PLAN_ONLY_MODE.key -> AnalyzeMode.name))(Map.empty) {
       withJdbcStatement() { statement =>
-        statement.execute(s"set ${KyuubiConf.OPERATION_PLAN_ONLY_MODE.key}=$PARSE")
+        statement.execute(s"set ${KyuubiConf.OPERATION_PLAN_ONLY_MODE.key}=${ParseMode.name}")
         val operationPlan = getOperationPlanWithStatement(statement)
         assert(operationPlan.startsWith("'Project"))
       }
@@ -62,7 +61,7 @@ class PlanOnlyOperationSuite extends WithKyuubiServer with HiveJDBCTestHelper {
   }
 
   test("KYUUBI #1919: Plan only operation with PHYSICAL mode") {
-    withSessionConf()(Map(KyuubiConf.OPERATION_PLAN_ONLY_MODE.key -> PHYSICAL.toString))(
+    withSessionConf()(Map(KyuubiConf.OPERATION_PLAN_ONLY_MODE.key -> PhysicalMode.name))(
       Map.empty) {
       withJdbcStatement() { statement =>
         val operationPlan = getOperationPlanWithStatement(statement)
@@ -72,7 +71,7 @@ class PlanOnlyOperationSuite extends WithKyuubiServer with HiveJDBCTestHelper {
   }
 
   test("KYUUBI #1919: Plan only operation with EXECUTION mode") {
-    withSessionConf()(Map(KyuubiConf.OPERATION_PLAN_ONLY_MODE.key -> EXECUTION.toString))(
+    withSessionConf()(Map(KyuubiConf.OPERATION_PLAN_ONLY_MODE.key -> ExecutionMode.name))(
       Map.empty) {
       withJdbcStatement() { statement =>
         val operationPlan = getOperationPlanWithStatement(statement)
@@ -83,10 +82,10 @@ class PlanOnlyOperationSuite extends WithKyuubiServer with HiveJDBCTestHelper {
   }
 
   test("KYUUBI #1920: Plan only operations with UseStatement or SetNamespaceCommand") {
-    withSessionConf()(Map(KyuubiConf.OPERATION_PLAN_ONLY_MODE.key -> NONE.toString))(Map.empty) {
+    withSessionConf()(Map(KyuubiConf.OPERATION_PLAN_ONLY_MODE.key -> NoneMode.name))(Map.empty) {
       withDatabases("test_database") { statement =>
         statement.execute("create database test_database")
-        statement.execute(s"set ${KyuubiConf.OPERATION_PLAN_ONLY_MODE.key}=$OPTIMIZE")
+        statement.execute(s"set ${KyuubiConf.OPERATION_PLAN_ONLY_MODE.key}=${OptimizeMode.name}")
         val result = statement.executeQuery("use test_database")
         assert(!result.next(), "In contrast to PlanOnly mode, it will returns an empty result")
       }
@@ -105,7 +104,7 @@ class PlanOnlyOperationSuite extends WithKyuubiServer with HiveJDBCTestHelper {
   }
 
   test("kyuubi #2565: Variable substitution should work in plan only mode") {
-    withSessionConf()(Map(KyuubiConf.OPERATION_PLAN_ONLY_MODE.key -> PARSE.toString))(Map.empty) {
+    withSessionConf()(Map(KyuubiConf.OPERATION_PLAN_ONLY_MODE.key -> ParseMode.name))(Map.empty) {
       withJdbcStatement() { statement =>
         statement.executeQuery("set x = y")
         val resultSet = statement.executeQuery("select '${x}'")
@@ -116,7 +115,7 @@ class PlanOnlyOperationSuite extends WithKyuubiServer with HiveJDBCTestHelper {
   }
 
   test("KYUUBI #3128: Support CostMode for PlanOnlyStatement") {
-    withSessionConf()(Map(KyuubiConf.OPERATION_PLAN_ONLY_MODE.key -> OPTIMIZE_WITH_STATS.toString))(
+    withSessionConf()(Map(KyuubiConf.OPERATION_PLAN_ONLY_MODE.key -> OptimizeWithStatsMode.name))(
       Map.empty) {
       withJdbcStatement() { statement =>
         val resultSet = statement.executeQuery(
@@ -131,9 +130,8 @@ class PlanOnlyOperationSuite extends WithKyuubiServer with HiveJDBCTestHelper {
 
   test("KYUUBI #3376 : Spark physical Plan outputs in plain style") {
     withSessionConf()(Map(
-      KyuubiConf.OPERATION_PLAN_ONLY_MODE.key -> PHYSICAL.toString,
-      KyuubiConf.OPERATION_PLAN_ONLY_OUT_STYLE.key -> PLAIN.toString))(
-      Map.empty) {
+      KyuubiConf.OPERATION_PLAN_ONLY_MODE.key -> PhysicalMode.name,
+      KyuubiConf.OPERATION_PLAN_ONLY_OUT_STYLE.key -> PlainStyle.name))() {
       withJdbcStatement() { statement =>
         val resultSet = statement.executeQuery(
           "SELECT * FROM VALUES(1),(2),(3) AS t(c1) DISTRIBUTE BY c1")
@@ -146,9 +144,8 @@ class PlanOnlyOperationSuite extends WithKyuubiServer with HiveJDBCTestHelper {
 
   test("KYUUBI #3376 : Spark physical Plan outputs in json style") {
     withSessionConf()(Map(
-      KyuubiConf.OPERATION_PLAN_ONLY_MODE.key -> PHYSICAL.toString,
-      KyuubiConf.OPERATION_PLAN_ONLY_OUT_STYLE.key -> JSON.toString))(
-      Map.empty) {
+      KyuubiConf.OPERATION_PLAN_ONLY_MODE.key -> PhysicalMode.name,
+      KyuubiConf.OPERATION_PLAN_ONLY_OUT_STYLE.key -> JsonStyle.name))() {
       withJdbcStatement() { statement =>
         val resultSet = statement.executeQuery(
           "SELECT * FROM VALUES(1),(2),(3) AS t(c1) DISTRIBUTE BY c1")
@@ -162,9 +159,8 @@ class PlanOnlyOperationSuite extends WithKyuubiServer with HiveJDBCTestHelper {
 
   test("KYUUBI #3376 : Spark optimized Plan outputs in json style") {
     withSessionConf()(Map(
-      KyuubiConf.OPERATION_PLAN_ONLY_MODE.key -> OPTIMIZE.toString,
-      KyuubiConf.OPERATION_PLAN_ONLY_OUT_STYLE.key -> JSON.toString))(
-      Map.empty) {
+      KyuubiConf.OPERATION_PLAN_ONLY_MODE.key -> OptimizeMode.name,
+      KyuubiConf.OPERATION_PLAN_ONLY_OUT_STYLE.key -> JsonStyle.name))() {
       withJdbcStatement() { statement =>
         val resultSet = statement.executeQuery(
           "SELECT * FROM VALUES(1),(2),(3) AS t(c1) DISTRIBUTE BY c1")
@@ -178,9 +174,8 @@ class PlanOnlyOperationSuite extends WithKyuubiServer with HiveJDBCTestHelper {
 
   test("KYUUBI #3376 : Spark parse Plan outputs in json style") {
     withSessionConf()(Map(
-      KyuubiConf.OPERATION_PLAN_ONLY_MODE.key -> PARSE.toString,
-      KyuubiConf.OPERATION_PLAN_ONLY_OUT_STYLE.key -> JSON.toString))(
-      Map.empty) {
+      KyuubiConf.OPERATION_PLAN_ONLY_MODE.key -> ParseMode.name,
+      KyuubiConf.OPERATION_PLAN_ONLY_OUT_STYLE.key -> JsonStyle.name))() {
       withJdbcStatement() { statement =>
         val resultSet = statement.executeQuery(
           "SELECT * FROM VALUES(1),(2),(3) AS t(c1) DISTRIBUTE BY c1")
@@ -196,8 +191,8 @@ class PlanOnlyOperationSuite extends WithKyuubiServer with HiveJDBCTestHelper {
     withSessionConf()(Map(KyuubiConf.OPERATION_PLAN_ONLY_MODE.key -> "parse"))(Map.empty) {
       withJdbcStatement() { statement =>
         statement.executeQuery(s"set ${KyuubiConf.OPERATION_PLAN_ONLY_MODE.key}=parser")
-        val e = intercept[SQLException](statement.executeQuery("select 1"))
-        assert(e.getMessage.contains("The operation mode UNKNOWN doesn't support"))
+        val e = intercept[KyuubiSQLException](statement.executeQuery("select 1"))
+        assert(e.getMessage.contains("Unknown planOnly mode: parser"))
         statement.executeQuery(s"set ${KyuubiConf.OPERATION_PLAN_ONLY_MODE.key}=parse")
         val result = statement.executeQuery("select 1")
         assert(result.next())