anishshri-db commented on code in PR #48401:
URL: https://github.com/apache/spark/pull/48401#discussion_r1837372697


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala:
##########
@@ -259,6 +259,19 @@ class IncrementalExecution(
     }
   }
 
+  object StateStoreColumnFamilySchemasRule extends SparkPlanPartialRule {

Review Comment:
   lets add a comment to explain what the rule does ?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MapStateImplWithTTL.scala:
##########
@@ -36,21 +36,27 @@ import org.apache.spark.util.NextIterator
  * @param ttlConfig  - the ttl configuration (time to live duration etc.)
  * @param batchTimestampMs - current batch processing timestamp.
  * @param metrics - metrics to be updated as part of stateful processing
+ * @param avroEnc - optional Avro serializer and deserializer for this state 
variable that
+ *                is used by the StateStore to encode state in Avro format
+ * @param secondaryIndexAvroEnc - optional Avro serializer and deserializer 
for TTL state that
+ *                is used by the StateStore to encode state in Avro format
  * @tparam K - type of key for map state variable
  * @tparam V - type of value for map state variable
  * @return - instance of MapState of type [K,V] that can be used to store 
state persistently
  */
 class MapStateImplWithTTL[K, V](
-    store: StateStore,
-    stateName: String,
-    keyExprEnc: ExpressionEncoder[Any],
-    userKeyEnc: ExpressionEncoder[Any],
-    valEncoder: ExpressionEncoder[Any],
-    ttlConfig: TTLConfig,
-    batchTimestampMs: Long,
-    metrics: Map[String, SQLMetric] = Map.empty)
+      store: StateStore,

Review Comment:
   indent is off ?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateExec.scala:
##########
@@ -580,8 +587,9 @@ case class TransformWithStateExec(
           new SerializableConfiguration(session.sessionState.newHadoopConf()))
         child.execute().mapPartitionsWithIndex[InternalRow](
           (i: Int, iter: Iterator[InternalRow]) => {
-            initNewStateStoreAndProcessData(i, hadoopConfBroadcast) { store =>
-              processData(store, iter)
+            initNewStateStoreAndProcessData(
+              i, hadoopConfBroadcast) { store =>

Review Comment:
   nit: lets revert to older style ?



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreSuite.scala:
##########
@@ -339,6 +339,82 @@ class RocksDBStateStoreSuite extends 
StateStoreSuiteBase[RocksDBStateStoreProvid
     }
   }
 
+  test("rocksdb range scan - fixed size non-ordering columns with Avro 
encoding") {
+

Review Comment:
   nit: extra newlines ?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MapStateImplWithTTL.scala:
##########
@@ -36,21 +36,27 @@ import org.apache.spark.util.NextIterator
  * @param ttlConfig  - the ttl configuration (time to live duration etc.)
  * @param batchTimestampMs - current batch processing timestamp.
  * @param metrics - metrics to be updated as part of stateful processing
+ * @param avroEnc - optional Avro serializer and deserializer for this state 
variable that
+ *                is used by the StateStore to encode state in Avro format

Review Comment:
   nit: lets confirm the expected style for these multi-line arg comments



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StateStoreColumnFamilySchemaUtils.scala:
##########
@@ -16,36 +16,132 @@
  */
 package org.apache.spark.sql.execution.streaming
 
+import org.apache.spark.internal.Logging
 import org.apache.spark.sql.Encoder
+import org.apache.spark.sql.avro.{AvroDeserializer, AvroOptions, 
AvroSerializer, SchemaConverters}
 import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
 import 
org.apache.spark.sql.execution.streaming.TransformWithStateKeyValueRowSchemaUtils._
-import 
org.apache.spark.sql.execution.streaming.state.{NoPrefixKeyStateEncoderSpec, 
PrefixKeyScanStateEncoderSpec, StateStoreColFamilySchema}
-import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.execution.streaming.state.{AvroEncoder, 
NoPrefixKeyStateEncoderSpec, PrefixKeyScanStateEncoderSpec, 
RangeKeyScanStateEncoderSpec, StateStoreColFamilySchema}
+import org.apache.spark.sql.types.{BinaryType, BooleanType, ByteType, 
DataType, DoubleType, FloatType, IntegerType, LongType, NullType, ShortType, 
StructField, StructType}

Review Comment:
   maybe just `types._` ?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateExec.scala:
##########
@@ -718,6 +732,22 @@ object TransformWithStateExec {
       stateStoreCkptIds = None
     )
 
+    val stateStoreEncoding = child.session.sessionState.conf.getConf(
+      SQLConf.STREAMING_STATE_STORE_ENCODING_FORMAT
+    )
+
+    def getDriverProcessorHandle(): DriverStatefulProcessorHandleImpl = {
+      val driverProcessorHandle = new DriverStatefulProcessorHandleImpl(
+        timeMode, keyEncoder, initializeAvroEnc =
+          stateStoreEncoding == StateStoreEncoding.Avro.toString)

Review Comment:
   same here



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateExec.scala:
##########
@@ -104,7 +106,10 @@ case class TransformWithStateExec(
    * @return a new instance of the driver processor handle
    */
   private def getDriverProcessorHandle(): DriverStatefulProcessorHandleImpl = {
-    val driverProcessorHandle = new 
DriverStatefulProcessorHandleImpl(timeMode, keyEncoder)
+
+    val driverProcessorHandle = new DriverStatefulProcessorHandleImpl(
+      timeMode, keyEncoder, initializeAvroEnc =
+        stateStoreEncoding == StateStoreEncoding.Avro.toString)

Review Comment:
   lets move to separate function ?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StateStoreColumnFamilySchemaUtils.scala:
##########
@@ -16,36 +16,132 @@
  */
 package org.apache.spark.sql.execution.streaming
 
+import org.apache.spark.internal.Logging
 import org.apache.spark.sql.Encoder
+import org.apache.spark.sql.avro.{AvroDeserializer, AvroOptions, 
AvroSerializer, SchemaConverters}
 import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
 import 
org.apache.spark.sql.execution.streaming.TransformWithStateKeyValueRowSchemaUtils._
-import 
org.apache.spark.sql.execution.streaming.state.{NoPrefixKeyStateEncoderSpec, 
PrefixKeyScanStateEncoderSpec, StateStoreColFamilySchema}
-import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.execution.streaming.state.{AvroEncoder, 
NoPrefixKeyStateEncoderSpec, PrefixKeyScanStateEncoderSpec, 
RangeKeyScanStateEncoderSpec, StateStoreColFamilySchema}
+import org.apache.spark.sql.types.{BinaryType, BooleanType, ByteType, 
DataType, DoubleType, FloatType, IntegerType, LongType, NullType, ShortType, 
StructField, StructType}
 
-object StateStoreColumnFamilySchemaUtils {
+object StateStoreColumnFamilySchemaUtils extends Serializable {
+
+  def apply(initializeAvroSerde: Boolean): StateStoreColumnFamilySchemaUtils =
+    new StateStoreColumnFamilySchemaUtils(initializeAvroSerde)
+
+  /**
+   * Avro uses zig-zag encoding for some fixed-length types, like Longs and 
Ints. For range scans
+   * we want to use big-endian encoding, so we need to convert the source 
schema to replace these
+   * types with BinaryType.
+   *
+   * @param schema The schema to convert
+   * @param ordinals If non-empty, only convert fields at these ordinals.
+   *                 If empty, convert all fields.
+   */
+  def convertForRangeScan(schema: StructType, ordinals: Seq[Int] = Seq.empty): 
StructType = {
+    val ordinalSet = ordinals.toSet
+
+    StructType(schema.fields.zipWithIndex.flatMap { case (field, idx) =>
+      if ((ordinals.isEmpty || ordinalSet.contains(idx)) && 
isFixedSize(field.dataType)) {
+        // For each numeric field, create two fields:
+        // 1. A boolean for sign (positive = true, negative = false)
+        // 2. The original numeric value in big-endian format
+        Seq(
+          StructField(s"${field.name}_marker", ByteType, nullable = false),
+          field.copy(name = s"${field.name}_value", BinaryType)
+        )
+      } else {
+        Seq(field)
+      }
+    })
+  }
+
+  private def isFixedSize(dataType: DataType): Boolean = dataType match {
+    case _: ByteType | _: BooleanType | _: ShortType | _: IntegerType | _: 
LongType |
+         _: FloatType | _: DoubleType => true
+    case _ => false
+  }
+
+  def getTtlColFamilyName(stateName: String): String = {
+    "$ttl_" + stateName
+  }
+}
+
+/**
+ *
+ * @param initializeAvroSerde Whether or not to create the Avro serializers 
and deserializers
+ *                            for this state type. This class is used to 
create the
+ *                            StateStoreColumnFamilySchema for each state 
variable from the driver
+ */
+class StateStoreColumnFamilySchemaUtils(initializeAvroSerde: Boolean)
+    extends Logging with Serializable {
+  private def getAvroSerializer(schema: StructType): AvroSerializer = {
+    val avroType = SchemaConverters.toAvroType(schema)
+    new AvroSerializer(schema, avroType, nullable = false)
+  }
+
+  private def getAvroDeserializer(schema: StructType): AvroDeserializer = {
+    val avroType = SchemaConverters.toAvroType(schema)
+    val avroOptions = AvroOptions(Map.empty)
+    new AvroDeserializer(avroType, schema,
+      avroOptions.datetimeRebaseModeInRead, 
avroOptions.useStableIdForUnionType,
+      avroOptions.stableIdPrefixForUnionType, 
avroOptions.recursiveFieldMaxDepth)
+  }
+
+  /**
+   * If initializeAvroSerde is true, this method will create an Avro 
Serializer and Deserializer
+   * for a particular key and value schema.
+   */
+  private[sql] def getAvroSerde(
+      keySchema: StructType,
+      valSchema: StructType,
+      suffixKeySchema: Option[StructType] = None
+  ): Option[AvroEncoder] = {
+    if (initializeAvroSerde) {
+

Review Comment:
   nit: extra newline ?



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