heyihong commented on code in PR #54056:
URL: https://github.com/apache/spark/pull/54056#discussion_r2795131742


##########
sql/connect/server/src/main/scala/org/apache/spark/sql/connect/IllegalStateErrors.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.connect
+
+import org.apache.spark.SparkIllegalStateException
+import org.apache.spark.sql.connect.service.{ExecuteHolder, ExecuteStatus, 
SessionHolder, SessionStatus}
+
+object IllegalStateErrors {
+
+  def streamLifecycleAlreadyCompleted(operation: String): 
SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = 
"SPARK_CONNECT_ILLEGAL_STATE.STREAM_LIFECYCLE_ALREADY_COMPLETED",
+      messageParameters = Map("operation" -> operation))
+
+  def cursorOutOfBounds(cursor: Long, batchSize: Long): 
SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = 
"SPARK_CONNECT_ILLEGAL_STATE.DATA_INTEGRITY_CURSOR_OUT_OF_BOUNDS",
+      messageParameters = Map(
+        "cursor" -> cursor.toString,
+        "batchSize" -> batchSize.toString))
+
+  def executionStateTransitionInvalidOperationStatus(
+      executeHolder: ExecuteHolder,
+      currentStatus: ExecuteStatus,
+      validStatuses: List[ExecuteStatus],
+      eventStatus: ExecuteStatus): SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = "SPARK_CONNECT_ILLEGAL_STATE." +
+        
"STATE_CONSISTENCY_EXECUTION_STATE_TRANSITION_INVALID_OPERATION_STATUS_MISMATCH",
+      messageParameters = Map(
+        "operationId" -> executeHolder.operationId,

Review Comment:
   In general, function or method parameters should be as strict (specific) as 
possible. In this case, we should pass the operationId string instead of 
executeHolder (consider that there might be another use case that wants to 
reuse this code but only has access to operationId and not executeHolder).



##########
sql/connect/server/src/main/scala/org/apache/spark/sql/connect/IllegalStateErrors.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.connect
+
+import org.apache.spark.SparkIllegalStateException
+import org.apache.spark.sql.connect.service.{ExecuteHolder, ExecuteStatus, 
SessionHolder, SessionStatus}
+
+object IllegalStateErrors {
+
+  def streamLifecycleAlreadyCompleted(operation: String): 
SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = 
"SPARK_CONNECT_ILLEGAL_STATE.STREAM_LIFECYCLE_ALREADY_COMPLETED",
+      messageParameters = Map("operation" -> operation))
+
+  def cursorOutOfBounds(cursor: Long, batchSize: Long): 
SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = 
"SPARK_CONNECT_ILLEGAL_STATE.DATA_INTEGRITY_CURSOR_OUT_OF_BOUNDS",
+      messageParameters = Map(
+        "cursor" -> cursor.toString,
+        "batchSize" -> batchSize.toString))
+
+  def executionStateTransitionInvalidOperationStatus(
+      executeHolder: ExecuteHolder,
+      currentStatus: ExecuteStatus,
+      validStatuses: List[ExecuteStatus],
+      eventStatus: ExecuteStatus): SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = "SPARK_CONNECT_ILLEGAL_STATE." +
+        
"STATE_CONSISTENCY_EXECUTION_STATE_TRANSITION_INVALID_OPERATION_STATUS_MISMATCH",
+      messageParameters = Map(
+        "operationId" -> executeHolder.operationId,
+        "currentStatus" -> currentStatus.toString,
+        "validStatuses" -> validStatuses.map(_.toString).mkString(", "),
+        "eventStatus" -> eventStatus.toString))
+
+  def executionStateTransitionInvalidSessionNotStarted(
+      sessionHolder: SessionHolder,
+      sessionStatus: SessionStatus,
+      eventStatus: ExecuteStatus): SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = "SPARK_CONNECT_ILLEGAL_STATE." +
+        
"STATE_CONSISTENCY_EXECUTION_STATE_TRANSITION_INVALID_SESSION_NOT_STARTED",
+      messageParameters = Map(
+        "sessionId" -> sessionHolder.sessionId,
+        "sessionStatus" -> sessionStatus.toString,
+        "eventStatus" -> eventStatus.toString))
+
+  def executeHolderAlreadyExists(operationId: String): 
SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = 
"SPARK_CONNECT_ILLEGAL_STATE.EXECUTION_STATE_EXECUTE_HOLDER_ALREADY_EXISTS",
+      messageParameters = Map("operationId" -> operationId))
+
+  def executeHolderAlreadyExistsGraphId(graphId: String): 
SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = "SPARK_CONNECT_ILLEGAL_STATE." +
+        "EXECUTION_STATE_EXECUTE_HOLDER_ALREADY_EXISTS_GRAPH",
+      messageParameters = Map("graphId" -> graphId))
+
+  def sessionAlreadyClosed(sessionHolder: SessionHolder): 
SparkIllegalStateException =

