viirya commented on a change in pull request #32928:
URL: https://github.com/apache/spark/pull/32928#discussion_r660296809



##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala
##########
@@ -0,0 +1,453 @@
+/*
+ * 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.File
+import java.util.Locale
+
+import scala.collection.{mutable, Map}
+import scala.ref.WeakReference
+import scala.util.Try
+
+import org.apache.hadoop.conf.Configuration
+import org.rocksdb.{RocksDB => NativeRocksDB, _}
+
+import org.apache.spark.TaskContext
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap
+import org.apache.spark.sql.util.RocksDBLoader
+import org.apache.spark.util.{NextIterator, Utils}
+
+/**
+ * Class representing a RocksDB instance that checkpoints version of data to 
DFS.
+ * After a set of updates, a new version can be committed by calling 
`commit()`.
+ * Any past version can be loaded by calling `load(version)`.
+ *
+ * @note This class is not thread-safe, so use it only from one thread.
+ * @see [[RocksDBFileManager]] to see how the files are laid out in local disk 
and DFS.
+ * @param dfsRootDir  Remote directory where checkpoints are going to be 
written
+ * @param conf         Configuration for RocksDB
+ * @param localRootDir Root directory in local disk that is used to working 
and checkpoing 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
+ */
+class RocksDB(
+    dfsRootDir: String,
+    val conf: RocksDBConf = RocksDBConf(),
+    localRootDir: File = Utils.createTempDir(),
+    hadoopConf: Configuration = new Configuration,
+    loggingId: String = "") extends Logging {
+
+  RocksDBLoader.loadLibrary()
+
+  // Java wrapper objects linking to native RocksDB objects
+  private val readOptions = new ReadOptions()  // used for gets
+  private val writeOptions = new WriteOptions().setSync(true)  // wait for 
batched write to complete
+  private val flushOptions = new FlushOptions().setWaitForFlush(true)  // wait 
for flush to complete
+  private val writeBatch = new WriteBatchWithIndex(true)  // overwrite 
multiple updates to a key
+
+  private val bloomFilter = new BloomFilter()
+  private val tableFormatConfig = new BlockBasedTableConfig()
+  tableFormatConfig.setBlockSize(conf.blockSizeKB * 1024)
+  tableFormatConfig.setBlockCache(new LRUCache(conf.blockCacheSizeMB * 1024 * 
1024))
+  tableFormatConfig.setFilterPolicy(bloomFilter)
+
+  private val dbOptions = new Options() // options to open the RocksDB
+  dbOptions.setCreateIfMissing(true)
+  dbOptions.setTableFormatConfig(tableFormatConfig)
+  private val dbLogger = createLogger() // for forwarding RocksDB native logs 
to log4j
+  dbOptions.setStatistics(new Statistics())
+
+  private val workingDir = createTempDir("workingDir")
+  private val fileManager = new RocksDBFileManager(
+    dfsRootDir, createTempDir("fileManager"), hadoopConf, loggingId = 
loggingId)
+  private val byteArrayPair = new ByteArrayPair()
+  private val commitLatencyMs = new mutable.HashMap[String, Long]()
+  private val acquireLock = new Object
+
+  @volatile private var db: NativeRocksDB = _
+  @volatile private var loadedVersion = -1L   // -1 = nothing valid is loaded
+  @volatile private var numCommittedKeys = 0L
+  @volatile private var numUncommittedKeys = 0L

Review comment:
       Maybe be `numKeysInCurrentVersion`?




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

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



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

Reply via email to