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

[SS][SPARK-47331] Serialization using case classes/primitives/POJO based on SQL encoder for Arbitrary State API v2. #45447

Closed
wants to merge 5 commits into from
Closed
Show file tree
Hide file tree
Changes from 2 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ package org.apache.spark.sql.streaming
import java.io.Serializable

import org.apache.spark.annotation.{Evolving, Experimental}
import org.apache.spark.sql.Encoder

/**
* Represents the operation handle provided to the stateful processor used in the
Expand All @@ -33,20 +34,22 @@ private[sql] trait StatefulProcessorHandle extends Serializable {
* The user must ensure to call this function only within the `init()` method of the
* StatefulProcessor.
* @param stateName - name of the state variable
* @param valEncoder - SQL encoder for state variable
* @tparam T - type of state variable
* @return - instance of ValueState of type T that can be used to store state persistently
*/
def getValueState[T](stateName: String): ValueState[T]
def getValueState[T](stateName: String, valEncoder: Encoder[T]): ValueState[T]

/**
* Creates new or returns existing list state associated with stateName.
* The ListState persists values of type T.
*
* @param stateName - name of the state variable
* @param valEncoder - SQL encoder for state variable
* @tparam T - type of state variable
* @return - instance of ListState of type T that can be used to store state persistently
*/
def getListState[T](stateName: String): ListState[T]
def getListState[T](stateName: String, valEncoder: Encoder[T]): ListState[T]

/** Function to return queryInfo for currently running task */
def getQueryInfo(): QueryInfo
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
package org.apache.spark.sql.execution.streaming