Review Comment:
   Ditto. Consider passing sessionKey instead of sessionHolder.



##########
sql/connect/server/src/main/scala/org/apache/spark/sql/connect/IllegalStateErrors.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.connect
+
+import org.apache.spark.SparkIllegalStateException
+import org.apache.spark.sql.connect.service.{ExecuteHolder, ExecuteStatus, 
SessionHolder, SessionStatus}
+
+object IllegalStateErrors {
+
+  def streamLifecycleAlreadyCompleted(operation: String): 
SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = 
"SPARK_CONNECT_ILLEGAL_STATE.STREAM_LIFECYCLE_ALREADY_COMPLETED",
+      messageParameters = Map("operation" -> operation))
+
+  def cursorOutOfBounds(cursor: Long, batchSize: Long): 
SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = 
"SPARK_CONNECT_ILLEGAL_STATE.DATA_INTEGRITY_CURSOR_OUT_OF_BOUNDS",
+      messageParameters = Map(
+        "cursor" -> cursor.toString,
+        "batchSize" -> batchSize.toString))
+
+  def executionStateTransitionInvalidOperationStatus(
+      executeHolder: ExecuteHolder,
+      currentStatus: ExecuteStatus,
+      validStatuses: List[ExecuteStatus],
+      eventStatus: ExecuteStatus): SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = "SPARK_CONNECT_ILLEGAL_STATE." +
+        
"STATE_CONSISTENCY_EXECUTION_STATE_TRANSITION_INVALID_OPERATION_STATUS_MISMATCH",
+      messageParameters = Map(
+        "operationId" -> executeHolder.operationId,
+        "currentStatus" -> currentStatus.toString,
+        "validStatuses" -> validStatuses.map(_.toString).mkString(", "),
+        "eventStatus" -> eventStatus.toString))
+
+  def executionStateTransitionInvalidSessionNotStarted(
+      sessionHolder: SessionHolder,
+      sessionStatus: SessionStatus,
+      eventStatus: ExecuteStatus): SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = "SPARK_CONNECT_ILLEGAL_STATE." +
+        
"STATE_CONSISTENCY_EXECUTION_STATE_TRANSITION_INVALID_SESSION_NOT_STARTED",
+      messageParameters = Map(
+        "sessionId" -> sessionHolder.sessionId,

Review Comment:
   Ditto. Consider passing sessionId instead of sessionHolder.



##########
sql/connect/server/src/main/scala/org/apache/spark/sql/connect/IllegalStateErrors.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.connect
+
+import org.apache.spark.SparkIllegalStateException
+import org.apache.spark.sql.connect.service.{ExecuteHolder, ExecuteStatus, 
SessionHolder, SessionStatus}
+
+object IllegalStateErrors {
+
+  def streamLifecycleAlreadyCompleted(operation: String): 
SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = 
"SPARK_CONNECT_ILLEGAL_STATE.STREAM_LIFECYCLE_ALREADY_COMPLETED",
+      messageParameters = Map("operation" -> operation))
+
+  def cursorOutOfBounds(cursor: Long, batchSize: Long): 
SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = 
"SPARK_CONNECT_ILLEGAL_STATE.DATA_INTEGRITY_CURSOR_OUT_OF_BOUNDS",
+      messageParameters = Map(
+        "cursor" -> cursor.toString,
+        "batchSize" -> batchSize.toString))
+
+  def executionStateTransitionInvalidOperationStatus(
+      executeHolder: ExecuteHolder,
+      currentStatus: ExecuteStatus,
+      validStatuses: List[ExecuteStatus],
+      eventStatus: ExecuteStatus): SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = "SPARK_CONNECT_ILLEGAL_STATE." +
+        
"STATE_CONSISTENCY_EXECUTION_STATE_TRANSITION_INVALID_OPERATION_STATUS_MISMATCH",
+      messageParameters = Map(
+        "operationId" -> executeHolder.operationId,
+        "currentStatus" -> currentStatus.toString,
+        "validStatuses" -> validStatuses.map(_.toString).mkString(", "),
+        "eventStatus" -> eventStatus.toString))
+
+  def executionStateTransitionInvalidSessionNotStarted(
+      sessionHolder: SessionHolder,
+      sessionStatus: SessionStatus,
+      eventStatus: ExecuteStatus): SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = "SPARK_CONNECT_ILLEGAL_STATE." +
+        
"STATE_CONSISTENCY_EXECUTION_STATE_TRANSITION_INVALID_SESSION_NOT_STARTED",
+      messageParameters = Map(
+        "sessionId" -> sessionHolder.sessionId,
+        "sessionStatus" -> sessionStatus.toString,
+        "eventStatus" -> eventStatus.toString))
+
+  def executeHolderAlreadyExists(operationId: String): 
SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = 
"SPARK_CONNECT_ILLEGAL_STATE.EXECUTION_STATE_EXECUTE_HOLDER_ALREADY_EXISTS",
+      messageParameters = Map("operationId" -> operationId))
+
+  def executeHolderAlreadyExistsGraphId(graphId: String): 
SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = "SPARK_CONNECT_ILLEGAL_STATE." +
+        "EXECUTION_STATE_EXECUTE_HOLDER_ALREADY_EXISTS_GRAPH",
+      messageParameters = Map("graphId" -> graphId))
+
+  def sessionAlreadyClosed(sessionHolder: SessionHolder): 
SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = 
"SPARK_CONNECT_ILLEGAL_STATE.SESSION_MANAGEMENT_SESSION_ALREADY_CLOSED",
+      messageParameters = Map("key" -> sessionHolder.key.toString))
+
+  def operationOrphaned(executeHolder: ExecuteHolder): 
SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = 
"SPARK_CONNECT_ILLEGAL_STATE.EXECUTION_STATE_OPERATION_ORPHANED",
+      messageParameters = Map("key" -> executeHolder.key.toString))
+
+  def sessionStateTransitionInvalid(
+      sessionHolder: SessionHolder,

Review Comment:
   ditto



##########
common/utils/src/main/resources/error/error-conditions.json:
##########
@@ -5487,6 +5409,149 @@
     ],
     "sqlState" : "42601"
   },
