Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SPARK-50714][SQL][SS] Enable schema evolution for TransformWithState when Avro encoding is used #49277

Open
wants to merge 25 commits into
base: master
Choose a base branch
from

Conversation

ericm-db
Copy link
Contributor

@ericm-db ericm-db commented Dec 23, 2024

What changes were proposed in this pull request?

This PR introduces stateful schema evolution for the TransformWithState operator when Avro is used.
We modified the StateStoreColumnFamilySchema and the StateSchemaV3 file to keep track of the key schema and value schema id in order to support versioning and lookups of schemas across query restarts.
The AvroStateEncoder now takes a StateSchemaProvider, which allows it to look up all of the active schemas in the StateStore for a given column family, allowing it to pass reader and writer schemas to the AvroEncoder class.
We have also added changes so that the StateDataSource can read from these rows with schema ID

Why are the changes needed?

Schema evolution is a critical feature for stateful stream processing applications that need to handle changing data schemas over time.

Does this PR introduce any user-facing change?

Yes - this change allows stateful schema evolution that was not previously possible for the TransformWithState operator.

How was this patch tested?

Unit and Integration tests in RocksDBStateStoreSuite and TransformWithStateSuite

Was this patch authored or co-authored using generative AI tooling?

readerSchema: Schema,
valueProj: UnsafeProjection): UnsafeRow = {
if (valueBytes != null) {
val reader = new GenericDatumReader[Any](writerSchema, readerSchema)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

lets add some comments here around the args

@ericm-db ericm-db changed the title Ssm [SPARK-50714] Enable schema evolution for TransformWithState when Avro encoding is used Jan 3, 2025
@HyukjinKwon HyukjinKwon changed the title [SPARK-50714] Enable schema evolution for TransformWithState when Avro encoding is used [SPARK-50714][SQL] Enable schema evolution for TransformWithState when Avro encoding is used Jan 3, 2025

dataType match {
// Basic types
case BooleanType => false
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Are these Avro defaults too ?


// Complex types
case ArrayType(elementType, _) =>
val defaultArray = new java.util.ArrayList[Any]()
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why not have empty collections ? i.e. empty array/map etc ?

assert(timeMode == TimeMode.EventTime.toString || timeMode == TimeMode.ProcessingTime.toString)
if (timeMode == TimeMode.EventTime.toString) {
val primaryIndex = if (timeMode == TimeMode.EventTime.toString) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: i guess this could also be split into a separate function ?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Oh I was trying to address this comment: #49277 (comment)

@@ -285,13 +285,13 @@ class TransformWithValueStateTTLSuite extends TransformWithStateTTLTest {
.add("expiryTimestampMs", LongType, nullable = false)
val schemaForValueRow: StructType = StructType(Array(StructField("__dummy__", NullType)))
val schema0 = StateStoreColFamilySchema(
TimerStateUtils.getTimerStateVarName(TimeMode.ProcessingTime().toString),
schemaForKeyRow,
TimerStateUtils.getTimerStateVarNames(TimeMode.ProcessingTime().toString)._1, 0,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Lets also add atleast a couple of tests for other composite types such as list, map etc ?

@anishshri-db
Copy link
Contributor

anishshri-db commented Jan 3, 2025

@ericm-db - can you also format the PR description and explain in more detail what functionality this PR adds. Thx

Also - this is a user facing change right ?


val result2 = inputData.toDS()
.groupByKey(x => x)
.transformWithState(new RunningCountStatefulProcessorNestedLongs(),
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could we also add a test case where the newly added column is also part of the output ? That way we can ensure that the default values are being picked up correctly ? Prob also similar for dropped columns ?

@ericm-db ericm-db changed the title [SPARK-50714][SQL] Enable schema evolution for TransformWithState when Avro encoding is used [SPARK-50714][SQL][SS] Enable schema evolution for TransformWithState when Avro encoding is used Jan 3, 2025
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

2 participants