import org.apache.spark.internal.Logging
import org.apache.spark.sql.Encoder
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
import org.apache.spark.sql.execution.streaming.StateKeyValueRowSchema.{KEY_ROW_SCHEMA, VALUE_ROW_SCHEMA}
import org.apache.spark.sql.execution.streaming.state.{StateStore, StateStoreErrors}
Expand All @@ -28,17 +29,20 @@ import org.apache.spark.sql.streaming.ListState
*
* @param store - reference to the StateStore instance to be used for storing state
* @param stateName - name of logical state partition
* @param keyEnc - Spark SQL encoder for key
* @param valEncoder - Spark SQL encoder for value
* @tparam S - data type of object that will be stored in the list
*/
class ListStateImpl[S](
store: StateStore,
stateName: String,
keyExprEnc: ExpressionEncoder[Any])
keyExprEnc: ExpressionEncoder[Any],
valEncoder: Encoder[S])
extends ListState[S] with Logging {

private val keySerializer = keyExprEnc.createSerializer()

private val stateTypesEncoder = StateTypesEncoder(keySerializer, stateName)
private val stateTypesEncoder = StateTypesEncoder(keySerializer, valEncoder, stateName)

store.createColFamilyIfAbsent(stateName, KEY_ROW_SCHEMA, numColsPrefixKey = 0,
VALUE_ROW_SCHEMA, useMultipleValuesPerKey = true)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,10 +17,10 @@

package org.apache.spark.sql.execution.streaming

import org.apache.commons.lang3.SerializationUtils

import org.apache.spark.sql.Encoder
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.Serializer
import org.apache.spark.sql.catalyst.encoders.encoderFor
import org.apache.spark.sql.catalyst.expressions.{UnsafeProjection, UnsafeRow}
import org.apache.spark.sql.execution.streaming.state.StateStoreErrors
import org.apache.spark.sql.types.{BinaryType, StructType}
Expand All @@ -41,17 +41,27 @@ object StateKeyValueRowSchema {
*
* @param keySerializer - serializer to serialize the grouping key of type `GK`
* to an [[InternalRow]]
* @param valEncoder - SQL encoder for value of type `S`
* @param stateName - name of logical state partition
* @tparam GK - grouping key type
* @tparam S - value type
Copy link
Contributor

Choose a reason for hiding this comment

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

Nit: Could we rename type param to V instead ?

*/
class StateTypesEncoder[GK](
class StateTypesEncoder[GK, S](
keySerializer: Serializer[GK],
valEncoder: Encoder[S],
stateName: String) {
import org.apache.spark.sql.execution.streaming.StateKeyValueRowSchema._

/** Variables reused for conversions between byte array and UnsafeRow */
private val keyProjection = UnsafeProjection.create(KEY_ROW_SCHEMA)
private val valueProjection = UnsafeProjection.create(VALUE_ROW_SCHEMA)

/** Variables reused for value conversions between spark sql and object */
private val valExpressionEnc = encoderFor(valEncoder)
private val objToRowSerializer = valExpressionEnc.createSerializer()
private val rowToObjDeserializer = valExpressionEnc.resolveAndBind().createDeserializer()
private val reuseRow = new UnsafeRow(valEncoder.schema.fields.length)

// TODO: validate places that are trying to encode the key and check if we can eliminate/
// add caching for some of these calls.
def encodeGroupingKey(): UnsafeRow = {
Expand All @@ -66,23 +76,26 @@ class StateTypesEncoder[GK](
keyRow
}

def encodeValue[S](value: S): UnsafeRow = {
val valueByteArr = SerializationUtils.serialize(value.asInstanceOf[Serializable])
val valueRow = valueProjection(InternalRow(valueByteArr))
valueRow
def encodeValue(value: S): UnsafeRow = {
val objRow: InternalRow = objToRowSerializer.apply(value)
val bytes = objRow.asInstanceOf[UnsafeRow].getBytes()
val valRow = valueProjection(InternalRow(bytes))
valRow
}

def decodeValue[S](row: UnsafeRow): S = {
SerializationUtils
.deserialize(row.getBinary(0))
.asInstanceOf[S]
def decodeValue(row: UnsafeRow): S = {
val bytes = row.getBinary(0)
reuseRow.pointTo(bytes, bytes.length)
val value = rowToObjDeserializer.apply(reuseRow)
value
}
}

object StateTypesEncoder {
def apply[GK](
def apply[GK, S](
keySerializer: Serializer[GK],
stateName: String): StateTypesEncoder[GK] = {
new StateTypesEncoder[GK](keySerializer, stateName)
valEncoder: Encoder[S],
stateName: String): StateTypesEncoder[GK, S] = {
new StateTypesEncoder[GK, S](keySerializer, valEncoder, stateName)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import java.util.UUID

import org.apache.spark.TaskContext
import org.apache.spark.internal.Logging
import org.apache.spark.sql.Encoder
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
import org.apache.spark.sql.execution.streaming.state.StateStore
import org.apache.spark.sql.streaming.{ListState, QueryInfo, StatefulProcessorHandle, ValueState}
Expand Down Expand Up @@ -112,10 +113,10 @@ class StatefulProcessorHandleImpl(

def getHandleState: StatefulProcessorHandleState = currState

override def getValueState[T](stateName: String): ValueState[T] = {
override def getValueState[T](stateName: String, valEncoder: Encoder[T]): ValueState[T] = {
verify(currState == CREATED, s"Cannot create state variable with name=$stateName after " +
"initialization is complete")
val resultState = new ValueStateImpl[T](store, stateName, keyEncoder)
val resultState = new ValueStateImpl[T](store, stateName, keyEncoder, valEncoder)
resultState
}

Expand All @@ -132,10 +133,10 @@ class StatefulProcessorHandleImpl(
store.removeColFamilyIfExists(stateName)
}

override def getListState[T](stateName: String): ListState[T] = {
override def getListState[T](stateName: String, valEncoder: Encoder[T]): ListState[T] = {
verify(currState == CREATED, s"Cannot create state variable with name=$stateName after " +
"initialization is complete")
val resultState = new ListStateImpl[T](store, stateName, keyEncoder)
val resultState = new ListStateImpl[T](store, stateName, keyEncoder, valEncoder)
resultState
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
package org.apache.spark.sql.execution.streaming

import org.apache.spark.internal.Logging
import org.apache.spark.sql.Encoder
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
import org.apache.spark.sql.catalyst.expressions.UnsafeRow
import org.apache.spark.sql.execution.streaming.StateKeyValueRowSchema.{KEY_ROW_SCHEMA, VALUE_ROW_SCHEMA}
Expand All @@ -29,16 +30,18 @@ import org.apache.spark.sql.streaming.ValueState
* @param store - reference to the StateStore instance to be used for storing state
* @param stateName - name of logical state partition
* @param keyEnc - Spark SQL encoder for key
* @param valEncoder - Spark SQL encoder for value
* @tparam S - data type of object that will be stored
*/
class ValueStateImpl[S](
store: StateStore,
stateName: String,
keyExprEnc: ExpressionEncoder[Any]) extends ValueState[S] with Logging {
keyExprEnc: ExpressionEncoder[Any],
valEncoder: Encoder[S]) extends ValueState[S] with Logging {

private val keySerializer = keyExprEnc.createSerializer()

private val stateTypesEncoder = StateTypesEncoder(keySerializer, stateName)
private val stateTypesEncoder = StateTypesEncoder(keySerializer, valEncoder, stateName)

store.createColFamilyIfAbsent(stateName, KEY_ROW_SCHEMA, numColsPrefixKey = 0,
VALUE_ROW_SCHEMA)
Expand All @@ -57,7 +60,7 @@ class ValueStateImpl[S](
override def get(): S = {
val retRow = getImpl()
if (retRow != null) {
stateTypesEncoder.decodeValue[S](retRow)
stateTypesEncoder.decodeValue(retRow)
} else {
null.asInstanceOf[S]
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,78 @@
/*
* 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;

/**
* A POJO class used for tests of arbitrary state SQL encoder.
*/
public class POJOTestClass {
// Fields
private String name;
private int id;

// Constructors
public POJOTestClass() {
// Default constructor
}

public POJOTestClass(String name, int id) {
this.name = name;
Copy link
Contributor

Choose a reason for hiding this comment

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

indent/spacing seems off ?

this.id = id;
}

// Getter methods
public String getName() {
return name;
}

public int getAge() {
return id;
}

// Setter methods
public void setName(String name) {
this.name = name;
}

public void setAge(int id) {
this.id = id;
}

// Additional methods if needed
public void incrementId() {
id++;
System.out.println(name + " is now " + id + "!");
}

// Override toString for better representation
@Override
public String toString() {
return "POJOTestClass{" +
"name='" + name + '\'' +
jingz-db marked this conversation as resolved.
Show resolved Hide resolved
", age=" + id +
'}';
}

// Override equals and hashCode for custom equality
@Override
public boolean equals(Object obj) {
POJOTestClass testObj = (POJOTestClass) obj;
return id == testObj.id && name.equals(testObj.name);
}
}

Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,9 @@ import org.apache.spark.sql.streaming.ValueState
import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types._

/** A case class for SQL encoder test purpose */
case class TestClass(var id: Long, var name: String)

/**
* Class that adds tests for single value ValueState types used in arbitrary stateful
* operators such as transformWithState
Expand Down Expand Up @@ -93,7 +96,7 @@ class ValueStateSuite extends SharedSparkSession
Encoders.STRING.asInstanceOf[ExpressionEncoder[Any]])

val stateName = "testState"
val testState: ValueState[Long] = handle.getValueState[Long]("testState")
val testState: ValueState[Long] = handle.getValueState[Long]("testState", Encoders.scalaLong)
Copy link
Contributor

Choose a reason for hiding this comment

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

Can we also add a test for other common types such as Double/String etc ?

assert(ImplicitGroupingKeyTracker.getImplicitKeyOption.isEmpty)
val ex = intercept[Exception] {
testState.update(123)
Expand Down Expand Up @@ -136,7 +139,7 @@ class ValueStateSuite extends SharedSparkSession
val handle = new StatefulProcessorHandleImpl(store, UUID.randomUUID(),
Encoders.STRING.asInstanceOf[ExpressionEncoder[Any]])

val testState: ValueState[Long] = handle.getValueState[Long]("testState")
val testState: ValueState[Long] = handle.getValueState[Long]("testState", Encoders.scalaLong)
ImplicitGroupingKeyTracker.setImplicitKey("test_key")
testState.update(123)
assert(testState.get() === 123)
Expand All @@ -162,8 +165,10 @@ class ValueStateSuite extends SharedSparkSession
val handle = new StatefulProcessorHandleImpl(store, UUID.randomUUID(),
Encoders.STRING.asInstanceOf[ExpressionEncoder[Any]])

val testState1: ValueState[Long] = handle.getValueState[Long]("testState1")
val testState2: ValueState[Long] = handle.getValueState[Long]("testState2")
val testState1: ValueState[Long] = handle.getValueState[Long](
"testState1", Encoders.scalaLong)
val testState2: ValueState[Long] = handle.getValueState[Long](
"testState2", Encoders.scalaLong)
ImplicitGroupingKeyTracker.setImplicitKey("test_key")
testState1.update(123)
assert(testState1.get() === 123)
Expand Down Expand Up @@ -217,4 +222,56 @@ class ValueStateSuite extends SharedSparkSession
matchPVals = true
)
}

test("test SQL encoder - Value state operations for case class instances") {
tryWithProviderResource(newStoreProviderWithValueState(true)) { provider =>
val store = provider.getStore(0)
val handle = new StatefulProcessorHandleImpl(store, UUID.randomUUID(),
Encoders.STRING.asInstanceOf[ExpressionEncoder[Any]])

val testState: ValueState[TestClass] = handle.getValueState[TestClass]("testState",
Encoders.product[TestClass])
ImplicitGroupingKeyTracker.setImplicitKey("test_key")
testState.update(TestClass(1, "testcase1"))
assert(testState.get().equals(TestClass(1, "testcase1")))
testState.clear()
assert(!testState.exists())
assert(testState.get() === null)

testState.update(TestClass(2, "testcase2"))
assert(testState.get() === TestClass(2, "testcase2"))
testState.update(TestClass(3, "testcase3"))
assert(testState.get() === TestClass(3, "testcase3"))

testState.clear()
assert(!testState.exists())
assert(testState.get() === null)
}
}

test("test SQL encoder - Value state operations for POJO instances") {
tryWithProviderResource(newStoreProviderWithValueState(true)) { provider =>
val store = provider.getStore(0)
val handle = new StatefulProcessorHandleImpl(store, UUID.randomUUID(),
Encoders.STRING.asInstanceOf[ExpressionEncoder[Any]])

val testState: ValueState[POJOTestClass] = handle.getValueState[POJOTestClass]("testState",
Encoders.bean(classOf[POJOTestClass]))
ImplicitGroupingKeyTracker.setImplicitKey("test_key")
testState.update(new POJOTestClass("testcase1", 1))
assert(testState.get().equals(new POJOTestClass("testcase1", 1)))
testState.clear()
assert(!testState.exists())
assert(testState.get() === null)

testState.update(new POJOTestClass("testcase2", 2))
assert(testState.get().equals(new POJOTestClass("testcase2", 2)))
testState.update(new POJOTestClass("testcase3", 3))
assert(testState.get().equals(new POJOTestClass("testcase3", 3)))

testState.clear()
assert(!testState.exists())
assert(testState.get() === null)
}
}
}