+  "SPARK_CONNECT_ILLEGAL_STATE" : {
+    "message" : [
+      "Spark Connect encountered an illegal state condition."
+    ],
+    "subClass" : {
+      "DATA_INTEGRITY_CHUNK_INDEX_MISMATCH" : {

Review Comment:
   My understanding is that illegal state errors are rare and should only occur 
when bugs are present. So, I was wondering whether it actually makes sense to 
have a subclass for each illegal state message instead of just using a general 
message <msg> (it may be a better idea to start with this). On the other hand, 
I feel that testing for error conditions is lacking, especially if we want to 
maintain so many different subclasses.



##########
sql/connect/server/src/main/scala/org/apache/spark/sql/connect/IllegalStateErrors.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.connect
+
+import org.apache.spark.SparkIllegalStateException
+import org.apache.spark.sql.connect.service.{ExecuteHolder, ExecuteStatus, 
SessionHolder, SessionStatus}
+
+object IllegalStateErrors {
+
+  def streamLifecycleAlreadyCompleted(operation: String): 
SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = 
"SPARK_CONNECT_ILLEGAL_STATE.STREAM_LIFECYCLE_ALREADY_COMPLETED",
+      messageParameters = Map("operation" -> operation))
+
+  def cursorOutOfBounds(cursor: Long, batchSize: Long): 
SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = 
"SPARK_CONNECT_ILLEGAL_STATE.DATA_INTEGRITY_CURSOR_OUT_OF_BOUNDS",
+      messageParameters = Map(
+        "cursor" -> cursor.toString,
+        "batchSize" -> batchSize.toString))
+
+  def executionStateTransitionInvalidOperationStatus(
+      executeHolder: ExecuteHolder,
+      currentStatus: ExecuteStatus,
+      validStatuses: List[ExecuteStatus],
+      eventStatus: ExecuteStatus): SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = "SPARK_CONNECT_ILLEGAL_STATE." +
+        
"STATE_CONSISTENCY_EXECUTION_STATE_TRANSITION_INVALID_OPERATION_STATUS_MISMATCH",
+      messageParameters = Map(
+        "operationId" -> executeHolder.operationId,
+        "currentStatus" -> currentStatus.toString,
+        "validStatuses" -> validStatuses.map(_.toString).mkString(", "),
+        "eventStatus" -> eventStatus.toString))
+
+  def executionStateTransitionInvalidSessionNotStarted(
+      sessionHolder: SessionHolder,
+      sessionStatus: SessionStatus,
+      eventStatus: ExecuteStatus): SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = "SPARK_CONNECT_ILLEGAL_STATE." +
+        
"STATE_CONSISTENCY_EXECUTION_STATE_TRANSITION_INVALID_SESSION_NOT_STARTED",
+      messageParameters = Map(
+        "sessionId" -> sessionHolder.sessionId,
+        "sessionStatus" -> sessionStatus.toString,
+        "eventStatus" -> eventStatus.toString))
+
+  def executeHolderAlreadyExists(operationId: String): 
SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = 
"SPARK_CONNECT_ILLEGAL_STATE.EXECUTION_STATE_EXECUTE_HOLDER_ALREADY_EXISTS",
+      messageParameters = Map("operationId" -> operationId))
+
+  def executeHolderAlreadyExistsGraphId(graphId: String): 
SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = "SPARK_CONNECT_ILLEGAL_STATE." +
+        "EXECUTION_STATE_EXECUTE_HOLDER_ALREADY_EXISTS_GRAPH",
+      messageParameters = Map("graphId" -> graphId))
+
+  def sessionAlreadyClosed(sessionHolder: SessionHolder): 
SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = 
"SPARK_CONNECT_ILLEGAL_STATE.SESSION_MANAGEMENT_SESSION_ALREADY_CLOSED",
+      messageParameters = Map("key" -> sessionHolder.key.toString))
+
+  def operationOrphaned(executeHolder: ExecuteHolder): 
SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = 
"SPARK_CONNECT_ILLEGAL_STATE.EXECUTION_STATE_OPERATION_ORPHANED",
+      messageParameters = Map("key" -> executeHolder.key.toString))
+
+  def sessionStateTransitionInvalid(
+      sessionHolder: SessionHolder,
+      fromState: SessionStatus,
+      toState: SessionStatus,
+      validStates: List[SessionStatus]): SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass =
+        
"SPARK_CONNECT_ILLEGAL_STATE.STATE_CONSISTENCY_SESSION_STATE_TRANSITION_INVALID",
+      messageParameters = Map(
+        "sessionId" -> sessionHolder.sessionId,
+        "fromState" -> fromState.toString,
+        "toState" -> toState.toString,
+        "validStates" -> validStates.map(_.toString).mkString(", ")))
+
+  def serviceNotStarted(): SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = 
"SPARK_CONNECT_ILLEGAL_STATE.SESSION_MANAGEMENT_SERVICE_NOT_STARTED",
+      messageParameters = Map.empty)
+
+  def streamingQueryUnexpectedReturnValue(
+      sessionHolder: SessionHolder,
+      value: Any,
+      context: String): SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = 
"SPARK_CONNECT_ILLEGAL_STATE.STREAMING_QUERY_UNEXPECTED_RETURN_VALUE",
+      messageParameters = Map(
+        "key" -> sessionHolder.key.toString,
+        "value" -> value.toString,
+        "context" -> context))
+
+  def cleanerAlreadySet(
+      sessionHolder: SessionHolder,
+      queryKey: Any): SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = 
"SPARK_CONNECT_ILLEGAL_STATE.STATE_CONSISTENCY_CLEANER_ALREADY_SET",
+      messageParameters = Map(
+        "key" -> sessionHolder.key.toString,
+        "queryKey" -> queryKey.toString))
+
+  def eventSendAfterShutdown(sessionHolder: SessionHolder): 
SparkIllegalStateException =

