MaxGekk commented on code in PR #44883:
URL: https://github.com/apache/spark/pull/44883#discussion_r1470709842


##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -1656,6 +1656,19 @@
     ],
     "sqlState" : "XX000"
   },
+  "INTERNAL_ERROR_TWS" : {
+    "message" : [
+      "Internal error using TransformWithStateOperator: "
+    ],
+    "subClass" : {
+      "IMPLICIT_KEY_NOT_FOUND" : {

Review Comment:
   Internal errors should be not visible to users, so, don't need to introduce 
any sub-classes. Please, remove it and move `<message>` to `"Internal error 
using TransformWithStateOperator: "`



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreErrors.scala:
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.execution.streaming.state
+
+import org.apache.spark.{SparkException, SparkUnsupportedOperationException}
+
+/**
+ * Object for grouping error messages from (most) exceptions thrown from State 
API V2
+ *
+ * ERROR_CLASS has a prefix of "TWS_" or "STATE_STORE_" to indicate where the 
error is from

Review Comment:
   Update the comment.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreErrors.scala:
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.execution.streaming.state
+
+import org.apache.spark.{SparkException, SparkUnsupportedOperationException}
+
+/**
+ * Object for grouping error messages from (most) exceptions thrown from State 
API V2
+ *
+ * ERROR_CLASS has a prefix of "TWS_" or "STATE_STORE_" to indicate where the 
error is from
+ */
+object StateStoreErrors {
+  def implicitKeyNotFound(
+    stateName: String): SparkException = {

Review Comment:
   ```suggestion
     def implicitKeyNotFound(stateName: String): SparkException = {
   ```



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ValueStateSuite.scala:
##########
@@ -184,4 +187,18 @@ class ValueStateSuite extends SharedSparkSession
       assert(testState2.get() === null)
     }
   }
+
+  test("colFamily with HDFSBackedStateStoreProvider should fail") {
+    val storeId = StateStoreId(newDir(), Random.nextInt(), 0)
+    val provider = new HDFSBackedStateStoreProvider()
+    val storeConf = new StateStoreConf(new SQLConf())
+    val ex = intercept[StateStoreMultipleColumnFamiliesNotSupportedException] {
+      provider.init(
+        storeId, keySchema, valueSchema, 0, useColumnFamilies = true,
+        storeConf, new Configuration)
+    }
+    assert(ex.getMessage.contains("Creating multiple column families" +
+      " with HDFSStateStoreProvider is not supported"))
+    assert(ex.getMessage.contains("[STATE_STORE_MULTIPLE_COLUMN_FAMILIES]"))

Review Comment:
   Please, use `checkError` here and other places.



##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -3230,6 +3243,24 @@
     ],
     "sqlState" : "0A000"
   },
+  "STATE_STORE_MULTIPLE_COLUMN_FAMILIES" : {
+    "message" : [
+      "Creating multiple column families with <stateStoreProvider> is not 
supported"

Review Comment:
   Cannot it be a sub-class of `UNSUPPORTED_FEATURE`?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreErrors.scala:
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.execution.streaming.state
+
+import org.apache.spark.{SparkException, SparkUnsupportedOperationException}
+
+/**
+ * Object for grouping error messages from (most) exceptions thrown from State 
API V2
+ *
+ * ERROR_CLASS has a prefix of "TWS_" or "STATE_STORE_" to indicate where the 
error is from
+ */
+object StateStoreErrors {
+  def implicitKeyNotFound(
+    stateName: String): SparkException = {
+    SparkException.internalError(
+      msg = s"Implicit key not found in state store for stateName=$stateName",
+      category = "TWS.IMPLICIT_KEY_NOT_FOUND"

Review Comment:
   Please, don't mix `category` and error sub-class.



##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -3230,6 +3243,24 @@
     ],
     "sqlState" : "0A000"
   },
+  "STATE_STORE_MULTIPLE_COLUMN_FAMILIES" : {
+    "message" : [
+      "Creating multiple column families with <stateStoreProvider> is not 
supported"

Review Comment:
   ```suggestion
         "Creating multiple column families with <stateStoreProvider> is not 
supported."
   ```



-- 
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