gaborgsomogyi commented on a change in pull request #24922: [SPARK-28120][SS] Rocksdb state storage implementation URL: https://github.com/apache/spark/pull/24922#discussion_r316222435
########## File path: sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreProvider.scala ########## @@ -0,0 +1,642 @@ +/* + * 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.io._ +import java.util +import java.util.Locale + +import scala.collection.JavaConverters._ +import scala.util.control.NonFatal + +import org.apache.commons.io.FileUtils +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs._ + +import org.apache.spark.{SparkConf, SparkEnv} +import org.apache.spark.internal.Logging +import org.apache.spark.io.FileUtility +import org.apache.spark.sql.catalyst.expressions.UnsafeRow +import org.apache.spark.sql.execution.streaming.CheckpointFileManager +import org.apache.spark.sql.types.StructType +import org.apache.spark.util.Utils + +private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Logging { + + /* Internal fields and methods */ + @volatile private var stateStoreId_ : StateStoreId = _ + @volatile private var keySchema: StructType = _ + @volatile private var valueSchema: StructType = _ + @volatile private var storeConf: StateStoreConf = _ + @volatile private var hadoopConf: Configuration = _ + @volatile private var numberOfVersionsToRetain: Int = _ + @volatile private var localDirectory: String = _ + + /* + * Additional configurations related to rocksDb. This will capture all configs in + * SQLConf that start with `spark.sql.streaming.stateStore.rocksDb` + */ + @volatile private var rocksDbConf: Map[String, String] = Map.empty[String, String] + + private lazy val baseDir: Path = stateStoreId.storeCheckpointLocation() + private lazy val fm = CheckpointFileManager.create(baseDir, hadoopConf) + private lazy val sparkConf = Option(SparkEnv.get).map(_.conf).getOrElse(new SparkConf) + + private case class StoreFile(version: Long, path: Path, isSnapshot: Boolean) + + import WALUtils._ + + /** Implementation of [[StateStore]] API which is backed by RocksDB and HDFS */ + class RocksDbStateStore(val version: Long) extends StateStore with Logging { + + /** Trait and classes representing the internal state of the store */ + trait STATE + + case object LOADED extends STATE + + case object UPDATING extends STATE + + case object COMMITTED extends STATE + + case object ABORTED extends STATE + + private val newVersion = version + 1 + @volatile private var state: STATE = LOADED + private val finalDeltaFile: Path = deltaFile(baseDir, newVersion) + private lazy val deltaFileStream = fm.createAtomic(finalDeltaFile, overwriteIfPossible = true) + private lazy val compressedStream = compressStream(deltaFileStream, sparkConf) + + override def id: StateStoreId = RocksDbStateStoreProvider.this.stateStoreId + + var rocksDbWriteInstance: OptimisticTransactionDbInstance = null + + /* + * numEntriesInDb and bytesUsedByDb are estimated value + * due to the nature of RocksDB implementation. + * see https://github.com/facebook/rocksdb/wiki/RocksDB-FAQ for more details + */ + var numEntriesInDb: Long = 0L + var bytesUsedByDb: Long = 0L + + private def initTransaction(): Unit = { + if (state == LOADED && rocksDbWriteInstance == null) { + logDebug(s"Creating Transactional DB for batch $version") + rocksDbWriteInstance = new OptimisticTransactionDbInstance( + keySchema, + valueSchema, + newVersion.toString, + rocksDbConf) + rocksDbWriteInstance.open(rocksDbPath) + state = UPDATING Review comment: Why setting the state before transaction started? `startTransactions` can throw exception. If for instance `put`called 2 times then in the second case `state = UPDATING` but no open transaction. ---------------------------------------------------------------- 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services --------------------------------------------------------------------- To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org