Review Comment:
   ditto



##########
sql/connect/server/src/main/scala/org/apache/spark/sql/connect/IllegalStateErrors.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.connect
+
+import org.apache.spark.SparkIllegalStateException
+import org.apache.spark.sql.connect.service.{ExecuteHolder, ExecuteStatus, 
SessionHolder, SessionStatus}
+
+object IllegalStateErrors {
+
+  def streamLifecycleAlreadyCompleted(operation: String): 
SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = 
"SPARK_CONNECT_ILLEGAL_STATE.STREAM_LIFECYCLE_ALREADY_COMPLETED",
+      messageParameters = Map("operation" -> operation))
+
+  def cursorOutOfBounds(cursor: Long, batchSize: Long): 
SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = 
"SPARK_CONNECT_ILLEGAL_STATE.DATA_INTEGRITY_CURSOR_OUT_OF_BOUNDS",
+      messageParameters = Map(
+        "cursor" -> cursor.toString,
+        "batchSize" -> batchSize.toString))
+
+  def executionStateTransitionInvalidOperationStatus(
+      executeHolder: ExecuteHolder,
+      currentStatus: ExecuteStatus,
+      validStatuses: List[ExecuteStatus],
+      eventStatus: ExecuteStatus): SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = "SPARK_CONNECT_ILLEGAL_STATE." +
+        
"STATE_CONSISTENCY_EXECUTION_STATE_TRANSITION_INVALID_OPERATION_STATUS_MISMATCH",
+      messageParameters = Map(
+        "operationId" -> executeHolder.operationId,
+        "currentStatus" -> currentStatus.toString,
+        "validStatuses" -> validStatuses.map(_.toString).mkString(", "),
+        "eventStatus" -> eventStatus.toString))
+
+  def executionStateTransitionInvalidSessionNotStarted(
+      sessionHolder: SessionHolder,
+      sessionStatus: SessionStatus,
+      eventStatus: ExecuteStatus): SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = "SPARK_CONNECT_ILLEGAL_STATE." +
+        
"STATE_CONSISTENCY_EXECUTION_STATE_TRANSITION_INVALID_SESSION_NOT_STARTED",
+      messageParameters = Map(
+        "sessionId" -> sessionHolder.sessionId,
+        "sessionStatus" -> sessionStatus.toString,
+        "eventStatus" -> eventStatus.toString))
+
+  def executeHolderAlreadyExists(operationId: String): 
SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = 
"SPARK_CONNECT_ILLEGAL_STATE.EXECUTION_STATE_EXECUTE_HOLDER_ALREADY_EXISTS",
+      messageParameters = Map("operationId" -> operationId))
+
+  def executeHolderAlreadyExistsGraphId(graphId: String): 
SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = "SPARK_CONNECT_ILLEGAL_STATE." +
+        "EXECUTION_STATE_EXECUTE_HOLDER_ALREADY_EXISTS_GRAPH",
+      messageParameters = Map("graphId" -> graphId))
+
+  def sessionAlreadyClosed(sessionHolder: SessionHolder): 
SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = 
"SPARK_CONNECT_ILLEGAL_STATE.SESSION_MANAGEMENT_SESSION_ALREADY_CLOSED",
+      messageParameters = Map("key" -> sessionHolder.key.toString))
+
+  def operationOrphaned(executeHolder: ExecuteHolder): 
SparkIllegalStateException =

