ericm-db commented on code in PR #49277:
URL: https://github.com/apache/spark/pull/49277#discussion_r1918070616


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreSuite.scala:
##########
@@ -496,6 +498,486 @@ class RocksDBStateStoreSuite extends 
StateStoreSuiteBase[RocksDBStateStoreProvid
     }
   }
 
+  test("AvroStateEncoder - add field") {
+    val keySchema = StructType(Seq(
+      StructField("k", StringType)
+    ))
+
+    val initialValueSchema = StructType(Seq(
+      StructField("value", IntegerType, false)
+    ))
+
+    val evolvedValueSchema = StructType(Seq(
+      StructField("value", IntegerType, false),
+      StructField("timestamp", LongType, true)
+    ))
+
+    // Create test state schema provider
+    val testProvider = new TestStateSchemaProvider()
+
+    // Add initial schema version
+    testProvider.captureSchema(
+      StateStore.DEFAULT_COL_FAMILY_NAME,
+      keySchema,
+      initialValueSchema,
+      valueSchemaId = 0
+    )
+
+    // Create encoder with initial schema
+    val encoder1 = new AvroStateEncoder(
+      NoPrefixKeyStateEncoderSpec(keySchema),
+      initialValueSchema,
+      Some(testProvider),
+      Some(ColumnFamilyInfo(StateStore.DEFAULT_COL_FAMILY_NAME, 1))
+    )
+
+    // Create test data
+    val proj = UnsafeProjection.create(initialValueSchema)
+    val row1 = proj.apply(InternalRow(1))
+
+    // Encode with schema v0
+    val encoded = encoder1.encodeValue(row1)
+
+    // Add evolved schema
+    testProvider.captureSchema(
+      StateStore.DEFAULT_COL_FAMILY_NAME,
+      keySchema,
+      evolvedValueSchema,
+      valueSchemaId = 1
+    )
+
+    // Create encoder with initial schema
+    val encoder2 = new AvroStateEncoder(
+      NoPrefixKeyStateEncoderSpec(keySchema),
+      evolvedValueSchema,
+      Some(testProvider),
+      Some(ColumnFamilyInfo(StateStore.DEFAULT_COL_FAMILY_NAME, 1))
+    )
+
+    // Decode with evolved schema
+    val decoded = encoder2.decodeValue(encoded)
+
+    // Should be able to read old format
+    assert(decoded.getInt(0) === 1)
+    assert(decoded.getLong(1) === 0L) // New field should be null

Review Comment:
   UnsafeRow doesn't have any checks, when we call getLong on a null, we return 0



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala:
##########
@@ -123,6 +126,25 @@ trait AlsoTestWithEncodingTypes extends SQLTestUtils {
       }
     }
   }
+
+  def usingAvroEncoding(): Boolean = {
+    SQLConf.get.getConf(

Review Comment:
   No, good point.



-- 
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: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org

Reply via email to