zifeif2 commented on code in PR #53459:
URL: https://github.com/apache/spark/pull/53459#discussion_r2666318846


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSource.scala:
##########
@@ -296,29 +315,33 @@ class StateDataSource extends TableProvider with 
DataSourceRegister with Logging
 
           if (sourceOptions.readRegisteredTimers) {
             stateVarName = TimerStateUtils.getTimerStateVarNames(timeMode)._1
+          } else if (sourceOptions.internalOnlyReadAllColumnFamilies) {
+            // When reading all column families (for repartitioning) for TWS 
operator,
+            // we will just choose a random state as placeholder for default 
column family,
+            // because we need to use matching stateVariableInfo and 
stateStoreColFamilySchemaOpt
+            // to inferSchema (partitionKey in particular) later
+            stateVarName = operatorProperties.stateVariables.head.stateName
           }
-          // When reading all column families (for repartitioning), we collect 
all state variable
-          // infos instead of validating a specific stateVarName. This skips 
the normal validation
-          // logic because we're not reading a specific state variable - we're 
reading all of them.
+
           if (sourceOptions.internalOnlyReadAllColumnFamilies) {
             stateVariableInfos = operatorProperties.stateVariables
-          } else {
-            var stateVarInfoList = operatorProperties.stateVariables
-              .filter(stateVar => stateVar.stateName == stateVarName)
-            if (stateVarInfoList.isEmpty &&
-              
StateStoreColumnFamilySchemaUtils.isTestingInternalColFamily(stateVarName)) {
-              // pass this dummy TWSStateVariableInfo for TWS internal column 
family during testing,
-              // because internalColumns are not register in 
operatorProperties.stateVariables,
-              // thus stateVarInfoList will be empty.
-              stateVarInfoList = List(TransformWithStateVariableInfo(
-                stateVarName, StateVariableType.ValueState, false
-              ))
-            }
-            require(stateVarInfoList.size == 1, s"Failed to find unique state 
variable info " +
-              s"for state variable $stateVarName in operator 
${sourceOptions.operatorId}")
-            val stateVarInfo = stateVarInfoList.head
-            transformWithStateVariableInfoOpt = Some(stateVarInfo)
           }
+          var stateVarInfoList = operatorProperties.stateVariables
+            .filter(stateVar => stateVar.stateName == stateVarName)
+          if (stateVarInfoList.isEmpty &&

Review Comment:
   It will be empty when it's a non-timer internal column. Updated the logic in 
the new version to make it more explicit 



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/timers/TimerStateImpl.scala:
##########
@@ -61,9 +61,14 @@ object TimerStateUtils {
   }
 
   def isTimerSecondaryIndexCF(colFamilyName: String): Boolean = {
-    assert(isTimerCFName(colFamilyName), s"Column family name must be for a 
timer: $colFamilyName")

Review Comment:
   Added it back. Was using it as a shortcut for the logic isTimerCFName && 
isTimerSecondaryIndexCF



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