Review Comment:
   ditto. Consider passing executeKey instead of executeHolder.



##########
sql/connect/server/src/main/scala/org/apache/spark/sql/connect/IllegalStateErrors.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.connect
+
+import org.apache.spark.SparkIllegalStateException
+import org.apache.spark.sql.connect.service.{ExecuteHolder, ExecuteStatus, 
SessionHolder, SessionStatus}
+
+object IllegalStateErrors {
+
+  def streamLifecycleAlreadyCompleted(operation: String): 
SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = 
"SPARK_CONNECT_ILLEGAL_STATE.STREAM_LIFECYCLE_ALREADY_COMPLETED",
+      messageParameters = Map("operation" -> operation))
+
+  def cursorOutOfBounds(cursor: Long, batchSize: Long): 
SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = 
"SPARK_CONNECT_ILLEGAL_STATE.DATA_INTEGRITY_CURSOR_OUT_OF_BOUNDS",
+      messageParameters = Map(
+        "cursor" -> cursor.toString,
+        "batchSize" -> batchSize.toString))
+
+  def executionStateTransitionInvalidOperationStatus(
+      executeHolder: ExecuteHolder,
+      currentStatus: ExecuteStatus,
+      validStatuses: List[ExecuteStatus],
+      eventStatus: ExecuteStatus): SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = "SPARK_CONNECT_ILLEGAL_STATE." +
+        
"STATE_CONSISTENCY_EXECUTION_STATE_TRANSITION_INVALID_OPERATION_STATUS_MISMATCH",
+      messageParameters = Map(
+        "operationId" -> executeHolder.operationId,
+        "currentStatus" -> currentStatus.toString,
+        "validStatuses" -> validStatuses.map(_.toString).mkString(", "),
+        "eventStatus" -> eventStatus.toString))
+
+  def executionStateTransitionInvalidSessionNotStarted(
+      sessionHolder: SessionHolder,
+      sessionStatus: SessionStatus,
+      eventStatus: ExecuteStatus): SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = "SPARK_CONNECT_ILLEGAL_STATE." +
+        
"STATE_CONSISTENCY_EXECUTION_STATE_TRANSITION_INVALID_SESSION_NOT_STARTED",
+      messageParameters = Map(
+        "sessionId" -> sessionHolder.sessionId,
+        "sessionStatus" -> sessionStatus.toString,
+        "eventStatus" -> eventStatus.toString))
+
+  def executeHolderAlreadyExists(operationId: String): 
SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = 
"SPARK_CONNECT_ILLEGAL_STATE.EXECUTION_STATE_EXECUTE_HOLDER_ALREADY_EXISTS",
+      messageParameters = Map("operationId" -> operationId))
+
+  def executeHolderAlreadyExistsGraphId(graphId: String): 
SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = "SPARK_CONNECT_ILLEGAL_STATE." +
+        "EXECUTION_STATE_EXECUTE_HOLDER_ALREADY_EXISTS_GRAPH",
+      messageParameters = Map("graphId" -> graphId))
+
+  def sessionAlreadyClosed(sessionHolder: SessionHolder): 
SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = 
"SPARK_CONNECT_ILLEGAL_STATE.SESSION_MANAGEMENT_SESSION_ALREADY_CLOSED",
+      messageParameters = Map("key" -> sessionHolder.key.toString))
+
+  def operationOrphaned(executeHolder: ExecuteHolder): 
SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = 
"SPARK_CONNECT_ILLEGAL_STATE.EXECUTION_STATE_OPERATION_ORPHANED",
+      messageParameters = Map("key" -> executeHolder.key.toString))
+
+  def sessionStateTransitionInvalid(
+      sessionHolder: SessionHolder,
+      fromState: SessionStatus,
+      toState: SessionStatus,
+      validStates: List[SessionStatus]): SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass =
+        
"SPARK_CONNECT_ILLEGAL_STATE.STATE_CONSISTENCY_SESSION_STATE_TRANSITION_INVALID",
+      messageParameters = Map(
+        "sessionId" -> sessionHolder.sessionId,
+        "fromState" -> fromState.toString,
+        "toState" -> toState.toString,
+        "validStates" -> validStates.map(_.toString).mkString(", ")))
+
+  def serviceNotStarted(): SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = 
"SPARK_CONNECT_ILLEGAL_STATE.SESSION_MANAGEMENT_SERVICE_NOT_STARTED",
+      messageParameters = Map.empty)
+
+  def streamingQueryUnexpectedReturnValue(
+      sessionHolder: SessionHolder,

Review Comment:
   ditto



##########
sql/connect/server/src/main/scala/org/apache/spark/sql/connect/IllegalStateErrors.scala:
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.connect
+
+import org.apache.spark.SparkIllegalStateException
+import org.apache.spark.sql.connect.service.{ExecuteHolder, ExecuteStatus, 
SessionHolder, SessionStatus}
+
+object IllegalStateErrors {
+
+  def streamLifecycleAlreadyCompleted(operation: String): 
SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = 
"SPARK_CONNECT_ILLEGAL_STATE.STREAM_LIFECYCLE_ALREADY_COMPLETED",
+      messageParameters = Map("operation" -> operation))
+
+  def cursorOutOfBounds(cursor: Long, batchSize: Long): 
SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = 
"SPARK_CONNECT_ILLEGAL_STATE.DATA_INTEGRITY_CURSOR_OUT_OF_BOUNDS",
+      messageParameters = Map(
+        "cursor" -> cursor.toString,
+        "batchSize" -> batchSize.toString))
+
+  def executionStateTransitionInvalidOperationStatus(
+      executeHolder: ExecuteHolder,
+      currentStatus: ExecuteStatus,
+      validStatuses: List[ExecuteStatus],
+      eventStatus: ExecuteStatus): SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = "SPARK_CONNECT_ILLEGAL_STATE." +
+        
"STATE_CONSISTENCY_EXECUTION_STATE_TRANSITION_INVALID_OPERATION_STATUS_MISMATCH",
+      messageParameters = Map(
+        "operationId" -> executeHolder.operationId,
+        "currentStatus" -> currentStatus.toString,
+        "validStatuses" -> validStatuses.map(_.toString).mkString(", "),
+        "eventStatus" -> eventStatus.toString))
+
+  def executionStateTransitionInvalidSessionNotStarted(
+      sessionHolder: SessionHolder,
+      sessionStatus: SessionStatus,
+      eventStatus: ExecuteStatus): SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = "SPARK_CONNECT_ILLEGAL_STATE." +
+        
"STATE_CONSISTENCY_EXECUTION_STATE_TRANSITION_INVALID_SESSION_NOT_STARTED",
+      messageParameters = Map(
+        "sessionId" -> sessionHolder.sessionId,
+        "sessionStatus" -> sessionStatus.toString,
+        "eventStatus" -> eventStatus.toString))
+
+  def executeHolderAlreadyExists(operationId: String): 
SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = 
"SPARK_CONNECT_ILLEGAL_STATE.EXECUTION_STATE_EXECUTE_HOLDER_ALREADY_EXISTS",
+      messageParameters = Map("operationId" -> operationId))
+
+  def executeHolderAlreadyExistsGraphId(graphId: String): 
SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = "SPARK_CONNECT_ILLEGAL_STATE." +
+        "EXECUTION_STATE_EXECUTE_HOLDER_ALREADY_EXISTS_GRAPH",
+      messageParameters = Map("graphId" -> graphId))
+
+  def sessionAlreadyClosed(sessionHolder: SessionHolder): 
SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = 
"SPARK_CONNECT_ILLEGAL_STATE.SESSION_MANAGEMENT_SESSION_ALREADY_CLOSED",
+      messageParameters = Map("key" -> sessionHolder.key.toString))
+
+  def operationOrphaned(executeHolder: ExecuteHolder): 
SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = 
"SPARK_CONNECT_ILLEGAL_STATE.EXECUTION_STATE_OPERATION_ORPHANED",
+      messageParameters = Map("key" -> executeHolder.key.toString))
+
+  def sessionStateTransitionInvalid(
+      sessionHolder: SessionHolder,
+      fromState: SessionStatus,
+      toState: SessionStatus,
+      validStates: List[SessionStatus]): SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass =
+        
"SPARK_CONNECT_ILLEGAL_STATE.STATE_CONSISTENCY_SESSION_STATE_TRANSITION_INVALID",
+      messageParameters = Map(
+        "sessionId" -> sessionHolder.sessionId,
+        "fromState" -> fromState.toString,
+        "toState" -> toState.toString,
+        "validStates" -> validStates.map(_.toString).mkString(", ")))
+
+  def serviceNotStarted(): SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = 
"SPARK_CONNECT_ILLEGAL_STATE.SESSION_MANAGEMENT_SERVICE_NOT_STARTED",
+      messageParameters = Map.empty)
+
+  def streamingQueryUnexpectedReturnValue(
+      sessionHolder: SessionHolder,
+      value: Any,
+      context: String): SparkIllegalStateException =
+    new SparkIllegalStateException(
+      errorClass = 
"SPARK_CONNECT_ILLEGAL_STATE.STREAMING_QUERY_UNEXPECTED_RETURN_VALUE",
+      messageParameters = Map(
+        "key" -> sessionHolder.key.toString,
+        "value" -> value.toString,
+        "context" -> context))
+
+  def cleanerAlreadySet(

Review Comment:
   ditto



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]


Reply via email to