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: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
