You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "anishshri-db (via GitHub)" <gi...@apache.org> on 2023/11/22 19:35:23 UTC

[PR] [SPIP-IN-PROGRESS][DO-NO-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider [spark]

anishshri-db opened a new pull request, #43961:
URL: https://github.com/apache/spark/pull/43961

   ### What changes were proposed in this pull request?
   This change adds the base changes for the new `transformWithState` streaming operator used for enhanced arbitrary stateful processing. As part of this base change, we add support only for RocksDB state store provider and only without changelog checkpointing enabled. Part of this, we introduce support for column families for this provider. Currently, this needs to be enabled/used on a per operator basis and the new operator will be the only one to use this right away. We also add support for basic single value ValueState types used for interacting with state. We also add signatures for the rest of the `StatefulProcessor` based APIs required for the new operator. 
   
   ### Why are the changes needed?
   These changes are part of the work around adding new stateful streaming operator for arbitrary state mgmt that provides a bunch of new features listed in the SPIP JIRA here - https://issues.apache.org/jira/browse/SPARK-45939
   
   ### Does this PR introduce _any_ user-facing change?
   Yes
   This PR introduces a new operator that users can use in their Spark streaming queries
   
   ### How was this patch tested?
   Added bunch of changes to existing tests around RocksDBStateStoreSuite and others. Also added new suites for ValueState, new operator etc
   
   ### Was this patch authored or co-authored using generative AI tooling?
   No
   


-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1429468098


##########
sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessor.scala:
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.streaming
+
+import java.io.Serializable
+
+import org.apache.spark.annotation.{Evolving, Experimental}
+
+/**
+ * Represents the arbitrary stateful logic that needs to be provided by the user to perform
+ * stateful manipulations on keyed streams.
+ */
+@Experimental
+@Evolving
+trait StatefulProcessor[K, I, O] extends Serializable {
+
+  /**
+   * Function that will be invoked as the first method that allows for users to
+   * initialize all their state variables and perform other init actions before handling data.
+   * @param handle - reference to the statefulProcessorHandle that the user can use to perform
+   *               future actions

Review Comment:
   nit: future actions is too generic. Are we going to explain what they can do in StatefulProcessorHandle? If then probably better to refer to the classdoc e.g. `[[StatefulProcessorHandle]]`



##########
sql/api/src/main/scala/org/apache/spark/sql/streaming/TimerValues.scala:
##########


Review Comment:
   We could address this in later review phase, but since this is a public API, we may want to put a considerable effort to describe the method doc. 



##########
sql/api/src/main/scala/org/apache/spark/sql/streaming/ValueState.scala:
##########
@@ -0,0 +1,46 @@
+/*
+ * 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.streaming
+
+import java.io.Serializable
+
+import org.apache.spark.annotation.{Evolving, Experimental}
+
+@Experimental
+@Evolving
+/**
+ * Interface used for arbitrary stateful operations with the v2 API to capture
+ * single value state.
+ */
+trait ValueState[S] extends Serializable {
+
+  /** Whether state exists or not. */
+  def exists(): Boolean
+
+  /** Get the state value if it exists, or throw NoSuchElementException. */

Review Comment:
   We can leverage annotation to describe known exceptions being thrown via `@throws[<Exception>](description)`



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala:
##########
@@ -0,0 +1,76 @@
+/*
+ * 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
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.execution.streaming.state.StateStore
+import org.apache.spark.sql.streaming.{StatefulProcessorHandle, ValueState}
+
+/**

Review Comment:
   I don't get why this is needed as of now, but I'll look into the code more and check the necessity.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImpl.scala:
##########
@@ -0,0 +1,105 @@
+/*
+ * 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
+
+import java.io.Serializable
+
+import org.apache.commons.lang3.SerializationUtils
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.execution.streaming.state.StateStore
+import org.apache.spark.sql.streaming.ValueState
+import org.apache.spark.sql.types._
+
+/**
+ * Class that provides a concrete implementation for a single value state associated with state
+ * variables used in the streaming transformWithState operator.
+ * @param store - reference to the StateStore instance to be used for storing state
+ * @param stateName - name of logical state partition
+ * @tparam S - data type of object that will be stored
+ */
+class ValueStateImpl[S](
+    store: StateStore,
+    stateName: String) extends ValueState[S] with Logging{
+
+  private def encodeKey(): UnsafeRow = {
+    val keyOption = ImplicitKeyTracker.getImplicitKeyOption
+    if (!keyOption.isDefined) {
+      throw new UnsupportedOperationException("Implicit key not found for operation on" +
+        s"stateName=$stateName")
+    }
+
+    val schemaForKeyRow: StructType = new StructType().add("key", BinaryType)
+    val keyByteArr = SerializationUtils.serialize(keyOption.get.asInstanceOf[Serializable])

Review Comment:
   Please correct me if I'm mistaken, but this enforces the type of key to ensure both compatibility with Spark SQL and Serializable. Can we enforce either one, preferably Spark SQL?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateExec.scala:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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
+
+import java.util.concurrent.TimeUnit.NANOSECONDS
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, Expression, SortOrder, UnsafeRow}
+import org.apache.spark.sql.catalyst.plans.physical.Distribution
+import org.apache.spark.sql.execution._
+import org.apache.spark.sql.execution.streaming.state._
+import org.apache.spark.sql.streaming.{OutputMode, StatefulProcessor}
+import org.apache.spark.sql.types._
+import org.apache.spark.util.CompletionIterator
+
+/**
+ * Physical operator for executing `TransformWithState`
+ *
+ * @param statefulProcessor processor methods called on underlying data
+ * @param keyDeserializer used to extract the key object for each group.
+ * @param valueDeserializer used to extract the items in the iterator from an input row.
+ * @param groupingAttributes used to group the data
+ * @param dataAttributes used to read the data
+ * @param outputObjAttr Defines the output object
+ * @param batchTimestampMs processing timestamp of the current batch.
+ * @param eventTimeWatermarkForLateEvents event time watermark for filtering late events
+ * @param eventTimeWatermarkForEviction event time watermark for state eviction
+ * @param child the physical plan for the underlying data
+ */
+case class TransformWithStateExec(
+    keyDeserializer: Expression,
+    valueDeserializer: Expression,
+    groupingAttributes: Seq[Attribute],
+    dataAttributes: Seq[Attribute],
+    statefulProcessor: StatefulProcessor[Any, Any, Any],
+    outputMode: OutputMode,
+    outputObjAttr: Attribute,
+    stateInfo: Option[StatefulOperatorStateInfo],
+    batchTimestampMs: Option[Long],
+    eventTimeWatermarkForLateEvents: Option[Long],
+    eventTimeWatermarkForEviction: Option[Long],
+    child: SparkPlan)
+  extends UnaryExecNode
+    with StateStoreWriter
+    with WatermarkSupport
+    with ObjectProducerExec {
+
+  override def shortName: String = "transformWithStateExec"
+
+  override def shouldRunAnotherBatch(newInputWatermark: Long): Boolean = false
+
+  override protected def withNewChildInternal(
+    newChild: SparkPlan): TransformWithStateExec = copy(child = newChild)
+
+  override def keyExpressions: Seq[Attribute] = groupingAttributes
+
+  protected val schemaForKeyRow: StructType = new StructType().add("key", BinaryType)
+
+  protected val schemaForValueRow: StructType = new StructType().add("value", BinaryType)
+
+  override def requiredChildDistribution: Seq[Distribution] = {
+    StatefulOperatorPartitioning.getCompatibleDistribution(groupingAttributes,
+      getStateInfo, conf) ::
+      Nil
+  }
+
+  override def requiredChildOrdering: Seq[Seq[SortOrder]] = Seq(
+    groupingAttributes.map(SortOrder(_, Ascending)))
+
+  private def handleInputRows(keyRow: UnsafeRow, valueRowIter: Iterator[InternalRow]):
+    Iterator[InternalRow] = {
+    val getKeyObj =
+      ObjectOperator.deserializeRowToObject(keyDeserializer, groupingAttributes)
+
+    val getValueObj =
+      ObjectOperator.deserializeRowToObject(valueDeserializer, dataAttributes)
+
+    val getOutputRow = ObjectOperator.wrapObjectToRow(outputObjectType)
+
+    val keyObj = getKeyObj(keyRow)  // convert key to objects
+    ImplicitKeyTracker.setImplicitKey(keyObj)

Review Comment:
   I see what you're trying to do. It'd be nice if we can figure out better approach, but that's OK if we can't find as I see the necessity.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImpl.scala:
##########
@@ -0,0 +1,105 @@
+/*
+ * 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
+
+import java.io.Serializable
+
+import org.apache.commons.lang3.SerializationUtils
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.execution.streaming.state.StateStore
+import org.apache.spark.sql.streaming.ValueState
+import org.apache.spark.sql.types._
+
+/**
+ * Class that provides a concrete implementation for a single value state associated with state
+ * variables used in the streaming transformWithState operator.
+ * @param store - reference to the StateStore instance to be used for storing state
+ * @param stateName - name of logical state partition
+ * @tparam S - data type of object that will be stored
+ */
+class ValueStateImpl[S](
+    store: StateStore,
+    stateName: String) extends ValueState[S] with Logging{
+
+  private def encodeKey(): UnsafeRow = {

Review Comment:
   This approach seems to end up with calling this multiple times. Should we employ some cache for latest key to the encoded key?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImpl.scala:
##########
@@ -0,0 +1,105 @@
+/*
+ * 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
+
+import java.io.Serializable
+
+import org.apache.commons.lang3.SerializationUtils
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.execution.streaming.state.StateStore
+import org.apache.spark.sql.streaming.ValueState
+import org.apache.spark.sql.types._
+
+/**
+ * Class that provides a concrete implementation for a single value state associated with state
+ * variables used in the streaming transformWithState operator.
+ * @param store - reference to the StateStore instance to be used for storing state
+ * @param stateName - name of logical state partition
+ * @tparam S - data type of object that will be stored
+ */
+class ValueStateImpl[S](
+    store: StateStore,
+    stateName: String) extends ValueState[S] with Logging{
+
+  private def encodeKey(): UnsafeRow = {
+    val keyOption = ImplicitKeyTracker.getImplicitKeyOption
+    if (!keyOption.isDefined) {
+      throw new UnsupportedOperationException("Implicit key not found for operation on" +
+        s"stateName=$stateName")
+    }
+
+    val schemaForKeyRow: StructType = new StructType().add("key", BinaryType)
+    val keyByteArr = SerializationUtils.serialize(keyOption.get.asInstanceOf[Serializable])
+    val keyEncoder = UnsafeProjection.create(schemaForKeyRow)
+    val keyRow = keyEncoder(InternalRow(keyByteArr))
+    keyRow
+  }
+
+  private def encodeValue(value: S): UnsafeRow = {
+    val schemaForValueRow: StructType = new StructType().add("value", BinaryType)
+    val valueByteArr = SerializationUtils.serialize(value.asInstanceOf[Serializable])
+    val valueEncoder = UnsafeProjection.create(schemaForValueRow)
+    val valueRow = valueEncoder(InternalRow(valueByteArr))
+    valueRow
+  }
+
+  /** Function to check if state exists. Returns true if present and false otherwise */
+  override def exists(): Boolean = {
+    val retRow = store.get(encodeKey(), stateName)
+    if (retRow == null) {
+      false
+    } else {
+      true
+    }
+  }
+
+  /** Function to return Option of value if exists and None otherwise */
+  override def getOption(): Option[S] = {
+    if (exists()) {

Review Comment:
   Shall we refine getOption() and get() to reduce unnecessary store.get() calls? It's now called twice but it should have been done at once.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreChangelog.scala:
##########
@@ -109,18 +119,94 @@ class StateStoreChangelogWriter(
   }
 }
 
+/**
+ * Write changes to the key value state store instance to a changelog file.
+ * There are 2 types of records, put and delete.
+ * A put record is written as: | key length | key content | value length | value content |
+ * A delete record is written as: | key length | key content | -1 |
+ * Write an Int -1 to signal the end of file.
+ * The overall changelog format is: | put record | delete record | ... | put record | -1 |
+ */
+class StateStoreChangelogWriterV1(
+    fm: CheckpointFileManager,
+    file: Path,
+    compressionCodec: CompressionCodec)
+    extends StateStoreChangelogWriter(fm, file, compressionCodec) {
+
+  override def put(key: Array[Byte], value: Array[Byte]): Unit = {
+    assert(compressedStream != null)
+    compressedStream.writeInt(key.size)
+    compressedStream.write(key)
+    compressedStream.writeInt(value.size)
+    compressedStream.write(value)
+    size += 1
+  }
+
+  override def delete(key: Array[Byte]): Unit = {
+    assert(compressedStream != null)
+    compressedStream.writeInt(key.size)
+    compressedStream.write(key)
+    // -1 in the value field means record deletion.
+    compressedStream.writeInt(-1)
+    size += 1
+  }
+}
 
 /**
- * Read an iterator of change record from the changelog file.
- * A record is represented by ByteArrayPair(key: Array[Byte], value: Array[Byte])
- * A put record is returned as a ByteArrayPair(key, value)
- * A delete record is return as a ByteArrayPair(key, null)
+ * Write changes to the key value state store instance to a changelog file.
+ * There are 2 types of records, put and delete.
+ * A put record is written as: | record type | key length
+ *    | key content | value length | value content | col family name length | col family name | -1 |
+ * A delete record is written as: | record type | key length | key content | -1
+ *    | col family name length | col family name | -1 |
+ * Write an Int -1 to signal the end of file.
+ * The overall changelog format is: | put record | delete record | ... | put record | -1 |
+ */
+class StateStoreChangelogWriterV2(
+    fm: CheckpointFileManager,
+    file: Path,
+    compressionCodec: CompressionCodec)
+    extends StateStoreChangelogWriter(fm, file, compressionCodec) {
+
+  override def put(key: Array[Byte], value: Array[Byte], colFamilyName: String): Unit = {
+    assert(compressedStream != null)
+    compressedStream.writeInt(RecordType.PUT_RECORD.toString.getBytes.size)

Review Comment:
   It looks like already non-trivial size just to mark whether it is a put or delete. This is applied to every changes. Let's just use numeric code, or even boolean.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreChangelog.scala:
##########
@@ -33,12 +33,20 @@ import org.apache.spark.sql.execution.streaming.CheckpointFileManager.Cancellabl
 import org.apache.spark.util.NextIterator
 
 /**
- * Write changes to the key value state store instance to a changelog file.
- * There are 2 types of records, put and delete.
- * A put record is written as: | key length | key content | value length | value content |
- * A delete record is written as: | key length | key content | -1 |
- * Write an Int -1 to signal the end of file.
- * The overall changelog format is: | put record | delete record | ... | put record | -1 |
+ * Enum used to write record types to changelog files used with RocksDBStateStoreProvider.
+ */
+object RecordType extends Enumeration {
+  type RecordType = Value
+
+  val PUT_RECORD = Value("put_record")
+  val DELETE_RECORD = Value("delete_record")
+}
+
+/**
+ * Base class for state store changelog writer
+ * @param fm - checkpoint file manager used to manage streaming query checkpoint
+ * @param file - name of file to use to write changelog
+ * @param compressionCodec - compression method using for writing changelog file
  */
 class StateStoreChangelogWriter(

Review Comment:
   We can make this be abstract if we only use this as base implementation - abstract for put and delete.



##########
sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessorHandle.scala:
##########
@@ -0,0 +1,33 @@
+/*
+ * 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.streaming
+
+import java.io.Serializable
+
+import org.apache.spark.annotation.{Evolving, Experimental}
+
+/**
+ * Represents the operation handle provided to the stateful processor used in the
+ * arbitrary state API v2.
+ */
+@Experimental
+@Evolving
+trait StatefulProcessorHandle extends Serializable {
+
+  /** Function to create new or return existing single value state variable of given type */

Review Comment:
   Are there requirements against stateName? E.g. uniqueness within the query or entire cluster, etcetc. Are there any characters which won't work?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImpl.scala:
##########
@@ -0,0 +1,105 @@
+/*
+ * 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
+
+import java.io.Serializable
+
+import org.apache.commons.lang3.SerializationUtils
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.execution.streaming.state.StateStore
+import org.apache.spark.sql.streaming.ValueState
+import org.apache.spark.sql.types._
+
+/**
+ * Class that provides a concrete implementation for a single value state associated with state
+ * variables used in the streaming transformWithState operator.
+ * @param store - reference to the StateStore instance to be used for storing state
+ * @param stateName - name of logical state partition
+ * @tparam S - data type of object that will be stored
+ */
+class ValueStateImpl[S](
+    store: StateStore,
+    stateName: String) extends ValueState[S] with Logging{
+
+  private def encodeKey(): UnsafeRow = {
+    val keyOption = ImplicitKeyTracker.getImplicitKeyOption
+    if (!keyOption.isDefined) {
+      throw new UnsupportedOperationException("Implicit key not found for operation on" +
+        s"stateName=$stateName")
+    }
+
+    val schemaForKeyRow: StructType = new StructType().add("key", BinaryType)
+    val keyByteArr = SerializationUtils.serialize(keyOption.get.asInstanceOf[Serializable])

Review Comment:
   You can rely on binary format of UnsafeRow, as we have been relying on this for existing state store. This is arguably an issue if we want to change UnsafeRow, but it's done already so at least isn't worse.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala:
##########
@@ -0,0 +1,76 @@
+/*
+ * 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
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.execution.streaming.state.StateStore
+import org.apache.spark.sql.streaming.{StatefulProcessorHandle, ValueState}
+
+/**
+ * Object used to assign/retrieve/remove grouping key passed implicitly for various state
+ * manipulation actions using the store handle.
+ */
+object ImplicitKeyTracker {
+  val implicitKey: InheritableThreadLocal[Any] = new InheritableThreadLocal[Any]
+
+  def getImplicitKeyOption: Option[Any] = Option(implicitKey.get())
+
+  def setImplicitKey(key: Any): Unit = implicitKey.set(key)
+
+  def removeImplicitKey(): Unit = implicitKey.remove()
+}
+
+/**
+ * Enum used to track valid states for the StatefulProcessorHandle
+ */
+object StatefulProcessorHandleState extends Enumeration {
+  type StatefulProcessorHandleState = Value
+  val CREATED, INITIALIZED, DATA_PROCESSED, CLOSED = Value
+}
+
+/**
+ * Class that provides a concrete implementation of a StatefulProcessorHandle. Note that we keep
+ * track of valid transitions as various functions are invoked to track object lifecycle.
+ * @param store - instance of state store
+ */
+class StatefulProcessorHandleImpl(store: StateStore)
+  extends StatefulProcessorHandle
+  with Logging {
+  import StatefulProcessorHandleState._
+
+  private var currState: StatefulProcessorHandleState = CREATED
+
+  private def verify(condition: => Boolean, msg: String): Unit = {
+    if (!condition) {
+      throw new IllegalStateException(msg)
+    }
+  }
+
+  def setHandleState(newState: StatefulProcessorHandleState): Unit = {
+    currState = newState
+  }
+
+  def getHandleState: StatefulProcessorHandleState = currState
+
+  override def getValueState[T](stateName: String): ValueState[T] = {
+    verify(currState == CREATED, s"Cannot create state variable with name=$stateName after " +

Review Comment:
   Shall we provide this (when user should define the state) as contract to users? I guess add contract to interface method doc would work.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateExec.scala:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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
+
+import java.util.concurrent.TimeUnit.NANOSECONDS
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, Expression, SortOrder, UnsafeRow}
+import org.apache.spark.sql.catalyst.plans.physical.Distribution
+import org.apache.spark.sql.execution._
+import org.apache.spark.sql.execution.streaming.state._
+import org.apache.spark.sql.streaming.{OutputMode, StatefulProcessor}
+import org.apache.spark.sql.types._
+import org.apache.spark.util.CompletionIterator
+
+/**
+ * Physical operator for executing `TransformWithState`
+ *
+ * @param statefulProcessor processor methods called on underlying data
+ * @param keyDeserializer used to extract the key object for each group.
+ * @param valueDeserializer used to extract the items in the iterator from an input row.
+ * @param groupingAttributes used to group the data
+ * @param dataAttributes used to read the data
+ * @param outputObjAttr Defines the output object
+ * @param batchTimestampMs processing timestamp of the current batch.
+ * @param eventTimeWatermarkForLateEvents event time watermark for filtering late events
+ * @param eventTimeWatermarkForEviction event time watermark for state eviction
+ * @param child the physical plan for the underlying data
+ */
+case class TransformWithStateExec(
+    keyDeserializer: Expression,
+    valueDeserializer: Expression,
+    groupingAttributes: Seq[Attribute],
+    dataAttributes: Seq[Attribute],
+    statefulProcessor: StatefulProcessor[Any, Any, Any],
+    outputMode: OutputMode,
+    outputObjAttr: Attribute,
+    stateInfo: Option[StatefulOperatorStateInfo],
+    batchTimestampMs: Option[Long],
+    eventTimeWatermarkForLateEvents: Option[Long],
+    eventTimeWatermarkForEviction: Option[Long],
+    child: SparkPlan)
+  extends UnaryExecNode
+    with StateStoreWriter
+    with WatermarkSupport
+    with ObjectProducerExec {
+
+  override def shortName: String = "transformWithStateExec"
+
+  override def shouldRunAnotherBatch(newInputWatermark: Long): Boolean = false

Review Comment:
   Are we going to disallow no-data batch for new API?



##########
sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessor.scala:
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.streaming
+
+import java.io.Serializable
+
+import org.apache.spark.annotation.{Evolving, Experimental}
+
+/**
+ * Represents the arbitrary stateful logic that needs to be provided by the user to perform
+ * stateful manipulations on keyed streams.
+ */
+@Experimental
+@Evolving
+trait StatefulProcessor[K, I, O] extends Serializable {
+
+  /**
+   * Function that will be invoked as the first method that allows for users to
+   * initialize all their state variables and perform other init actions before handling data.
+   * @param handle - reference to the statefulProcessorHandle that the user can use to perform
+   *               future actions
+   * @param outputMode - output mode for the stateful processor
+   */
+  def init(
+    handle: StatefulProcessorHandle,
+    outputMode: OutputMode): Unit
+
+  /**
+   * Function that will allow users to interact with input data rows along with the grouping key
+   * and current timer values and optionally provide output rows.
+   * @param key - grouping key
+   * @param inputRows - iterator of input rows associated with grouping key
+   * @param timerValues - instance of TimerValues that provides access to current processing/event
+   *                    time if available
+   * @return - Zero or more output rows
+   */
+  def handleInputRows(

Review Comment:
   As I stated in SPIP doc, I'd like to see the ability of chaining as one of MVP. We'd need to have an origin input row for each output row to tag the event time timestamp, and then we should either 1) require users to provide the origin input row per each output row, or 2) change the method signature to handle an input row, so that we can implicitly associate the origin input row to the output rows in the same method call.
   
   Well known approach is 2), but I'm also open to 1) if handling multiple values are considered as great UX advantage.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreChangelog.scala:
##########
@@ -133,11 +219,30 @@ class StateStoreChangelogReader(
     case f: FileNotFoundException =>
       throw QueryExecutionErrors.failedToReadStreamingStateFileError(fileToRead, f)
   }
-  private val input: DataInputStream = decompressStream(sourceStream)
+  protected val input: DataInputStream = decompressStream(sourceStream)
 
   def close(): Unit = { if (input != null) input.close() }
 
-  override def getNext(): (Array[Byte], Array[Byte]) = {
+  override def getNext(): (RecordType.Value, Array[Byte], Array[Byte], String) = {
+    throw new UnsupportedOperationException("Iterator operations not supported on base " +
+      "changelog reader implementation")
+  }
+}
+
+/**
+ * Read an iterator of change record from the changelog file.
+ * A record is represented by ByteArrayPair(recordType: RecordType.Value,
+ *  key: Array[Byte], value: Array[Byte], colFamilyName: String)
+ * A put record is returned as a ByteArrayPair(recordType, key, value, colFamilyName)
+ * A delete record is return as a ByteArrayPair(recordType, key, null, colFamilyName)
+ */
+class StateStoreChangelogReaderV1(
+    fm: CheckpointFileManager,
+    fileToRead: Path,
+    compressionCodec: CompressionCodec) extends StateStoreChangelogReader(fm, fileToRead,

Review Comment:
   nit: looks like you can pull extends part down one line and make everything one liner.



##########
sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala:
##########
@@ -642,6 +642,37 @@ class KeyValueGroupedDataset[K, V] private[sql](
       outputMode, timeoutConf, initialState)(f)(stateEncoder, outputEncoder)
   }
 
+  /**
+   * (Scala-specific)
+   * Invokes methods defined in the stateful processor used in arbitrary state API v2.
+   * We allow the user to act on per-group set of input rows along with keyed state and the
+   * user can choose to output/return 0 or more rows.
+   * For a static/batch dataset, this operator is not supported and will throw an exception.
+   * For a streaming dataframe, we will repeatedly invoke the interface methods for new rows
+   * in each trigger and the user's state/state variables will be stored persistently across
+   * invocations.
+   *
+   * @tparam U The type of the output objects. Must be encodable to Spark SQL types.
+   * @param statefulProcessor Instance of statefulProcessor whose functions will be invoked by the
+   *                          operator.
+   * @param outputMode The output mode of the stateful processor. Defaults to APPEND mode.
+   *
+   */
+  def transformWithState[U: Encoder]
+    (statefulProcessor: StatefulProcessor[K, V, U],

Review Comment:
   nit: place `(` in above line and 4 spaces for params



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala:
##########
@@ -569,6 +569,41 @@ case class FlatMapGroupsWithState(
     copy(child = newLeft, initialState = newRight)
 }
 
+object TransformWithState {
+  def apply[K: Encoder, V: Encoder, U: Encoder](
+    groupingAttributes: Seq[Attribute],

Review Comment:
   nit: 2 more spaces for params



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TimerValuesImpl.scala:
##########
@@ -0,0 +1,44 @@
+/*
+ * 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
+
+import org.apache.spark.sql.streaming.TimerValues
+
+/**
+ * Class that provides a concrete implementation for TimerValues used for fetching
+ * processing time and event time (watermark).
+ * @param currentProcessingTimeOpt - option to current processing time
+ * @param currentWatermarkOpt - option to current watermark
+ */
+class TimerValuesImpl(
+    currentProcessingTimeOpt: Option[Long],
+    currentWatermarkOpt: Option[Long]) extends TimerValues {
+
+  // Return available processing time or -1 otherwise
+  override def getCurrentProcessingTimeInMs(): Long = if (currentProcessingTimeOpt.isDefined) {

Review Comment:
   nit: `currentProcessingTimeOpt.getOrElse(-1L)`



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateExec.scala:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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
+
+import java.util.concurrent.TimeUnit.NANOSECONDS
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, Expression, SortOrder, UnsafeRow}
+import org.apache.spark.sql.catalyst.plans.physical.Distribution
+import org.apache.spark.sql.execution._
+import org.apache.spark.sql.execution.streaming.state._
+import org.apache.spark.sql.streaming.{OutputMode, StatefulProcessor}
+import org.apache.spark.sql.types._
+import org.apache.spark.util.CompletionIterator
+
+/**
+ * Physical operator for executing `TransformWithState`
+ *
+ * @param statefulProcessor processor methods called on underlying data
+ * @param keyDeserializer used to extract the key object for each group.
+ * @param valueDeserializer used to extract the items in the iterator from an input row.
+ * @param groupingAttributes used to group the data
+ * @param dataAttributes used to read the data
+ * @param outputObjAttr Defines the output object
+ * @param batchTimestampMs processing timestamp of the current batch.
+ * @param eventTimeWatermarkForLateEvents event time watermark for filtering late events
+ * @param eventTimeWatermarkForEviction event time watermark for state eviction
+ * @param child the physical plan for the underlying data
+ */
+case class TransformWithStateExec(
+    keyDeserializer: Expression,
+    valueDeserializer: Expression,
+    groupingAttributes: Seq[Attribute],
+    dataAttributes: Seq[Attribute],
+    statefulProcessor: StatefulProcessor[Any, Any, Any],
+    outputMode: OutputMode,
+    outputObjAttr: Attribute,
+    stateInfo: Option[StatefulOperatorStateInfo],
+    batchTimestampMs: Option[Long],
+    eventTimeWatermarkForLateEvents: Option[Long],
+    eventTimeWatermarkForEviction: Option[Long],
+    child: SparkPlan)
+  extends UnaryExecNode
+    with StateStoreWriter

Review Comment:
   nit: 2 spaces for extends/with



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TimerValuesImpl.scala:
##########
@@ -0,0 +1,44 @@
+/*
+ * 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
+
+import org.apache.spark.sql.streaming.TimerValues
+
+/**
+ * Class that provides a concrete implementation for TimerValues used for fetching
+ * processing time and event time (watermark).
+ * @param currentProcessingTimeOpt - option to current processing time
+ * @param currentWatermarkOpt - option to current watermark
+ */
+class TimerValuesImpl(
+    currentProcessingTimeOpt: Option[Long],
+    currentWatermarkOpt: Option[Long]) extends TimerValues {
+
+  // Return available processing time or -1 otherwise
+  override def getCurrentProcessingTimeInMs(): Long = if (currentProcessingTimeOpt.isDefined) {
+    currentProcessingTimeOpt.get
+  } else {
+    -1L
+  }
+
+  // Return available watermark or -1 otherwise
+  override def getCurrentWatermarkInMs(): Long = if (currentWatermarkOpt.isDefined) {

Review Comment:
   ditto



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImpl.scala:
##########
@@ -0,0 +1,105 @@
+/*
+ * 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
+
+import java.io.Serializable
+
+import org.apache.commons.lang3.SerializationUtils
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.execution.streaming.state.StateStore
+import org.apache.spark.sql.streaming.ValueState
+import org.apache.spark.sql.types._
+
+/**
+ * Class that provides a concrete implementation for a single value state associated with state
+ * variables used in the streaming transformWithState operator.
+ * @param store - reference to the StateStore instance to be used for storing state
+ * @param stateName - name of logical state partition
+ * @tparam S - data type of object that will be stored
+ */
+class ValueStateImpl[S](
+    store: StateStore,
+    stateName: String) extends ValueState[S] with Logging{

Review Comment:
   nit: space between g and {



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateExec.scala:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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
+
+import java.util.concurrent.TimeUnit.NANOSECONDS
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, Expression, SortOrder, UnsafeRow}
+import org.apache.spark.sql.catalyst.plans.physical.Distribution
+import org.apache.spark.sql.execution._
+import org.apache.spark.sql.execution.streaming.state._
+import org.apache.spark.sql.streaming.{OutputMode, StatefulProcessor}
+import org.apache.spark.sql.types._
+import org.apache.spark.util.CompletionIterator
+
+/**
+ * Physical operator for executing `TransformWithState`
+ *
+ * @param statefulProcessor processor methods called on underlying data
+ * @param keyDeserializer used to extract the key object for each group.
+ * @param valueDeserializer used to extract the items in the iterator from an input row.
+ * @param groupingAttributes used to group the data
+ * @param dataAttributes used to read the data
+ * @param outputObjAttr Defines the output object
+ * @param batchTimestampMs processing timestamp of the current batch.
+ * @param eventTimeWatermarkForLateEvents event time watermark for filtering late events
+ * @param eventTimeWatermarkForEviction event time watermark for state eviction
+ * @param child the physical plan for the underlying data
+ */
+case class TransformWithStateExec(
+    keyDeserializer: Expression,
+    valueDeserializer: Expression,
+    groupingAttributes: Seq[Attribute],
+    dataAttributes: Seq[Attribute],
+    statefulProcessor: StatefulProcessor[Any, Any, Any],
+    outputMode: OutputMode,
+    outputObjAttr: Attribute,
+    stateInfo: Option[StatefulOperatorStateInfo],
+    batchTimestampMs: Option[Long],
+    eventTimeWatermarkForLateEvents: Option[Long],
+    eventTimeWatermarkForEviction: Option[Long],
+    child: SparkPlan)
+  extends UnaryExecNode
+    with StateStoreWriter
+    with WatermarkSupport
+    with ObjectProducerExec {
+
+  override def shortName: String = "transformWithStateExec"
+
+  override def shouldRunAnotherBatch(newInputWatermark: Long): Boolean = false
+
+  override protected def withNewChildInternal(
+    newChild: SparkPlan): TransformWithStateExec = copy(child = newChild)
+
+  override def keyExpressions: Seq[Attribute] = groupingAttributes
+
+  protected val schemaForKeyRow: StructType = new StructType().add("key", BinaryType)
+
+  protected val schemaForValueRow: StructType = new StructType().add("value", BinaryType)
+
+  override def requiredChildDistribution: Seq[Distribution] = {
+    StatefulOperatorPartitioning.getCompatibleDistribution(groupingAttributes,
+      getStateInfo, conf) ::
+      Nil
+  }
+
+  override def requiredChildOrdering: Seq[Seq[SortOrder]] = Seq(
+    groupingAttributes.map(SortOrder(_, Ascending)))
+
+  private def handleInputRows(keyRow: UnsafeRow, valueRowIter: Iterator[InternalRow]):
+    Iterator[InternalRow] = {
+    val getKeyObj =
+      ObjectOperator.deserializeRowToObject(keyDeserializer, groupingAttributes)
+
+    val getValueObj =
+      ObjectOperator.deserializeRowToObject(valueDeserializer, dataAttributes)
+
+    val getOutputRow = ObjectOperator.wrapObjectToRow(outputObjectType)
+
+    val keyObj = getKeyObj(keyRow)  // convert key to objects
+    ImplicitKeyTracker.setImplicitKey(keyObj)
+    val valueObjIter = valueRowIter.map(getValueObj.apply)
+    val mappedIterator = statefulProcessor.handleInputRows(keyObj, valueObjIter,
+      new TimerValuesImpl(batchTimestampMs, eventTimeWatermarkForLateEvents)).map { obj =>
+        getOutputRow(obj)
+    }
+    ImplicitKeyTracker.removeImplicitKey()
+    mappedIterator
+  }
+
+  private def processNewData(dataIter: Iterator[InternalRow]): Iterator[InternalRow] = {
+    val groupedIter = GroupedIterator(dataIter, groupingAttributes, child.output)
+    groupedIter.flatMap { case (keyRow, valueRowIter) =>
+      val keyUnsafeRow = keyRow.asInstanceOf[UnsafeRow]
+      handleInputRows(keyUnsafeRow, valueRowIter)
+    }
+  }
+
+  private def processDataWithPartition(
+      iter: Iterator[InternalRow],
+      store: StateStore,
+      processorHandle: StatefulProcessorHandleImpl):
+    CompletionIterator[InternalRow, Iterator[InternalRow]] = {
+    val allUpdatesTimeMs = longMetric("allUpdatesTimeMs")
+    val commitTimeMs = longMetric("commitTimeMs")
+
+    val currentTimeNs = System.nanoTime
+    val updatesStartTimeNs = currentTimeNs
+
+    // If timeout is based on event time, then filter late data based on watermark
+    val filteredIter = watermarkPredicateForDataForLateEvents match {
+      case Some(predicate) =>
+        applyRemovingRowsOlderThanWatermark(iter, predicate)
+      case _ =>
+        iter
+    }
+
+    val outputIterator = processNewData(filteredIter)
+    processorHandle.setHandleState(StatefulProcessorHandleState.DATA_PROCESSED)
+    // Return an iterator of all the rows generated by all the keys, such that when fully
+    // consumed, all the state updates will be committed by the state store
+    CompletionIterator[InternalRow, Iterator[InternalRow]](outputIterator, {
+      // Note: Due to the iterator lazy execution, this metric also captures the time taken
+      // by the upstream (consumer) operators in addition to the processing in this operator.
+      allUpdatesTimeMs += NANOSECONDS.toMillis(System.nanoTime - updatesStartTimeNs)
+      commitTimeMs += timeTakenMs {
+        store.commit()
+      }
+      setStoreMetrics(store)
+      setOperatorMetrics()
+      statefulProcessor.close()
+      processorHandle.setHandleState(StatefulProcessorHandleState.CLOSED)
+    })
+  }
+
+  override protected def doExecute(): RDD[InternalRow] = {
+    metrics

Review Comment:
   nit: Is this to enforce initialization? If then please leave a code comment. Otherwise remove this.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala:
##########
@@ -118,6 +121,10 @@ class RocksDB(
     dbOptions.setWriteBufferManager(writeBufferManager)
   }
 
+  // Maintain mapping of column family name to handle
+  @volatile private var colFamilyNameToHandleMap =

Review Comment:
   Do we ever change the reference? I'm not sure you need `@volatile` and `var` here. If you meant to get synchronization on accessing map, either you need synchronization block (for synchronizing multiple accesses) or ConcurrentHashMap from Java collection (for single access including CAS). 



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala:
##########
@@ -219,49 +232,122 @@ class RocksDB(
     loadedVersion = endVersion
   }
 
+  private def checkColFamilyExists(colFamilyName: String): Boolean = {
+    colFamilyNameToHandleMap.contains(colFamilyName)
+  }
+
+  /**
+   * Create RocksDB column family, if not created already
+   */
+  def createColFamilyIfAbsent(colFamilyName: String): Unit = {
+    if (colFamilyName == StateStore.DEFAULT_COL_FAMILY_NAME) {
+      throw new UnsupportedOperationException("Failed to create column family with reserved " +
+        s"name=$colFamilyName")
+    }
+
+    if (!checkColFamilyExists(colFamilyName)) {
+      assert(db != null)
+      val descriptor = new ColumnFamilyDescriptor(colFamilyName.getBytes, columnFamilyOptions)
+      val handle = db.createColumnFamily(descriptor)
+      colFamilyNameToHandleMap(handle.getName.map(_.toChar).mkString) = handle
+    }
+  }
+
   /**
    * Get the value for the given key if present, or null.
    * @note This will return the last written value even if it was uncommitted.
    */
-  def get(key: Array[Byte]): Array[Byte] = {
-    db.get(readOptions, key)
+  def get(key: Array[Byte],

Review Comment:
   nit: if it does not fit to the one liner, it is clearer to follow multi-lines style, even with 2 lines. pull key to below line, 4 spaces for params in overall



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala:
##########
@@ -219,49 +232,122 @@ class RocksDB(
     loadedVersion = endVersion
   }
 
+  private def checkColFamilyExists(colFamilyName: String): Boolean = {
+    colFamilyNameToHandleMap.contains(colFamilyName)
+  }
+
+  /**
+   * Create RocksDB column family, if not created already
+   */
+  def createColFamilyIfAbsent(colFamilyName: String): Unit = {
+    if (colFamilyName == StateStore.DEFAULT_COL_FAMILY_NAME) {
+      throw new UnsupportedOperationException("Failed to create column family with reserved " +
+        s"name=$colFamilyName")
+    }
+
+    if (!checkColFamilyExists(colFamilyName)) {
+      assert(db != null)
+      val descriptor = new ColumnFamilyDescriptor(colFamilyName.getBytes, columnFamilyOptions)
+      val handle = db.createColumnFamily(descriptor)
+      colFamilyNameToHandleMap(handle.getName.map(_.toChar).mkString) = handle
+    }
+  }
+
   /**
    * Get the value for the given key if present, or null.
    * @note This will return the last written value even if it was uncommitted.
    */
-  def get(key: Array[Byte]): Array[Byte] = {
-    db.get(readOptions, key)
+  def get(key: Array[Byte],
+    colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Array[Byte] = {
+    if (useColumnFamilies) {
+      // if col family is not created, throw an exception
+      if (!checkColFamilyExists(colFamilyName)) {
+        throw new RuntimeException(s"Column family with name=$colFamilyName does not exist")
+      }
+      db.get(colFamilyNameToHandleMap(colFamilyName), readOptions, key)
+    } else {
+      db.get(readOptions, key)
+    }
   }
 
   /**
    * Put the given value for the given key.
    * @note This update is not committed to disk until commit() is called.
    */
-  def put(key: Array[Byte], value: Array[Byte]): Unit = {
-    if (conf.trackTotalNumberOfRows) {
-      val oldValue = db.get(readOptions, key)
-      if (oldValue == null) {
-        numKeysOnWritingVersion += 1
+  def put(key: Array[Byte], value: Array[Byte],

Review Comment:
   ditto



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImpl.scala:
##########
@@ -0,0 +1,105 @@
+/*
+ * 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
+
+import java.io.Serializable
+
+import org.apache.commons.lang3.SerializationUtils
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.execution.streaming.state.StateStore
+import org.apache.spark.sql.streaming.ValueState
+import org.apache.spark.sql.types._
+
+/**
+ * Class that provides a concrete implementation for a single value state associated with state
+ * variables used in the streaming transformWithState operator.
+ * @param store - reference to the StateStore instance to be used for storing state
+ * @param stateName - name of logical state partition
+ * @tparam S - data type of object that will be stored
+ */
+class ValueStateImpl[S](
+    store: StateStore,
+    stateName: String) extends ValueState[S] with Logging{
+
+  private def encodeKey(): UnsafeRow = {
+    val keyOption = ImplicitKeyTracker.getImplicitKeyOption
+    if (!keyOption.isDefined) {
+      throw new UnsupportedOperationException("Implicit key not found for operation on" +
+        s"stateName=$stateName")
+    }
+
+    val schemaForKeyRow: StructType = new StructType().add("key", BinaryType)
+    val keyByteArr = SerializationUtils.serialize(keyOption.get.asInstanceOf[Serializable])
+    val keyEncoder = UnsafeProjection.create(schemaForKeyRow)
+    val keyRow = keyEncoder(InternalRow(keyByteArr))
+    keyRow
+  }
+
+  private def encodeValue(value: S): UnsafeRow = {
+    val schemaForValueRow: StructType = new StructType().add("value", BinaryType)
+    val valueByteArr = SerializationUtils.serialize(value.asInstanceOf[Serializable])
+    val valueEncoder = UnsafeProjection.create(schemaForValueRow)
+    val valueRow = valueEncoder(InternalRow(valueByteArr))
+    valueRow
+  }
+
+  /** Function to check if state exists. Returns true if present and false otherwise */
+  override def exists(): Boolean = {
+    val retRow = store.get(encodeKey(), stateName)
+    if (retRow == null) {

Review Comment:
   nit: `retRow != null` one liner



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala:
##########
@@ -204,12 +211,18 @@ class RocksDB(
     for (v <- loadedVersion + 1 to endVersion) {
       var changelogReader: StateStoreChangelogReader = null
       try {
-        changelogReader = fileManager.getChangelogReader(v)
-        changelogReader.foreach { case (key, value) =>
-          if (value != null) {
-            put(key, value)
-          } else {
-            remove(key)
+        changelogReader = fileManager.getChangelogReader(v, useColumnFamilies)
+        changelogReader.foreach { case (recordType, key, value, colFamilyName) =>

Review Comment:
   Will we allow putting `null` in value in state store? Just to check we need additional recordType or not.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala:
##########
@@ -604,17 +728,57 @@ class RocksDB(
   }
 
   private def getDBProperty(property: String): Long = {
-    db.getProperty(property).toLong
+    if (useColumnFamilies) {
+      // get cumulative sum across all available column families
+      assert(!colFamilyNameToHandleMap.isEmpty)
+      colFamilyNameToHandleMap
+        .values
+        .map(handle => db.getProperty(handle, property).toLong)
+        .sum
+    } else {
+      db.getProperty(property).toLong
+    }
   }
 
   private def openDB(): Unit = {
     assert(db == null)
-    db = NativeRocksDB.open(dbOptions, workingDir.toString)
-    logInfo(s"Opened DB with conf ${conf}")
+    if (useColumnFamilies) {
+      val colFamilies = NativeRocksDB.listColumnFamilies(dbOptions, workingDir.toString)
+
+      var colFamilyDescriptors: Seq[ColumnFamilyDescriptor] = Seq.empty[ColumnFamilyDescriptor]
+      // populate the list of available col family descriptors
+      colFamilies.asScala.toList.foreach(family => {
+        val descriptor = new ColumnFamilyDescriptor(family, columnFamilyOptions)
+        colFamilyDescriptors = colFamilyDescriptors :+ descriptor
+      })
+
+      if (colFamilyDescriptors.isEmpty) {
+        colFamilyDescriptors = colFamilyDescriptors :+
+          new ColumnFamilyDescriptor(NativeRocksDB.DEFAULT_COLUMN_FAMILY, columnFamilyOptions)
+      }
+
+      val colFamilyHandles = new java.util.ArrayList[ColumnFamilyHandle]()
+      db = NativeRocksDB.open(new DBOptions(dbOptions), workingDir.toString,
+        colFamilyDescriptors.asJava, colFamilyHandles)
+
+      // Store the mapping of names to handles in the internal map
+      colFamilyHandles.asScala.toList.map(

Review Comment:
   nit: foreach? also `{ handle =>`. you can save a couple of lines.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala:
##########
@@ -604,17 +728,57 @@ class RocksDB(
   }
 
   private def getDBProperty(property: String): Long = {
-    db.getProperty(property).toLong
+    if (useColumnFamilies) {
+      // get cumulative sum across all available column families
+      assert(!colFamilyNameToHandleMap.isEmpty)
+      colFamilyNameToHandleMap
+        .values
+        .map(handle => db.getProperty(handle, property).toLong)
+        .sum
+    } else {
+      db.getProperty(property).toLong
+    }
   }
 
   private def openDB(): Unit = {
     assert(db == null)
-    db = NativeRocksDB.open(dbOptions, workingDir.toString)
-    logInfo(s"Opened DB with conf ${conf}")
+    if (useColumnFamilies) {
+      val colFamilies = NativeRocksDB.listColumnFamilies(dbOptions, workingDir.toString)
+
+      var colFamilyDescriptors: Seq[ColumnFamilyDescriptor] = Seq.empty[ColumnFamilyDescriptor]
+      // populate the list of available col family descriptors
+      colFamilies.asScala.toList.foreach(family => {

Review Comment:
   nit: `{ family =>` should be sufficient, no need to do `(family => {`



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala:
##########
@@ -149,21 +149,31 @@ class RocksDBFileManager(
 
   @volatile private var rootDirChecked: Boolean = false
 
-  def getChangeLogWriter(version: Long): StateStoreChangelogWriter = {
+  def getChangeLogWriter(version: Long,

Review Comment:
   ditto about style



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala:
##########
@@ -219,49 +232,122 @@ class RocksDB(
     loadedVersion = endVersion
   }
 
+  private def checkColFamilyExists(colFamilyName: String): Boolean = {
+    colFamilyNameToHandleMap.contains(colFamilyName)
+  }
+
+  /**
+   * Create RocksDB column family, if not created already
+   */
+  def createColFamilyIfAbsent(colFamilyName: String): Unit = {
+    if (colFamilyName == StateStore.DEFAULT_COL_FAMILY_NAME) {
+      throw new UnsupportedOperationException("Failed to create column family with reserved " +
+        s"name=$colFamilyName")
+    }
+
+    if (!checkColFamilyExists(colFamilyName)) {
+      assert(db != null)
+      val descriptor = new ColumnFamilyDescriptor(colFamilyName.getBytes, columnFamilyOptions)
+      val handle = db.createColumnFamily(descriptor)
+      colFamilyNameToHandleMap(handle.getName.map(_.toChar).mkString) = handle
+    }
+  }
+
   /**
    * Get the value for the given key if present, or null.
    * @note This will return the last written value even if it was uncommitted.
    */
-  def get(key: Array[Byte]): Array[Byte] = {
-    db.get(readOptions, key)
+  def get(key: Array[Byte],
+    colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Array[Byte] = {
+    if (useColumnFamilies) {
+      // if col family is not created, throw an exception
+      if (!checkColFamilyExists(colFamilyName)) {
+        throw new RuntimeException(s"Column family with name=$colFamilyName does not exist")
+      }
+      db.get(colFamilyNameToHandleMap(colFamilyName), readOptions, key)
+    } else {
+      db.get(readOptions, key)
+    }
   }
 
   /**
    * Put the given value for the given key.
    * @note This update is not committed to disk until commit() is called.
    */
-  def put(key: Array[Byte], value: Array[Byte]): Unit = {
-    if (conf.trackTotalNumberOfRows) {
-      val oldValue = db.get(readOptions, key)
-      if (oldValue == null) {
-        numKeysOnWritingVersion += 1
+  def put(key: Array[Byte], value: Array[Byte],
+    colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Unit = {
+    if (useColumnFamilies) {
+      // if col family is not created, throw an exception
+      if (!checkColFamilyExists(colFamilyName)) {
+        throw new RuntimeException(s"Column family with name=$colFamilyName does not exist")
+      }
+
+      if (conf.trackTotalNumberOfRows) {
+        val oldValue = db.get(colFamilyNameToHandleMap(colFamilyName), readOptions, key)
+        if (oldValue == null) {
+          numKeysOnWritingVersion += 1
+        }
+      }
+      db.put(colFamilyNameToHandleMap(colFamilyName), writeOptions, key, value)
+      changelogWriter.foreach(_.put(key, value, colFamilyName))
+    } else {
+      if (conf.trackTotalNumberOfRows) {
+        val oldValue = db.get(readOptions, key)
+        if (oldValue == null) {
+          numKeysOnWritingVersion += 1
+        }
       }
+      db.put(writeOptions, key, value)
+      changelogWriter.foreach(_.put(key, value))
     }
-    db.put(writeOptions, key, value)
-    changelogWriter.foreach(_.put(key, value))
   }
 
   /**
    * Remove the key if present.
    * @note This update is not committed to disk until commit() is called.
    */
-  def remove(key: Array[Byte]): Unit = {
-    if (conf.trackTotalNumberOfRows) {
-      val value = db.get(readOptions, key)
-      if (value != null) {
-        numKeysOnWritingVersion -= 1
+  def remove(key: Array[Byte], colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Unit = {

Review Comment:
   ditto



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreChangelog.scala:
##########
@@ -153,12 +258,70 @@ class StateStoreChangelogReader(
       val valueSize = input.readInt()
       if (valueSize < 0) {
         // A deletion record
-        (keyBuffer, null)
+        (RecordType.DELETE_RECORD, keyBuffer, null, StateStore.DEFAULT_COL_FAMILY_NAME)
       } else {
         val valueBuffer = new Array[Byte](valueSize)
         ByteStreams.readFully(input, valueBuffer, 0, valueSize)
         // A put record.
-        (keyBuffer, valueBuffer)
+        (RecordType.PUT_RECORD, keyBuffer, valueBuffer, StateStore.DEFAULT_COL_FAMILY_NAME)
+      }
+    }
+  }
+}
+
+/**
+ * Read an iterator of change record from the changelog file.
+ * A record is represented by ByteArrayPair(recordType: RecordType.Value,
+ *  key: Array[Byte], value: Array[Byte], colFamilyName: String)
+ * A put record is returned as a ByteArrayPair(recordType, key, value, colFamilyName)
+ * A delete record is return as a ByteArrayPair(recordType, key, null, colFamilyName)
+ */
+class StateStoreChangelogReaderV2(
+    fm: CheckpointFileManager,
+    fileToRead: Path,
+    compressionCodec: CompressionCodec) extends StateStoreChangelogReader(fm, fileToRead,
+      compressionCodec) {
+
+  private def parseBuffer(input: DataInputStream): Array[Byte] = {
+    val blockSize = input.readInt()
+    val blockBuffer = new Array[Byte](blockSize)
+    ByteStreams.readFully(input, blockBuffer, 0, blockSize)
+    blockBuffer
+  }
+
+  override def getNext(): (RecordType.Value, Array[Byte], Array[Byte], String) = {
+    val recordTypeSize = input.readInt()
+    // A -1 key size mean end of file.
+    if (recordTypeSize == -1) {
+      finished = true
+      null
+    } else if (recordTypeSize < 0) {
+      throw new IOException(
+        s"Error reading streaming state file $fileToRead: " +
+        s"record type size cannot be $recordTypeSize")
+    } else {
+      val recordTypeBuffer = new Array[Byte](recordTypeSize)

Review Comment:
   This proves that the overhead is significant.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreChangelog.scala:
##########
@@ -109,18 +119,94 @@ class StateStoreChangelogWriter(
   }
 }
 
+/**
+ * Write changes to the key value state store instance to a changelog file.
+ * There are 2 types of records, put and delete.
+ * A put record is written as: | key length | key content | value length | value content |
+ * A delete record is written as: | key length | key content | -1 |
+ * Write an Int -1 to signal the end of file.
+ * The overall changelog format is: | put record | delete record | ... | put record | -1 |
+ */
+class StateStoreChangelogWriterV1(
+    fm: CheckpointFileManager,
+    file: Path,
+    compressionCodec: CompressionCodec)
+    extends StateStoreChangelogWriter(fm, file, compressionCodec) {
+
+  override def put(key: Array[Byte], value: Array[Byte]): Unit = {
+    assert(compressedStream != null)
+    compressedStream.writeInt(key.size)
+    compressedStream.write(key)
+    compressedStream.writeInt(value.size)
+    compressedStream.write(value)
+    size += 1
+  }
+
+  override def delete(key: Array[Byte]): Unit = {
+    assert(compressedStream != null)
+    compressedStream.writeInt(key.size)
+    compressedStream.write(key)
+    // -1 in the value field means record deletion.
+    compressedStream.writeInt(-1)
+    size += 1
+  }
+}
 
 /**
- * Read an iterator of change record from the changelog file.
- * A record is represented by ByteArrayPair(key: Array[Byte], value: Array[Byte])
- * A put record is returned as a ByteArrayPair(key, value)
- * A delete record is return as a ByteArrayPair(key, null)
+ * Write changes to the key value state store instance to a changelog file.
+ * There are 2 types of records, put and delete.
+ * A put record is written as: | record type | key length
+ *    | key content | value length | value content | col family name length | col family name | -1 |
+ * A delete record is written as: | record type | key length | key content | -1
+ *    | col family name length | col family name | -1 |
+ * Write an Int -1 to signal the end of file.
+ * The overall changelog format is: | put record | delete record | ... | put record | -1 |
+ */
+class StateStoreChangelogWriterV2(
+    fm: CheckpointFileManager,
+    file: Path,
+    compressionCodec: CompressionCodec)
+    extends StateStoreChangelogWriter(fm, file, compressionCodec) {

Review Comment:
   nit: 2 spaces



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreChangelog.scala:
##########
@@ -109,18 +119,94 @@ class StateStoreChangelogWriter(
   }
 }
 
+/**
+ * Write changes to the key value state store instance to a changelog file.
+ * There are 2 types of records, put and delete.
+ * A put record is written as: | key length | key content | value length | value content |
+ * A delete record is written as: | key length | key content | -1 |
+ * Write an Int -1 to signal the end of file.
+ * The overall changelog format is: | put record | delete record | ... | put record | -1 |
+ */
+class StateStoreChangelogWriterV1(
+    fm: CheckpointFileManager,
+    file: Path,
+    compressionCodec: CompressionCodec)
+    extends StateStoreChangelogWriter(fm, file, compressionCodec) {
+
+  override def put(key: Array[Byte], value: Array[Byte]): Unit = {
+    assert(compressedStream != null)
+    compressedStream.writeInt(key.size)
+    compressedStream.write(key)
+    compressedStream.writeInt(value.size)
+    compressedStream.write(value)
+    size += 1
+  }
+
+  override def delete(key: Array[Byte]): Unit = {
+    assert(compressedStream != null)
+    compressedStream.writeInt(key.size)
+    compressedStream.write(key)
+    // -1 in the value field means record deletion.
+    compressedStream.writeInt(-1)
+    size += 1
+  }
+}
 
 /**
- * Read an iterator of change record from the changelog file.
- * A record is represented by ByteArrayPair(key: Array[Byte], value: Array[Byte])
- * A put record is returned as a ByteArrayPair(key, value)
- * A delete record is return as a ByteArrayPair(key, null)
+ * Write changes to the key value state store instance to a changelog file.
+ * There are 2 types of records, put and delete.
+ * A put record is written as: | record type | key length
+ *    | key content | value length | value content | col family name length | col family name | -1 |
+ * A delete record is written as: | record type | key length | key content | -1
+ *    | col family name length | col family name | -1 |
+ * Write an Int -1 to signal the end of file.
+ * The overall changelog format is: | put record | delete record | ... | put record | -1 |
+ */
+class StateStoreChangelogWriterV2(
+    fm: CheckpointFileManager,
+    file: Path,
+    compressionCodec: CompressionCodec)
+    extends StateStoreChangelogWriter(fm, file, compressionCodec) {
+
+  override def put(key: Array[Byte], value: Array[Byte], colFamilyName: String): Unit = {
+    assert(compressedStream != null)
+    compressedStream.writeInt(RecordType.PUT_RECORD.toString.getBytes.size)
+    compressedStream.write(RecordType.PUT_RECORD.toString.getBytes)
+    compressedStream.writeInt(key.size)
+    compressedStream.write(key)
+    compressedStream.writeInt(value.size)
+    compressedStream.write(value)
+    compressedStream.writeInt(colFamilyName.getBytes.size)
+    compressedStream.write(colFamilyName.getBytes)
+    size += 1
+  }
+
+  override def delete(key: Array[Byte], colFamilyName: String): Unit = {
+    assert(compressedStream != null)
+    compressedStream.writeInt(RecordType.DELETE_RECORD.toString.getBytes.size)
+    compressedStream.write(RecordType.DELETE_RECORD.toString.getBytes)
+    compressedStream.writeInt(key.size)
+    compressedStream.write(key)
+    // -1 in the value field means record deletion.
+    compressedStream.writeInt(-1)
+    compressedStream.writeInt(colFamilyName.getBytes.size)
+    compressedStream.write(colFamilyName.getBytes)
+    size += 1
+  }
+}
+
+/**
+ * Base class for state store changelog reader
+ * @param fm - checkpoint file manager used to manage streaming query checkpoint
+ * @param fileToRead - name of file to use to read changelog
+ * @param compressionCodec - de-compression method using for reading changelog file
  */
 class StateStoreChangelogReader(
     fm: CheckpointFileManager,
     fileToRead: Path,
     compressionCodec: CompressionCodec)
-  extends NextIterator[(Array[Byte], Array[Byte])] with Logging {
+    extends NextIterator[(RecordType.Value, Array[Byte], Array[Byte], String)]

Review Comment:
   nit: 2 spaces



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala:
##########
@@ -149,21 +149,31 @@ class RocksDBFileManager(
 
   @volatile private var rootDirChecked: Boolean = false
 
-  def getChangeLogWriter(version: Long): StateStoreChangelogWriter = {
+  def getChangeLogWriter(version: Long,
+    useColumnFamilies: Boolean = false): StateStoreChangelogWriter = {
     val changelogFile = dfsChangelogFile(version)
     if (!rootDirChecked) {
       val rootDir = new Path(dfsRootDir)
       if (!fm.exists(rootDir)) fm.mkdirs(rootDir)
       rootDirChecked = true
     }
-    val changelogWriter = new StateStoreChangelogWriter(fm, changelogFile, codec)
+    val changelogWriter = if (useColumnFamilies) {
+      new StateStoreChangelogWriterV2(fm, changelogFile, codec)
+    } else {
+      new StateStoreChangelogWriterV1(fm, changelogFile, codec)
+    }
     changelogWriter
   }
 
   // Get the changelog file at version
-  def getChangelogReader(version: Long): StateStoreChangelogReader = {
+  def getChangelogReader(version: Long,

Review Comment:
   ditto about style



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreChangelog.scala:
##########
@@ -109,18 +119,94 @@ class StateStoreChangelogWriter(
   }
 }
 
+/**
+ * Write changes to the key value state store instance to a changelog file.
+ * There are 2 types of records, put and delete.
+ * A put record is written as: | key length | key content | value length | value content |
+ * A delete record is written as: | key length | key content | -1 |
+ * Write an Int -1 to signal the end of file.
+ * The overall changelog format is: | put record | delete record | ... | put record | -1 |
+ */
+class StateStoreChangelogWriterV1(
+    fm: CheckpointFileManager,
+    file: Path,
+    compressionCodec: CompressionCodec)
+    extends StateStoreChangelogWriter(fm, file, compressionCodec) {

Review Comment:
   nit: 2 spaces



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreChangelog.scala:
##########
@@ -153,12 +258,70 @@ class StateStoreChangelogReader(
       val valueSize = input.readInt()
       if (valueSize < 0) {
         // A deletion record
-        (keyBuffer, null)
+        (RecordType.DELETE_RECORD, keyBuffer, null, StateStore.DEFAULT_COL_FAMILY_NAME)
       } else {
         val valueBuffer = new Array[Byte](valueSize)
         ByteStreams.readFully(input, valueBuffer, 0, valueSize)
         // A put record.
-        (keyBuffer, valueBuffer)
+        (RecordType.PUT_RECORD, keyBuffer, valueBuffer, StateStore.DEFAULT_COL_FAMILY_NAME)
+      }
+    }
+  }
+}
+
+/**
+ * Read an iterator of change record from the changelog file.
+ * A record is represented by ByteArrayPair(recordType: RecordType.Value,
+ *  key: Array[Byte], value: Array[Byte], colFamilyName: String)
+ * A put record is returned as a ByteArrayPair(recordType, key, value, colFamilyName)
+ * A delete record is return as a ByteArrayPair(recordType, key, null, colFamilyName)
+ */
+class StateStoreChangelogReaderV2(
+    fm: CheckpointFileManager,
+    fileToRead: Path,
+    compressionCodec: CompressionCodec) extends StateStoreChangelogReader(fm, fileToRead,

Review Comment:
   ditto



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1430755921


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateExec.scala:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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
+
+import java.util.concurrent.TimeUnit.NANOSECONDS
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, Expression, SortOrder, UnsafeRow}
+import org.apache.spark.sql.catalyst.plans.physical.Distribution
+import org.apache.spark.sql.execution._
+import org.apache.spark.sql.execution.streaming.state._
+import org.apache.spark.sql.streaming.{OutputMode, StatefulProcessor}
+import org.apache.spark.sql.types._
+import org.apache.spark.util.CompletionIterator
+
+/**
+ * Physical operator for executing `TransformWithState`
+ *
+ * @param statefulProcessor processor methods called on underlying data
+ * @param keyDeserializer used to extract the key object for each group.
+ * @param valueDeserializer used to extract the items in the iterator from an input row.
+ * @param groupingAttributes used to group the data
+ * @param dataAttributes used to read the data
+ * @param outputObjAttr Defines the output object
+ * @param batchTimestampMs processing timestamp of the current batch.
+ * @param eventTimeWatermarkForLateEvents event time watermark for filtering late events
+ * @param eventTimeWatermarkForEviction event time watermark for state eviction
+ * @param child the physical plan for the underlying data
+ */
+case class TransformWithStateExec(
+    keyDeserializer: Expression,
+    valueDeserializer: Expression,
+    groupingAttributes: Seq[Attribute],
+    dataAttributes: Seq[Attribute],
+    statefulProcessor: StatefulProcessor[Any, Any, Any],
+    outputMode: OutputMode,
+    outputObjAttr: Attribute,
+    stateInfo: Option[StatefulOperatorStateInfo],
+    batchTimestampMs: Option[Long],
+    eventTimeWatermarkForLateEvents: Option[Long],
+    eventTimeWatermarkForEviction: Option[Long],
+    child: SparkPlan)
+  extends UnaryExecNode
+    with StateStoreWriter
+    with WatermarkSupport
+    with ObjectProducerExec {
+
+  override def shortName: String = "transformWithStateExec"
+
+  override def shouldRunAnotherBatch(newInputWatermark: Long): Boolean = false
+
+  override protected def withNewChildInternal(
+    newChild: SparkPlan): TransformWithStateExec = copy(child = newChild)
+
+  override def keyExpressions: Seq[Attribute] = groupingAttributes
+
+  protected val schemaForKeyRow: StructType = new StructType().add("key", BinaryType)
+
+  protected val schemaForValueRow: StructType = new StructType().add("value", BinaryType)
+
+  override def requiredChildDistribution: Seq[Distribution] = {
+    StatefulOperatorPartitioning.getCompatibleDistribution(groupingAttributes,
+      getStateInfo, conf) ::
+      Nil
+  }
+
+  override def requiredChildOrdering: Seq[Seq[SortOrder]] = Seq(
+    groupingAttributes.map(SortOrder(_, Ascending)))
+
+  private def handleInputRows(keyRow: UnsafeRow, valueRowIter: Iterator[InternalRow]):
+    Iterator[InternalRow] = {
+    val getKeyObj =
+      ObjectOperator.deserializeRowToObject(keyDeserializer, groupingAttributes)
+
+    val getValueObj =
+      ObjectOperator.deserializeRowToObject(valueDeserializer, dataAttributes)
+
+    val getOutputRow = ObjectOperator.wrapObjectToRow(outputObjectType)
+
+    val keyObj = getKeyObj(keyRow)  // convert key to objects
+    ImplicitKeyTracker.setImplicitKey(keyObj)
+    val valueObjIter = valueRowIter.map(getValueObj.apply)
+    val mappedIterator = statefulProcessor.handleInputRows(keyObj, valueObjIter,
+      new TimerValuesImpl(batchTimestampMs, eventTimeWatermarkForLateEvents)).map { obj =>
+        getOutputRow(obj)
+    }
+    ImplicitKeyTracker.removeImplicitKey()
+    mappedIterator
+  }
+
+  private def processNewData(dataIter: Iterator[InternalRow]): Iterator[InternalRow] = {
+    val groupedIter = GroupedIterator(dataIter, groupingAttributes, child.output)
+    groupedIter.flatMap { case (keyRow, valueRowIter) =>
+      val keyUnsafeRow = keyRow.asInstanceOf[UnsafeRow]
+      handleInputRows(keyUnsafeRow, valueRowIter)
+    }
+  }
+
+  private def processDataWithPartition(
+      iter: Iterator[InternalRow],
+      store: StateStore,
+      processorHandle: StatefulProcessorHandleImpl):
+    CompletionIterator[InternalRow, Iterator[InternalRow]] = {
+    val allUpdatesTimeMs = longMetric("allUpdatesTimeMs")
+    val commitTimeMs = longMetric("commitTimeMs")
+
+    val currentTimeNs = System.nanoTime
+    val updatesStartTimeNs = currentTimeNs
+
+    // If timeout is based on event time, then filter late data based on watermark
+    val filteredIter = watermarkPredicateForDataForLateEvents match {
+      case Some(predicate) =>
+        applyRemovingRowsOlderThanWatermark(iter, predicate)
+      case _ =>
+        iter
+    }
+
+    val outputIterator = processNewData(filteredIter)
+    processorHandle.setHandleState(StatefulProcessorHandleState.DATA_PROCESSED)
+    // Return an iterator of all the rows generated by all the keys, such that when fully
+    // consumed, all the state updates will be committed by the state store
+    CompletionIterator[InternalRow, Iterator[InternalRow]](outputIterator, {
+      // Note: Due to the iterator lazy execution, this metric also captures the time taken
+      // by the upstream (consumer) operators in addition to the processing in this operator.
+      allUpdatesTimeMs += NANOSECONDS.toMillis(System.nanoTime - updatesStartTimeNs)
+      commitTimeMs += timeTakenMs {
+        store.commit()
+      }
+      setStoreMetrics(store)
+      setOperatorMetrics()
+      statefulProcessor.close()
+      processorHandle.setHandleState(StatefulProcessorHandleState.CLOSED)
+    })
+  }
+
+  override protected def doExecute(): RDD[InternalRow] = {
+    metrics

Review Comment:
   Done - added comment



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SS][SPARK-46816] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1463844823


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImpl.scala:
##########
@@ -0,0 +1,109 @@
+/*
+ * 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
+
+import java.io.Serializable
+
+import org.apache.commons.lang3.SerializationUtils
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.execution.streaming.state.StateStore
+import org.apache.spark.sql.streaming.ValueState
+import org.apache.spark.sql.types._
+
+/**
+ * Class that provides a concrete implementation for a single value state associated with state
+ * variables used in the streaming transformWithState operator.
+ * @param store - reference to the StateStore instance to be used for storing state
+ * @param stateName - name of logical state partition
+ * @tparam S - data type of object that will be stored
+ */
+class ValueStateImpl[S](
+    store: StateStore,
+    stateName: String) extends ValueState[S] with Logging {
+
+  // TODO: validate places that are trying to encode the key and check if we can eliminate/
+  // add caching for some of these calls.
+  private def encodeKey(): UnsafeRow = {
+    val keyOption = ImplicitKeyTracker.getImplicitKeyOption
+    if (!keyOption.isDefined) {
+      throw new UnsupportedOperationException("Implicit key not found for operation on" +
+        s"stateName=$stateName")
+    }
+
+    val schemaForKeyRow: StructType = new StructType().add("key", BinaryType)
+    val keyByteArr = SerializationUtils.serialize(keyOption.get.asInstanceOf[Serializable])
+    val keyEncoder = UnsafeProjection.create(schemaForKeyRow)
+    val keyRow = keyEncoder(InternalRow(keyByteArr))
+    keyRow
+  }
+
+  private def encodeValue(value: S): UnsafeRow = {
+    val schemaForValueRow: StructType = new StructType().add("value", BinaryType)
+    val valueByteArr = SerializationUtils.serialize(value.asInstanceOf[Serializable])
+    val valueEncoder = UnsafeProjection.create(schemaForValueRow)
+    val valueRow = valueEncoder(InternalRow(valueByteArr))
+    valueRow
+  }
+
+  /** Function to check if state exists. Returns true if present and false otherwise */
+  override def exists(): Boolean = {
+    getImpl() != null
+  }
+
+  /** Function to return Option of value if exists and None otherwise */
+  override def getOption(): Option[S] = {
+    val retRow = getImpl()
+    if (retRow != null) {
+      val resState = SerializationUtils
+        .deserialize(retRow.getBinary(0))
+        .asInstanceOf[S]
+      Some(resState)
+    } else {
+      None
+    }
+  }
+
+  /** Function to return associated value with key if exists and null otherwise */
+  override def get(): S = {

Review Comment:
   Not really. Most APIs for get queries would return null. The expectation is for users to check if the key exists first. But if it doesn't - then users can still filter on null being returned. 
   
   Similar approach for RocksDB - https://javadoc.io/doc/org.rocksdb/rocksdbjni/6.8.1/org/rocksdb/RocksDB.html#get(byte[])



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1440183858


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala:
##########
@@ -896,63 +918,69 @@ abstract class StateStoreSuiteBase[ProviderClass <: StateStoreProvider]
   protected val keySchema: StructType = StateStoreTestsHelper.keySchema
   protected val valueSchema: StructType = StateStoreTestsHelper.valueSchema
 
-  testWithAllCodec("get, put, remove, commit, and all data iterator") {
-    tryWithProviderResource(newStoreProvider()) { provider =>
-      // Verify state before starting a new set of updates
-      assert(getLatestData(provider).isEmpty)
-
-      val store = provider.getStore(0)
-      assert(!store.hasCommitted)
-      assert(get(store, "a", 0) === None)
-      assert(store.iterator().isEmpty)
-      assert(store.metrics.numKeys === 0)
-
-      // Verify state after updating
-      put(store, "a", 0, 1)
-      assert(get(store, "a", 0) === Some(1))
-
-      assert(store.iterator().nonEmpty)
-      assert(getLatestData(provider).isEmpty)
-
-      // Make updates, commit and then verify state
-      put(store, "b", 0, 2)
-      put(store, "aa", 0, 3)
-      remove(store, _._1.startsWith("a"))
-      assert(store.commit() === 1)
-
-      assert(store.hasCommitted)
-      assert(rowPairsToDataSet(store.iterator()) === Set(("b", 0) -> 2))
-      assert(getLatestData(provider) === Set(("b", 0) -> 2))
-
-      // Trying to get newer versions should fail
-      var e = intercept[SparkException] {
-        provider.getStore(2)
-      }
-      assert(e.getCause.isInstanceOf[SparkException])
-      assert(e.getCause.getMessage.contains("does not exist"))
-
-      e = intercept[SparkException] {
-        getData(provider, 2)
-      }
-      assert(e.getCause.isInstanceOf[SparkException])
-      assert(e.getCause.getMessage.contains("does not exist"))
+  Seq(true, false).foreach { useColumnFamilies =>

Review Comment:
   Updated



##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.streaming
+
+import org.apache.spark.SparkException
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{AnalysisException, SaveMode}
+import org.apache.spark.sql.execution.streaming._
+import org.apache.spark.sql.execution.streaming.state.{AlsoTestWithChangelogCheckpointingEnabled, RocksDBStateStoreProvider}
+import org.apache.spark.sql.internal.SQLConf
+
+class RunningCountStatefulProcessor extends StatefulProcessor[String, String, (String, String)]
+  with Logging {
+  @transient private var _countState: ValueState[Long] = _
+  @transient var _processorHandle: StatefulProcessorHandle = _
+
+  override def init(handle: StatefulProcessorHandle,
+    outputMode: OutputMode) : Unit = {
+    _processorHandle = handle
+    assert(handle.getQueryInfo().getBatchId >= 0)
+    assert(handle.getQueryInfo().getOperatorId == 0)
+    assert(handle.getQueryInfo().getPartitionId >= 0 && handle.getQueryInfo().getPartitionId < 5)

Review Comment:
   Done



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1430640516


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala:
##########
@@ -569,6 +569,41 @@ case class FlatMapGroupsWithState(
     copy(child = newLeft, initialState = newRight)
 }
 
+object TransformWithState {
+  def apply[K: Encoder, V: Encoder, U: Encoder](
+    groupingAttributes: Seq[Attribute],

Review Comment:
   Done



##########
sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala:
##########
@@ -642,6 +642,37 @@ class KeyValueGroupedDataset[K, V] private[sql](
       outputMode, timeoutConf, initialState)(f)(stateEncoder, outputEncoder)
   }
 
+  /**
+   * (Scala-specific)
+   * Invokes methods defined in the stateful processor used in arbitrary state API v2.
+   * We allow the user to act on per-group set of input rows along with keyed state and the
+   * user can choose to output/return 0 or more rows.
+   * For a static/batch dataset, this operator is not supported and will throw an exception.
+   * For a streaming dataframe, we will repeatedly invoke the interface methods for new rows
+   * in each trigger and the user's state/state variables will be stored persistently across
+   * invocations.
+   *
+   * @tparam U The type of the output objects. Must be encodable to Spark SQL types.
+   * @param statefulProcessor Instance of statefulProcessor whose functions will be invoked by the
+   *                          operator.
+   * @param outputMode The output mode of the stateful processor. Defaults to APPEND mode.
+   *
+   */
+  def transformWithState[U: Encoder]
+    (statefulProcessor: StatefulProcessor[K, V, U],

Review Comment:
   Done



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1430858026


##########
sql/api/src/main/scala/org/apache/spark/sql/streaming/ValueState.scala:
##########
@@ -0,0 +1,46 @@
+/*
+ * 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.streaming
+
+import java.io.Serializable
+
+import org.apache.spark.annotation.{Evolving, Experimental}
+
+@Experimental
+@Evolving
+/**
+ * Interface used for arbitrary stateful operations with the v2 API to capture
+ * single value state.
+ */
+trait ValueState[S] extends Serializable {
+
+  /** Whether state exists or not. */
+  def exists(): Boolean
+
+  /** Get the state value if it exists, or throw NoSuchElementException. */

Review Comment:
   Done



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1439280714


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala:
##########
@@ -604,17 +734,56 @@ class RocksDB(
   }
 
   private def getDBProperty(property: String): Long = {
-    db.getProperty(property).toLong
+    if (useColumnFamilies) {
+      // get cumulative sum across all available column families
+      assert(!colFamilyNameToHandleMap.isEmpty)
+      colFamilyNameToHandleMap
+        .values
+        .map(handle => db.getProperty(handle, property).toLong)
+        .sum

Review Comment:
   This is actually only an issue only for multiple col families at this layer - upper layers will accumulate across partitions



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1432080835


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreChangelog.scala:
##########
@@ -33,12 +33,20 @@ import org.apache.spark.sql.execution.streaming.CheckpointFileManager.Cancellabl
 import org.apache.spark.util.NextIterator
 
 /**
- * Write changes to the key value state store instance to a changelog file.
- * There are 2 types of records, put and delete.
- * A put record is written as: | key length | key content | value length | value content |
- * A delete record is written as: | key length | key content | -1 |
- * Write an Int -1 to signal the end of file.
- * The overall changelog format is: | put record | delete record | ... | put record | -1 |
+ * Enum used to write record types to changelog files used with RocksDBStateStoreProvider.
+ */
+object RecordType extends Enumeration {
+  type RecordType = Value
+
+  val PUT_RECORD = Value("put_record")

Review Comment:
   I still kept the `enum` rather than passing strings around - just for the in-memory representation. Lmk if you still have a strong pref towards removing this



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1430755761


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala:
##########
@@ -204,12 +211,18 @@ class RocksDB(
     for (v <- loadedVersion + 1 to endVersion) {
       var changelogReader: StateStoreChangelogReader = null
       try {
-        changelogReader = fileManager.getChangelogReader(v)
-        changelogReader.foreach { case (key, value) =>
-          if (value != null) {
-            put(key, value)
-          } else {
-            remove(key)
+        changelogReader = fileManager.getChangelogReader(v, useColumnFamilies)
+        changelogReader.foreach { case (recordType, key, value, colFamilyName) =>

Review Comment:
   No - we won't allow null for values in state store



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1440220667


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ValueStateSuite.scala:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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
+
+import java.util.UUID
+
+import scala.util.Random
+
+import org.apache.hadoop.conf.Configuration
+import org.scalatest.BeforeAndAfter
+
+import org.apache.spark.sql.execution.streaming.{ImplicitKeyTracker, StatefulProcessorHandleImpl}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.streaming.ValueState
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types._
+
+/**
+ * Class that adds tests for single value ValueState types used in arbitrary stateful
+ * operators such as transformWithState
+ */
+class ValueStateSuite extends SharedSparkSession
+  with BeforeAndAfter {
+
+  before {
+    StateStore.stop()
+    require(!StateStore.isMaintenanceRunning)
+  }
+
+  after {
+    StateStore.stop()
+    require(!StateStore.isMaintenanceRunning)
+  }
+
+  import StateStoreTestsHelper._
+
+  val schemaForKeyRow: StructType = new StructType().add("key", BinaryType)
+
+  val schemaForValueRow: StructType = new StructType().add("value", BinaryType)
+
+  private def newStoreProviderWithValueState(useColumnFamilies: Boolean):
+    RocksDBStateStoreProvider = {
+    newStoreProviderWithValueState(StateStoreId(newDir(), Random.nextInt(), 0),
+      numColsPrefixKey = 0,
+      useColumnFamilies = useColumnFamilies)
+  }
+
+  private def newStoreProviderWithValueState(
+    storeId: StateStoreId,
+    numColsPrefixKey: Int,
+    sqlConf: Option[SQLConf] = None,

Review Comment:
   Done



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1439281777


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala:
##########
@@ -604,17 +734,56 @@ class RocksDB(
   }
 
   private def getDBProperty(property: String): Long = {
-    db.getProperty(property).toLong
+    if (useColumnFamilies) {
+      // get cumulative sum across all available column families
+      assert(!colFamilyNameToHandleMap.isEmpty)
+      colFamilyNameToHandleMap
+        .values
+        .map(handle => db.getProperty(handle, property).toLong)
+        .sum
+    } else {
+      db.getProperty(property).toLong
+    }
   }
 
   private def openDB(): Unit = {
     assert(db == null)
-    db = NativeRocksDB.open(dbOptions, workingDir.toString)
-    logInfo(s"Opened DB with conf ${conf}")
+    if (useColumnFamilies) {
+      val colFamilies = NativeRocksDB.listColumnFamilies(dbOptions, workingDir.toString)
+
+      var colFamilyDescriptors: Seq[ColumnFamilyDescriptor] = Seq.empty[ColumnFamilyDescriptor]
+      // populate the list of available col family descriptors
+      colFamilies.asScala.toList.foreach { family =>
+        val descriptor = new ColumnFamilyDescriptor(family, columnFamilyOptions)
+        colFamilyDescriptors = colFamilyDescriptors :+ descriptor
+      }
+
+      if (colFamilyDescriptors.isEmpty) {
+        colFamilyDescriptors = colFamilyDescriptors :+
+          new ColumnFamilyDescriptor(NativeRocksDB.DEFAULT_COLUMN_FAMILY, columnFamilyOptions)

Review Comment:
   Haha I did try this - but I was running into an exception with `NativeRocksDB.open`. I believe it expects to have the colFamilyDesc to atleast be initialized with the default column family handle  



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPARK-46816][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1465833578


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImpl.scala:
##########
@@ -32,11 +34,14 @@ import org.apache.spark.sql.types._
  * variables used in the streaming transformWithState operator.
  * @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
+ * @tparam K - data type of key
  * @tparam S - data type of object that will be stored
  */
-class ValueStateImpl[S](
+class ValueStateImpl[K, S](

Review Comment:
   Same here. Created JIRA - https://issues.apache.org/jira/browse/SPARK-46853



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1440183657


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala:
##########
@@ -836,14 +850,16 @@ class StateStoreSuite extends StateStoreSuiteBase[HDFSBackedStateStoreProvider]
       dir: String = newDir(),
       minDeltasForSnapshot: Int = SQLConf.STATE_STORE_MIN_DELTAS_FOR_SNAPSHOT.defaultValue.get,
       numOfVersToRetainInMemory: Int = SQLConf.MAX_BATCHES_TO_RETAIN_IN_MEMORY.defaultValue.get,
-      hadoopConf: Configuration = new Configuration): HDFSBackedStateStoreProvider = {
+      hadoopConf: Configuration = new Configuration,
+      useColumnFamilies: Boolean = false): HDFSBackedStateStoreProvider = {

Review Comment:
   Done - removed this



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPARK-46816][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1465833909


##########
sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessor.scala:
##########
@@ -50,9 +50,9 @@ trait StatefulProcessor[K, I, O] extends Serializable {
    *                    time if available
    * @return - Zero or more output rows
    */
-  def handleInputRow(
+  def handleInputRows(

Review Comment:
   Created JIRA - https://issues.apache.org/jira/browse/SPARK-46854



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1432198470


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala:
##########
@@ -377,119 +380,247 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared
     }
   }
 
-  test("RocksDB: compression conf") {
-    val remoteDir = Utils.createTempDir().toString
-    new File(remoteDir).delete()  // to make sure that the directory gets created
+  Seq(true, false).foreach { useColumnFamilies =>
+    test(s"RocksDB: compression conf with useColumnFamilies=$useColumnFamilies") {
+      val remoteDir = Utils.createTempDir().toString
+      new File(remoteDir).delete() // to make sure that the directory gets created
 
-    val conf = RocksDBConf().copy(compression = "zstd")
-    withDB(remoteDir, conf = conf) { db =>
-      assert(db.columnFamilyOptions.compressionType() == CompressionType.ZSTD_COMPRESSION)
-    }
+      val conf = RocksDBConf().copy(compression = "zstd")
+      withDB(remoteDir, conf = conf, useColumnFamilies = useColumnFamilies) { db =>
+        assert(db.columnFamilyOptions.compressionType() == CompressionType.ZSTD_COMPRESSION)
+      }
 
-    // Test the default is LZ4
-    withDB(remoteDir, conf = RocksDBConf().copy()) { db =>
-      assert(db.columnFamilyOptions.compressionType() == CompressionType.LZ4_COMPRESSION)
+      // Test the default is LZ4
+      withDB(remoteDir, conf = RocksDBConf().copy(), useColumnFamilies = useColumnFamilies) { db =>
+        assert(db.columnFamilyOptions.compressionType() == CompressionType.LZ4_COMPRESSION)
+      }
     }
   }
 
-  test("RocksDB: get, put, iterator, commit, load") {
-    def testOps(compactOnCommit: Boolean): Unit = {
-      val remoteDir = Utils.createTempDir().toString
-      new File(remoteDir).delete()  // to make sure that the directory gets created
+  Seq(true, false).foreach { useColumnFamilies =>
+    test(s"RocksDB: get, put, iterator, commit, load with " +

Review Comment:
   I thought it was calling `testWithChangelogCheckpointingEnabled`.



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1430727325


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateExec.scala:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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
+
+import java.util.concurrent.TimeUnit.NANOSECONDS
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, Expression, SortOrder, UnsafeRow}
+import org.apache.spark.sql.catalyst.plans.physical.Distribution
+import org.apache.spark.sql.execution._
+import org.apache.spark.sql.execution.streaming.state._
+import org.apache.spark.sql.streaming.{OutputMode, StatefulProcessor}
+import org.apache.spark.sql.types._
+import org.apache.spark.util.CompletionIterator
+
+/**
+ * Physical operator for executing `TransformWithState`
+ *
+ * @param statefulProcessor processor methods called on underlying data
+ * @param keyDeserializer used to extract the key object for each group.
+ * @param valueDeserializer used to extract the items in the iterator from an input row.
+ * @param groupingAttributes used to group the data
+ * @param dataAttributes used to read the data
+ * @param outputObjAttr Defines the output object
+ * @param batchTimestampMs processing timestamp of the current batch.
+ * @param eventTimeWatermarkForLateEvents event time watermark for filtering late events
+ * @param eventTimeWatermarkForEviction event time watermark for state eviction
+ * @param child the physical plan for the underlying data
+ */
+case class TransformWithStateExec(
+    keyDeserializer: Expression,
+    valueDeserializer: Expression,
+    groupingAttributes: Seq[Attribute],
+    dataAttributes: Seq[Attribute],
+    statefulProcessor: StatefulProcessor[Any, Any, Any],
+    outputMode: OutputMode,
+    outputObjAttr: Attribute,
+    stateInfo: Option[StatefulOperatorStateInfo],
+    batchTimestampMs: Option[Long],
+    eventTimeWatermarkForLateEvents: Option[Long],
+    eventTimeWatermarkForEviction: Option[Long],
+    child: SparkPlan)
+  extends UnaryExecNode
+    with StateStoreWriter
+    with WatermarkSupport
+    with ObjectProducerExec {
+
+  override def shortName: String = "transformWithStateExec"
+
+  override def shouldRunAnotherBatch(newInputWatermark: Long): Boolean = false

Review Comment:
   @HeartSaVioR - we will allow it once we add support for timers (timeouts). Basically this operator will ask for batch to be run if it has valid registered timeouts



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1431027492


##########
sql/api/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TransformWithStateTimeoutModes.scala:
##########
@@ -0,0 +1,22 @@
+/*
+ * 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.catalyst.plans.logical
+
+import org.apache.spark.sql.streaming.TimeoutMode
+
+/** Types of timeouts used in tranformWithState operator */
+case object noTimeouts extends TimeoutMode

Review Comment:
   Hmm - had to change this because I was getting a style error for the Java method name saying that it did not match this regex -
   
   ```
   ^[a-z][a-z0-9][a-zA-Z0-9_]*$
   ```



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1431961555


##########
sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessorHandle.scala:
##########
@@ -0,0 +1,43 @@
+/*
+ * 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.streaming
+
+import java.io.Serializable
+
+import org.apache.spark.annotation.{Evolving, Experimental}
+
+/**
+ * Represents the operation handle provided to the stateful processor used in the
+ * arbitrary state API v2.
+ */
+@Experimental
+@Evolving
+trait StatefulProcessorHandle extends Serializable {
+
+  /**
+   * Function to create new or return existing single value state variable of given type
+   * The user must ensure to call this function only within the `init()` method of the
+   * StatefulProcessor.
+   * @param stateName - name of the 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]
+
+  /** Function to return queryInfo for currently running task */
+  def getQueryInfo(): QueryInfo

Review Comment:
   Yes, this is part of the handle right ? sorry, not sure I understand the question



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1431963749


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImpl.scala:
##########
@@ -0,0 +1,109 @@
+/*
+ * 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
+
+import java.io.Serializable
+
+import org.apache.commons.lang3.SerializationUtils
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.execution.streaming.state.StateStore
+import org.apache.spark.sql.streaming.ValueState
+import org.apache.spark.sql.types._
+
+/**
+ * Class that provides a concrete implementation for a single value state associated with state
+ * variables used in the streaming transformWithState operator.
+ * @param store - reference to the StateStore instance to be used for storing state
+ * @param stateName - name of logical state partition
+ * @tparam S - data type of object that will be stored
+ */
+class ValueStateImpl[S](
+    store: StateStore,
+    stateName: String) extends ValueState[S] with Logging {
+
+  // TODO: validate places that are trying to encode the key and check if we can eliminate/
+  // add caching for some of these calls.
+  private def encodeKey(): UnsafeRow = {
+    val keyOption = ImplicitKeyTracker.getImplicitKeyOption
+    if (!keyOption.isDefined) {
+      throw new UnsupportedOperationException("Implicit key not found for operation on" +
+        s"stateName=$stateName")
+    }
+
+    val schemaForKeyRow: StructType = new StructType().add("key", BinaryType)
+    val keyByteArr = SerializationUtils.serialize(keyOption.get.asInstanceOf[Serializable])
+    val keyEncoder = UnsafeProjection.create(schemaForKeyRow)
+    val keyRow = keyEncoder(InternalRow(keyByteArr))
+    keyRow
+  }
+
+  private def encodeValue(value: S): UnsafeRow = {
+    val schemaForValueRow: StructType = new StructType().add("value", BinaryType)
+    val valueByteArr = SerializationUtils.serialize(value.asInstanceOf[Serializable])
+    val valueEncoder = UnsafeProjection.create(schemaForValueRow)
+    val valueRow = valueEncoder(InternalRow(valueByteArr))
+    valueRow
+  }
+
+  /** Function to check if state exists. Returns true if present and false otherwise */
+  override def exists(): Boolean = {
+    getImpl() != null
+  }
+
+  /** Function to return Option of value if exists and None otherwise */
+  override def getOption(): Option[S] = {
+    val retRow = getImpl()
+    if (retRow != null) {
+      val resState = SerializationUtils
+        .deserialize(retRow.getBinary(0))
+        .asInstanceOf[S]
+      Some(resState)
+    } else {
+      None
+    }
+  }
+
+  /** Function to return associated value with key if exists and null otherwise */
+  override def get(): S = {
+    val retRow = getImpl()
+    if (retRow != null) {
+      val resState = SerializationUtils
+        .deserialize(retRow.getBinary(0))
+        .asInstanceOf[S]
+      resState
+    } else {
+      null.asInstanceOf[S]

Review Comment:
   Yea I had to do it to keep the compiler happy :) 



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1432036860


##########
sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessorHandle.scala:
##########
@@ -0,0 +1,43 @@
+/*
+ * 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.streaming
+
+import java.io.Serializable
+
+import org.apache.spark.annotation.{Evolving, Experimental}
+
+/**
+ * Represents the operation handle provided to the stateful processor used in the
+ * arbitrary state API v2.
+ */
+@Experimental
+@Evolving
+trait StatefulProcessorHandle extends Serializable {
+
+  /**
+   * Function to create new or return existing single value state variable of given type
+   * The user must ensure to call this function only within the `init()` method of the
+   * StatefulProcessor.
+   * @param stateName - name of the 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]
+
+  /** Function to return queryInfo for currently running task */
+  def getQueryInfo(): QueryInfo

Review Comment:
   Oh this is meant for 2 primary reasons:
   - to easily identify retry attempts (user could query batch id to ensure calls to external services could be made idempotent)
   - to aid in debugging (i.e. to understand and isolate any issues for specific query/run/batch/partition)



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1432080835


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreChangelog.scala:
##########
@@ -33,12 +33,20 @@ import org.apache.spark.sql.execution.streaming.CheckpointFileManager.Cancellabl
 import org.apache.spark.util.NextIterator
 
 /**
- * Write changes to the key value state store instance to a changelog file.
- * There are 2 types of records, put and delete.
- * A put record is written as: | key length | key content | value length | value content |
- * A delete record is written as: | key length | key content | -1 |
- * Write an Int -1 to signal the end of file.
- * The overall changelog format is: | put record | delete record | ... | put record | -1 |
+ * Enum used to write record types to changelog files used with RocksDBStateStoreProvider.
+ */
+object RecordType extends Enumeration {
+  type RecordType = Value
+
+  val PUT_RECORD = Value("put_record")

Review Comment:
   I still kept the `enum` rather than passing strings around - just for the in-memory representation. For the changelog, we just use a single byte marker for record types. Lmk if you still have a strong pref towards removing this



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreChangelog.scala:
##########
@@ -109,18 +119,93 @@ class StateStoreChangelogWriter(
   }
 }
 
+/**
+ * Write changes to the key value state store instance to a changelog file.
+ * There are 2 types of records, put and delete.
+ * A put record is written as: | key length | key content | value length | value content |
+ * A delete record is written as: | key length | key content | -1 |
+ * Write an Int -1 to signal the end of file.
+ * The overall changelog format is: | put record | delete record | ... | put record | -1 |
+ */
+class StateStoreChangelogWriterV1(
+    fm: CheckpointFileManager,
+    file: Path,
+    compressionCodec: CompressionCodec)
+  extends StateStoreChangelogWriter(fm, file, compressionCodec) {
+
+  override def put(key: Array[Byte], value: Array[Byte]): Unit = {
+    assert(compressedStream != null)
+    compressedStream.writeInt(key.size)
+    compressedStream.write(key)
+    compressedStream.writeInt(value.size)
+    compressedStream.write(value)
+    size += 1
+  }
+
+  override def delete(key: Array[Byte]): Unit = {
+    assert(compressedStream != null)
+    compressedStream.writeInt(key.size)
+    compressedStream.write(key)
+    // -1 in the value field means record deletion.
+    compressedStream.writeInt(-1)
+    size += 1
+  }
+}
 
 /**
- * Read an iterator of change record from the changelog file.
- * A record is represented by ByteArrayPair(key: Array[Byte], value: Array[Byte])
- * A put record is returned as a ByteArrayPair(key, value)
- * A delete record is return as a ByteArrayPair(key, null)
+ * Write changes to the key value state store instance to a changelog file.
+ * There are 2 types of records, put and delete.
+ * A put record is written as: | record type | key length
+ *    | key content | value length | value content | col family name length | col family name | -1 |
+ * A delete record is written as: | record type | key length | key content | -1
+ *    | col family name length | col family name | -1 |
+ * Write an Int -1 to signal the end of file.
+ * The overall changelog format is: | put record | delete record | ... | put record | -1 |
+ */
+class StateStoreChangelogWriterV2(
+    fm: CheckpointFileManager,
+    file: Path,
+    compressionCodec: CompressionCodec)
+  extends StateStoreChangelogWriter(fm, file, compressionCodec) {
+
+  override def put(key: Array[Byte], value: Array[Byte], colFamilyName: String): Unit = {
+    assert(compressedStream != null)
+    compressedStream.writeInt(RecordType.PUT_RECORD.toString.getBytes.size)
+    compressedStream.write(RecordType.PUT_RECORD.toString.getBytes)
+    compressedStream.writeInt(key.size)
+    compressedStream.write(key)
+    compressedStream.writeInt(value.size)
+    compressedStream.write(value)
+    compressedStream.writeInt(colFamilyName.getBytes.size)
+    compressedStream.write(colFamilyName.getBytes)
+    size += 1
+  }
+
+  override def delete(key: Array[Byte], colFamilyName: String): Unit = {
+    assert(compressedStream != null)
+    compressedStream.writeInt(RecordType.DELETE_RECORD.toString.getBytes.size)
+    compressedStream.write(RecordType.DELETE_RECORD.toString.getBytes)
+    compressedStream.writeInt(key.size)
+    compressedStream.write(key)
+    // -1 in the value field means record deletion.
+    compressedStream.writeInt(-1)
+    compressedStream.writeInt(colFamilyName.getBytes.size)
+    compressedStream.write(colFamilyName.getBytes)
+    size += 1
+  }
+}
+
+/**
+ * Base class for state store changelog reader
+ * @param fm - checkpoint file manager used to manage streaming query checkpoint
+ * @param fileToRead - name of file to use to read changelog
+ * @param compressionCodec - de-compression method using for reading changelog file
  */
 class StateStoreChangelogReader(

Review Comment:
   Done



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1445746872


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImpl.scala:
##########
@@ -0,0 +1,105 @@
+/*
+ * 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
+
+import java.io.Serializable
+
+import org.apache.commons.lang3.SerializationUtils
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.execution.streaming.state.StateStore
+import org.apache.spark.sql.streaming.ValueState
+import org.apache.spark.sql.types._
+
+/**
+ * Class that provides a concrete implementation for a single value state associated with state
+ * variables used in the streaming transformWithState operator.
+ * @param store - reference to the StateStore instance to be used for storing state
+ * @param stateName - name of logical state partition
+ * @tparam S - data type of object that will be stored
+ */
+class ValueStateImpl[S](
+    store: StateStore,
+    stateName: String) extends ValueState[S] with Logging{
+
+  private def encodeKey(): UnsafeRow = {
+    val keyOption = ImplicitKeyTracker.getImplicitKeyOption
+    if (!keyOption.isDefined) {
+      throw new UnsupportedOperationException("Implicit key not found for operation on" +
+        s"stateName=$stateName")
+    }
+
+    val schemaForKeyRow: StructType = new StructType().add("key", BinaryType)
+    val keyByteArr = SerializationUtils.serialize(keyOption.get.asInstanceOf[Serializable])

Review Comment:
   Note that this cannot be TODO unless the JIRA ticket for TODO is marked as blocker for release. The serialization format for persistence is really painful to change.



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPARK-46816][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on PR #43961:
URL: https://github.com/apache/spark/pull/43961#issuecomment-1909336819

   (Pending CI before merging this.)


-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SS][SPARK-46816] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1465821940


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala:
##########
@@ -115,11 +116,11 @@ class StatefulProcessorHandleImpl(store: StateStore, runId: UUID)
 
   def getHandleState: StatefulProcessorHandleState = currState
 
-  override def getValueState[T](stateName: String): ValueState[T] = {
+  override def getValueState[K, T](stateName: String, keyEncoder: Encoder[K]): ValueState[T] = {

Review Comment:
   Definitely would be better to provide this for users, because the type of the key is known this time and we even have a key encoder in KeyValueGroupedDataset. 
   
   AFAIK, FlatMapGroupsWithState passes StateEncoder over driver to executor. Maybe you need to extract out serializer and deserializer expressions to serialize over the wire but at least it is technically feasible you can pass the keyEncoder rather than requiring users to provide every time.
   
   We can file a JIRA ticket to deal with this, as this PR is holding for a while and we have more works to do before calling to be done. I won't block this PR due to this.



##########
sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessor.scala:
##########
@@ -50,9 +50,9 @@ trait StatefulProcessor[K, I, O] extends Serializable {
    *                    time if available
    * @return - Zero or more output rows
    */
-  def handleInputRow(
+  def handleInputRows(

Review Comment:
   We are not yet enabling the ability of chaining, right? Then it's OK to go ahead. Probably need to file a JIRA ticket to deal with this.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImpl.scala:
##########
@@ -32,11 +34,14 @@ import org.apache.spark.sql.types._
  * variables used in the streaming transformWithState operator.
  * @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
+ * @tparam K - data type of key
  * @tparam S - data type of object that will be stored
  */
-class ValueStateImpl[S](
+class ValueStateImpl[K, S](

Review Comment:
   My intuition is that value can be serialized with Spark SQL, but again you can file a JIRA ticket for this as well. (But that ticket should be a blocker on releasing this API.) We don't release the API as it is, so it's OK.



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1439498065


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala:
##########
@@ -50,13 +51,15 @@ import org.apache.spark.util.{NextIterator, Utils}
  * @param localRootDir Root directory in local disk that is used to working and checkpointing dirs
  * @param hadoopConf   Hadoop configuration for talking to the remote file system
  * @param loggingId    Id that will be prepended in logs for isolating concurrent RocksDBs
+ * @param useColumnFamilies Used to determine whether a single or multiple column families are used
  */
 class RocksDB(
     dfsRootDir: String,
     val conf: RocksDBConf,
     localRootDir: File = Utils.createTempDir(),
     hadoopConf: Configuration = new Configuration,
-    loggingId: String = "") extends Logging {
+    loggingId: String = "",
+    useColumnFamilies: Boolean = false) extends Logging {

Review Comment:
   Refactored this further and now the use is quite small. PTAL 



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1430716288


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala:
##########
@@ -0,0 +1,76 @@
+/*
+ * 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
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.execution.streaming.state.StateStore
+import org.apache.spark.sql.streaming.{StatefulProcessorHandle, ValueState}
+
+/**

Review Comment:
   Primarily for asserting the state transition(s) values are valid basically



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1430887896


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImpl.scala:
##########
@@ -0,0 +1,105 @@
+/*
+ * 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
+
+import java.io.Serializable
+
+import org.apache.commons.lang3.SerializationUtils
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.execution.streaming.state.StateStore
+import org.apache.spark.sql.streaming.ValueState
+import org.apache.spark.sql.types._
+
+/**
+ * Class that provides a concrete implementation for a single value state associated with state
+ * variables used in the streaming transformWithState operator.
+ * @param store - reference to the StateStore instance to be used for storing state
+ * @param stateName - name of logical state partition
+ * @tparam S - data type of object that will be stored
+ */
+class ValueStateImpl[S](
+    store: StateStore,
+    stateName: String) extends ValueState[S] with Logging{
+
+  private def encodeKey(): UnsafeRow = {
+    val keyOption = ImplicitKeyTracker.getImplicitKeyOption
+    if (!keyOption.isDefined) {
+      throw new UnsupportedOperationException("Implicit key not found for operation on" +
+        s"stateName=$stateName")
+    }
+
+    val schemaForKeyRow: StructType = new StructType().add("key", BinaryType)
+    val keyByteArr = SerializationUtils.serialize(keyOption.get.asInstanceOf[Serializable])

Review Comment:
   You mean for key type - you prefer to use encoding to Spark SQL ? this means we could only encode keys with primitive types and case classes only though, right ?



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1430886414


##########
sql/api/src/main/scala/org/apache/spark/sql/streaming/TimerValues.scala:
##########


Review Comment:
   Added some more info - can update more as we add more timer functionality as well



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1430849634


##########
sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessor.scala:
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.streaming
+
+import java.io.Serializable
+
+import org.apache.spark.annotation.{Evolving, Experimental}
+
+/**
+ * Represents the arbitrary stateful logic that needs to be provided by the user to perform
+ * stateful manipulations on keyed streams.
+ */
+@Experimental
+@Evolving
+trait StatefulProcessor[K, I, O] extends Serializable {
+
+  /**
+   * Function that will be invoked as the first method that allows for users to
+   * initialize all their state variables and perform other init actions before handling data.
+   * @param handle - reference to the statefulProcessorHandle that the user can use to perform
+   *               future actions

Review Comment:
   Done



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SS][SPARK-46816] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1465408102


##########
sql/api/src/main/scala/org/apache/spark/sql/streaming/QueryInfo.scala:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.streaming
+
+import java.io.Serializable
+import java.util.UUID
+
+import org.apache.spark.annotation.{Evolving, Experimental}
+
+/**
+ * Represents the query info provided to the stateful processor used in the
+ * arbitrary state API v2 to easily identify task retries on the same partition.
+ */
+@Experimental
+@Evolving
+private[sql] trait QueryInfo extends Serializable {
+
+  /** Returns the streaming query id associated with stateful operator */
+  def getQueryId: UUID
+
+  /** Returns the streaming query runId associated with stateful operator */
+  def getRunId: UUID
+
+  /** Returns the batch id associated with stateful operator */
+  def getBatchId: Long
+
+  /** Returns the operator id associated with stateful operator */
+  def getOperatorId: Long
+
+  /** Returns the partition id associated with stateful operator */
+  def getPartitionId: Int

Review Comment:
   I dont think we need to expose these to the user. 



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPARK-46816][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR closed pull request #43961: [SPARK-46816][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing
URL: https://github.com/apache/spark/pull/43961


-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1432210084


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreChangelog.scala:
##########
@@ -109,18 +119,94 @@ class StateStoreChangelogWriter(
   }
 }
 
+/**
+ * Write changes to the key value state store instance to a changelog file.
+ * There are 2 types of records, put and delete.
+ * A put record is written as: | key length | key content | value length | value content |
+ * A delete record is written as: | key length | key content | -1 |
+ * Write an Int -1 to signal the end of file.
+ * The overall changelog format is: | put record | delete record | ... | put record | -1 |
+ */
+class StateStoreChangelogWriterV1(
+    fm: CheckpointFileManager,
+    file: Path,
+    compressionCodec: CompressionCodec)
+    extends StateStoreChangelogWriter(fm, file, compressionCodec) {
+
+  override def put(key: Array[Byte], value: Array[Byte]): Unit = {
+    assert(compressedStream != null)
+    compressedStream.writeInt(key.size)
+    compressedStream.write(key)
+    compressedStream.writeInt(value.size)
+    compressedStream.write(value)
+    size += 1
+  }
+
+  override def delete(key: Array[Byte]): Unit = {
+    assert(compressedStream != null)
+    compressedStream.writeInt(key.size)
+    compressedStream.write(key)
+    // -1 in the value field means record deletion.
+    compressedStream.writeInt(-1)
+    size += 1
+  }
+}
 
 /**
- * Read an iterator of change record from the changelog file.
- * A record is represented by ByteArrayPair(key: Array[Byte], value: Array[Byte])
- * A put record is returned as a ByteArrayPair(key, value)
- * A delete record is return as a ByteArrayPair(key, null)
+ * Write changes to the key value state store instance to a changelog file.
+ * There are 2 types of records, put and delete.
+ * A put record is written as: | record type | key length
+ *    | key content | value length | value content | col family name length | col family name | -1 |
+ * A delete record is written as: | record type | key length | key content | -1
+ *    | col family name length | col family name | -1 |
+ * Write an Int -1 to signal the end of file.
+ * The overall changelog format is: | put record | delete record | ... | put record | -1 |
+ */
+class StateStoreChangelogWriterV2(
+    fm: CheckpointFileManager,
+    file: Path,
+    compressionCodec: CompressionCodec)
+    extends StateStoreChangelogWriter(fm, file, compressionCodec) {
+
+  override def put(key: Array[Byte], value: Array[Byte], colFamilyName: String): Unit = {
+    assert(compressedStream != null)
+    compressedStream.writeInt(RecordType.PUT_RECORD.toString.getBytes.size)

Review Comment:
   Updated this



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1431963454


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImpl.scala:
##########
@@ -0,0 +1,109 @@
+/*
+ * 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
+
+import java.io.Serializable
+
+import org.apache.commons.lang3.SerializationUtils
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.execution.streaming.state.StateStore
+import org.apache.spark.sql.streaming.ValueState
+import org.apache.spark.sql.types._
+
+/**
+ * Class that provides a concrete implementation for a single value state associated with state
+ * variables used in the streaming transformWithState operator.
+ * @param store - reference to the StateStore instance to be used for storing state
+ * @param stateName - name of logical state partition
+ * @tparam S - data type of object that will be stored
+ */
+class ValueStateImpl[S](
+    store: StateStore,
+    stateName: String) extends ValueState[S] with Logging {
+
+  // TODO: validate places that are trying to encode the key and check if we can eliminate/
+  // add caching for some of these calls.
+  private def encodeKey(): UnsafeRow = {
+    val keyOption = ImplicitKeyTracker.getImplicitKeyOption
+    if (!keyOption.isDefined) {
+      throw new UnsupportedOperationException("Implicit key not found for operation on" +
+        s"stateName=$stateName")
+    }
+
+    val schemaForKeyRow: StructType = new StructType().add("key", BinaryType)
+    val keyByteArr = SerializationUtils.serialize(keyOption.get.asInstanceOf[Serializable])
+    val keyEncoder = UnsafeProjection.create(schemaForKeyRow)
+    val keyRow = keyEncoder(InternalRow(keyByteArr))
+    keyRow
+  }
+
+  private def encodeValue(value: S): UnsafeRow = {
+    val schemaForValueRow: StructType = new StructType().add("value", BinaryType)
+    val valueByteArr = SerializationUtils.serialize(value.asInstanceOf[Serializable])
+    val valueEncoder = UnsafeProjection.create(schemaForValueRow)
+    val valueRow = valueEncoder(InternalRow(valueByteArr))
+    valueRow
+  }
+
+  /** Function to check if state exists. Returns true if present and false otherwise */
+  override def exists(): Boolean = {
+    getImpl() != null
+  }
+
+  /** Function to return Option of value if exists and None otherwise */
+  override def getOption(): Option[S] = {
+    val retRow = getImpl()
+    if (retRow != null) {
+      val resState = SerializationUtils
+        .deserialize(retRow.getBinary(0))
+        .asInstanceOf[S]

Review Comment:
   Yup thx



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "ericm-db (via GitHub)" <gi...@apache.org>.
ericm-db commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1448001913


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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
+
+import java.util.UUID
+
+import org.apache.spark.TaskContext
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.execution.streaming.state.StateStore
+import org.apache.spark.sql.streaming.{QueryInfo, StatefulProcessorHandle, ValueState}
+import org.apache.spark.util.Utils
+
+/**
+ * Object used to assign/retrieve/remove grouping key passed implicitly for various state
+ * manipulation actions using the store handle.
+ */
+object ImplicitKeyTracker {
+  val implicitKey: InheritableThreadLocal[Any] = new InheritableThreadLocal[Any]
+
+  def getImplicitKeyOption: Option[Any] = Option(implicitKey.get())
+
+  def setImplicitKey(key: Any): Unit = implicitKey.set(key)
+
+  def removeImplicitKey(): Unit = implicitKey.remove()
+}
+
+/**
+ * Enum used to track valid states for the StatefulProcessorHandle
+ */
+object StatefulProcessorHandleState extends Enumeration {
+  type StatefulProcessorHandleState = Value
+  val CREATED, INITIALIZED, DATA_PROCESSED, CLOSED = Value
+}
+
+class QueryInfoImpl(
+    val queryId: UUID,
+    val runId: UUID,
+    val batchId: Long,
+    val operatorId: Long,
+    val partitionId: Int) extends QueryInfo {
+
+  override def getQueryId: UUID = queryId
+
+  override def getRunId: UUID = runId
+
+  override def getBatchId: Long = batchId
+
+  override def getOperatorId: Long = operatorId
+
+  override def getPartitionId: Int = partitionId
+
+  override def toString: String = {
+    s"QueryInfo(queryId=$queryId, runId=$runId, batchId=$batchId, operatorId=$operatorId, " +
+      s"partitionId=$partitionId)"
+  }
+}
+
+/**
+ * Class that provides a concrete implementation of a StatefulProcessorHandle. Note that we keep
+ * track of valid transitions as various functions are invoked to track object lifecycle.
+ * @param store - instance of state store
+ */
+class StatefulProcessorHandleImpl(store: StateStore, runId: UUID)
+  extends StatefulProcessorHandle with Logging {
+  import StatefulProcessorHandleState._
+
+  private def buildQueryInfo(): QueryInfo = {
+    val taskCtxOpt = Option(TaskContext.get())
+    // Task context is not available in tests, so we generate a random query id and batch id here
+    val queryId = if (taskCtxOpt.isDefined) {
+      taskCtxOpt.get.getLocalProperty(StreamExecution.QUERY_ID_KEY)
+    } else {
+      assert(Utils.isTesting)

Review Comment:
   Do we want to add a message here?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImpl.scala:
##########
@@ -0,0 +1,109 @@
+/*
+ * 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
+
+import java.io.Serializable
+
+import org.apache.commons.lang3.SerializationUtils
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.execution.streaming.state.StateStore
+import org.apache.spark.sql.streaming.ValueState
+import org.apache.spark.sql.types._
+
+/**
+ * Class that provides a concrete implementation for a single value state associated with state
+ * variables used in the streaming transformWithState operator.
+ * @param store - reference to the StateStore instance to be used for storing state
+ * @param stateName - name of logical state partition
+ * @tparam S - data type of object that will be stored
+ */
+class ValueStateImpl[S](
+    store: StateStore,
+    stateName: String) extends ValueState[S] with Logging {
+
+  // TODO: validate places that are trying to encode the key and check if we can eliminate/
+  // add caching for some of these calls.
+  private def encodeKey(): UnsafeRow = {
+    val keyOption = ImplicitKeyTracker.getImplicitKeyOption
+    if (!keyOption.isDefined) {
+      throw new UnsupportedOperationException("Implicit key not found for operation on" +
+        s"stateName=$stateName")
+    }
+
+    val schemaForKeyRow: StructType = new StructType().add("key", BinaryType)
+    val keyByteArr = SerializationUtils.serialize(keyOption.get.asInstanceOf[Serializable])
+    val keyEncoder = UnsafeProjection.create(schemaForKeyRow)
+    val keyRow = keyEncoder(InternalRow(keyByteArr))
+    keyRow
+  }
+
+  private def encodeValue(value: S): UnsafeRow = {
+    val schemaForValueRow: StructType = new StructType().add("value", BinaryType)
+    val valueByteArr = SerializationUtils.serialize(value.asInstanceOf[Serializable])
+    val valueEncoder = UnsafeProjection.create(schemaForValueRow)
+    val valueRow = valueEncoder(InternalRow(valueByteArr))
+    valueRow
+  }
+
+  /** Function to check if state exists. Returns true if present and false otherwise */
+  override def exists(): Boolean = {
+    getImpl() != null
+  }
+
+  /** Function to return Option of value if exists and None otherwise */
+  override def getOption(): Option[S] = {
+    val retRow = getImpl()
+    if (retRow != null) {
+      val resState = SerializationUtils
+        .deserialize(retRow.getBinary(0))
+        .asInstanceOf[S]
+      Some(resState)
+    } else {
+      None
+    }
+  }
+
+  /** Function to return associated value with key if exists and null otherwise */
+  override def get(): S = {

Review Comment:
   Isn't the contract usually to throw an error if no such row/value exists? I think that makes more sense than returning null...



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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
+
+import java.util.UUID
+
+import org.apache.spark.TaskContext
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.execution.streaming.state.StateStore
+import org.apache.spark.sql.streaming.{QueryInfo, StatefulProcessorHandle, ValueState}
+import org.apache.spark.util.Utils
+
+/**
+ * Object used to assign/retrieve/remove grouping key passed implicitly for various state
+ * manipulation actions using the store handle.
+ */
+object ImplicitKeyTracker {

Review Comment:
   @sahnib suggested we rename this `ImplicitGroupingKeyTracker` and I agree that it definitely helps with readability



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPARK-46816][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on PR #43961:
URL: https://github.com/apache/spark/pull/43961#issuecomment-1909444472

   @anishshri-db 
   I just realized you mentioned "Yes" in the section `Was this patch authored or co-authored using generative AI tooling?`. This requires an attention in ASF project.
   
   https://www.apache.org/legal/generative-tooling.html
   If you weren't confused with answering the section, could you please go through and check yourself it won't hit copyright issue? ASF requires every contribution to be copyrightable so that ASF has copyright on the codebase, AFAIK.


-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1439278954


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala:
##########
@@ -50,13 +51,15 @@ import org.apache.spark.util.{NextIterator, Utils}
  * @param localRootDir Root directory in local disk that is used to working and checkpointing dirs
  * @param hadoopConf   Hadoop configuration for talking to the remote file system
  * @param loggingId    Id that will be prepended in logs for isolating concurrent RocksDBs
+ * @param useColumnFamilies Used to determine whether a single or multiple column families are used
  */
 class RocksDB(
     dfsRootDir: String,
     val conf: RocksDBConf,
     localRootDir: File = Utils.createTempDir(),
     hadoopConf: Configuration = new Configuration,
-    loggingId: String = "") extends Logging {
+    loggingId: String = "",
+    useColumnFamilies: Boolean = false) extends Logging {

Review Comment:
   I thought about this actually - but the reason I added this flag is 2 fold:
   - one is to isolate users of this flag - basically in the current impl, this flag is set to true only for the `transformWithState` operator. We are not touching any other operators - so we would limit the impact surface
   - second is to identify which changelog writer format to use
   
   If we distinguish old vs new based on just the `default` column family name - either we won't be able use the `default` col family with the new operator or we won't be able to identify which writers/formats to use



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1439537332


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala:
##########
@@ -219,49 +233,127 @@ class RocksDB(
     loadedVersion = endVersion
   }
 
+  private def checkColFamilyExists(colFamilyName: String): Boolean = {
+    colFamilyNameToHandleMap.contains(colFamilyName)
+  }
+
+  /**
+   * Create RocksDB column family, if not created already
+   */
+  def createColFamilyIfAbsent(colFamilyName: String): Unit = {
+    if (colFamilyName == StateStore.DEFAULT_COL_FAMILY_NAME) {
+      throw new UnsupportedOperationException("Failed to create column family with reserved " +
+        s"name=$colFamilyName")
+    }
+
+    if (!checkColFamilyExists(colFamilyName)) {
+      assert(db != null)
+      val descriptor = new ColumnFamilyDescriptor(colFamilyName.getBytes, columnFamilyOptions)
+      val handle = db.createColumnFamily(descriptor)
+      colFamilyNameToHandleMap(handle.getName.map(_.toChar).mkString) = handle
+    }
+  }
+
   /**
    * Get the value for the given key if present, or null.
    * @note This will return the last written value even if it was uncommitted.
    */
-  def get(key: Array[Byte]): Array[Byte] = {
-    db.get(readOptions, key)
+  def get(
+      key: Array[Byte],
+      colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Array[Byte] = {
+    if (useColumnFamilies) {
+      // if col family is not created, throw an exception
+      if (!checkColFamilyExists(colFamilyName)) {
+        throw new RuntimeException(s"Column family with name=$colFamilyName does not exist")
+      }
+      db.get(colFamilyNameToHandleMap(colFamilyName), readOptions, key)
+    } else {
+      db.get(readOptions, key)
+    }
   }
 
   /**
    * Put the given value for the given key.
    * @note This update is not committed to disk until commit() is called.
    */
-  def put(key: Array[Byte], value: Array[Byte]): Unit = {
-    if (conf.trackTotalNumberOfRows) {
-      val oldValue = db.get(readOptions, key)
-      if (oldValue == null) {
-        numKeysOnWritingVersion += 1
+  def put(
+      key: Array[Byte],
+      value: Array[Byte],
+      colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Unit = {
+    if (useColumnFamilies) {
+      // if col family is not created, throw an exception
+      if (!checkColFamilyExists(colFamilyName)) {
+        throw new RuntimeException(s"Column family with name=$colFamilyName does not exist")
+      }
+
+      if (conf.trackTotalNumberOfRows) {
+        val oldValue = db.get(colFamilyNameToHandleMap(colFamilyName), readOptions, key)
+        if (oldValue == null) {
+          numKeysOnWritingVersion += 1
+        }
+      }
+      db.put(colFamilyNameToHandleMap(colFamilyName), writeOptions, key, value)
+      changelogWriter.foreach(_.put(key, value, colFamilyName))
+    } else {
+      if (conf.trackTotalNumberOfRows) {
+        val oldValue = db.get(readOptions, key)
+        if (oldValue == null) {
+          numKeysOnWritingVersion += 1
+        }
       }
+      db.put(writeOptions, key, value)
+      changelogWriter.foreach(_.put(key, value))
     }
-    db.put(writeOptions, key, value)
-    changelogWriter.foreach(_.put(key, value))
   }
 
   /**
    * Remove the key if present.
    * @note This update is not committed to disk until commit() is called.
    */
-  def remove(key: Array[Byte]): Unit = {
-    if (conf.trackTotalNumberOfRows) {
-      val value = db.get(readOptions, key)
-      if (value != null) {
-        numKeysOnWritingVersion -= 1
+  def remove(
+      key: Array[Byte],
+      colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Unit = {
+    if (useColumnFamilies) {
+      // if col family is not created, throw an exception
+      if (!checkColFamilyExists(colFamilyName)) {

Review Comment:
   Done



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1439533439


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala:
##########
@@ -604,17 +734,56 @@ class RocksDB(
   }
 
   private def getDBProperty(property: String): Long = {
-    db.getProperty(property).toLong
+    if (useColumnFamilies) {
+      // get cumulative sum across all available column families
+      assert(!colFamilyNameToHandleMap.isEmpty)
+      colFamilyNameToHandleMap
+        .values
+        .map(handle => db.getProperty(handle, property).toLong)
+        .sum
+    } else {
+      db.getProperty(property).toLong
+    }
   }
 
   private def openDB(): Unit = {
     assert(db == null)
-    db = NativeRocksDB.open(dbOptions, workingDir.toString)
-    logInfo(s"Opened DB with conf ${conf}")
+    if (useColumnFamilies) {
+      val colFamilies = NativeRocksDB.listColumnFamilies(dbOptions, workingDir.toString)
+
+      var colFamilyDescriptors: Seq[ColumnFamilyDescriptor] = Seq.empty[ColumnFamilyDescriptor]
+      // populate the list of available col family descriptors
+      colFamilies.asScala.toList.foreach { family =>
+        val descriptor = new ColumnFamilyDescriptor(family, columnFamilyOptions)
+        colFamilyDescriptors = colFamilyDescriptors :+ descriptor

Review Comment:
   Seems like this is linear since it relies/uses underlying immutable collection impl. Switched to `ArrayBuffer` instead which should give us amortized constant time for append



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1439497538


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImpl.scala:
##########
@@ -0,0 +1,105 @@
+/*
+ * 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
+
+import java.io.Serializable
+
+import org.apache.commons.lang3.SerializationUtils
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.execution.streaming.state.StateStore
+import org.apache.spark.sql.streaming.ValueState
+import org.apache.spark.sql.types._
+
+/**
+ * Class that provides a concrete implementation for a single value state associated with state
+ * variables used in the streaming transformWithState operator.
+ * @param store - reference to the StateStore instance to be used for storing state
+ * @param stateName - name of logical state partition
+ * @tparam S - data type of object that will be stored
+ */
+class ValueStateImpl[S](
+    store: StateStore,
+    stateName: String) extends ValueState[S] with Logging{
+
+  private def encodeKey(): UnsafeRow = {
+    val keyOption = ImplicitKeyTracker.getImplicitKeyOption
+    if (!keyOption.isDefined) {
+      throw new UnsupportedOperationException("Implicit key not found for operation on" +
+        s"stateName=$stateName")
+    }
+
+    val schemaForKeyRow: StructType = new StructType().add("key", BinaryType)
+    val keyByteArr = SerializationUtils.serialize(keyOption.get.asInstanceOf[Serializable])

Review Comment:
   Would it be ok to keep this as it is for now ? I can add a TODO for this - but basically with `mapState`, I am not sure whether we can fall back to `Spark SQL Encoder` - given that we might flatten the grouping key and map key



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1445744234


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImpl.scala:
##########
@@ -0,0 +1,105 @@
+/*
+ * 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
+
+import java.io.Serializable
+
+import org.apache.commons.lang3.SerializationUtils
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.execution.streaming.state.StateStore
+import org.apache.spark.sql.streaming.ValueState
+import org.apache.spark.sql.types._
+
+/**
+ * Class that provides a concrete implementation for a single value state associated with state
+ * variables used in the streaming transformWithState operator.
+ * @param store - reference to the StateStore instance to be used for storing state
+ * @param stateName - name of logical state partition
+ * @tparam S - data type of object that will be stored
+ */
+class ValueStateImpl[S](
+    store: StateStore,
+    stateName: String) extends ValueState[S] with Logging{
+
+  private def encodeKey(): UnsafeRow = {
+    val keyOption = ImplicitKeyTracker.getImplicitKeyOption
+    if (!keyOption.isDefined) {
+      throw new UnsupportedOperationException("Implicit key not found for operation on" +
+        s"stateName=$stateName")
+    }
+
+    val schemaForKeyRow: StructType = new StructType().add("key", BinaryType)
+    val keyByteArr = SerializationUtils.serialize(keyOption.get.asInstanceOf[Serializable])

Review Comment:
   cc. @cloud-fan It would be awesome to hear your voice on this since you seem to work on typed API. Thanks in advance.



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1439500960


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala:
##########
@@ -219,49 +233,127 @@ class RocksDB(
     loadedVersion = endVersion
   }
 
+  private def checkColFamilyExists(colFamilyName: String): Boolean = {
+    colFamilyNameToHandleMap.contains(colFamilyName)
+  }
+
+  /**
+   * Create RocksDB column family, if not created already
+   */
+  def createColFamilyIfAbsent(colFamilyName: String): Unit = {
+    if (colFamilyName == StateStore.DEFAULT_COL_FAMILY_NAME) {
+      throw new UnsupportedOperationException("Failed to create column family with reserved " +
+        s"name=$colFamilyName")
+    }
+
+    if (!checkColFamilyExists(colFamilyName)) {
+      assert(db != null)
+      val descriptor = new ColumnFamilyDescriptor(colFamilyName.getBytes, columnFamilyOptions)
+      val handle = db.createColumnFamily(descriptor)
+      colFamilyNameToHandleMap(handle.getName.map(_.toChar).mkString) = handle
+    }
+  }
+
   /**
    * Get the value for the given key if present, or null.
    * @note This will return the last written value even if it was uncommitted.
    */
-  def get(key: Array[Byte]): Array[Byte] = {
-    db.get(readOptions, key)
+  def get(
+      key: Array[Byte],
+      colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Array[Byte] = {
+    if (useColumnFamilies) {
+      // if col family is not created, throw an exception
+      if (!checkColFamilyExists(colFamilyName)) {
+        throw new RuntimeException(s"Column family with name=$colFamilyName does not exist")
+      }
+      db.get(colFamilyNameToHandleMap(colFamilyName), readOptions, key)
+    } else {
+      db.get(readOptions, key)
+    }
   }
 
   /**
    * Put the given value for the given key.
    * @note This update is not committed to disk until commit() is called.
    */
-  def put(key: Array[Byte], value: Array[Byte]): Unit = {
-    if (conf.trackTotalNumberOfRows) {
-      val oldValue = db.get(readOptions, key)
-      if (oldValue == null) {
-        numKeysOnWritingVersion += 1
+  def put(
+      key: Array[Byte],
+      value: Array[Byte],
+      colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Unit = {
+    if (useColumnFamilies) {
+      // if col family is not created, throw an exception
+      if (!checkColFamilyExists(colFamilyName)) {
+        throw new RuntimeException(s"Column family with name=$colFamilyName does not exist")
+      }
+
+      if (conf.trackTotalNumberOfRows) {
+        val oldValue = db.get(colFamilyNameToHandleMap(colFamilyName), readOptions, key)
+        if (oldValue == null) {
+          numKeysOnWritingVersion += 1
+        }
+      }
+      db.put(colFamilyNameToHandleMap(colFamilyName), writeOptions, key, value)
+      changelogWriter.foreach(_.put(key, value, colFamilyName))
+    } else {
+      if (conf.trackTotalNumberOfRows) {
+        val oldValue = db.get(readOptions, key)
+        if (oldValue == null) {
+          numKeysOnWritingVersion += 1
+        }

Review Comment:
   Updated - PTAL



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1439581709


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateExec.scala:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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
+
+import java.util.concurrent.TimeUnit.NANOSECONDS
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, Expression, SortOrder, UnsafeRow}
+import org.apache.spark.sql.catalyst.plans.physical.Distribution
+import org.apache.spark.sql.execution._
+import org.apache.spark.sql.execution.streaming.state._
+import org.apache.spark.sql.streaming.{OutputMode, StatefulProcessor}
+import org.apache.spark.sql.types._
+import org.apache.spark.util.CompletionIterator
+
+/**
+ * Physical operator for executing `TransformWithState`
+ *
+ * @param statefulProcessor processor methods called on underlying data
+ * @param keyDeserializer used to extract the key object for each group.
+ * @param valueDeserializer used to extract the items in the iterator from an input row.
+ * @param groupingAttributes used to group the data
+ * @param dataAttributes used to read the data
+ * @param outputObjAttr Defines the output object
+ * @param batchTimestampMs processing timestamp of the current batch.
+ * @param eventTimeWatermarkForLateEvents event time watermark for filtering late events
+ * @param eventTimeWatermarkForEviction event time watermark for state eviction
+ * @param child the physical plan for the underlying data
+ */
+case class TransformWithStateExec(
+    keyDeserializer: Expression,
+    valueDeserializer: Expression,
+    groupingAttributes: Seq[Attribute],
+    dataAttributes: Seq[Attribute],
+    statefulProcessor: StatefulProcessor[Any, Any, Any],
+    outputMode: OutputMode,
+    outputObjAttr: Attribute,
+    stateInfo: Option[StatefulOperatorStateInfo],
+    batchTimestampMs: Option[Long],
+    eventTimeWatermarkForLateEvents: Option[Long],
+    eventTimeWatermarkForEviction: Option[Long],
+    child: SparkPlan)
+  extends UnaryExecNode
+    with StateStoreWriter
+    with WatermarkSupport
+    with ObjectProducerExec {
+
+  override def shortName: String = "transformWithStateExec"
+
+  override def shouldRunAnotherBatch(newInputWatermark: Long): Boolean = false
+
+  override protected def withNewChildInternal(
+    newChild: SparkPlan): TransformWithStateExec = copy(child = newChild)
+
+  override def keyExpressions: Seq[Attribute] = groupingAttributes
+
+  protected val schemaForKeyRow: StructType = new StructType().add("key", BinaryType)
+
+  protected val schemaForValueRow: StructType = new StructType().add("value", BinaryType)
+
+  override def requiredChildDistribution: Seq[Distribution] = {
+    StatefulOperatorPartitioning.getCompatibleDistribution(groupingAttributes,
+      getStateInfo, conf) ::
+      Nil
+  }
+
+  override def requiredChildOrdering: Seq[Seq[SortOrder]] = Seq(
+    groupingAttributes.map(SortOrder(_, Ascending)))
+
+  private def handleInputRows(keyRow: UnsafeRow, valueRowIter: Iterator[InternalRow]):
+    Iterator[InternalRow] = {
+    val getKeyObj =
+      ObjectOperator.deserializeRowToObject(keyDeserializer, groupingAttributes)
+
+    val getValueObj =
+      ObjectOperator.deserializeRowToObject(valueDeserializer, dataAttributes)
+
+    val getOutputRow = ObjectOperator.wrapObjectToRow(outputObjectType)
+
+    val keyObj = getKeyObj(keyRow)  // convert key to objects
+    ImplicitKeyTracker.setImplicitKey(keyObj)

Review Comment:
   Yea can't think of a better/different way atleast for now



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1440222266


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreChangelog.scala:
##########
@@ -153,12 +265,70 @@ class StateStoreChangelogReader(
       val valueSize = input.readInt()
       if (valueSize < 0) {
         // A deletion record
-        (keyBuffer, null)
+        (RecordType.DELETE_RECORD, keyBuffer, null, StateStore.DEFAULT_COL_FAMILY_NAME)
       } else {
         val valueBuffer = new Array[Byte](valueSize)
         ByteStreams.readFully(input, valueBuffer, 0, valueSize)
         // A put record.
-        (keyBuffer, valueBuffer)
+        (RecordType.PUT_RECORD, keyBuffer, valueBuffer, StateStore.DEFAULT_COL_FAMILY_NAME)
+      }
+    }
+  }
+}
+
+/**
+ * Read an iterator of change record from the changelog file.
+ * A record is represented by ByteArrayPair(recordType: RecordType.Value,
+ *  key: Array[Byte], value: Array[Byte], colFamilyName: String)
+ * A put record is returned as a ByteArrayPair(recordType, key, value, colFamilyName)
+ * A delete record is return as a ByteArrayPair(recordType, key, null, colFamilyName)
+ */
+class StateStoreChangelogReaderV2(
+    fm: CheckpointFileManager,
+    fileToRead: Path,
+    compressionCodec: CompressionCodec)
+  extends StateStoreChangelogReader(fm, fileToRead, compressionCodec) {
+
+  private def parseBuffer(input: DataInputStream): Array[Byte] = {
+    val blockSize = input.readInt()
+    val blockBuffer = new Array[Byte](blockSize)
+    ByteStreams.readFully(input, blockBuffer, 0, blockSize)
+    blockBuffer
+  }
+
+  override def getNext(): (RecordType.Value, Array[Byte], Array[Byte], String) = {
+    val recordTypeSize = input.readInt()
+    // A -1 key size mean end of file.
+    if (recordTypeSize == -1) {
+      finished = true
+      null
+    } else if (recordTypeSize < 0) {
+      throw new IOException(
+        s"Error reading streaming state file $fileToRead: " +
+        s"record type size cannot be $recordTypeSize")
+    } else {
+      val recordTypeBuffer = new Array[Byte](recordTypeSize)
+      ByteStreams.readFully(input, recordTypeBuffer, 0, recordTypeSize)

Review Comment:
   Just used the already existing pattern for `StateStoreChangelog`. Seems like its based on Google's guava package - https://github.com/google/guava/blob/master/guava/src/com/google/common/io/ByteStreams.java 



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPARK-46816][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on PR #43961:
URL: https://github.com/apache/spark/pull/43961#issuecomment-1909432912

   Thanks! Merging to master.


-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SS][SPARK-46816] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1464244769


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImpl.scala:
##########
@@ -0,0 +1,105 @@
+/*
+ * 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
+
+import java.io.Serializable
+
+import org.apache.commons.lang3.SerializationUtils
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.execution.streaming.state.StateStore
+import org.apache.spark.sql.streaming.ValueState
+import org.apache.spark.sql.types._
+
+/**
+ * Class that provides a concrete implementation for a single value state associated with state
+ * variables used in the streaming transformWithState operator.
+ * @param store - reference to the StateStore instance to be used for storing state
+ * @param stateName - name of logical state partition
+ * @tparam S - data type of object that will be stored
+ */
+class ValueStateImpl[S](
+    store: StateStore,
+    stateName: String) extends ValueState[S] with Logging{
+
+  private def encodeKey(): UnsafeRow = {
+    val keyOption = ImplicitKeyTracker.getImplicitKeyOption
+    if (!keyOption.isDefined) {
+      throw new UnsupportedOperationException("Implicit key not found for operation on" +
+        s"stateName=$stateName")
+    }
+
+    val schemaForKeyRow: StructType = new StructType().add("key", BinaryType)
+    val keyByteArr = SerializationUtils.serialize(keyOption.get.asInstanceOf[Serializable])

Review Comment:
   Done - using Spark SQL encoder now



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SS][SPARK-46816] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1465419732


##########
sql/api/src/main/scala/org/apache/spark/sql/streaming/QueryInfo.scala:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.streaming
+
+import java.io.Serializable
+import java.util.UUID
+
+import org.apache.spark.annotation.{Evolving, Experimental}
+
+/**
+ * Represents the query info provided to the stateful processor used in the
+ * arbitrary state API v2 to easily identify task retries on the same partition.
+ */
+@Experimental
+@Evolving
+private[sql] trait QueryInfo extends Serializable {
+
+  /** Returns the streaming query id associated with stateful operator */
+  def getQueryId: UUID
+
+  /** Returns the streaming query runId associated with stateful operator */
+  def getRunId: UUID
+
+  /** Returns the batch id associated with stateful operator */
+  def getBatchId: Long
+
+  /** Returns the operator id associated with stateful operator */
+  def getOperatorId: Long
+
+  /** Returns the partition id associated with stateful operator */
+  def getPartitionId: Int

Review Comment:
   Done - thx



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1430886666


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImpl.scala:
##########
@@ -0,0 +1,105 @@
+/*
+ * 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
+
+import java.io.Serializable
+
+import org.apache.commons.lang3.SerializationUtils
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.execution.streaming.state.StateStore
+import org.apache.spark.sql.streaming.ValueState
+import org.apache.spark.sql.types._
+
+/**
+ * Class that provides a concrete implementation for a single value state associated with state
+ * variables used in the streaming transformWithState operator.
+ * @param store - reference to the StateStore instance to be used for storing state
+ * @param stateName - name of logical state partition
+ * @tparam S - data type of object that will be stored
+ */
+class ValueStateImpl[S](
+    store: StateStore,
+    stateName: String) extends ValueState[S] with Logging{
+
+  private def encodeKey(): UnsafeRow = {

Review Comment:
   Added a TODO. We are already addressing this in a subsequent PR



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImpl.scala:
##########
@@ -0,0 +1,105 @@
+/*
+ * 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
+
+import java.io.Serializable
+
+import org.apache.commons.lang3.SerializationUtils
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.execution.streaming.state.StateStore
+import org.apache.spark.sql.streaming.ValueState
+import org.apache.spark.sql.types._
+
+/**
+ * Class that provides a concrete implementation for a single value state associated with state
+ * variables used in the streaming transformWithState operator.
+ * @param store - reference to the StateStore instance to be used for storing state
+ * @param stateName - name of logical state partition
+ * @tparam S - data type of object that will be stored
+ */
+class ValueStateImpl[S](
+    store: StateStore,
+    stateName: String) extends ValueState[S] with Logging{
+
+  private def encodeKey(): UnsafeRow = {
+    val keyOption = ImplicitKeyTracker.getImplicitKeyOption
+    if (!keyOption.isDefined) {
+      throw new UnsupportedOperationException("Implicit key not found for operation on" +
+        s"stateName=$stateName")
+    }
+
+    val schemaForKeyRow: StructType = new StructType().add("key", BinaryType)
+    val keyByteArr = SerializationUtils.serialize(keyOption.get.asInstanceOf[Serializable])
+    val keyEncoder = UnsafeProjection.create(schemaForKeyRow)
+    val keyRow = keyEncoder(InternalRow(keyByteArr))
+    keyRow
+  }
+
+  private def encodeValue(value: S): UnsafeRow = {
+    val schemaForValueRow: StructType = new StructType().add("value", BinaryType)
+    val valueByteArr = SerializationUtils.serialize(value.asInstanceOf[Serializable])
+    val valueEncoder = UnsafeProjection.create(schemaForValueRow)
+    val valueRow = valueEncoder(InternalRow(valueByteArr))
+    valueRow
+  }
+
+  /** Function to check if state exists. Returns true if present and false otherwise */
+  override def exists(): Boolean = {
+    val retRow = store.get(encodeKey(), stateName)
+    if (retRow == null) {
+      false
+    } else {
+      true
+    }
+  }
+
+  /** Function to return Option of value if exists and None otherwise */
+  override def getOption(): Option[S] = {
+    if (exists()) {

Review Comment:
   Done



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1430857638


##########
sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessorHandle.scala:
##########
@@ -0,0 +1,33 @@
+/*
+ * 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.streaming
+
+import java.io.Serializable
+
+import org.apache.spark.annotation.{Evolving, Experimental}
+
+/**
+ * Represents the operation handle provided to the stateful processor used in the
+ * arbitrary state API v2.
+ */
+@Experimental
+@Evolving
+trait StatefulProcessorHandle extends Serializable {
+
+  /** Function to create new or return existing single value state variable of given type */

Review Comment:
   So within the same query, we expect different state vars to have different names. I'll probably add a check for this (might need to keep some local state within the handle) For the cluster, I don't think we have a restriction as such. Basically these are like logical partitions within the same RocksDB instance - so they are already scoped to that instance
   
   Based on the RocksDB doc I read so far, I don't think have come across any such limitation on the allowed chars



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1430858707


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala:
##########
@@ -118,6 +121,10 @@ class RocksDB(
     dbOptions.setWriteBufferManager(writeBufferManager)
   }
 
+  // Maintain mapping of column family name to handle
+  @volatile private var colFamilyNameToHandleMap =

Review Comment:
   We don't change the reference. I changed it to `private val`. But its basically guarded by the DB instance lock - I think that should be sufficient here



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1430755529


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala:
##########
@@ -604,17 +728,57 @@ class RocksDB(
   }
 
   private def getDBProperty(property: String): Long = {
-    db.getProperty(property).toLong
+    if (useColumnFamilies) {
+      // get cumulative sum across all available column families
+      assert(!colFamilyNameToHandleMap.isEmpty)
+      colFamilyNameToHandleMap
+        .values
+        .map(handle => db.getProperty(handle, property).toLong)
+        .sum
+    } else {
+      db.getProperty(property).toLong
+    }
   }
 
   private def openDB(): Unit = {
     assert(db == null)
-    db = NativeRocksDB.open(dbOptions, workingDir.toString)
-    logInfo(s"Opened DB with conf ${conf}")
+    if (useColumnFamilies) {
+      val colFamilies = NativeRocksDB.listColumnFamilies(dbOptions, workingDir.toString)
+
+      var colFamilyDescriptors: Seq[ColumnFamilyDescriptor] = Seq.empty[ColumnFamilyDescriptor]
+      // populate the list of available col family descriptors
+      colFamilies.asScala.toList.foreach(family => {
+        val descriptor = new ColumnFamilyDescriptor(family, columnFamilyOptions)
+        colFamilyDescriptors = colFamilyDescriptors :+ descriptor
+      })
+
+      if (colFamilyDescriptors.isEmpty) {
+        colFamilyDescriptors = colFamilyDescriptors :+
+          new ColumnFamilyDescriptor(NativeRocksDB.DEFAULT_COLUMN_FAMILY, columnFamilyOptions)
+      }
+
+      val colFamilyHandles = new java.util.ArrayList[ColumnFamilyHandle]()
+      db = NativeRocksDB.open(new DBOptions(dbOptions), workingDir.toString,
+        colFamilyDescriptors.asJava, colFamilyHandles)
+
+      // Store the mapping of names to handles in the internal map
+      colFamilyHandles.asScala.toList.map(

Review Comment:
   Done



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala:
##########
@@ -604,17 +728,57 @@ class RocksDB(
   }
 
   private def getDBProperty(property: String): Long = {
-    db.getProperty(property).toLong
+    if (useColumnFamilies) {
+      // get cumulative sum across all available column families
+      assert(!colFamilyNameToHandleMap.isEmpty)
+      colFamilyNameToHandleMap
+        .values
+        .map(handle => db.getProperty(handle, property).toLong)
+        .sum
+    } else {
+      db.getProperty(property).toLong
+    }
   }
 
   private def openDB(): Unit = {
     assert(db == null)
-    db = NativeRocksDB.open(dbOptions, workingDir.toString)
-    logInfo(s"Opened DB with conf ${conf}")
+    if (useColumnFamilies) {
+      val colFamilies = NativeRocksDB.listColumnFamilies(dbOptions, workingDir.toString)
+
+      var colFamilyDescriptors: Seq[ColumnFamilyDescriptor] = Seq.empty[ColumnFamilyDescriptor]
+      // populate the list of available col family descriptors
+      colFamilies.asScala.toList.foreach(family => {

Review Comment:
   Done



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1445740654


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImpl.scala:
##########
@@ -0,0 +1,105 @@
+/*
+ * 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
+
+import java.io.Serializable
+
+import org.apache.commons.lang3.SerializationUtils
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.execution.streaming.state.StateStore
+import org.apache.spark.sql.streaming.ValueState
+import org.apache.spark.sql.types._
+
+/**
+ * Class that provides a concrete implementation for a single value state associated with state
+ * variables used in the streaming transformWithState operator.
+ * @param store - reference to the StateStore instance to be used for storing state
+ * @param stateName - name of logical state partition
+ * @tparam S - data type of object that will be stored
+ */
+class ValueStateImpl[S](
+    store: StateStore,
+    stateName: String) extends ValueState[S] with Logging{
+
+  private def encodeKey(): UnsafeRow = {
+    val keyOption = ImplicitKeyTracker.getImplicitKeyOption
+    if (!keyOption.isDefined) {
+      throw new UnsupportedOperationException("Implicit key not found for operation on" +
+        s"stateName=$stateName")
+    }
+
+    val schemaForKeyRow: StructType = new StructType().add("key", BinaryType)
+    val keyByteArr = SerializationUtils.serialize(keyOption.get.asInstanceOf[Serializable])

Review Comment:
   https://www.databricks.com/blog/2016/01/04/introducing-apache-spark-datasets.html
   
   Check `Lightning-fast Serialization with Encoders`. You can see how much performance benefit we get on serde if we make the custom types be bound to Spark SQL type system. Combining two UnsafeRows into one could be also done via either flattening schema and data (concatenate) or nested model.
   
   Users may have to provide the encoder for types on state (value for all types of state, key for map state) which may be burdensome, but that is what Flink requires users to do as well - it requires type information (at least class), which is to perform the magic behind the scene. They build their own type system, not just simply going through Java's Serializable.
   
   ```
       @Override
       public void open(OpenContext openContext) throws Exception {
           state = getRuntimeContext().getState(new ValueStateDescriptor<>("myState", CountWithTimestamp.class));
       }
   ```
   
   https://nightlies.apache.org/flink/flink-docs-release-1.18/docs/dev/datastream/fault-tolerance/serialization/types_serialization/#data-types--serialization



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1445746872


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImpl.scala:
##########
@@ -0,0 +1,105 @@
+/*
+ * 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
+
+import java.io.Serializable
+
+import org.apache.commons.lang3.SerializationUtils
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.execution.streaming.state.StateStore
+import org.apache.spark.sql.streaming.ValueState
+import org.apache.spark.sql.types._
+
+/**
+ * Class that provides a concrete implementation for a single value state associated with state
+ * variables used in the streaming transformWithState operator.
+ * @param store - reference to the StateStore instance to be used for storing state
+ * @param stateName - name of logical state partition
+ * @tparam S - data type of object that will be stored
+ */
+class ValueStateImpl[S](
+    store: StateStore,
+    stateName: String) extends ValueState[S] with Logging{
+
+  private def encodeKey(): UnsafeRow = {
+    val keyOption = ImplicitKeyTracker.getImplicitKeyOption
+    if (!keyOption.isDefined) {
+      throw new UnsupportedOperationException("Implicit key not found for operation on" +
+        s"stateName=$stateName")
+    }
+
+    val schemaForKeyRow: StructType = new StructType().add("key", BinaryType)
+    val keyByteArr = SerializationUtils.serialize(keyOption.get.asInstanceOf[Serializable])

Review Comment:
   Note that this cannot be TODO unless the JIRA ticket for TODO is marked as blocker. The serialization format for persistence is really painful to change.



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1432080018


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala:
##########
@@ -204,12 +211,18 @@ class RocksDB(
     for (v <- loadedVersion + 1 to endVersion) {
       var changelogReader: StateStoreChangelogReader = null
       try {
-        changelogReader = fileManager.getChangelogReader(v)
-        changelogReader.foreach { case (key, value) =>
-          if (value != null) {
-            put(key, value)
-          } else {
-            remove(key)
+        changelogReader = fileManager.getChangelogReader(v, useColumnFamilies)
+        changelogReader.foreach { case (recordType, key, value, colFamilyName) =>

Review Comment:
   Updated this to use a single byte



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1432193162


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala:
##########
@@ -377,119 +380,247 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared
     }
   }
 
-  test("RocksDB: compression conf") {
-    val remoteDir = Utils.createTempDir().toString
-    new File(remoteDir).delete()  // to make sure that the directory gets created
+  Seq(true, false).foreach { useColumnFamilies =>
+    test(s"RocksDB: compression conf with useColumnFamilies=$useColumnFamilies") {
+      val remoteDir = Utils.createTempDir().toString
+      new File(remoteDir).delete() // to make sure that the directory gets created
 
-    val conf = RocksDBConf().copy(compression = "zstd")
-    withDB(remoteDir, conf = conf) { db =>
-      assert(db.columnFamilyOptions.compressionType() == CompressionType.ZSTD_COMPRESSION)
-    }
+      val conf = RocksDBConf().copy(compression = "zstd")
+      withDB(remoteDir, conf = conf, useColumnFamilies = useColumnFamilies) { db =>
+        assert(db.columnFamilyOptions.compressionType() == CompressionType.ZSTD_COMPRESSION)
+      }
 
-    // Test the default is LZ4
-    withDB(remoteDir, conf = RocksDBConf().copy()) { db =>
-      assert(db.columnFamilyOptions.compressionType() == CompressionType.LZ4_COMPRESSION)
+      // Test the default is LZ4
+      withDB(remoteDir, conf = RocksDBConf().copy(), useColumnFamilies = useColumnFamilies) { db =>
+        assert(db.columnFamilyOptions.compressionType() == CompressionType.LZ4_COMPRESSION)
+      }
     }
   }
 
-  test("RocksDB: get, put, iterator, commit, load") {
-    def testOps(compactOnCommit: Boolean): Unit = {
-      val remoteDir = Utils.createTempDir().toString
-      new File(remoteDir).delete()  // to make sure that the directory gets created
+  Seq(true, false).foreach { useColumnFamilies =>
+    test(s"RocksDB: get, put, iterator, commit, load with " +

Review Comment:
   Not sure what you mean ? Its calling `test` itself right - this one is for zero or single col family. The other one is for multiple column families



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1430758033


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala:
##########
@@ -204,12 +211,18 @@ class RocksDB(
     for (v <- loadedVersion + 1 to endVersion) {
       var changelogReader: StateStoreChangelogReader = null
       try {
-        changelogReader = fileManager.getChangelogReader(v)
-        changelogReader.foreach { case (key, value) =>
-          if (value != null) {
-            put(key, value)
-          } else {
-            remove(key)
+        changelogReader = fileManager.getChangelogReader(v, useColumnFamilies)
+        changelogReader.foreach { case (recordType, key, value, colFamilyName) =>

Review Comment:
   Then can we use the approach of tombstone like we did instead of adding a huge overhead on recordType? Do you foresee multiple types?



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1430872986


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala:
##########
@@ -204,12 +211,18 @@ class RocksDB(
     for (v <- loadedVersion + 1 to endVersion) {
       var changelogReader: StateStoreChangelogReader = null
       try {
-        changelogReader = fileManager.getChangelogReader(v)
-        changelogReader.foreach { case (key, value) =>
-          if (value != null) {
-            put(key, value)
-          } else {
-            remove(key)
+        changelogReader = fileManager.getChangelogReader(v, useColumnFamilies)
+        changelogReader.foreach { case (recordType, key, value, colFamilyName) =>

Review Comment:
   We will also have to add the `merge` record type in the future



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1430748052


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateExec.scala:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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
+
+import java.util.concurrent.TimeUnit.NANOSECONDS
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, Expression, SortOrder, UnsafeRow}
+import org.apache.spark.sql.catalyst.plans.physical.Distribution
+import org.apache.spark.sql.execution._
+import org.apache.spark.sql.execution.streaming.state._
+import org.apache.spark.sql.streaming.{OutputMode, StatefulProcessor}
+import org.apache.spark.sql.types._
+import org.apache.spark.util.CompletionIterator
+
+/**
+ * Physical operator for executing `TransformWithState`
+ *
+ * @param statefulProcessor processor methods called on underlying data
+ * @param keyDeserializer used to extract the key object for each group.
+ * @param valueDeserializer used to extract the items in the iterator from an input row.
+ * @param groupingAttributes used to group the data
+ * @param dataAttributes used to read the data
+ * @param outputObjAttr Defines the output object
+ * @param batchTimestampMs processing timestamp of the current batch.
+ * @param eventTimeWatermarkForLateEvents event time watermark for filtering late events
+ * @param eventTimeWatermarkForEviction event time watermark for state eviction
+ * @param child the physical plan for the underlying data
+ */
+case class TransformWithStateExec(
+    keyDeserializer: Expression,
+    valueDeserializer: Expression,
+    groupingAttributes: Seq[Attribute],
+    dataAttributes: Seq[Attribute],
+    statefulProcessor: StatefulProcessor[Any, Any, Any],
+    outputMode: OutputMode,
+    outputObjAttr: Attribute,
+    stateInfo: Option[StatefulOperatorStateInfo],
+    batchTimestampMs: Option[Long],
+    eventTimeWatermarkForLateEvents: Option[Long],
+    eventTimeWatermarkForEviction: Option[Long],
+    child: SparkPlan)
+  extends UnaryExecNode
+    with StateStoreWriter
+    with WatermarkSupport
+    with ObjectProducerExec {
+
+  override def shortName: String = "transformWithStateExec"
+
+  override def shouldRunAnotherBatch(newInputWatermark: Long): Boolean = false

Review Comment:
   Ah OK, no timer now. Could you please add TODO comment so that we don't forget? No need to add JIRA ticket as of now, as we need to pass SPIP to move forward.



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1431076826


##########
sql/api/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TransformWithStateTimeoutModes.scala:
##########
@@ -0,0 +1,22 @@
+/*
+ * 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.catalyst.plans.logical
+
+import org.apache.spark.sql.streaming.TimeoutMode
+
+/** Types of timeouts used in tranformWithState operator */
+case object noTimeouts extends TimeoutMode

Review Comment:
   That is super odd - don't understand why this is caught by Java style at all.



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1440220450


##########
sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessor.scala:
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.streaming
+
+import java.io.Serializable
+
+import org.apache.spark.annotation.{Evolving, Experimental}
+
+/**
+ * Represents the arbitrary stateful logic that needs to be provided by the user to perform
+ * stateful manipulations on keyed streams.
+ */
+@Experimental
+@Evolving
+trait StatefulProcessor[K, I, O] extends Serializable {
+
+  /**
+   * Function that will be invoked as the first method that allows for users to
+   * initialize all their state variables and perform other init actions before handling data.
+   * @param handle - reference to the statefulProcessorHandle that the user can use to perform
+   *               future actions
+   * @param outputMode - output mode for the stateful processor
+   */
+  def init(
+    handle: StatefulProcessorHandle,
+    outputMode: OutputMode): Unit
+
+  /**
+   * Function that will allow users to interact with input data rows along with the grouping key
+   * and current timer values and optionally provide output rows.
+   * @param key - grouping key
+   * @param inputRows - iterator of input rows associated with grouping key
+   * @param timerValues - instance of TimerValues that provides access to current processing/event
+   *                    time if available
+   * @return - Zero or more output rows
+   */
+  def handleInputRows(

Review Comment:
   Updated to use `handleInputRow`



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1438622114


##########
sql/api/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TransformWithStateTimeoutModes.scala:
##########
@@ -0,0 +1,22 @@
+/*
+ * 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.catalyst.plans.logical
+
+import org.apache.spark.sql.streaming.TimeoutMode
+
+/** Types of timeouts used in tranformWithState operator */
+case object noTimeouts extends TimeoutMode

Review Comment:
   Had to add a suppression - fixed



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1439275819


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala:
##########
@@ -78,9 +78,9 @@ private[sql] class HDFSBackedStateStoreProvider extends StateStoreProvider with
 
     override def id: StateStoreId = HDFSBackedStateStoreProvider.this.stateStoreId
 
-    override def get(key: UnsafeRow): UnsafeRow = map.get(key)
+    override def get(key: UnsafeRow, colFamilyName: String): UnsafeRow = map.get(key)
 
-    override def iterator(): Iterator[UnsafeRowPair] = {
+    override def iterator(colFamilyName: String): Iterator[UnsafeRowPair] = {

Review Comment:
   Would prefer to keep this as it is - basically we would add support for hdfs provider eventually anyway I think



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1430858354


##########
sql/api/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TransformWithStateTimeoutModes.scala:
##########
@@ -0,0 +1,22 @@
+/*
+ * 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.catalyst.plans.logical
+
+import org.apache.spark.sql.streaming.TimeoutMode
+
+/** Types of timeouts used in tranformWithState operator */
+case object noTimeouts extends TimeoutMode

Review Comment:
   nit: CamelCase



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala:
##########
@@ -719,6 +719,36 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
     }
   }
 
+  /**
+   * Strategy to convert [[TransformWithState]] logical operator to physical operator
+   * in streaming plans.
+   */
+   object TransformWithStateStrategy extends Strategy {
+    override def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {

Review Comment:
   nit: 2 spaces (looks to be 1 space)



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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
+
+import java.util.UUID
+
+import org.apache.spark.TaskContext
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.execution.streaming.state.StateStore
+import org.apache.spark.sql.streaming.{QueryInfo, StatefulProcessorHandle, ValueState}
+import org.apache.spark.util.Utils
+
+/**
+ * Object used to assign/retrieve/remove grouping key passed implicitly for various state
+ * manipulation actions using the store handle.
+ */
+object ImplicitKeyTracker {
+  val implicitKey: InheritableThreadLocal[Any] = new InheritableThreadLocal[Any]
+
+  def getImplicitKeyOption: Option[Any] = Option(implicitKey.get())
+
+  def setImplicitKey(key: Any): Unit = implicitKey.set(key)
+
+  def removeImplicitKey(): Unit = implicitKey.remove()
+}
+
+/**
+ * Enum used to track valid states for the StatefulProcessorHandle
+ */
+object StatefulProcessorHandleState extends Enumeration {
+  type StatefulProcessorHandleState = Value
+  val CREATED, INITIALIZED, DATA_PROCESSED, CLOSED = Value
+}
+
+class QueryInfoImpl(
+    val queryId: UUID,
+    val runId: UUID,
+    val batchId: Long,
+    val operatorId: Long,
+    val partitionId: Int) extends QueryInfo {
+
+  override def getQueryId: UUID = queryId
+
+  override def getRunId: UUID = runId
+
+  override def getBatchId: Long = batchId
+
+  override def getOperatorId: Long = operatorId
+
+  override def getPartitionId: Int = partitionId
+
+  override def toString: String = {
+    s"QueryInfo(queryId=$queryId, runId=$runId, batchId=$batchId, operatorId=$operatorId, " +
+      s"partitionId=$partitionId)"
+  }
+}
+
+/**
+ * Class that provides a concrete implementation of a StatefulProcessorHandle. Note that we keep
+ * track of valid transitions as various functions are invoked to track object lifecycle.
+ * @param store - instance of state store
+ */
+class StatefulProcessorHandleImpl(store: StateStore, runId: UUID)

Review Comment:
   If we can pass over the params to constructor or so, let's do that. You can even add companion object with apply method to do the magic with TaskContext. Let's avoid the instance itself be tied to the thread.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreChangelog.scala:
##########
@@ -33,12 +33,20 @@ import org.apache.spark.sql.execution.streaming.CheckpointFileManager.Cancellabl
 import org.apache.spark.util.NextIterator
 
 /**
- * Write changes to the key value state store instance to a changelog file.
- * There are 2 types of records, put and delete.
- * A put record is written as: | key length | key content | value length | value content |
- * A delete record is written as: | key length | key content | -1 |
- * Write an Int -1 to signal the end of file.
- * The overall changelog format is: | put record | delete record | ... | put record | -1 |
+ * Enum used to write record types to changelog files used with RocksDBStateStoreProvider.
+ */
+object RecordType extends Enumeration {
+  type RecordType = Value
+
+  val PUT_RECORD = Value("put_record")

Review Comment:
   Maybe you no longer need to have string representation if we go with 1 byte marker.



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala:
##########
@@ -503,11 +634,79 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared
     changelogWriter.commit()
     val changelogReader = fileManager.getChangelogReader(1)
     val entries = changelogReader.toSeq
-    val expectedEntries = (1 to 5).map(i => (i.toString.getBytes, i.toString.getBytes)) ++
-      (2 to 4).map(j => (j.toString.getBytes, null))
+    val expectedEntries = (1 to 5).map(i =>
+      (RecordType.PUT_RECORD, i.toString.getBytes,
+        i.toString.getBytes, StateStore.DEFAULT_COL_FAMILY_NAME)) ++
+      (2 to 4).map(j => (RecordType.DELETE_RECORD, j.toString.getBytes,
+        null, StateStore.DEFAULT_COL_FAMILY_NAME))
+    assert(entries.size == expectedEntries.size)
+    entries.zip(expectedEntries).map{
+      case (e1, e2) => assert(e1._1 === e2._1 && e1._2 === e2._2 &&
+        e1._3 === e2._3 && e1._4 === e2._4)
+    }
+  }
+
+  testWithChangelogCheckpointingEnabled(
+    "RocksDBFileManager: read and write v2 changelog with default col family") {
+    val dfsRootDir = new File(Utils.createTempDir().getAbsolutePath + "/state/1/1")
+    val fileManager = new RocksDBFileManager(
+      dfsRootDir.getAbsolutePath, Utils.createTempDir(), new Configuration)
+    val changelogWriter = fileManager.getChangeLogWriter(1, true)
+    for (i <- 1 to 5) changelogWriter.put(i.toString, i.toString,
+      StateStore.DEFAULT_COL_FAMILY_NAME)
+    for (j <- 2 to 4) changelogWriter.delete(j.toString, StateStore.DEFAULT_COL_FAMILY_NAME)
+    changelogWriter.commit()
+    val changelogReader = fileManager.getChangelogReader(1, true)
+    val entries = changelogReader.toSeq
+    val expectedEntries = (1 to 5).map(i =>
+      (RecordType.PUT_RECORD, i.toString.getBytes,
+        i.toString.getBytes, StateStore.DEFAULT_COL_FAMILY_NAME)) ++
+      (2 to 4).map(j => (RecordType.DELETE_RECORD, j.toString.getBytes,
+        null, StateStore.DEFAULT_COL_FAMILY_NAME))
+    assert(entries.size == expectedEntries.size)
+    entries.zip(expectedEntries).map{
+      case (e1, e2) => assert(e1._1 === e2._1 && e1._2 === e2._2 &&
+        e1._3 === e2._3 && e1._4 === e2._4)
+    }
+  }
+
+  testWithChangelogCheckpointingEnabled(
+    "RocksDBFileManager: read and write v2 changelog with multiple col families") {
+    val dfsRootDir = new File(Utils.createTempDir().getAbsolutePath + "/state/1/1")
+    val testColFamily1: String = "testColFamily1"
+    val testColFamily2: String = "testColFamily2"
+    val fileManager = new RocksDBFileManager(
+      dfsRootDir.getAbsolutePath, Utils.createTempDir(), new Configuration)
+    val changelogWriter = fileManager.getChangeLogWriter(1, true)
+    for (i <- 1 to 5) changelogWriter.put(i.toString, i.toString,
+      testColFamily1)
+    for (j <- 2 to 4) changelogWriter.delete(j.toString, testColFamily1)
+
+    for (i <- 1 to 5) changelogWriter.put(i.toString, i.toString,
+      testColFamily2)
+    for (j <- 2 to 4) changelogWriter.delete(j.toString, testColFamily2)
+
+    changelogWriter.commit()
+    val changelogReader = fileManager.getChangelogReader(1, true)
+    val entries = changelogReader.toSeq
+    val expectedEntriesForColFamily1 = (1 to 5).map(i =>
+      (RecordType.PUT_RECORD, i.toString.getBytes,
+        i.toString.getBytes, testColFamily1)) ++
+      (2 to 4).map(j => (RecordType.DELETE_RECORD, j.toString.getBytes,

Review Comment:
   ditto



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala:
##########
@@ -503,11 +634,79 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared
     changelogWriter.commit()
     val changelogReader = fileManager.getChangelogReader(1)
     val entries = changelogReader.toSeq
-    val expectedEntries = (1 to 5).map(i => (i.toString.getBytes, i.toString.getBytes)) ++
-      (2 to 4).map(j => (j.toString.getBytes, null))
+    val expectedEntries = (1 to 5).map(i =>
+      (RecordType.PUT_RECORD, i.toString.getBytes,
+        i.toString.getBytes, StateStore.DEFAULT_COL_FAMILY_NAME)) ++
+      (2 to 4).map(j => (RecordType.DELETE_RECORD, j.toString.getBytes,
+        null, StateStore.DEFAULT_COL_FAMILY_NAME))
+    assert(entries.size == expectedEntries.size)
+    entries.zip(expectedEntries).map{
+      case (e1, e2) => assert(e1._1 === e2._1 && e1._2 === e2._2 &&
+        e1._3 === e2._3 && e1._4 === e2._4)
+    }
+  }
+
+  testWithChangelogCheckpointingEnabled(
+    "RocksDBFileManager: read and write v2 changelog with default col family") {
+    val dfsRootDir = new File(Utils.createTempDir().getAbsolutePath + "/state/1/1")
+    val fileManager = new RocksDBFileManager(
+      dfsRootDir.getAbsolutePath, Utils.createTempDir(), new Configuration)
+    val changelogWriter = fileManager.getChangeLogWriter(1, true)
+    for (i <- 1 to 5) changelogWriter.put(i.toString, i.toString,
+      StateStore.DEFAULT_COL_FAMILY_NAME)
+    for (j <- 2 to 4) changelogWriter.delete(j.toString, StateStore.DEFAULT_COL_FAMILY_NAME)
+    changelogWriter.commit()
+    val changelogReader = fileManager.getChangelogReader(1, true)
+    val entries = changelogReader.toSeq
+    val expectedEntries = (1 to 5).map(i =>
+      (RecordType.PUT_RECORD, i.toString.getBytes,
+        i.toString.getBytes, StateStore.DEFAULT_COL_FAMILY_NAME)) ++
+      (2 to 4).map(j => (RecordType.DELETE_RECORD, j.toString.getBytes,
+        null, StateStore.DEFAULT_COL_FAMILY_NAME))
+    assert(entries.size == expectedEntries.size)
+    entries.zip(expectedEntries).map{
+      case (e1, e2) => assert(e1._1 === e2._1 && e1._2 === e2._2 &&
+        e1._3 === e2._3 && e1._4 === e2._4)
+    }
+  }
+
+  testWithChangelogCheckpointingEnabled(
+    "RocksDBFileManager: read and write v2 changelog with multiple col families") {
+    val dfsRootDir = new File(Utils.createTempDir().getAbsolutePath + "/state/1/1")
+    val testColFamily1: String = "testColFamily1"
+    val testColFamily2: String = "testColFamily2"
+    val fileManager = new RocksDBFileManager(
+      dfsRootDir.getAbsolutePath, Utils.createTempDir(), new Configuration)
+    val changelogWriter = fileManager.getChangeLogWriter(1, true)
+    for (i <- 1 to 5) changelogWriter.put(i.toString, i.toString,
+      testColFamily1)
+    for (j <- 2 to 4) changelogWriter.delete(j.toString, testColFamily1)
+
+    for (i <- 1 to 5) changelogWriter.put(i.toString, i.toString,
+      testColFamily2)
+    for (j <- 2 to 4) changelogWriter.delete(j.toString, testColFamily2)
+
+    changelogWriter.commit()
+    val changelogReader = fileManager.getChangelogReader(1, true)
+    val entries = changelogReader.toSeq
+    val expectedEntriesForColFamily1 = (1 to 5).map(i =>
+      (RecordType.PUT_RECORD, i.toString.getBytes,
+        i.toString.getBytes, testColFamily1)) ++
+      (2 to 4).map(j => (RecordType.DELETE_RECORD, j.toString.getBytes,
+        null, testColFamily1))
+
+    val expectedEntriesForColFamily2 = (1 to 5).map(i =>
+      (RecordType.PUT_RECORD, i.toString.getBytes,
+        i.toString.getBytes, testColFamily2)) ++
+      (2 to 4).map(j => (RecordType.DELETE_RECORD, j.toString.getBytes,

Review Comment:
   ditto



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala:
##########
@@ -503,11 +634,79 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared
     changelogWriter.commit()
     val changelogReader = fileManager.getChangelogReader(1)
     val entries = changelogReader.toSeq
-    val expectedEntries = (1 to 5).map(i => (i.toString.getBytes, i.toString.getBytes)) ++
-      (2 to 4).map(j => (j.toString.getBytes, null))
+    val expectedEntries = (1 to 5).map(i =>
+      (RecordType.PUT_RECORD, i.toString.getBytes,
+        i.toString.getBytes, StateStore.DEFAULT_COL_FAMILY_NAME)) ++
+      (2 to 4).map(j => (RecordType.DELETE_RECORD, j.toString.getBytes,
+        null, StateStore.DEFAULT_COL_FAMILY_NAME))
+    assert(entries.size == expectedEntries.size)
+    entries.zip(expectedEntries).map{
+      case (e1, e2) => assert(e1._1 === e2._1 && e1._2 === e2._2 &&
+        e1._3 === e2._3 && e1._4 === e2._4)
+    }
+  }
+
+  testWithChangelogCheckpointingEnabled(
+    "RocksDBFileManager: read and write v2 changelog with default col family") {
+    val dfsRootDir = new File(Utils.createTempDir().getAbsolutePath + "/state/1/1")
+    val fileManager = new RocksDBFileManager(
+      dfsRootDir.getAbsolutePath, Utils.createTempDir(), new Configuration)
+    val changelogWriter = fileManager.getChangeLogWriter(1, true)
+    for (i <- 1 to 5) changelogWriter.put(i.toString, i.toString,
+      StateStore.DEFAULT_COL_FAMILY_NAME)
+    for (j <- 2 to 4) changelogWriter.delete(j.toString, StateStore.DEFAULT_COL_FAMILY_NAME)
+    changelogWriter.commit()
+    val changelogReader = fileManager.getChangelogReader(1, true)
+    val entries = changelogReader.toSeq
+    val expectedEntries = (1 to 5).map(i =>
+      (RecordType.PUT_RECORD, i.toString.getBytes,
+        i.toString.getBytes, StateStore.DEFAULT_COL_FAMILY_NAME)) ++
+      (2 to 4).map(j => (RecordType.DELETE_RECORD, j.toString.getBytes,

Review Comment:
   ditto



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala:
##########
@@ -896,63 +918,69 @@ abstract class StateStoreSuiteBase[ProviderClass <: StateStoreProvider]
   protected val keySchema: StructType = StateStoreTestsHelper.keySchema
   protected val valueSchema: StructType = StateStoreTestsHelper.valueSchema
 
-  testWithAllCodec("get, put, remove, commit, and all data iterator") {
-    tryWithProviderResource(newStoreProvider()) { provider =>
-      // Verify state before starting a new set of updates
-      assert(getLatestData(provider).isEmpty)
-
-      val store = provider.getStore(0)
-      assert(!store.hasCommitted)
-      assert(get(store, "a", 0) === None)
-      assert(store.iterator().isEmpty)
-      assert(store.metrics.numKeys === 0)
-
-      // Verify state after updating
-      put(store, "a", 0, 1)
-      assert(get(store, "a", 0) === Some(1))
-
-      assert(store.iterator().nonEmpty)
-      assert(getLatestData(provider).isEmpty)
-
-      // Make updates, commit and then verify state
-      put(store, "b", 0, 2)
-      put(store, "aa", 0, 3)
-      remove(store, _._1.startsWith("a"))
-      assert(store.commit() === 1)
-
-      assert(store.hasCommitted)
-      assert(rowPairsToDataSet(store.iterator()) === Set(("b", 0) -> 2))
-      assert(getLatestData(provider) === Set(("b", 0) -> 2))
-
-      // Trying to get newer versions should fail
-      var e = intercept[SparkException] {
-        provider.getStore(2)
-      }
-      assert(e.getCause.isInstanceOf[SparkException])
-      assert(e.getCause.getMessage.contains("does not exist"))
-
-      e = intercept[SparkException] {
-        getData(provider, 2)
-      }
-      assert(e.getCause.isInstanceOf[SparkException])
-      assert(e.getCause.getMessage.contains("does not exist"))
+  Seq(true, false).foreach { useColumnFamilies =>

Review Comment:
   Ideally we should figure out how to bring up test for matrix test, but let's put this aside for now.



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala:
##########
@@ -503,11 +634,79 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared
     changelogWriter.commit()
     val changelogReader = fileManager.getChangelogReader(1)
     val entries = changelogReader.toSeq
-    val expectedEntries = (1 to 5).map(i => (i.toString.getBytes, i.toString.getBytes)) ++
-      (2 to 4).map(j => (j.toString.getBytes, null))
+    val expectedEntries = (1 to 5).map(i =>
+      (RecordType.PUT_RECORD, i.toString.getBytes,
+        i.toString.getBytes, StateStore.DEFAULT_COL_FAMILY_NAME)) ++
+      (2 to 4).map(j => (RecordType.DELETE_RECORD, j.toString.getBytes,
+        null, StateStore.DEFAULT_COL_FAMILY_NAME))
+    assert(entries.size == expectedEntries.size)
+    entries.zip(expectedEntries).map{
+      case (e1, e2) => assert(e1._1 === e2._1 && e1._2 === e2._2 &&
+        e1._3 === e2._3 && e1._4 === e2._4)
+    }
+  }
+
+  testWithChangelogCheckpointingEnabled(
+    "RocksDBFileManager: read and write v2 changelog with default col family") {
+    val dfsRootDir = new File(Utils.createTempDir().getAbsolutePath + "/state/1/1")
+    val fileManager = new RocksDBFileManager(
+      dfsRootDir.getAbsolutePath, Utils.createTempDir(), new Configuration)
+    val changelogWriter = fileManager.getChangeLogWriter(1, true)
+    for (i <- 1 to 5) changelogWriter.put(i.toString, i.toString,
+      StateStore.DEFAULT_COL_FAMILY_NAME)
+    for (j <- 2 to 4) changelogWriter.delete(j.toString, StateStore.DEFAULT_COL_FAMILY_NAME)
+    changelogWriter.commit()
+    val changelogReader = fileManager.getChangelogReader(1, true)
+    val entries = changelogReader.toSeq
+    val expectedEntries = (1 to 5).map(i =>
+      (RecordType.PUT_RECORD, i.toString.getBytes,
+        i.toString.getBytes, StateStore.DEFAULT_COL_FAMILY_NAME)) ++
+      (2 to 4).map(j => (RecordType.DELETE_RECORD, j.toString.getBytes,
+        null, StateStore.DEFAULT_COL_FAMILY_NAME))
+    assert(entries.size == expectedEntries.size)
+    entries.zip(expectedEntries).map{
+      case (e1, e2) => assert(e1._1 === e2._1 && e1._2 === e2._2 &&
+        e1._3 === e2._3 && e1._4 === e2._4)
+    }
+  }
+
+  testWithChangelogCheckpointingEnabled(
+    "RocksDBFileManager: read and write v2 changelog with multiple col families") {
+    val dfsRootDir = new File(Utils.createTempDir().getAbsolutePath + "/state/1/1")
+    val testColFamily1: String = "testColFamily1"
+    val testColFamily2: String = "testColFamily2"
+    val fileManager = new RocksDBFileManager(
+      dfsRootDir.getAbsolutePath, Utils.createTempDir(), new Configuration)
+    val changelogWriter = fileManager.getChangeLogWriter(1, true)
+    for (i <- 1 to 5) changelogWriter.put(i.toString, i.toString,

Review Comment:
   ditto for following 4 lines



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ValueStateSuite.scala:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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
+
+import java.util.UUID
+
+import scala.util.Random
+
+import org.apache.hadoop.conf.Configuration
+import org.scalatest.BeforeAndAfter
+
+import org.apache.spark.sql.execution.streaming.{ImplicitKeyTracker, StatefulProcessorHandleImpl}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.streaming.ValueState
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types._
+
+/**
+ * Class that adds tests for single value ValueState types used in arbitrary stateful
+ * operators such as transformWithState
+ */
+class ValueStateSuite extends SharedSparkSession
+  with BeforeAndAfter {
+
+  before {
+    StateStore.stop()
+    require(!StateStore.isMaintenanceRunning)
+  }
+
+  after {
+    StateStore.stop()
+    require(!StateStore.isMaintenanceRunning)
+  }
+
+  import StateStoreTestsHelper._
+
+  val schemaForKeyRow: StructType = new StructType().add("key", BinaryType)
+
+  val schemaForValueRow: StructType = new StructType().add("value", BinaryType)
+
+  private def newStoreProviderWithValueState(useColumnFamilies: Boolean):
+    RocksDBStateStoreProvider = {
+    newStoreProviderWithValueState(StateStoreId(newDir(), Random.nextInt(), 0),
+      numColsPrefixKey = 0,
+      useColumnFamilies = useColumnFamilies)
+  }
+
+  private def newStoreProviderWithValueState(
+    storeId: StateStoreId,
+    numColsPrefixKey: Int,
+    sqlConf: Option[SQLConf] = None,
+    conf: Configuration = new Configuration,
+    useColumnFamilies: Boolean = false): RocksDBStateStoreProvider = {
+    val provider = new RocksDBStateStoreProvider()
+    provider.init(
+      storeId, schemaForKeyRow, schemaForValueRow, numColsPrefixKey = numColsPrefixKey,
+      useColumnFamilies,
+      new StateStoreConf(sqlConf.getOrElse(SQLConf.get)), conf)
+    provider
+  }
+
+  private def tryWithProviderResource[T](
+    provider: StateStoreProvider)(f: StateStoreProvider => T): T = {
+    try {
+      f(provider)
+    } finally {
+      provider.close()
+    }
+  }
+
+  test("Implicit key operations") {
+    tryWithProviderResource(newStoreProviderWithValueState(true)) { provider =>
+      val store = provider.getStore(0)
+      val handle = new StatefulProcessorHandleImpl(store, UUID.randomUUID())
+      assert(handle.getQueryInfo().getPartitionId === 0)
+
+      val testState: ValueState[Long] = handle.getValueState[Long]("testState")
+      assert(ImplicitKeyTracker.getImplicitKeyOption.isEmpty)
+      val ex = intercept[Exception] {
+        testState.update(123)
+      }
+
+      assert(ex.isInstanceOf[UnsupportedOperationException])
+      assert(ex.getMessage.contains("Implicit key not found"))
+      ImplicitKeyTracker.setImplicitKey("test_key")
+      assert(ImplicitKeyTracker.getImplicitKeyOption.isDefined)
+      testState.update(123)

Review Comment:
   read back to verify the update?



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ValueStateSuite.scala:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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
+
+import java.util.UUID
+
+import scala.util.Random
+
+import org.apache.hadoop.conf.Configuration
+import org.scalatest.BeforeAndAfter
+
+import org.apache.spark.sql.execution.streaming.{ImplicitKeyTracker, StatefulProcessorHandleImpl}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.streaming.ValueState
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types._
+
+/**
+ * Class that adds tests for single value ValueState types used in arbitrary stateful
+ * operators such as transformWithState
+ */
+class ValueStateSuite extends SharedSparkSession
+  with BeforeAndAfter {
+
+  before {
+    StateStore.stop()
+    require(!StateStore.isMaintenanceRunning)
+  }
+
+  after {
+    StateStore.stop()
+    require(!StateStore.isMaintenanceRunning)
+  }
+
+  import StateStoreTestsHelper._
+
+  val schemaForKeyRow: StructType = new StructType().add("key", BinaryType)
+
+  val schemaForValueRow: StructType = new StructType().add("value", BinaryType)
+
+  private def newStoreProviderWithValueState(useColumnFamilies: Boolean):
+    RocksDBStateStoreProvider = {
+    newStoreProviderWithValueState(StateStoreId(newDir(), Random.nextInt(), 0),
+      numColsPrefixKey = 0,
+      useColumnFamilies = useColumnFamilies)
+  }
+
+  private def newStoreProviderWithValueState(
+    storeId: StateStoreId,

Review Comment:
   nit: 4 spaces



##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.streaming
+
+import org.apache.spark.SparkException
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{AnalysisException, SaveMode}
+import org.apache.spark.sql.execution.streaming._
+import org.apache.spark.sql.execution.streaming.state.{AlsoTestWithChangelogCheckpointingEnabled, RocksDBStateStoreProvider}
+import org.apache.spark.sql.internal.SQLConf
+
+class RunningCountStatefulProcessor extends StatefulProcessor[String, String, (String, String)]
+  with Logging {
+  @transient private var _countState: ValueState[Long] = _
+  @transient var _processorHandle: StatefulProcessorHandle = _
+
+  override def init(handle: StatefulProcessorHandle,
+    outputMode: OutputMode) : Unit = {
+    _processorHandle = handle
+    assert(handle.getQueryInfo().getBatchId >= 0)
+    assert(handle.getQueryInfo().getOperatorId == 0)
+    assert(handle.getQueryInfo().getPartitionId >= 0 && handle.getQueryInfo().getPartitionId < 5)
+    _countState = _processorHandle.getValueState[Long]("countState")
+  }
+
+  override def handleInputRows(
+      key: String,
+      inputRows: Iterator[String],
+      timerValues: TimerValues): Iterator[(String, String)] = {
+    val count = _countState.getOption().getOrElse(0L) + inputRows.size
+    if (count == 3) {
+      _countState.remove()
+      Iterator.empty
+    } else {
+      _countState.update(count)
+      Iterator((key, count.toString))
+    }
+  }
+
+  override def close(): Unit = {}
+}
+
+class RunningCountStatefulProcessorWithError extends RunningCountStatefulProcessor {
+  @transient private var _tempState: ValueState[Long] = _
+
+  override def handleInputRows(key: String,

Review Comment:
   nit: follow multi-line style



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreChangelog.scala:
##########
@@ -109,18 +119,94 @@ class StateStoreChangelogWriter(
   }
 }
 
+/**
+ * Write changes to the key value state store instance to a changelog file.
+ * There are 2 types of records, put and delete.
+ * A put record is written as: | key length | key content | value length | value content |
+ * A delete record is written as: | key length | key content | -1 |
+ * Write an Int -1 to signal the end of file.
+ * The overall changelog format is: | put record | delete record | ... | put record | -1 |
+ */
+class StateStoreChangelogWriterV1(
+    fm: CheckpointFileManager,
+    file: Path,
+    compressionCodec: CompressionCodec)
+    extends StateStoreChangelogWriter(fm, file, compressionCodec) {
+
+  override def put(key: Array[Byte], value: Array[Byte]): Unit = {
+    assert(compressedStream != null)
+    compressedStream.writeInt(key.size)
+    compressedStream.write(key)
+    compressedStream.writeInt(value.size)
+    compressedStream.write(value)
+    size += 1
+  }
+
+  override def delete(key: Array[Byte]): Unit = {
+    assert(compressedStream != null)
+    compressedStream.writeInt(key.size)
+    compressedStream.write(key)
+    // -1 in the value field means record deletion.
+    compressedStream.writeInt(-1)
+    size += 1
+  }
+}
 
 /**
- * Read an iterator of change record from the changelog file.
- * A record is represented by ByteArrayPair(key: Array[Byte], value: Array[Byte])
- * A put record is returned as a ByteArrayPair(key, value)
- * A delete record is return as a ByteArrayPair(key, null)
+ * Write changes to the key value state store instance to a changelog file.
+ * There are 2 types of records, put and delete.
+ * A put record is written as: | record type | key length
+ *    | key content | value length | value content | col family name length | col family name | -1 |
+ * A delete record is written as: | record type | key length | key content | -1
+ *    | col family name length | col family name | -1 |
+ * Write an Int -1 to signal the end of file.
+ * The overall changelog format is: | put record | delete record | ... | put record | -1 |
+ */
+class StateStoreChangelogWriterV2(
+    fm: CheckpointFileManager,
+    file: Path,
+    compressionCodec: CompressionCodec)
+    extends StateStoreChangelogWriter(fm, file, compressionCodec) {
+
+  override def put(key: Array[Byte], value: Array[Byte], colFamilyName: String): Unit = {
+    assert(compressedStream != null)
+    compressedStream.writeInt(RecordType.PUT_RECORD.toString.getBytes.size)

Review Comment:
   Discussed offline: decided to use a 1 byte marker.



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala:
##########
@@ -294,8 +297,8 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared
     // Now enable changelog checkpointing in a checkpoint created by a state store
     // that disable changelog checkpointing.
     val enableChangelogCheckpointingConf =
-      dbConf.copy(enableChangelogCheckpointing = true, minVersionsToRetain = 30,
-        minDeltasForSnapshot = 1)
+    dbConf.copy(enableChangelogCheckpointing = true, minVersionsToRetain = 30,

Review Comment:
   nit: prev. indentation seems more proper



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala:
##########
@@ -503,11 +634,79 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared
     changelogWriter.commit()
     val changelogReader = fileManager.getChangelogReader(1)
     val entries = changelogReader.toSeq
-    val expectedEntries = (1 to 5).map(i => (i.toString.getBytes, i.toString.getBytes)) ++
-      (2 to 4).map(j => (j.toString.getBytes, null))
+    val expectedEntries = (1 to 5).map(i =>
+      (RecordType.PUT_RECORD, i.toString.getBytes,
+        i.toString.getBytes, StateStore.DEFAULT_COL_FAMILY_NAME)) ++
+      (2 to 4).map(j => (RecordType.DELETE_RECORD, j.toString.getBytes,
+        null, StateStore.DEFAULT_COL_FAMILY_NAME))
+    assert(entries.size == expectedEntries.size)
+    entries.zip(expectedEntries).map{
+      case (e1, e2) => assert(e1._1 === e2._1 && e1._2 === e2._2 &&
+        e1._3 === e2._3 && e1._4 === e2._4)
+    }
+  }
+
+  testWithChangelogCheckpointingEnabled(
+    "RocksDBFileManager: read and write v2 changelog with default col family") {
+    val dfsRootDir = new File(Utils.createTempDir().getAbsolutePath + "/state/1/1")
+    val fileManager = new RocksDBFileManager(
+      dfsRootDir.getAbsolutePath, Utils.createTempDir(), new Configuration)
+    val changelogWriter = fileManager.getChangeLogWriter(1, true)
+    for (i <- 1 to 5) changelogWriter.put(i.toString, i.toString,
+      StateStore.DEFAULT_COL_FAMILY_NAME)
+    for (j <- 2 to 4) changelogWriter.delete(j.toString, StateStore.DEFAULT_COL_FAMILY_NAME)
+    changelogWriter.commit()
+    val changelogReader = fileManager.getChangelogReader(1, true)
+    val entries = changelogReader.toSeq
+    val expectedEntries = (1 to 5).map(i =>
+      (RecordType.PUT_RECORD, i.toString.getBytes,
+        i.toString.getBytes, StateStore.DEFAULT_COL_FAMILY_NAME)) ++
+      (2 to 4).map(j => (RecordType.DELETE_RECORD, j.toString.getBytes,
+        null, StateStore.DEFAULT_COL_FAMILY_NAME))
+    assert(entries.size == expectedEntries.size)
+    entries.zip(expectedEntries).map{
+      case (e1, e2) => assert(e1._1 === e2._1 && e1._2 === e2._2 &&
+        e1._3 === e2._3 && e1._4 === e2._4)
+    }
+  }
+
+  testWithChangelogCheckpointingEnabled(
+    "RocksDBFileManager: read and write v2 changelog with multiple col families") {
+    val dfsRootDir = new File(Utils.createTempDir().getAbsolutePath + "/state/1/1")
+    val testColFamily1: String = "testColFamily1"
+    val testColFamily2: String = "testColFamily2"
+    val fileManager = new RocksDBFileManager(
+      dfsRootDir.getAbsolutePath, Utils.createTempDir(), new Configuration)
+    val changelogWriter = fileManager.getChangeLogWriter(1, true)
+    for (i <- 1 to 5) changelogWriter.put(i.toString, i.toString,
+      testColFamily1)
+    for (j <- 2 to 4) changelogWriter.delete(j.toString, testColFamily1)
+
+    for (i <- 1 to 5) changelogWriter.put(i.toString, i.toString,
+      testColFamily2)
+    for (j <- 2 to 4) changelogWriter.delete(j.toString, testColFamily2)
+
+    changelogWriter.commit()
+    val changelogReader = fileManager.getChangelogReader(1, true)
+    val entries = changelogReader.toSeq
+    val expectedEntriesForColFamily1 = (1 to 5).map(i =>
+      (RecordType.PUT_RECORD, i.toString.getBytes,
+        i.toString.getBytes, testColFamily1)) ++
+      (2 to 4).map(j => (RecordType.DELETE_RECORD, j.toString.getBytes,
+        null, testColFamily1))
+
+    val expectedEntriesForColFamily2 = (1 to 5).map(i =>

Review Comment:
   ditto



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala:
##########
@@ -503,11 +634,79 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared
     changelogWriter.commit()
     val changelogReader = fileManager.getChangelogReader(1)
     val entries = changelogReader.toSeq
-    val expectedEntries = (1 to 5).map(i => (i.toString.getBytes, i.toString.getBytes)) ++
-      (2 to 4).map(j => (j.toString.getBytes, null))
+    val expectedEntries = (1 to 5).map(i =>
+      (RecordType.PUT_RECORD, i.toString.getBytes,
+        i.toString.getBytes, StateStore.DEFAULT_COL_FAMILY_NAME)) ++
+      (2 to 4).map(j => (RecordType.DELETE_RECORD, j.toString.getBytes,
+        null, StateStore.DEFAULT_COL_FAMILY_NAME))
+    assert(entries.size == expectedEntries.size)
+    entries.zip(expectedEntries).map{
+      case (e1, e2) => assert(e1._1 === e2._1 && e1._2 === e2._2 &&
+        e1._3 === e2._3 && e1._4 === e2._4)
+    }
+  }
+
+  testWithChangelogCheckpointingEnabled(
+    "RocksDBFileManager: read and write v2 changelog with default col family") {
+    val dfsRootDir = new File(Utils.createTempDir().getAbsolutePath + "/state/1/1")
+    val fileManager = new RocksDBFileManager(
+      dfsRootDir.getAbsolutePath, Utils.createTempDir(), new Configuration)
+    val changelogWriter = fileManager.getChangeLogWriter(1, true)
+    for (i <- 1 to 5) changelogWriter.put(i.toString, i.toString,
+      StateStore.DEFAULT_COL_FAMILY_NAME)
+    for (j <- 2 to 4) changelogWriter.delete(j.toString, StateStore.DEFAULT_COL_FAMILY_NAME)
+    changelogWriter.commit()
+    val changelogReader = fileManager.getChangelogReader(1, true)
+    val entries = changelogReader.toSeq
+    val expectedEntries = (1 to 5).map(i =>

Review Comment:
   nit: { for multi-line, `{ i =>`



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala:
##########
@@ -219,49 +233,127 @@ class RocksDB(
     loadedVersion = endVersion
   }
 
+  private def checkColFamilyExists(colFamilyName: String): Boolean = {
+    colFamilyNameToHandleMap.contains(colFamilyName)
+  }
+
+  /**
+   * Create RocksDB column family, if not created already
+   */
+  def createColFamilyIfAbsent(colFamilyName: String): Unit = {
+    if (colFamilyName == StateStore.DEFAULT_COL_FAMILY_NAME) {
+      throw new UnsupportedOperationException("Failed to create column family with reserved " +
+        s"name=$colFamilyName")
+    }
+
+    if (!checkColFamilyExists(colFamilyName)) {
+      assert(db != null)
+      val descriptor = new ColumnFamilyDescriptor(colFamilyName.getBytes, columnFamilyOptions)
+      val handle = db.createColumnFamily(descriptor)
+      colFamilyNameToHandleMap(handle.getName.map(_.toChar).mkString) = handle
+    }
+  }
+
   /**
    * Get the value for the given key if present, or null.
    * @note This will return the last written value even if it was uncommitted.
    */
-  def get(key: Array[Byte]): Array[Byte] = {
-    db.get(readOptions, key)
+  def get(
+      key: Array[Byte],
+      colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Array[Byte] = {
+    if (useColumnFamilies) {
+      // if col family is not created, throw an exception
+      if (!checkColFamilyExists(colFamilyName)) {
+        throw new RuntimeException(s"Column family with name=$colFamilyName does not exist")
+      }
+      db.get(colFamilyNameToHandleMap(colFamilyName), readOptions, key)
+    } else {
+      db.get(readOptions, key)
+    }
   }
 
   /**
    * Put the given value for the given key.
    * @note This update is not committed to disk until commit() is called.
    */
-  def put(key: Array[Byte], value: Array[Byte]): Unit = {
-    if (conf.trackTotalNumberOfRows) {
-      val oldValue = db.get(readOptions, key)
-      if (oldValue == null) {
-        numKeysOnWritingVersion += 1
+  def put(
+      key: Array[Byte],
+      value: Array[Byte],
+      colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Unit = {
+    if (useColumnFamilies) {
+      // if col family is not created, throw an exception
+      if (!checkColFamilyExists(colFamilyName)) {
+        throw new RuntimeException(s"Column family with name=$colFamilyName does not exist")
+      }
+
+      if (conf.trackTotalNumberOfRows) {
+        val oldValue = db.get(colFamilyNameToHandleMap(colFamilyName), readOptions, key)
+        if (oldValue == null) {
+          numKeysOnWritingVersion += 1
+        }
+      }
+      db.put(colFamilyNameToHandleMap(colFamilyName), writeOptions, key, value)
+      changelogWriter.foreach(_.put(key, value, colFamilyName))
+    } else {
+      if (conf.trackTotalNumberOfRows) {
+        val oldValue = db.get(readOptions, key)
+        if (oldValue == null) {
+          numKeysOnWritingVersion += 1
+        }
       }
+      db.put(writeOptions, key, value)
+      changelogWriter.foreach(_.put(key, value))
     }
-    db.put(writeOptions, key, value)
-    changelogWriter.foreach(_.put(key, value))
   }
 
   /**
    * Remove the key if present.
    * @note This update is not committed to disk until commit() is called.
    */
-  def remove(key: Array[Byte]): Unit = {
-    if (conf.trackTotalNumberOfRows) {
-      val value = db.get(readOptions, key)
-      if (value != null) {
-        numKeysOnWritingVersion -= 1
+  def remove(
+      key: Array[Byte],
+      colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Unit = {
+    if (useColumnFamilies) {
+      // if col family is not created, throw an exception
+      if (!checkColFamilyExists(colFamilyName)) {

Review Comment:
   Maybe have a new method starting with assert - we see lots of places doing the same check and throwing the same exception.



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala:
##########


Review Comment:
   I'd rather define a new test method to do matrix test, for this case, something along the line, `testWithColumnFamilyEnabled()` to handle `func: Boolean => `. 
   
   If you do that, your change won't be significant as indentation won't be changed. Same with other test suites if applies.
   
   ```
     def executeFuncWithStateVersionSQLConf(
         stateVersion: Int,
         confPairs: Seq[(String, String)],
         func: => Any): Unit = {
       withSQLConf(confPairs ++
         Seq(SQLConf.STREAMING_AGGREGATION_STATE_FORMAT_VERSION.key -> stateVersion.toString): _*) {
         func
       }
     }
   
     def testWithAllStateVersions(name: String, confPairs: (String, String)*)
                                 (func: => Any): Unit = {
       for (version <- StreamingAggregationStateManager.supportedVersions) {
         test(s"$name - state format version $version") {
           executeFuncWithStateVersionSQLConf(version, confPairs, func)
         }
       }
     }
   ```



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreChangelog.scala:
##########
@@ -109,18 +119,93 @@ class StateStoreChangelogWriter(
   }
 }
 
+/**
+ * Write changes to the key value state store instance to a changelog file.
+ * There are 2 types of records, put and delete.
+ * A put record is written as: | key length | key content | value length | value content |
+ * A delete record is written as: | key length | key content | -1 |
+ * Write an Int -1 to signal the end of file.
+ * The overall changelog format is: | put record | delete record | ... | put record | -1 |
+ */
+class StateStoreChangelogWriterV1(
+    fm: CheckpointFileManager,
+    file: Path,
+    compressionCodec: CompressionCodec)
+  extends StateStoreChangelogWriter(fm, file, compressionCodec) {
+
+  override def put(key: Array[Byte], value: Array[Byte]): Unit = {
+    assert(compressedStream != null)
+    compressedStream.writeInt(key.size)
+    compressedStream.write(key)
+    compressedStream.writeInt(value.size)
+    compressedStream.write(value)
+    size += 1
+  }
+
+  override def delete(key: Array[Byte]): Unit = {
+    assert(compressedStream != null)
+    compressedStream.writeInt(key.size)
+    compressedStream.write(key)
+    // -1 in the value field means record deletion.
+    compressedStream.writeInt(-1)
+    size += 1
+  }
+}
 
 /**
- * Read an iterator of change record from the changelog file.
- * A record is represented by ByteArrayPair(key: Array[Byte], value: Array[Byte])
- * A put record is returned as a ByteArrayPair(key, value)
- * A delete record is return as a ByteArrayPair(key, null)
+ * Write changes to the key value state store instance to a changelog file.
+ * There are 2 types of records, put and delete.
+ * A put record is written as: | record type | key length
+ *    | key content | value length | value content | col family name length | col family name | -1 |
+ * A delete record is written as: | record type | key length | key content | -1
+ *    | col family name length | col family name | -1 |
+ * Write an Int -1 to signal the end of file.
+ * The overall changelog format is: | put record | delete record | ... | put record | -1 |
+ */
+class StateStoreChangelogWriterV2(
+    fm: CheckpointFileManager,
+    file: Path,
+    compressionCodec: CompressionCodec)
+  extends StateStoreChangelogWriter(fm, file, compressionCodec) {
+
+  override def put(key: Array[Byte], value: Array[Byte], colFamilyName: String): Unit = {
+    assert(compressedStream != null)
+    compressedStream.writeInt(RecordType.PUT_RECORD.toString.getBytes.size)
+    compressedStream.write(RecordType.PUT_RECORD.toString.getBytes)
+    compressedStream.writeInt(key.size)
+    compressedStream.write(key)
+    compressedStream.writeInt(value.size)
+    compressedStream.write(value)
+    compressedStream.writeInt(colFamilyName.getBytes.size)
+    compressedStream.write(colFamilyName.getBytes)
+    size += 1
+  }
+
+  override def delete(key: Array[Byte], colFamilyName: String): Unit = {
+    assert(compressedStream != null)
+    compressedStream.writeInt(RecordType.DELETE_RECORD.toString.getBytes.size)
+    compressedStream.write(RecordType.DELETE_RECORD.toString.getBytes)
+    compressedStream.writeInt(key.size)
+    compressedStream.write(key)
+    // -1 in the value field means record deletion.
+    compressedStream.writeInt(-1)
+    compressedStream.writeInt(colFamilyName.getBytes.size)
+    compressedStream.write(colFamilyName.getBytes)
+    size += 1
+  }
+}
+
+/**
+ * Base class for state store changelog reader
+ * @param fm - checkpoint file manager used to manage streaming query checkpoint
+ * @param fileToRead - name of file to use to read changelog
+ * @param compressionCodec - de-compression method using for reading changelog file
  */
 class StateStoreChangelogReader(

Review Comment:
   Same, consider explicitly using abstract class with abstract method rather than opening a chance to use the wrong class.



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala:
##########
@@ -109,12 +109,12 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared
     "RocksDB: check changelog and snapshot version") {
     val remoteDir = Utils.createTempDir().toString
     val conf = dbConf.copy(minDeltasForSnapshot = 1)
-    new File(remoteDir).delete()  // to make sure that the directory gets created
+    new File(remoteDir).delete() // to make sure that the directory gets created
     for (version <- 0 to 49) {
       withDB(remoteDir, version = version, conf = conf) { db =>
-          db.put(version.toString, version.toString)
-          db.commit()
-          if ((version + 1) % 5 == 0) db.doMaintenance()
+        db.put(version.toString, version.toString)

Review Comment:
   nice catch on indentation!



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala:
##########
@@ -377,119 +380,247 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared
     }
   }
 
-  test("RocksDB: compression conf") {
-    val remoteDir = Utils.createTempDir().toString
-    new File(remoteDir).delete()  // to make sure that the directory gets created
+  Seq(true, false).foreach { useColumnFamilies =>
+    test(s"RocksDB: compression conf with useColumnFamilies=$useColumnFamilies") {
+      val remoteDir = Utils.createTempDir().toString
+      new File(remoteDir).delete() // to make sure that the directory gets created
 
-    val conf = RocksDBConf().copy(compression = "zstd")
-    withDB(remoteDir, conf = conf) { db =>
-      assert(db.columnFamilyOptions.compressionType() == CompressionType.ZSTD_COMPRESSION)
-    }
+      val conf = RocksDBConf().copy(compression = "zstd")
+      withDB(remoteDir, conf = conf, useColumnFamilies = useColumnFamilies) { db =>
+        assert(db.columnFamilyOptions.compressionType() == CompressionType.ZSTD_COMPRESSION)
+      }
 
-    // Test the default is LZ4
-    withDB(remoteDir, conf = RocksDBConf().copy()) { db =>
-      assert(db.columnFamilyOptions.compressionType() == CompressionType.LZ4_COMPRESSION)
+      // Test the default is LZ4
+      withDB(remoteDir, conf = RocksDBConf().copy(), useColumnFamilies = useColumnFamilies) { db =>
+        assert(db.columnFamilyOptions.compressionType() == CompressionType.LZ4_COMPRESSION)
+      }
     }
   }
 
-  test("RocksDB: get, put, iterator, commit, load") {
-    def testOps(compactOnCommit: Boolean): Unit = {
-      val remoteDir = Utils.createTempDir().toString
-      new File(remoteDir).delete()  // to make sure that the directory gets created
+  Seq(true, false).foreach { useColumnFamilies =>
+    test(s"RocksDB: get, put, iterator, commit, load with " +
+      s"useColumnFamilies=$useColumnFamilies") {
+      def testOps(compactOnCommit: Boolean): Unit = {
+        val remoteDir = Utils.createTempDir().toString
+        new File(remoteDir).delete() // to make sure that the directory gets created
 
-      val conf = RocksDBConf().copy(compactOnCommit = compactOnCommit)
-      withDB(remoteDir, conf = conf) { db =>
-        assert(db.get("a") === null)
-        assert(iterator(db).isEmpty)
+        val conf = RocksDBConf().copy(compactOnCommit = compactOnCommit)
+        withDB(remoteDir, conf = conf, useColumnFamilies = useColumnFamilies) { db =>
+          assert(db.get("a") === null)
+          assert(iterator(db).isEmpty)
 
-        db.put("a", "1")
-        assert(toStr(db.get("a")) === "1")
-        db.commit()
-      }
+          db.put("a", "1")
+          assert(toStr(db.get("a")) === "1")
+          db.commit()
+        }
 
-      withDB(remoteDir, conf = conf, version = 0) { db =>
-        // version 0 can be loaded again
-        assert(toStr(db.get("a")) === null)
-        assert(iterator(db).isEmpty)
-      }
+        withDB(remoteDir, conf = conf, version = 0, useColumnFamilies = useColumnFamilies) { db =>
+          // version 0 can be loaded again
+          assert(toStr(db.get("a")) === null)
+          assert(iterator(db).isEmpty)
+        }
 
-      withDB(remoteDir, conf = conf, version = 1) { db =>
-        // version 1 data recovered correctly
-        assert(toStr(db.get("a")) === "1")
-        assert(db.iterator().map(toStr).toSet === Set(("a", "1")))
+        withDB(remoteDir, conf = conf, version = 1, useColumnFamilies = useColumnFamilies) { db =>
+          // version 1 data recovered correctly
+          assert(toStr(db.get("a")) === "1")
+          assert(db.iterator().map(toStr).toSet === Set(("a", "1")))
 
-        // make changes but do not commit version 2
-        db.put("b", "2")
-        assert(toStr(db.get("b")) === "2")
-        assert(db.iterator().map(toStr).toSet === Set(("a", "1"), ("b", "2")))
-      }
+          // make changes but do not commit version 2
+          db.put("b", "2")
+          assert(toStr(db.get("b")) === "2")
+          assert(db.iterator().map(toStr).toSet === Set(("a", "1"), ("b", "2")))
+        }
 
-      withDB(remoteDir, conf = conf, version = 1) { db =>
-        // version 1 data not changed
-        assert(toStr(db.get("a")) === "1")
-        assert(db.get("b") === null)
-        assert(db.iterator().map(toStr).toSet === Set(("a", "1")))
+        withDB(remoteDir, conf = conf, version = 1, useColumnFamilies = useColumnFamilies) { db =>
+          // version 1 data not changed
+          assert(toStr(db.get("a")) === "1")
+          assert(db.get("b") === null)
+          assert(db.iterator().map(toStr).toSet === Set(("a", "1")))
 
-        // commit version 2
-        db.put("b", "2")
-        assert(toStr(db.get("b")) === "2")
-        db.commit()
-        assert(db.iterator().map(toStr).toSet === Set(("a", "1"), ("b", "2")))
-      }
+          // commit version 2
+          db.put("b", "2")
+          assert(toStr(db.get("b")) === "2")
+          db.commit()
+          assert(db.iterator().map(toStr).toSet === Set(("a", "1"), ("b", "2")))
+        }
+
+        withDB(remoteDir, conf = conf, version = 1, useColumnFamilies = useColumnFamilies) { db =>
+          // version 1 data not changed
+          assert(toStr(db.get("a")) === "1")
+          assert(db.get("b") === null)
+        }
 
-      withDB(remoteDir, conf = conf, version = 1) { db =>
-        // version 1 data not changed
-        assert(toStr(db.get("a")) === "1")
-        assert(db.get("b") === null)
+        withDB(remoteDir, conf = conf, version = 2, useColumnFamilies = useColumnFamilies) { db =>
+          // version 2 can be loaded again
+          assert(toStr(db.get("b")) === "2")
+          assert(db.iterator().map(toStr).toSet === Set(("a", "1"), ("b", "2")))
+
+          db.load(1)
+          assert(toStr(db.get("b")) === null)
+          assert(db.iterator().map(toStr).toSet === Set(("a", "1")))
+        }
       }
 
-      withDB(remoteDir, conf = conf, version = 2) { db =>
-        // version 2 can be loaded again
-        assert(toStr(db.get("b")) === "2")
-        assert(db.iterator().map(toStr).toSet === Set(("a", "1"), ("b", "2")))
+      for (compactOnCommit <- Seq(false, true)) {
+        withClue(s"compactOnCommit = $compactOnCommit") {
+          testOps(compactOnCommit)
+        }
+      }
+    }
+  }
 
-        db.load(1)
-        assert(toStr(db.get("b")) === null)
-        assert(db.iterator().map(toStr).toSet === Set(("a", "1")))
+  testWithChangelogCheckpointingEnabled(s"RocksDB: get, put, iterator, commit, laod " +

Review Comment:
   see comment above



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala:
##########
@@ -503,11 +634,79 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared
     changelogWriter.commit()
     val changelogReader = fileManager.getChangelogReader(1)
     val entries = changelogReader.toSeq
-    val expectedEntries = (1 to 5).map(i => (i.toString.getBytes, i.toString.getBytes)) ++
-      (2 to 4).map(j => (j.toString.getBytes, null))
+    val expectedEntries = (1 to 5).map(i =>
+      (RecordType.PUT_RECORD, i.toString.getBytes,
+        i.toString.getBytes, StateStore.DEFAULT_COL_FAMILY_NAME)) ++
+      (2 to 4).map(j => (RecordType.DELETE_RECORD, j.toString.getBytes,
+        null, StateStore.DEFAULT_COL_FAMILY_NAME))
+    assert(entries.size == expectedEntries.size)
+    entries.zip(expectedEntries).map{
+      case (e1, e2) => assert(e1._1 === e2._1 && e1._2 === e2._2 &&
+        e1._3 === e2._3 && e1._4 === e2._4)
+    }
+  }
+
+  testWithChangelogCheckpointingEnabled(
+    "RocksDBFileManager: read and write v2 changelog with default col family") {
+    val dfsRootDir = new File(Utils.createTempDir().getAbsolutePath + "/state/1/1")
+    val fileManager = new RocksDBFileManager(
+      dfsRootDir.getAbsolutePath, Utils.createTempDir(), new Configuration)
+    val changelogWriter = fileManager.getChangeLogWriter(1, true)
+    for (i <- 1 to 5) changelogWriter.put(i.toString, i.toString,
+      StateStore.DEFAULT_COL_FAMILY_NAME)
+    for (j <- 2 to 4) changelogWriter.delete(j.toString, StateStore.DEFAULT_COL_FAMILY_NAME)
+    changelogWriter.commit()
+    val changelogReader = fileManager.getChangelogReader(1, true)
+    val entries = changelogReader.toSeq
+    val expectedEntries = (1 to 5).map(i =>
+      (RecordType.PUT_RECORD, i.toString.getBytes,
+        i.toString.getBytes, StateStore.DEFAULT_COL_FAMILY_NAME)) ++
+      (2 to 4).map(j => (RecordType.DELETE_RECORD, j.toString.getBytes,
+        null, StateStore.DEFAULT_COL_FAMILY_NAME))
+    assert(entries.size == expectedEntries.size)
+    entries.zip(expectedEntries).map{
+      case (e1, e2) => assert(e1._1 === e2._1 && e1._2 === e2._2 &&
+        e1._3 === e2._3 && e1._4 === e2._4)
+    }
+  }
+
+  testWithChangelogCheckpointingEnabled(
+    "RocksDBFileManager: read and write v2 changelog with multiple col families") {
+    val dfsRootDir = new File(Utils.createTempDir().getAbsolutePath + "/state/1/1")
+    val testColFamily1: String = "testColFamily1"
+    val testColFamily2: String = "testColFamily2"
+    val fileManager = new RocksDBFileManager(
+      dfsRootDir.getAbsolutePath, Utils.createTempDir(), new Configuration)
+    val changelogWriter = fileManager.getChangeLogWriter(1, true)
+    for (i <- 1 to 5) changelogWriter.put(i.toString, i.toString,
+      testColFamily1)
+    for (j <- 2 to 4) changelogWriter.delete(j.toString, testColFamily1)
+
+    for (i <- 1 to 5) changelogWriter.put(i.toString, i.toString,
+      testColFamily2)
+    for (j <- 2 to 4) changelogWriter.delete(j.toString, testColFamily2)
+
+    changelogWriter.commit()
+    val changelogReader = fileManager.getChangelogReader(1, true)
+    val entries = changelogReader.toSeq
+    val expectedEntriesForColFamily1 = (1 to 5).map(i =>

Review Comment:
   ditto



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala:
##########
@@ -377,119 +380,247 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared
     }
   }
 
-  test("RocksDB: compression conf") {
-    val remoteDir = Utils.createTempDir().toString
-    new File(remoteDir).delete()  // to make sure that the directory gets created
+  Seq(true, false).foreach { useColumnFamilies =>
+    test(s"RocksDB: compression conf with useColumnFamilies=$useColumnFamilies") {
+      val remoteDir = Utils.createTempDir().toString
+      new File(remoteDir).delete() // to make sure that the directory gets created
 
-    val conf = RocksDBConf().copy(compression = "zstd")
-    withDB(remoteDir, conf = conf) { db =>
-      assert(db.columnFamilyOptions.compressionType() == CompressionType.ZSTD_COMPRESSION)
-    }
+      val conf = RocksDBConf().copy(compression = "zstd")
+      withDB(remoteDir, conf = conf, useColumnFamilies = useColumnFamilies) { db =>
+        assert(db.columnFamilyOptions.compressionType() == CompressionType.ZSTD_COMPRESSION)
+      }
 
-    // Test the default is LZ4
-    withDB(remoteDir, conf = RocksDBConf().copy()) { db =>
-      assert(db.columnFamilyOptions.compressionType() == CompressionType.LZ4_COMPRESSION)
+      // Test the default is LZ4
+      withDB(remoteDir, conf = RocksDBConf().copy(), useColumnFamilies = useColumnFamilies) { db =>
+        assert(db.columnFamilyOptions.compressionType() == CompressionType.LZ4_COMPRESSION)
+      }
     }
   }
 
-  test("RocksDB: get, put, iterator, commit, load") {
-    def testOps(compactOnCommit: Boolean): Unit = {
-      val remoteDir = Utils.createTempDir().toString
-      new File(remoteDir).delete()  // to make sure that the directory gets created
+  Seq(true, false).foreach { useColumnFamilies =>
+    test(s"RocksDB: get, put, iterator, commit, load with " +

Review Comment:
   AFAIK, this is tested with and without changelog checkpointing. See the definition of AlsoTestWithChangelogCheckpointingEnabled. 
   
   Or do you have something to additionally check for changelog checkpointing in below test case?



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala:
##########
@@ -503,11 +634,79 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared
     changelogWriter.commit()
     val changelogReader = fileManager.getChangelogReader(1)
     val entries = changelogReader.toSeq
-    val expectedEntries = (1 to 5).map(i => (i.toString.getBytes, i.toString.getBytes)) ++
-      (2 to 4).map(j => (j.toString.getBytes, null))
+    val expectedEntries = (1 to 5).map(i =>
+      (RecordType.PUT_RECORD, i.toString.getBytes,
+        i.toString.getBytes, StateStore.DEFAULT_COL_FAMILY_NAME)) ++
+      (2 to 4).map(j => (RecordType.DELETE_RECORD, j.toString.getBytes,
+        null, StateStore.DEFAULT_COL_FAMILY_NAME))
+    assert(entries.size == expectedEntries.size)
+    entries.zip(expectedEntries).map{
+      case (e1, e2) => assert(e1._1 === e2._1 && e1._2 === e2._2 &&
+        e1._3 === e2._3 && e1._4 === e2._4)
+    }
+  }
+
+  testWithChangelogCheckpointingEnabled(
+    "RocksDBFileManager: read and write v2 changelog with default col family") {
+    val dfsRootDir = new File(Utils.createTempDir().getAbsolutePath + "/state/1/1")
+    val fileManager = new RocksDBFileManager(
+      dfsRootDir.getAbsolutePath, Utils.createTempDir(), new Configuration)
+    val changelogWriter = fileManager.getChangeLogWriter(1, true)
+    for (i <- 1 to 5) changelogWriter.put(i.toString, i.toString,

Review Comment:
   nit: Shall we make this be consistent? `(1 to 5).foreach { i => `



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala:
##########
@@ -503,11 +634,79 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared
     changelogWriter.commit()
     val changelogReader = fileManager.getChangelogReader(1)
     val entries = changelogReader.toSeq
-    val expectedEntries = (1 to 5).map(i => (i.toString.getBytes, i.toString.getBytes)) ++
-      (2 to 4).map(j => (j.toString.getBytes, null))
+    val expectedEntries = (1 to 5).map(i =>
+      (RecordType.PUT_RECORD, i.toString.getBytes,
+        i.toString.getBytes, StateStore.DEFAULT_COL_FAMILY_NAME)) ++
+      (2 to 4).map(j => (RecordType.DELETE_RECORD, j.toString.getBytes,
+        null, StateStore.DEFAULT_COL_FAMILY_NAME))
+    assert(entries.size == expectedEntries.size)
+    entries.zip(expectedEntries).map{
+      case (e1, e2) => assert(e1._1 === e2._1 && e1._2 === e2._2 &&
+        e1._3 === e2._3 && e1._4 === e2._4)
+    }
+  }
+
+  testWithChangelogCheckpointingEnabled(
+    "RocksDBFileManager: read and write v2 changelog with default col family") {
+    val dfsRootDir = new File(Utils.createTempDir().getAbsolutePath + "/state/1/1")
+    val fileManager = new RocksDBFileManager(
+      dfsRootDir.getAbsolutePath, Utils.createTempDir(), new Configuration)
+    val changelogWriter = fileManager.getChangeLogWriter(1, true)
+    for (i <- 1 to 5) changelogWriter.put(i.toString, i.toString,
+      StateStore.DEFAULT_COL_FAMILY_NAME)
+    for (j <- 2 to 4) changelogWriter.delete(j.toString, StateStore.DEFAULT_COL_FAMILY_NAME)

Review Comment:
   ditto



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala:
##########
@@ -836,14 +850,16 @@ class StateStoreSuite extends StateStoreSuiteBase[HDFSBackedStateStoreProvider]
       dir: String = newDir(),
       minDeltasForSnapshot: Int = SQLConf.STATE_STORE_MIN_DELTAS_FOR_SNAPSHOT.defaultValue.get,
       numOfVersToRetainInMemory: Int = SQLConf.MAX_BATCHES_TO_RETAIN_IN_MEMORY.defaultValue.get,
-      hadoopConf: Configuration = new Configuration): HDFSBackedStateStoreProvider = {
+      hadoopConf: Configuration = new Configuration,
+      useColumnFamilies: Boolean = false): HDFSBackedStateStoreProvider = {

Review Comment:
   If this method does not override anything then I wouldn't add the parameter which won't be effective.



##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.streaming
+
+import org.apache.spark.SparkException
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{AnalysisException, SaveMode}
+import org.apache.spark.sql.execution.streaming._
+import org.apache.spark.sql.execution.streaming.state.{AlsoTestWithChangelogCheckpointingEnabled, RocksDBStateStoreProvider}
+import org.apache.spark.sql.internal.SQLConf
+
+class RunningCountStatefulProcessor extends StatefulProcessor[String, String, (String, String)]
+  with Logging {
+  @transient private var _countState: ValueState[Long] = _
+  @transient var _processorHandle: StatefulProcessorHandle = _
+
+  override def init(handle: StatefulProcessorHandle,

Review Comment:
   nit: let's follow multi-line style



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala:
##########
@@ -785,6 +791,11 @@ class StateStoreSuite extends StateStoreSuiteBase[HDFSBackedStateStoreProvider]
     newStoreProvider(storeId.operatorId, storeId.partitionId, dir = storeId.checkpointRootLocation)
   }
 
+  override def newStoreProvider(storeId: StateStoreId,

Review Comment:
   nit: let's follow the multi-line style



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ValueStateSuite.scala:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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
+
+import java.util.UUID
+
+import scala.util.Random
+
+import org.apache.hadoop.conf.Configuration
+import org.scalatest.BeforeAndAfter
+
+import org.apache.spark.sql.execution.streaming.{ImplicitKeyTracker, StatefulProcessorHandleImpl}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.streaming.ValueState
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types._
+
+/**
+ * Class that adds tests for single value ValueState types used in arbitrary stateful
+ * operators such as transformWithState
+ */
+class ValueStateSuite extends SharedSparkSession
+  with BeforeAndAfter {
+
+  before {
+    StateStore.stop()
+    require(!StateStore.isMaintenanceRunning)
+  }
+
+  after {
+    StateStore.stop()
+    require(!StateStore.isMaintenanceRunning)
+  }
+
+  import StateStoreTestsHelper._
+
+  val schemaForKeyRow: StructType = new StructType().add("key", BinaryType)
+
+  val schemaForValueRow: StructType = new StructType().add("value", BinaryType)
+
+  private def newStoreProviderWithValueState(useColumnFamilies: Boolean):
+    RocksDBStateStoreProvider = {
+    newStoreProviderWithValueState(StateStoreId(newDir(), Random.nextInt(), 0),
+      numColsPrefixKey = 0,
+      useColumnFamilies = useColumnFamilies)
+  }
+
+  private def newStoreProviderWithValueState(
+    storeId: StateStoreId,
+    numColsPrefixKey: Int,
+    sqlConf: Option[SQLConf] = None,
+    conf: Configuration = new Configuration,
+    useColumnFamilies: Boolean = false): RocksDBStateStoreProvider = {
+    val provider = new RocksDBStateStoreProvider()
+    provider.init(
+      storeId, schemaForKeyRow, schemaForValueRow, numColsPrefixKey = numColsPrefixKey,
+      useColumnFamilies,
+      new StateStoreConf(sqlConf.getOrElse(SQLConf.get)), conf)
+    provider
+  }
+
+  private def tryWithProviderResource[T](
+    provider: StateStoreProvider)(f: StateStoreProvider => T): T = {

Review Comment:
   nit: 4 spaces



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ValueStateSuite.scala:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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
+
+import java.util.UUID
+
+import scala.util.Random
+
+import org.apache.hadoop.conf.Configuration
+import org.scalatest.BeforeAndAfter
+
+import org.apache.spark.sql.execution.streaming.{ImplicitKeyTracker, StatefulProcessorHandleImpl}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.streaming.ValueState
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types._
+
+/**
+ * Class that adds tests for single value ValueState types used in arbitrary stateful
+ * operators such as transformWithState
+ */
+class ValueStateSuite extends SharedSparkSession
+  with BeforeAndAfter {
+
+  before {
+    StateStore.stop()
+    require(!StateStore.isMaintenanceRunning)
+  }
+
+  after {
+    StateStore.stop()
+    require(!StateStore.isMaintenanceRunning)
+  }
+
+  import StateStoreTestsHelper._
+
+  val schemaForKeyRow: StructType = new StructType().add("key", BinaryType)
+
+  val schemaForValueRow: StructType = new StructType().add("value", BinaryType)
+
+  private def newStoreProviderWithValueState(useColumnFamilies: Boolean):
+    RocksDBStateStoreProvider = {
+    newStoreProviderWithValueState(StateStoreId(newDir(), Random.nextInt(), 0),
+      numColsPrefixKey = 0,
+      useColumnFamilies = useColumnFamilies)
+  }
+
+  private def newStoreProviderWithValueState(
+    storeId: StateStoreId,
+    numColsPrefixKey: Int,
+    sqlConf: Option[SQLConf] = None,

Review Comment:
   Would having `SQLConf.get` as default value work?



##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.streaming
+
+import org.apache.spark.SparkException
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{AnalysisException, SaveMode}
+import org.apache.spark.sql.execution.streaming._
+import org.apache.spark.sql.execution.streaming.state.{AlsoTestWithChangelogCheckpointingEnabled, RocksDBStateStoreProvider}
+import org.apache.spark.sql.internal.SQLConf
+
+class RunningCountStatefulProcessor extends StatefulProcessor[String, String, (String, String)]
+  with Logging {
+  @transient private var _countState: ValueState[Long] = _
+  @transient var _processorHandle: StatefulProcessorHandle = _
+
+  override def init(handle: StatefulProcessorHandle,
+    outputMode: OutputMode) : Unit = {
+    _processorHandle = handle
+    assert(handle.getQueryInfo().getBatchId >= 0)
+    assert(handle.getQueryInfo().getOperatorId == 0)
+    assert(handle.getQueryInfo().getPartitionId >= 0 && handle.getQueryInfo().getPartitionId < 5)

Review Comment:
   nit: shall we explicitly get the number of shuffle partitions rather than magic number?



##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala:
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.streaming
+
+import org.apache.spark.SparkException
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{AnalysisException, SaveMode}
+import org.apache.spark.sql.execution.streaming._
+import org.apache.spark.sql.execution.streaming.state.{AlsoTestWithChangelogCheckpointingEnabled, RocksDBStateStoreProvider}
+import org.apache.spark.sql.internal.SQLConf
+
+class RunningCountStatefulProcessor extends StatefulProcessor[String, String, (String, String)]
+  with Logging {
+  @transient private var _countState: ValueState[Long] = _
+  @transient var _processorHandle: StatefulProcessorHandle = _
+
+  override def init(handle: StatefulProcessorHandle,
+    outputMode: OutputMode) : Unit = {
+    _processorHandle = handle
+    assert(handle.getQueryInfo().getBatchId >= 0)
+    assert(handle.getQueryInfo().getOperatorId == 0)
+    assert(handle.getQueryInfo().getPartitionId >= 0 && handle.getQueryInfo().getPartitionId < 5)
+    _countState = _processorHandle.getValueState[Long]("countState")
+  }
+
+  override def handleInputRows(
+      key: String,
+      inputRows: Iterator[String],
+      timerValues: TimerValues): Iterator[(String, String)] = {
+    val count = _countState.getOption().getOrElse(0L) + inputRows.size
+    if (count == 3) {
+      _countState.remove()
+      Iterator.empty
+    } else {
+      _countState.update(count)
+      Iterator((key, count.toString))
+    }
+  }
+
+  override def close(): Unit = {}
+}
+
+class RunningCountStatefulProcessorWithError extends RunningCountStatefulProcessor {
+  @transient private var _tempState: ValueState[Long] = _
+
+  override def handleInputRows(key: String,
+    inputRows: Iterator[String],

Review Comment:
   nit: 4 spaces



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala:
##########
@@ -204,12 +211,18 @@ class RocksDB(
     for (v <- loadedVersion + 1 to endVersion) {
       var changelogReader: StateStoreChangelogReader = null
       try {
-        changelogReader = fileManager.getChangelogReader(v)
-        changelogReader.foreach { case (key, value) =>
-          if (value != null) {
-            put(key, value)
-          } else {
-            remove(key)
+        changelogReader = fileManager.getChangelogReader(v, useColumnFamilies)
+        changelogReader.foreach { case (recordType, key, value, colFamilyName) =>

Review Comment:
   discussed offline: there could be several types in future but we cannot imagine the types to be exceed 1 byte.



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala:
##########
@@ -896,63 +918,69 @@ abstract class StateStoreSuiteBase[ProviderClass <: StateStoreProvider]
   protected val keySchema: StructType = StateStoreTestsHelper.keySchema
   protected val valueSchema: StructType = StateStoreTestsHelper.valueSchema
 
-  testWithAllCodec("get, put, remove, commit, and all data iterator") {
-    tryWithProviderResource(newStoreProvider()) { provider =>
-      // Verify state before starting a new set of updates
-      assert(getLatestData(provider).isEmpty)
-
-      val store = provider.getStore(0)
-      assert(!store.hasCommitted)
-      assert(get(store, "a", 0) === None)
-      assert(store.iterator().isEmpty)
-      assert(store.metrics.numKeys === 0)
-
-      // Verify state after updating
-      put(store, "a", 0, 1)
-      assert(get(store, "a", 0) === Some(1))
-
-      assert(store.iterator().nonEmpty)
-      assert(getLatestData(provider).isEmpty)
-
-      // Make updates, commit and then verify state
-      put(store, "b", 0, 2)
-      put(store, "aa", 0, 3)
-      remove(store, _._1.startsWith("a"))
-      assert(store.commit() === 1)
-
-      assert(store.hasCommitted)
-      assert(rowPairsToDataSet(store.iterator()) === Set(("b", 0) -> 2))
-      assert(getLatestData(provider) === Set(("b", 0) -> 2))
-
-      // Trying to get newer versions should fail
-      var e = intercept[SparkException] {
-        provider.getStore(2)
-      }
-      assert(e.getCause.isInstanceOf[SparkException])
-      assert(e.getCause.getMessage.contains("does not exist"))
-
-      e = intercept[SparkException] {
-        getData(provider, 2)
-      }
-      assert(e.getCause.isInstanceOf[SparkException])
-      assert(e.getCause.getMessage.contains("does not exist"))
+  Seq(true, false).foreach { useColumnFamilies =>

Review Comment:
   I take my word back. There are so many test cases falling this case - shall we check whether modifying testWithAllCodec would simply work?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImpl.scala:
##########
@@ -0,0 +1,105 @@
+/*
+ * 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
+
+import java.io.Serializable
+
+import org.apache.commons.lang3.SerializationUtils
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.execution.streaming.state.StateStore
+import org.apache.spark.sql.streaming.ValueState
+import org.apache.spark.sql.types._
+
+/**
+ * Class that provides a concrete implementation for a single value state associated with state
+ * variables used in the streaming transformWithState operator.
+ * @param store - reference to the StateStore instance to be used for storing state
+ * @param stateName - name of logical state partition
+ * @tparam S - data type of object that will be stored
+ */
+class ValueStateImpl[S](
+    store: StateStore,
+    stateName: String) extends ValueState[S] with Logging{
+
+  private def encodeKey(): UnsafeRow = {
+    val keyOption = ImplicitKeyTracker.getImplicitKeyOption
+    if (!keyOption.isDefined) {
+      throw new UnsupportedOperationException("Implicit key not found for operation on" +
+        s"stateName=$stateName")
+    }
+
+    val schemaForKeyRow: StructType = new StructType().add("key", BinaryType)
+    val keyByteArr = SerializationUtils.serialize(keyOption.get.asInstanceOf[Serializable])

Review Comment:
   The key is always the case, right? The type being returned from groupByKey() should be encodable with Spark SQL expression encoder, so even you have more types to handle, as long as you use the key as prefix of the entire key, they can be just done with single source of truth for serialization, Spark SQL.



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1432032049


##########
sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessorHandle.scala:
##########
@@ -0,0 +1,43 @@
+/*
+ * 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.streaming
+
+import java.io.Serializable
+
+import org.apache.spark.annotation.{Evolving, Experimental}
+
+/**
+ * Represents the operation handle provided to the stateful processor used in the
+ * arbitrary state API v2.
+ */
+@Experimental
+@Evolving
+trait StatefulProcessorHandle extends Serializable {
+
+  /**
+   * Function to create new or return existing single value state variable of given type
+   * The user must ensure to call this function only within the `init()` method of the
+   * StatefulProcessor.
+   * @param stateName - name of the 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]
+
+  /** Function to return queryInfo for currently running task */
+  def getQueryInfo(): QueryInfo

Review Comment:
   I think he meant about the use cases - in other words, would users find this information be useful, especially something affecting the logic in user function?



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1431945142


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala:
##########
@@ -219,49 +233,127 @@ class RocksDB(
     loadedVersion = endVersion
   }
 
+  private def checkColFamilyExists(colFamilyName: String): Boolean = {
+    colFamilyNameToHandleMap.contains(colFamilyName)
+  }
+
+  /**
+   * Create RocksDB column family, if not created already
+   */
+  def createColFamilyIfAbsent(colFamilyName: String): Unit = {
+    if (colFamilyName == StateStore.DEFAULT_COL_FAMILY_NAME) {
+      throw new UnsupportedOperationException("Failed to create column family with reserved " +
+        s"name=$colFamilyName")
+    }
+
+    if (!checkColFamilyExists(colFamilyName)) {
+      assert(db != null)
+      val descriptor = new ColumnFamilyDescriptor(colFamilyName.getBytes, columnFamilyOptions)
+      val handle = db.createColumnFamily(descriptor)
+      colFamilyNameToHandleMap(handle.getName.map(_.toChar).mkString) = handle
+    }
+  }
+
   /**
    * Get the value for the given key if present, or null.
    * @note This will return the last written value even if it was uncommitted.
    */
-  def get(key: Array[Byte]): Array[Byte] = {
-    db.get(readOptions, key)
+  def get(
+      key: Array[Byte],
+      colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Array[Byte] = {
+    if (useColumnFamilies) {
+      // if col family is not created, throw an exception
+      if (!checkColFamilyExists(colFamilyName)) {
+        throw new RuntimeException(s"Column family with name=$colFamilyName does not exist")
+      }
+      db.get(colFamilyNameToHandleMap(colFamilyName), readOptions, key)
+    } else {
+      db.get(readOptions, key)
+    }
   }
 
   /**
    * Put the given value for the given key.
    * @note This update is not committed to disk until commit() is called.
    */
-  def put(key: Array[Byte], value: Array[Byte]): Unit = {
-    if (conf.trackTotalNumberOfRows) {
-      val oldValue = db.get(readOptions, key)
-      if (oldValue == null) {
-        numKeysOnWritingVersion += 1
+  def put(
+      key: Array[Byte],
+      value: Array[Byte],
+      colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Unit = {
+    if (useColumnFamilies) {
+      // if col family is not created, throw an exception
+      if (!checkColFamilyExists(colFamilyName)) {
+        throw new RuntimeException(s"Column family with name=$colFamilyName does not exist")
+      }
+
+      if (conf.trackTotalNumberOfRows) {
+        val oldValue = db.get(colFamilyNameToHandleMap(colFamilyName), readOptions, key)
+        if (oldValue == null) {
+          numKeysOnWritingVersion += 1
+        }
+      }
+      db.put(colFamilyNameToHandleMap(colFamilyName), writeOptions, key, value)
+      changelogWriter.foreach(_.put(key, value, colFamilyName))
+    } else {
+      if (conf.trackTotalNumberOfRows) {
+        val oldValue = db.get(readOptions, key)
+        if (oldValue == null) {
+          numKeysOnWritingVersion += 1
+        }

Review Comment:
   There seems to be various viable approaches to deduplicate code 1) consider adding default CF to colFamilyNameToHandleMap and do lookup 2) create private methods which receives "handle" and caller figures out the handle. 
   
   There are redundant lookup against the map already if trackTotalNumberOfRows is enabled - lookup result (handle) can be just stored as local val and lookup could just be done once.



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1431946666


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala:
##########
@@ -604,17 +734,56 @@ class RocksDB(
   }
 
   private def getDBProperty(property: String): Long = {
-    db.getProperty(property).toLong
+    if (useColumnFamilies) {
+      // get cumulative sum across all available column families
+      assert(!colFamilyNameToHandleMap.isEmpty)
+      colFamilyNameToHandleMap
+        .values
+        .map(handle => db.getProperty(handle, property).toLong)
+        .sum

Review Comment:
   Maybe we should have done the check as it should be existing issue if there is any since we accumulate values from all partitions. But good to double check, and probably leave a code comment (even in current codebase).



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1430640213


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreChangelog.scala:
##########
@@ -133,11 +219,30 @@ class StateStoreChangelogReader(
     case f: FileNotFoundException =>
       throw QueryExecutionErrors.failedToReadStreamingStateFileError(fileToRead, f)
   }
-  private val input: DataInputStream = decompressStream(sourceStream)
+  protected val input: DataInputStream = decompressStream(sourceStream)
 
   def close(): Unit = { if (input != null) input.close() }
 
-  override def getNext(): (Array[Byte], Array[Byte]) = {
+  override def getNext(): (RecordType.Value, Array[Byte], Array[Byte], String) = {
+    throw new UnsupportedOperationException("Iterator operations not supported on base " +
+      "changelog reader implementation")
+  }
+}
+
+/**
+ * Read an iterator of change record from the changelog file.
+ * A record is represented by ByteArrayPair(recordType: RecordType.Value,
+ *  key: Array[Byte], value: Array[Byte], colFamilyName: String)
+ * A put record is returned as a ByteArrayPair(recordType, key, value, colFamilyName)
+ * A delete record is return as a ByteArrayPair(recordType, key, null, colFamilyName)
+ */
+class StateStoreChangelogReaderV1(
+    fm: CheckpointFileManager,
+    fileToRead: Path,
+    compressionCodec: CompressionCodec) extends StateStoreChangelogReader(fm, fileToRead,

Review Comment:
   Done



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreChangelog.scala:
##########
@@ -153,12 +258,70 @@ class StateStoreChangelogReader(
       val valueSize = input.readInt()
       if (valueSize < 0) {
         // A deletion record
-        (keyBuffer, null)
+        (RecordType.DELETE_RECORD, keyBuffer, null, StateStore.DEFAULT_COL_FAMILY_NAME)
       } else {
         val valueBuffer = new Array[Byte](valueSize)
         ByteStreams.readFully(input, valueBuffer, 0, valueSize)
         // A put record.
-        (keyBuffer, valueBuffer)
+        (RecordType.PUT_RECORD, keyBuffer, valueBuffer, StateStore.DEFAULT_COL_FAMILY_NAME)
+      }
+    }
+  }
+}
+
+/**
+ * Read an iterator of change record from the changelog file.
+ * A record is represented by ByteArrayPair(recordType: RecordType.Value,
+ *  key: Array[Byte], value: Array[Byte], colFamilyName: String)
+ * A put record is returned as a ByteArrayPair(recordType, key, value, colFamilyName)
+ * A delete record is return as a ByteArrayPair(recordType, key, null, colFamilyName)
+ */
+class StateStoreChangelogReaderV2(
+    fm: CheckpointFileManager,
+    fileToRead: Path,
+    compressionCodec: CompressionCodec) extends StateStoreChangelogReader(fm, fileToRead,

Review Comment:
   Done



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1431766707


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala:
##########
@@ -604,17 +734,56 @@ class RocksDB(
   }
 
   private def getDBProperty(property: String): Long = {
-    db.getProperty(property).toLong
+    if (useColumnFamilies) {
+      // get cumulative sum across all available column families
+      assert(!colFamilyNameToHandleMap.isEmpty)
+      colFamilyNameToHandleMap
+        .values
+        .map(handle => db.getProperty(handle, property).toLong)
+        .sum
+    } else {
+      db.getProperty(property).toLong
+    }
   }
 
   private def openDB(): Unit = {
     assert(db == null)
-    db = NativeRocksDB.open(dbOptions, workingDir.toString)
-    logInfo(s"Opened DB with conf ${conf}")
+    if (useColumnFamilies) {
+      val colFamilies = NativeRocksDB.listColumnFamilies(dbOptions, workingDir.toString)
+
+      var colFamilyDescriptors: Seq[ColumnFamilyDescriptor] = Seq.empty[ColumnFamilyDescriptor]
+      // populate the list of available col family descriptors
+      colFamilies.asScala.toList.foreach { family =>
+        val descriptor = new ColumnFamilyDescriptor(family, columnFamilyOptions)
+        colFamilyDescriptors = colFamilyDescriptors :+ descriptor

Review Comment:
   Is `:+` O(1) operation on `Seq`?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImpl.scala:
##########
@@ -0,0 +1,109 @@
+/*
+ * 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
+
+import java.io.Serializable
+
+import org.apache.commons.lang3.SerializationUtils
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.execution.streaming.state.StateStore
+import org.apache.spark.sql.streaming.ValueState
+import org.apache.spark.sql.types._
+
+/**
+ * Class that provides a concrete implementation for a single value state associated with state
+ * variables used in the streaming transformWithState operator.
+ * @param store - reference to the StateStore instance to be used for storing state
+ * @param stateName - name of logical state partition
+ * @tparam S - data type of object that will be stored
+ */
+class ValueStateImpl[S](
+    store: StateStore,
+    stateName: String) extends ValueState[S] with Logging {
+
+  // TODO: validate places that are trying to encode the key and check if we can eliminate/
+  // add caching for some of these calls.
+  private def encodeKey(): UnsafeRow = {
+    val keyOption = ImplicitKeyTracker.getImplicitKeyOption
+    if (!keyOption.isDefined) {
+      throw new UnsupportedOperationException("Implicit key not found for operation on" +
+        s"stateName=$stateName")
+    }
+
+    val schemaForKeyRow: StructType = new StructType().add("key", BinaryType)
+    val keyByteArr = SerializationUtils.serialize(keyOption.get.asInstanceOf[Serializable])
+    val keyEncoder = UnsafeProjection.create(schemaForKeyRow)
+    val keyRow = keyEncoder(InternalRow(keyByteArr))
+    keyRow
+  }
+
+  private def encodeValue(value: S): UnsafeRow = {
+    val schemaForValueRow: StructType = new StructType().add("value", BinaryType)
+    val valueByteArr = SerializationUtils.serialize(value.asInstanceOf[Serializable])
+    val valueEncoder = UnsafeProjection.create(schemaForValueRow)
+    val valueRow = valueEncoder(InternalRow(valueByteArr))
+    valueRow
+  }
+
+  /** Function to check if state exists. Returns true if present and false otherwise */
+  override def exists(): Boolean = {
+    getImpl() != null
+  }
+
+  /** Function to return Option of value if exists and None otherwise */
+  override def getOption(): Option[S] = {
+    val retRow = getImpl()
+    if (retRow != null) {
+      val resState = SerializationUtils
+        .deserialize(retRow.getBinary(0))
+        .asInstanceOf[S]

Review Comment:
   Just FYI, in List State PR I extracted out the key/value encoding to a separate object for ease of use across different state types.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreChangelog.scala:
##########
@@ -153,12 +265,70 @@ class StateStoreChangelogReader(
       val valueSize = input.readInt()
       if (valueSize < 0) {
         // A deletion record
-        (keyBuffer, null)
+        (RecordType.DELETE_RECORD, keyBuffer, null, StateStore.DEFAULT_COL_FAMILY_NAME)
       } else {
         val valueBuffer = new Array[Byte](valueSize)
         ByteStreams.readFully(input, valueBuffer, 0, valueSize)
         // A put record.
-        (keyBuffer, valueBuffer)
+        (RecordType.PUT_RECORD, keyBuffer, valueBuffer, StateStore.DEFAULT_COL_FAMILY_NAME)
+      }
+    }
+  }
+}
+
+/**
+ * Read an iterator of change record from the changelog file.
+ * A record is represented by ByteArrayPair(recordType: RecordType.Value,
+ *  key: Array[Byte], value: Array[Byte], colFamilyName: String)
+ * A put record is returned as a ByteArrayPair(recordType, key, value, colFamilyName)
+ * A delete record is return as a ByteArrayPair(recordType, key, null, colFamilyName)
+ */
+class StateStoreChangelogReaderV2(
+    fm: CheckpointFileManager,
+    fileToRead: Path,
+    compressionCodec: CompressionCodec)
+  extends StateStoreChangelogReader(fm, fileToRead, compressionCodec) {
+
+  private def parseBuffer(input: DataInputStream): Array[Byte] = {
+    val blockSize = input.readInt()
+    val blockBuffer = new Array[Byte](blockSize)
+    ByteStreams.readFully(input, blockBuffer, 0, blockSize)
+    blockBuffer
+  }
+
+  override def getNext(): (RecordType.Value, Array[Byte], Array[Byte], String) = {
+    val recordTypeSize = input.readInt()
+    // A -1 key size mean end of file.
+    if (recordTypeSize == -1) {
+      finished = true
+      null
+    } else if (recordTypeSize < 0) {
+      throw new IOException(
+        s"Error reading streaming state file $fileToRead: " +
+        s"record type size cannot be $recordTypeSize")
+    } else {
+      val recordTypeBuffer = new Array[Byte](recordTypeSize)
+      ByteStreams.readFully(input, recordTypeBuffer, 0, recordTypeSize)

Review Comment:
   Should we use org.apache.spark.unsafe.Platform.copyMemory to copy these bytes? Not sure what ByteStreams does under the hood, but unsafe copy probably would be more efficient.  



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala:
##########
@@ -604,17 +734,56 @@ class RocksDB(
   }
 
   private def getDBProperty(property: String): Long = {
-    db.getProperty(property).toLong
+    if (useColumnFamilies) {
+      // get cumulative sum across all available column families
+      assert(!colFamilyNameToHandleMap.isEmpty)
+      colFamilyNameToHandleMap
+        .values
+        .map(handle => db.getProperty(handle, property).toLong)
+        .sum

Review Comment:
   Curious, are all RocksDB properties additive across column families? (I guess sstables, and stats are - but I have not looked at exhaustive list). 



##########
sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessorHandle.scala:
##########
@@ -0,0 +1,43 @@
+/*
+ * 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.streaming
+
+import java.io.Serializable
+
+import org.apache.spark.annotation.{Evolving, Experimental}
+
+/**
+ * Represents the operation handle provided to the stateful processor used in the
+ * arbitrary state API v2.
+ */
+@Experimental
+@Evolving
+trait StatefulProcessorHandle extends Serializable {
+
+  /**
+   * Function to create new or return existing single value state variable of given type
+   * The user must ensure to call this function only within the `init()` method of the
+   * StatefulProcessor.
+   * @param stateName - name of the 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]
+
+  /** Function to return queryInfo for currently running task */
+  def getQueryInfo(): QueryInfo

Review Comment:
   How do we expect users to use the QueryInfo? Do we need to expose this in the processor handle? 



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala:
##########
@@ -50,13 +51,15 @@ import org.apache.spark.util.{NextIterator, Utils}
  * @param localRootDir Root directory in local disk that is used to working and checkpointing dirs
  * @param hadoopConf   Hadoop configuration for talking to the remote file system
  * @param loggingId    Id that will be prepended in logs for isolating concurrent RocksDBs
+ * @param useColumnFamilies Used to determine whether a single or multiple column families are used
  */
 class RocksDB(
     dfsRootDir: String,
     val conf: RocksDBConf,
     localRootDir: File = Utils.createTempDir(),
     hadoopConf: Configuration = new Configuration,
-    loggingId: String = "") extends Logging {
+    loggingId: String = "",
+    useColumnFamilies: Boolean = false) extends Logging {

Review Comment:
   Given that we don't use column families in Spark today, and every get/put operation defaults to ColumnFamily `DEFAULT`, do we need the additional `useColumnFamilies` guard? Not having the guard prevents a bunch of additional validation across changelog and RocksDB, so I am curious if its absolutely necessary to have this flag.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala:
##########
@@ -204,12 +211,18 @@ class RocksDB(
     for (v <- loadedVersion + 1 to endVersion) {
       var changelogReader: StateStoreChangelogReader = null
       try {
-        changelogReader = fileManager.getChangelogReader(v)
-        changelogReader.foreach { case (key, value) =>
-          if (value != null) {
-            put(key, value)
-          } else {
-            remove(key)
+        changelogReader = fileManager.getChangelogReader(v, useColumnFamilies)
+        changelogReader.foreach { case (recordType, key, value, colFamilyName) =>

Review Comment:
   Do we write the actual recordType string in changelog file (`put_record', 'delete_record`)? If yes, can we encode it to an Int (cons: mapping order matters) to save some I/O?  



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala:
##########
@@ -604,17 +734,56 @@ class RocksDB(
   }
 
   private def getDBProperty(property: String): Long = {
-    db.getProperty(property).toLong
+    if (useColumnFamilies) {
+      // get cumulative sum across all available column families
+      assert(!colFamilyNameToHandleMap.isEmpty)
+      colFamilyNameToHandleMap
+        .values
+        .map(handle => db.getProperty(handle, property).toLong)
+        .sum
+    } else {
+      db.getProperty(property).toLong
+    }
   }
 
   private def openDB(): Unit = {
     assert(db == null)
-    db = NativeRocksDB.open(dbOptions, workingDir.toString)
-    logInfo(s"Opened DB with conf ${conf}")
+    if (useColumnFamilies) {
+      val colFamilies = NativeRocksDB.listColumnFamilies(dbOptions, workingDir.toString)
+
+      var colFamilyDescriptors: Seq[ColumnFamilyDescriptor] = Seq.empty[ColumnFamilyDescriptor]
+      // populate the list of available col family descriptors
+      colFamilies.asScala.toList.foreach { family =>
+        val descriptor = new ColumnFamilyDescriptor(family, columnFamilyOptions)
+        colFamilyDescriptors = colFamilyDescriptors :+ descriptor
+      }
+
+      if (colFamilyDescriptors.isEmpty) {
+        colFamilyDescriptors = colFamilyDescriptors :+
+          new ColumnFamilyDescriptor(NativeRocksDB.DEFAULT_COLUMN_FAMILY, columnFamilyOptions)

Review Comment:
   If column families are present, we do not need the `Default` handle anymore? No one would ever query/modify anything against the Default handle? 



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala:
##########
@@ -78,9 +78,9 @@ private[sql] class HDFSBackedStateStoreProvider extends StateStoreProvider with
 
     override def id: StateStoreId = HDFSBackedStateStoreProvider.this.stateStoreId
 
-    override def get(key: UnsafeRow): UnsafeRow = map.get(key)
+    override def get(key: UnsafeRow, colFamilyName: String): UnsafeRow = map.get(key)
 
-    override def iterator(): Iterator[UnsafeRowPair] = {
+    override def iterator(colFamilyName: String): Iterator[UnsafeRowPair] = {

Review Comment:
   I guess we never expect `HDFSBackedStateStoreProvider`'s get or iterator to be called with a colFamilyName as non-empty, but should we assert that colFamilyName is empty (as technically its not supported)?  



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImpl.scala:
##########
@@ -0,0 +1,109 @@
+/*
+ * 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
+
+import java.io.Serializable
+
+import org.apache.commons.lang3.SerializationUtils
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.execution.streaming.state.StateStore
+import org.apache.spark.sql.streaming.ValueState
+import org.apache.spark.sql.types._
+
+/**
+ * Class that provides a concrete implementation for a single value state associated with state
+ * variables used in the streaming transformWithState operator.
+ * @param store - reference to the StateStore instance to be used for storing state
+ * @param stateName - name of logical state partition
+ * @tparam S - data type of object that will be stored
+ */
+class ValueStateImpl[S](
+    store: StateStore,
+    stateName: String) extends ValueState[S] with Logging {
+
+  // TODO: validate places that are trying to encode the key and check if we can eliminate/
+  // add caching for some of these calls.
+  private def encodeKey(): UnsafeRow = {
+    val keyOption = ImplicitKeyTracker.getImplicitKeyOption
+    if (!keyOption.isDefined) {
+      throw new UnsupportedOperationException("Implicit key not found for operation on" +
+        s"stateName=$stateName")
+    }
+
+    val schemaForKeyRow: StructType = new StructType().add("key", BinaryType)
+    val keyByteArr = SerializationUtils.serialize(keyOption.get.asInstanceOf[Serializable])
+    val keyEncoder = UnsafeProjection.create(schemaForKeyRow)
+    val keyRow = keyEncoder(InternalRow(keyByteArr))
+    keyRow
+  }
+
+  private def encodeValue(value: S): UnsafeRow = {
+    val schemaForValueRow: StructType = new StructType().add("value", BinaryType)
+    val valueByteArr = SerializationUtils.serialize(value.asInstanceOf[Serializable])
+    val valueEncoder = UnsafeProjection.create(schemaForValueRow)
+    val valueRow = valueEncoder(InternalRow(valueByteArr))
+    valueRow
+  }
+
+  /** Function to check if state exists. Returns true if present and false otherwise */
+  override def exists(): Boolean = {
+    getImpl() != null
+  }
+
+  /** Function to return Option of value if exists and None otherwise */
+  override def getOption(): Option[S] = {
+    val retRow = getImpl()
+    if (retRow != null) {
+      val resState = SerializationUtils
+        .deserialize(retRow.getBinary(0))
+        .asInstanceOf[S]
+      Some(resState)
+    } else {
+      None
+    }
+  }
+
+  /** Function to return associated value with key if exists and null otherwise */
+  override def get(): S = {
+    val retRow = getImpl()
+    if (retRow != null) {
+      val resState = SerializationUtils
+        .deserialize(retRow.getBinary(0))
+        .asInstanceOf[S]
+      resState
+    } else {
+      null.asInstanceOf[S]

Review Comment:
   [nit] do we need `asInstanceOf` here?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala:
##########
@@ -219,49 +233,127 @@ class RocksDB(
     loadedVersion = endVersion
   }
 
+  private def checkColFamilyExists(colFamilyName: String): Boolean = {
+    colFamilyNameToHandleMap.contains(colFamilyName)
+  }
+
+  /**
+   * Create RocksDB column family, if not created already
+   */
+  def createColFamilyIfAbsent(colFamilyName: String): Unit = {
+    if (colFamilyName == StateStore.DEFAULT_COL_FAMILY_NAME) {
+      throw new UnsupportedOperationException("Failed to create column family with reserved " +
+        s"name=$colFamilyName")
+    }
+
+    if (!checkColFamilyExists(colFamilyName)) {
+      assert(db != null)
+      val descriptor = new ColumnFamilyDescriptor(colFamilyName.getBytes, columnFamilyOptions)
+      val handle = db.createColumnFamily(descriptor)
+      colFamilyNameToHandleMap(handle.getName.map(_.toChar).mkString) = handle
+    }
+  }
+
   /**
    * Get the value for the given key if present, or null.
    * @note This will return the last written value even if it was uncommitted.
    */
-  def get(key: Array[Byte]): Array[Byte] = {
-    db.get(readOptions, key)
+  def get(
+      key: Array[Byte],
+      colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Array[Byte] = {
+    if (useColumnFamilies) {
+      // if col family is not created, throw an exception
+      if (!checkColFamilyExists(colFamilyName)) {
+        throw new RuntimeException(s"Column family with name=$colFamilyName does not exist")
+      }
+      db.get(colFamilyNameToHandleMap(colFamilyName), readOptions, key)
+    } else {
+      db.get(readOptions, key)
+    }
   }
 
   /**
    * Put the given value for the given key.
    * @note This update is not committed to disk until commit() is called.
    */
-  def put(key: Array[Byte], value: Array[Byte]): Unit = {
-    if (conf.trackTotalNumberOfRows) {
-      val oldValue = db.get(readOptions, key)
-      if (oldValue == null) {
-        numKeysOnWritingVersion += 1
+  def put(
+      key: Array[Byte],
+      value: Array[Byte],
+      colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Unit = {
+    if (useColumnFamilies) {
+      // if col family is not created, throw an exception
+      if (!checkColFamilyExists(colFamilyName)) {
+        throw new RuntimeException(s"Column family with name=$colFamilyName does not exist")
+      }
+
+      if (conf.trackTotalNumberOfRows) {
+        val oldValue = db.get(colFamilyNameToHandleMap(colFamilyName), readOptions, key)
+        if (oldValue == null) {
+          numKeysOnWritingVersion += 1
+        }
+      }
+      db.put(colFamilyNameToHandleMap(colFamilyName), writeOptions, key, value)
+      changelogWriter.foreach(_.put(key, value, colFamilyName))
+    } else {
+      if (conf.trackTotalNumberOfRows) {
+        val oldValue = db.get(readOptions, key)
+        if (oldValue == null) {
+          numKeysOnWritingVersion += 1
+        }

Review Comment:
   Addition of column family ends up in similar code across both `if` and `else` branches. One way to avoid duplication would be to add additional `doGet`, `doPut`, `writeChangelog` operations which would check for column family handle and call the appropriate RocksDB operation. (cons: It adds additional if..else checks in the same code-base, as an example in this code - we would validate 3 times instead of 1). 
   
   Do you think its worth the additional validation overhead for code simplification here? 



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreChangelog.scala:
##########
@@ -109,18 +119,94 @@ class StateStoreChangelogWriter(
   }
 }
 
+/**
+ * Write changes to the key value state store instance to a changelog file.
+ * There are 2 types of records, put and delete.
+ * A put record is written as: | key length | key content | value length | value content |
+ * A delete record is written as: | key length | key content | -1 |
+ * Write an Int -1 to signal the end of file.
+ * The overall changelog format is: | put record | delete record | ... | put record | -1 |
+ */
+class StateStoreChangelogWriterV1(
+    fm: CheckpointFileManager,
+    file: Path,
+    compressionCodec: CompressionCodec)
+    extends StateStoreChangelogWriter(fm, file, compressionCodec) {
+
+  override def put(key: Array[Byte], value: Array[Byte]): Unit = {
+    assert(compressedStream != null)
+    compressedStream.writeInt(key.size)
+    compressedStream.write(key)
+    compressedStream.writeInt(value.size)
+    compressedStream.write(value)
+    size += 1
+  }
+
+  override def delete(key: Array[Byte]): Unit = {
+    assert(compressedStream != null)
+    compressedStream.writeInt(key.size)
+    compressedStream.write(key)
+    // -1 in the value field means record deletion.
+    compressedStream.writeInt(-1)
+    size += 1
+  }
+}
 
 /**
- * Read an iterator of change record from the changelog file.
- * A record is represented by ByteArrayPair(key: Array[Byte], value: Array[Byte])
- * A put record is returned as a ByteArrayPair(key, value)
- * A delete record is return as a ByteArrayPair(key, null)
+ * Write changes to the key value state store instance to a changelog file.
+ * There are 2 types of records, put and delete.
+ * A put record is written as: | record type | key length
+ *    | key content | value length | value content | col family name length | col family name | -1 |
+ * A delete record is written as: | record type | key length | key content | -1
+ *    | col family name length | col family name | -1 |
+ * Write an Int -1 to signal the end of file.
+ * The overall changelog format is: | put record | delete record | ... | put record | -1 |
+ */
+class StateStoreChangelogWriterV2(
+    fm: CheckpointFileManager,
+    file: Path,
+    compressionCodec: CompressionCodec)
+    extends StateStoreChangelogWriter(fm, file, compressionCodec) {
+
+  override def put(key: Array[Byte], value: Array[Byte], colFamilyName: String): Unit = {
+    assert(compressedStream != null)
+    compressedStream.writeInt(RecordType.PUT_RECORD.toString.getBytes.size)

Review Comment:
   +1 to numeric code. I gave the same comment above, just doing +1 here.  



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala:
##########
@@ -204,12 +211,18 @@ class RocksDB(
     for (v <- loadedVersion + 1 to endVersion) {
       var changelogReader: StateStoreChangelogReader = null
       try {
-        changelogReader = fileManager.getChangelogReader(v)
-        changelogReader.foreach { case (key, value) =>
-          if (value != null) {
-            put(key, value)
-          } else {
-            remove(key)
+        changelogReader = fileManager.getChangelogReader(v, useColumnFamilies)
+        changelogReader.foreach { case (recordType, key, value, colFamilyName) =>

Review Comment:
   Looked further downstream, and seems like we write the entire string. 
   
   ```
       compressedStream.writeInt(RecordType.PUT_RECORD.toString.getBytes.size)
       compressedStream.write(RecordType.PUT_RECORD.toString.getBytes)
   
   ```



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1445740654


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImpl.scala:
##########
@@ -0,0 +1,105 @@
+/*
+ * 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
+
+import java.io.Serializable
+
+import org.apache.commons.lang3.SerializationUtils
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.execution.streaming.state.StateStore
+import org.apache.spark.sql.streaming.ValueState
+import org.apache.spark.sql.types._
+
+/**
+ * Class that provides a concrete implementation for a single value state associated with state
+ * variables used in the streaming transformWithState operator.
+ * @param store - reference to the StateStore instance to be used for storing state
+ * @param stateName - name of logical state partition
+ * @tparam S - data type of object that will be stored
+ */
+class ValueStateImpl[S](
+    store: StateStore,
+    stateName: String) extends ValueState[S] with Logging{
+
+  private def encodeKey(): UnsafeRow = {
+    val keyOption = ImplicitKeyTracker.getImplicitKeyOption
+    if (!keyOption.isDefined) {
+      throw new UnsupportedOperationException("Implicit key not found for operation on" +
+        s"stateName=$stateName")
+    }
+
+    val schemaForKeyRow: StructType = new StructType().add("key", BinaryType)
+    val keyByteArr = SerializationUtils.serialize(keyOption.get.asInstanceOf[Serializable])

Review Comment:
   https://www.databricks.com/blog/2016/01/04/introducing-apache-spark-datasets.html
   
   Check `Lightning-fast Serialization with Encoders`. You can see how much performance benefit we get on serde if we make the custom types be bound to Spark SQL type system. Combining two UnsafeRows into one could be also done via either flattening schema and data (concatenate) or nested model.
   
   Users may have to provide the encoder for types on state (value for all types of state, key for map state) which may be burdensome, but that is what Flink requires users to do as well - it requires type information (at least class type), which is to perform the magic behind the scene. They build their own type system, not just simply going through Java's Serializable.
   
   ```
       @Override
       public void open(OpenContext openContext) throws Exception {
           state = getRuntimeContext().getState(new ValueStateDescriptor<>("myState", CountWithTimestamp.class));
       }
   ```
   
   https://nightlies.apache.org/flink/flink-docs-release-1.18/docs/dev/datastream/fault-tolerance/serialization/types_serialization/#data-types--serialization



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1445740654


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImpl.scala:
##########
@@ -0,0 +1,105 @@
+/*
+ * 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
+
+import java.io.Serializable
+
+import org.apache.commons.lang3.SerializationUtils
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.execution.streaming.state.StateStore
+import org.apache.spark.sql.streaming.ValueState
+import org.apache.spark.sql.types._
+
+/**
+ * Class that provides a concrete implementation for a single value state associated with state
+ * variables used in the streaming transformWithState operator.
+ * @param store - reference to the StateStore instance to be used for storing state
+ * @param stateName - name of logical state partition
+ * @tparam S - data type of object that will be stored
+ */
+class ValueStateImpl[S](
+    store: StateStore,
+    stateName: String) extends ValueState[S] with Logging{
+
+  private def encodeKey(): UnsafeRow = {
+    val keyOption = ImplicitKeyTracker.getImplicitKeyOption
+    if (!keyOption.isDefined) {
+      throw new UnsupportedOperationException("Implicit key not found for operation on" +
+        s"stateName=$stateName")
+    }
+
+    val schemaForKeyRow: StructType = new StructType().add("key", BinaryType)
+    val keyByteArr = SerializationUtils.serialize(keyOption.get.asInstanceOf[Serializable])

Review Comment:
   https://www.databricks.com/blog/2016/01/04/introducing-apache-spark-datasets.html
   
   Check `Lightning-fast Serialization with Encoders`. You can see how much performance benefit we get on serde if we make the custom types be bound to Spark SQL type system. Combining two UnsafeRows into one could be also done via either flattening schema and data (concatenate) or nested model.
   
   Users may have to provide the encoder for types on state (value for all types of state, key for map state) which may be burdensome, but that is what Flink requires users to do as well - it requires type information (at least class type), which is to perform the magic behind the scene. They build their own type system and serializer, not just simply going through Java's Serializable.
   
   ```
       @Override
       public void open(OpenContext openContext) throws Exception {
           state = getRuntimeContext().getState(new ValueStateDescriptor<>("myState", CountWithTimestamp.class));
       }
   ```
   
   https://nightlies.apache.org/flink/flink-docs-release-1.18/docs/dev/datastream/fault-tolerance/serialization/types_serialization/#data-types--serialization



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1445746872


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImpl.scala:
##########
@@ -0,0 +1,105 @@
+/*
+ * 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
+
+import java.io.Serializable
+
+import org.apache.commons.lang3.SerializationUtils
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.execution.streaming.state.StateStore
+import org.apache.spark.sql.streaming.ValueState
+import org.apache.spark.sql.types._
+
+/**
+ * Class that provides a concrete implementation for a single value state associated with state
+ * variables used in the streaming transformWithState operator.
+ * @param store - reference to the StateStore instance to be used for storing state
+ * @param stateName - name of logical state partition
+ * @tparam S - data type of object that will be stored
+ */
+class ValueStateImpl[S](
+    store: StateStore,
+    stateName: String) extends ValueState[S] with Logging{
+
+  private def encodeKey(): UnsafeRow = {
+    val keyOption = ImplicitKeyTracker.getImplicitKeyOption
+    if (!keyOption.isDefined) {
+      throw new UnsupportedOperationException("Implicit key not found for operation on" +
+        s"stateName=$stateName")
+    }
+
+    val schemaForKeyRow: StructType = new StructType().add("key", BinaryType)
+    val keyByteArr = SerializationUtils.serialize(keyOption.get.asInstanceOf[Serializable])

Review Comment:
   Note that this cannot be TODO unless the JIRA ticket for TODO is marked as blocker for release. The serialization format for persistence is really painful to change later once it is shipped.



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPARK-46816][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1465833378


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala:
##########
@@ -115,11 +116,11 @@ class StatefulProcessorHandleImpl(store: StateStore, runId: UUID)
 
   def getHandleState: StatefulProcessorHandleState = currState
 
-  override def getValueState[T](stateName: String): ValueState[T] = {
+  override def getValueState[K, T](stateName: String, keyEncoder: Encoder[K]): ValueState[T] = {

Review Comment:
   Sure - created JIRA here - https://issues.apache.org/jira/browse/SPARK-46852



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1440110409


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala:
##########


Review Comment:
   Done



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1439278954


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala:
##########
@@ -50,13 +51,15 @@ import org.apache.spark.util.{NextIterator, Utils}
  * @param localRootDir Root directory in local disk that is used to working and checkpointing dirs
  * @param hadoopConf   Hadoop configuration for talking to the remote file system
  * @param loggingId    Id that will be prepended in logs for isolating concurrent RocksDBs
+ * @param useColumnFamilies Used to determine whether a single or multiple column families are used
  */
 class RocksDB(
     dfsRootDir: String,
     val conf: RocksDBConf,
     localRootDir: File = Utils.createTempDir(),
     hadoopConf: Configuration = new Configuration,
-    loggingId: String = "") extends Logging {
+    loggingId: String = "",
+    useColumnFamilies: Boolean = false) extends Logging {

Review Comment:
   I thought about this actually - but the reason I added this flag is 2 fold:
   - one is to isolate users of this flag - basically in the current impl, this flag is set to true only for the `transformWithState` operator. We are not touching any other operators - so we would limit the impact surface, in case there is any regression
   - second is to identify which changelog writer format to use
   
   If we distinguish old vs new based on just the `default` column family name - either we won't be able use the `default` col family with the new operator or we won't be able to identify which writers/formats to use



-- 
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-unsubscribe@spark.apache.org

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


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


Re: [PR] [SPIP-IN-PROGRESS][DO-NOT-MERGE][SS] Add base support for new arbitrary state management operator, single valueState type, multiple state variables and underlying support for column families for RocksDBStateStoreProvider with/without changelog checkpointing [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #43961:
URL: https://github.com/apache/spark/pull/43961#discussion_r1439495940


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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
+
+import java.util.UUID
+
+import org.apache.spark.TaskContext
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.execution.streaming.state.StateStore
+import org.apache.spark.sql.streaming.{QueryInfo, StatefulProcessorHandle, ValueState}
+import org.apache.spark.util.Utils
+
+/**
+ * Object used to assign/retrieve/remove grouping key passed implicitly for various state
+ * manipulation actions using the store handle.
+ */
+object ImplicitKeyTracker {
+  val implicitKey: InheritableThreadLocal[Any] = new InheritableThreadLocal[Any]
+
+  def getImplicitKeyOption: Option[Any] = Option(implicitKey.get())
+
+  def setImplicitKey(key: Any): Unit = implicitKey.set(key)
+
+  def removeImplicitKey(): Unit = implicitKey.remove()
+}
+
+/**
+ * Enum used to track valid states for the StatefulProcessorHandle
+ */
+object StatefulProcessorHandleState extends Enumeration {
+  type StatefulProcessorHandleState = Value
+  val CREATED, INITIALIZED, DATA_PROCESSED, CLOSED = Value
+}
+
+class QueryInfoImpl(
+    val queryId: UUID,
+    val runId: UUID,
+    val batchId: Long,
+    val operatorId: Long,
+    val partitionId: Int) extends QueryInfo {
+
+  override def getQueryId: UUID = queryId
+
+  override def getRunId: UUID = runId
+
+  override def getBatchId: Long = batchId
+
+  override def getOperatorId: Long = operatorId
+
+  override def getPartitionId: Int = partitionId
+
+  override def toString: String = {
+    s"QueryInfo(queryId=$queryId, runId=$runId, batchId=$batchId, operatorId=$operatorId, " +
+      s"partitionId=$partitionId)"
+  }
+}
+
+/**
+ * Class that provides a concrete implementation of a StatefulProcessorHandle. Note that we keep
+ * track of valid transitions as various functions are invoked to track object lifecycle.
+ * @param store - instance of state store
+ */
+class StatefulProcessorHandleImpl(store: StateStore, runId: UUID)

Review Comment:
   Not sure I understand what you mean - could you please clarify further ?



-- 
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-unsubscribe@spark.apache.org

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


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