From f59d88142c0c73d014a6c7c23f853bc62b8eec74 Mon Sep 17 00:00:00 2001 From: Vikram Agrawal Date: Thu, 20 Jun 2019 14:06:19 +0530 Subject: [PATCH 01/26] Rocksdb state storage implementation --- .../java/org/apache/spark/io/FileUtility.java | 88 +++ sql/core/pom.xml | 6 + .../streaming/state/RocksDbInstance.scala | 423 ++++++++++++ .../state/RocksDbStateStoreProvider.scala | 625 ++++++++++++++++++ .../execution/streaming/state/WALUtils.scala | 286 ++++++++ .../state/RocksDbStateStoreSuite.scala | 516 +++++++++++++++ 6 files changed, 1944 insertions(+) create mode 100644 core/src/main/java/org/apache/spark/io/FileUtility.java create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbInstance.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreProvider.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/WALUtils.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreSuite.scala diff --git a/core/src/main/java/org/apache/spark/io/FileUtility.java b/core/src/main/java/org/apache/spark/io/FileUtility.java new file mode 100644 index 000000000000..a62185ddfb5f --- /dev/null +++ b/core/src/main/java/org/apache/spark/io/FileUtility.java @@ -0,0 +1,88 @@ +package org.apache.spark.io; + +import org.apache.commons.compress.archivers.ArchiveException; +import org.apache.commons.compress.archivers.ArchiveStreamFactory; +import org.apache.commons.compress.archivers.tar.TarArchiveEntry; +import org.apache.commons.compress.archivers.tar.TarArchiveInputStream; +import org.apache.commons.compress.archivers.tar.TarArchiveOutputStream; +import org.apache.commons.compress.utils.IOUtils; + +import java.io.*; + +public class FileUtility { + + /** + * Untar an input file into an output file. + * + * The output file is created in the output folder, having the same name as + * the input file, minus the '.tar' extension. + * + * @param inputFile the input .tar file + * @throws IOException + * + * @throws ArchiveException + */ + public static void unTar(final File inputFile) + throws IOException, ArchiveException { + + String outputDir = inputFile.getAbsolutePath().split(".tar")[0]; + File outputTarDir = new File(outputDir); + outputTarDir.mkdir(); + final InputStream is = new FileInputStream(inputFile); + final TarArchiveInputStream debInputStream = (TarArchiveInputStream) new ArchiveStreamFactory().createArchiveInputStream( + "tar", is); + TarArchiveEntry entry = null; + while ((entry = (TarArchiveEntry) debInputStream.getNextEntry()) != null) { + final File outputFile = new File(outputDir, entry.getName()); + if (entry.isDirectory()) { + if (!outputFile.exists()) { + if (!outputFile.mkdirs()) { + throw new IllegalStateException(String.format( + "Couldn't create directory %s.", outputFile.getAbsolutePath())); + } + } + } else { + final OutputStream outputFileStream = new FileOutputStream(outputFile); + IOUtils.copy(debInputStream, outputFileStream); + outputFileStream.close(); + } + } + debInputStream.close(); + } + + public static void createTarFile(String source, String destFileName) throws Exception { + TarArchiveOutputStream tarOs = null; + File f = new File(destFileName); + if (f.exists()) { + f.delete(); + } + try { + FileOutputStream fos = new FileOutputStream(destFileName); + tarOs = (TarArchiveOutputStream) new ArchiveStreamFactory().createArchiveOutputStream("tar", fos); + tarOs = new TarArchiveOutputStream(fos); + File folder = new File(source); + File[] fileNames = folder.listFiles(); + for(File file : fileNames){ + TarArchiveEntry tar_file = new TarArchiveEntry(file.getName()); + tar_file.setSize(file.length()); + tarOs.putArchiveEntry(tar_file); + BufferedInputStream bis = new BufferedInputStream(new FileInputStream(file)); + IOUtils.copy(bis, tarOs); + bis.close(); + tarOs.closeArchiveEntry(); + } + } catch (IOException e) { + throw new IllegalStateException(String.format( + "createTarFile failed with exception %s.", e.getMessage())); + } finally { + try { + tarOs.finish(); + tarOs.close(); + } catch (IOException e) { + e.printStackTrace(); + } + } + } + + +} diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 02ed6f8adaa6..af42ebeb12af 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -147,6 +147,12 @@ mockito-core test + + + org.rocksdb + rocksdbjni + 6.0.1 + target/scala-${scala.binary.version}/classes diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbInstance.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbInstance.scala new file mode 100644 index 000000000000..5e031960905e --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbInstance.scala @@ -0,0 +1,423 @@ +/* + * 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 org.apache.commons.io.FileUtils +import org.rocksdb._ +import org.rocksdb.RocksDB +import org.rocksdb.util.SizeUnit + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.expressions.UnsafeRow +import org.apache.spark.sql.types.StructType +import org.apache.spark.util.Utils + + +class RocksDbInstance(keySchema: StructType, + valueSchema: StructType, + identifier: String) extends Logging { + + import RocksDbInstance._ + RocksDB.loadLibrary() + + protected var db: RocksDB = null + protected var dbPath: String = _ + protected val readOptions: ReadOptions = new ReadOptions() + protected val writeOptions: WriteOptions = new WriteOptions() + protected val table_options = new BlockBasedTableConfig + protected val options: Options = new Options() + + def isOpen(): Boolean = { + db != null + } + + def open(path: String, conf: Map[String, String], readOnly: Boolean): Unit = { + verify(db == null, "Another rocksDb instance is already actve") + try { + setOptions(conf) + db = readOnly match { + case true => options.setCreateIfMissing(false) + RocksDB.openReadOnly(options, path) + case false => options.setCreateIfMissing(true) + RocksDB.open(options, path) + } + dbPath = path + } catch { + case e: Throwable => + throw new IllegalStateException(s"Error while creating rocksDb instance ${e.getMessage}", e) + } + } + + def get(key: UnsafeRow): UnsafeRow = { + verify(isOpen(), "Open rocksDb instance before any operation") + Option(db.get(readOptions, key.getBytes)) match { + case Some(valueInBytes) => + val value = new UnsafeRow(valueSchema.fields.length) + value.pointTo(valueInBytes, valueInBytes.length) + value + case None => null + } + } + + def put(key: UnsafeRow, value: UnsafeRow): Unit = { + verify(isOpen(), "Open rocksDb instance before any operation") + db.put(key.getBytes, value.getBytes) + } + + def remove(key: UnsafeRow): Unit = { + verify(isOpen(), "Open rocksDb instance before any operation") + db.delete(key.getBytes) + } + + def commit(backupPath: Option[String] = None): Unit = { + backupPath.foreach(f => createCheckpoint(db, f)) + } + + def abort: Unit = { + // no-op + } + + def close(): Unit = { + readOptions.close() + writeOptions.close() + logDebug("Closing the db") + db.close() + db = null + } + + def iterator(closeDbOnCompletion: Boolean): Iterator[UnsafeRowPair] = { + verify(isOpen(), "Open rocksDb instance before any operation") + Option(db.getSnapshot) match { + case Some(snapshot) => + logDebug(s"Inside rockdDB iterator function") + var snapshotReadOptions: ReadOptions = new ReadOptions().setSnapshot(snapshot) + val itr = db.newIterator(snapshotReadOptions) + createUnsafeRowPairIterator(itr, snapshotReadOptions, closeDbOnCompletion) + case None => + Iterator.empty + } + } + + protected def createUnsafeRowPairIterator( + itr: RocksIterator, + itrReadOptions: ReadOptions, + closeDbOnCompletion: Boolean): Iterator[UnsafeRowPair] = { + + itr.seekToFirst() + + new Iterator[ UnsafeRowPair ] { + override def hasNext: Boolean = { + if ( itr.isValid ) { + true + } else { + itrReadOptions.close() + if ( closeDbOnCompletion ) { + close() + } + logDebug(s"read from DB completed") + false + } + } + + override def next(): UnsafeRowPair = { + val keyBytes = itr.key + val key = new UnsafeRow(keySchema.fields.length) + key.pointTo(keyBytes, keyBytes.length) + val valueBytes = itr.value + val value = new UnsafeRow(valueSchema.fields.length) + value.pointTo(valueBytes, valueBytes.length) + itr.next() + new UnsafeRowPair(key, value) + } + } + } + + + def printStats: Unit = { + verify(isOpen(), "Open rocksDb instance before any operation") + try { + val stats = db.getProperty("rocksdb.stats") + logInfo(s"Stats = $stats") + } catch { + case e: Exception => + logWarning("Exception while getting stats") + } + } + + def setOptions(conf: Map[String, String]): Unit = { + + // Read options + readOptions.setFillCache(false) + + // Write options + writeOptions.setSync(false) + writeOptions.setDisableWAL(true) + + val dataBlockSize = conf.getOrElse( + "spark.sql.streaming.stateStore.rocksDb.blockSizeInKB". + toLowerCase(Locale.ROOT), "64").toInt + + val metadataBlockSize = conf.getOrElse( + "spark.sql.streaming.stateStore.rocksDb.metadataBlockSizeInKB". + toLowerCase(Locale.ROOT), "4").toInt + + // Table configs + // https://github.com/facebook/rocksdb/wiki/Partitioned-Index-Filters + table_options + .setBlockSize(dataBlockSize) + .setBlockSizeDeviation(5) + .setMetadataBlockSize(metadataBlockSize) + .setFilterPolicy(new BloomFilter(10, false)) + .setPartitionFilters(true) + .setIndexType(IndexType.kTwoLevelIndexSearch) + .setBlockCache(lRUCache) + .setCacheIndexAndFilterBlocks(true) + .setPinTopLevelIndexAndFilter(true) + .setCacheIndexAndFilterBlocksWithHighPriority(true) + .setPinL0FilterAndIndexBlocksInCache(true) + .setFormatVersion(4) // https://rocksdb.org/blog/2019/03/08/format-version-4.html + .setIndexBlockRestartInterval(16) + + var bufferNumber = conf.getOrElse( + "spark.sql.streaming.stateStore.rocksDb.bufferNumber".toLowerCase(Locale.ROOT), "5").toInt + + bufferNumber = Math.max(bufferNumber, 3) + + val bufferNumberToMaintain = Math.max(bufferNumber - 2, 3) + + logInfo(s"Using Max Buffer Name = $bufferNumber & " + + s"max buffer number to maintain = $bufferNumberToMaintain") + + // DB Options + options.setCreateIfMissing(true) + .setMaxWriteBufferNumber(bufferNumber) + .setMaxWriteBufferNumberToMaintain(bufferNumberToMaintain) + .setMaxBackgroundCompactions(4) + .setMaxBackgroundFlushes(2) + .setMaxOpenFiles(-1) + .setMaxFileOpeningThreads(4) + .setWriteBufferSize(256 * SizeUnit.MB) + .setTargetFileSizeBase(256 * SizeUnit.MB) + .setLevelZeroFileNumCompactionTrigger(10) + .setLevelZeroSlowdownWritesTrigger(20) + .setLevelZeroStopWritesTrigger(40) + .setMaxBytesForLevelBase(2 * SizeUnit.GB) + .setTableFormatConfig(table_options) + + } + + def createCheckpoint(rocksDb: RocksDB, dir: String): Unit = { + verify(isOpen(), "Open rocksDb instance before any operation") + val (result, elapsedMs) = Utils.timeTakenMs { + val c = Checkpoint.create(rocksDb) + val f: File = new File(dir) + if (f.exists()) { + FileUtils.deleteDirectory(f) + } + c.createCheckpoint(dir) + c.close() + } + logDebug(s"Creating createCheckpoint at $dir took $elapsedMs ms.") + } + + def createBackup(dir: String): Unit = { + verify(isOpen(), "Open rocksDb instance before any operation") + val (result, elapsedMs) = Utils.timeTakenMs { + val backupableDBOptions = new BackupableDBOptions(dir) + backupableDBOptions.setDestroyOldData(true) + val env: Env = Env.getDefault + env.setBackgroundThreads(2) + val be = BackupEngine.open(env, backupableDBOptions) + be.createNewBackup(db, true) // + backupableDBOptions.close() + env.close() + be.close() + } + logInfo(s"Creating backup at $dir takes $elapsedMs ms.") + } +} + +class OptimisticTransactionDbInstance( + keySchema: StructType, valueSchema: StructType, identifier: String) + extends RocksDbInstance( + keySchema: StructType, valueSchema: StructType, identifier: String) { + + import RocksDbInstance._ + RocksDB.loadLibrary() + + var otdb: OptimisticTransactionDB = null + var txn: Transaction = null + + override def isOpen(): Boolean = { + otdb != null + } + + def open(path: String, conf: Map[String, String]): Unit = { + open(path, conf, false) + } + + override def open(path: String, conf: Map[String, String], readOnly: Boolean): Unit = { + verify(otdb == null, "Another OptimisticTransactionDbInstance instance is already actve") + verify(readOnly == false, "Cannot open OptimisticTransactionDbInstance in Readonly mode") + try { + setOptions(conf) + options.setCreateIfMissing(true) + otdb = OptimisticTransactionDB.open(options, path) + dbPath = path + } catch { + case e: Throwable => + throw new IllegalStateException(s"Error while creating OptimisticTransactionDb instance" + + s" ${e.getMessage}", e) + } + } + + def startTransactions(): Unit = { + verify(isOpen(), "Open OptimisticTransactionDbInstance before performing any operation") + Option(txn) match { + case None => + val optimisticTransactionOptions = new OptimisticTransactionOptions() + txn = otdb.beginTransaction(writeOptions, optimisticTransactionOptions) + txn.setSavePoint() + case Some(x) => + throw new IllegalStateException(s"Already started a transaction") + } + } + + override def put(key: UnsafeRow, value: UnsafeRow): Unit = { + verify(txn != null, "Start Transaction before inserting any key") + txn.put(key.getBytes, value.getBytes) + } + + override def remove(key: UnsafeRow): Unit = { + verify(txn != null, "Start Transaction before deleting any key") + txn.delete(key.getBytes) + } + + override def get(key: UnsafeRow): UnsafeRow = { + verify(txn != null, "Start Transaction before fetching any key-value") + Option(txn.get(readOptions, key.getBytes)) match { + case Some(valueInBytes) => + val value = new UnsafeRow(valueSchema.fields.length) + value.pointTo(valueInBytes, valueInBytes.length) + value + case None => + null + } + } + + override def commit(backupPath: Option[String] = None): Unit = { + verify(txn != null, "Start Transaction before fetching any key-value") + // printTrxStats + try { + val file = new File(dbPath, identifier.toUpperCase(Locale.ROOT)) + file.createNewFile() + txn.commit() + txn.close() + txn = null + backupPath.foreach(f => createCheckpoint(otdb.asInstanceOf[ RocksDB ], f)) + } catch { + case e: Exception => + log.error(s"Unable to commit the transactions. Error message = ${e.getMessage}") + throw e + } + } + + def printTrxStats(): Unit = { + verify(txn != null, "No open Transaction") + logInfo( + s""" + | deletes = ${txn.getNumDeletes} + | numKeys = ${txn.getNumKeys} + | puts = ${txn.getNumPuts} + | time = ${txn.getElapsedTime} + """.stripMargin) + } + + + override def abort(): Unit = { + verify(txn != null, "No Transaction to abort") + txn.rollbackToSavePoint() + txn.close() + txn = null + } + + override def close(): Unit = { + verify(isOpen(), "No DB to close") + readOptions.close() + writeOptions.close() + logDebug("Closing the transaction db") + otdb.close() + otdb = null + } + + override def iterator(closeDbOnCompletion: Boolean): Iterator[UnsafeRowPair] = { + verify(txn != null, "Transaction is not set") + verify(closeDbOnCompletion == false, + "Cannot close a DB without aborting/commiting the transactions") + val readOptions = new ReadOptions() + val itr: RocksIterator = txn.getIterator(readOptions) + Option(itr) match { + case Some(i) => + logDebug(s"creating iterator from transaction DB") + createUnsafeRowPairIterator(i, readOptions, false) + case None => + Iterator.empty + } + } + +} + +object RocksDbInstance { + + RocksDB.loadLibrary() + + private val destroyOptions: Options = new Options() + + val lRUCache = new LRUCache(1024 * 1024 * 1024, 6, + false, 0.05) + + def destroyDB(path: String): Unit = { + val f: File = new File(path) + if (f.exists()) { + RocksDB.destroyDB(path, destroyOptions) + FileUtils.deleteDirectory(f) + } + } + + def restoreFromBackup(backupDir: String, dbDir: String): Unit = { + val (result, elapsedMs) = Utils.timeTakenMs { + val backupableDBOptions = new BackupableDBOptions(backupDir) + val be = BackupEngine.open(Env.getDefault, backupableDBOptions) + val restoreOptions = new RestoreOptions(false) + be.restoreDbFromLatestBackup(dbDir, dbDir, restoreOptions) + restoreOptions.close() + backupableDBOptions.close() + be.close() + } + } + + def verify(condition: => Boolean, msg: String): Unit = { + if (!condition) { + throw new IllegalStateException(msg) + } + } + +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreProvider.scala new file mode 100644 index 000000000000..2c7b43e24800 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreProvider.scala @@ -0,0 +1,625 @@ +/* + * 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 org.apache.commons.io.FileUtils +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs._ +import scala.collection.JavaConverters._ +import scala.util.control.NonFatal + +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 = _ + + /* + * 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 + 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) + rocksDbWriteInstance.open(rocksDbPath, rocksDbConf) + state = UPDATING + rocksDbWriteInstance.startTransactions() + } + } + + override def get(key: UnsafeRow): UnsafeRow = { + initTransaction + rocksDbWriteInstance.get(key) + } + + override def put(key: UnsafeRow, value: UnsafeRow): Unit = { + initTransaction + verify(state == UPDATING, s"Current state of the store is $state. " + + s"Cannot put after already committed or aborted") + val keyCopy = key.copy() + val valueCopy = value.copy() + rocksDbWriteInstance.put(keyCopy, valueCopy) + writeUpdateToDeltaFile(compressedStream, keyCopy, valueCopy) + } + + override def remove(key: UnsafeRow): Unit = { + initTransaction + verify(state == UPDATING, "Cannot remove after already committed or aborted") + rocksDbWriteInstance.remove(key) + // TODO check if removed value is null + writeRemoveToDeltaFile(compressedStream, key) + } + + override def getRange( start: Option[UnsafeRow], + end: Option[UnsafeRow]): Iterator[UnsafeRowPair] = { + verify(state == UPDATING, "Cannot getRange after already committed or aborted") + iterator() + } + + /** Commit all the updates that have been made to the store, and return the new version. */ + override def commit(): Long = { + initTransaction + verify(state == UPDATING, s"Current state of the store is $state " + + s"Cannot commit after already committed or aborted") + try { + state = COMMITTED + synchronized { + rocksDbWriteInstance.commit(Some(getBackupPath(newVersion))) + finalizeDeltaFile(compressedStream) + } + numEntriesInDb = rocksDbWriteInstance.otdb.getLongProperty("rocksdb.estimate-num-keys") + bytesUsedByDb = numEntriesInDb * (keySchema.defaultSize + valueSchema.defaultSize) + newVersion + } catch { + case NonFatal(e) => + throw new IllegalStateException( + s"Error committing version $newVersion into $this", e) + } finally { + storeMap.remove(version) + close() + } + } + + /* + * Abort all the updates made on this store. This store will not be usable any more. + */ + override def abort(): Unit = { + // This if statement is to ensure that files are deleted only if there are changes to the + // StateStore. We have two StateStores for each task, one which is used only for reading, and + // the other used for read+write. We don't want the read-only to delete state files. + try { + if (state == UPDATING) { + state = ABORTED + synchronized { + rocksDbWriteInstance.abort() + cancelDeltaFile(compressedStream, deltaFileStream) + } + logInfo(s"Aborted version $newVersion for $this") + } else { + state = ABORTED + } + } catch { + case NonFatal(e) => + throw new IllegalStateException(s"Error aborting version $newVersion into $this", e) + } finally { + storeMap.remove(version) + close() + } + } + + def close(): Unit = { + if ( rocksDbWriteInstance != null ) { + rocksDbWriteInstance.close() + rocksDbWriteInstance = null + } + } + + /* + * Get an iterator of all the store data. + * This can be called only after committing all the updates made in the current thread. + */ + override def iterator(): Iterator[UnsafeRowPair] = { + state match { + case UPDATING => + logDebug("state = updating using transaction DB") + // We need to use current db to read uncommitted transactions + rocksDbWriteInstance.iterator(closeDbOnCompletion = false) + + case LOADED | ABORTED => + // use checkpointed db for previous version + logDebug(s"state = loaded/aborted using checkpointed DB with version $version") + if (version == 0) { + Iterator.empty + } else { + val path = getBackupPath(version) + val r: RocksDbInstance = new RocksDbInstance(keySchema, valueSchema, version.toString) + r.open(path, rocksDbConf, readOnly = true) + r.iterator(closeDbOnCompletion = true) + } + case COMMITTED => + logDebug(s"state = committed using checkpointed DB with version $newVersion") + // use checkpointed db for current updated version + val path = getBackupPath(newVersion) + val r: RocksDbInstance = new RocksDbInstance(keySchema, valueSchema, newVersion.toString) + r.open(path, rocksDbConf, readOnly = true) + r.iterator(closeDbOnCompletion = true) + + case _ => Iterator.empty + } + } + + override def metrics: StateStoreMetrics = { + val metricsFromProvider: Map[String, Long] = getMetricsForProvider() + val customMetrics = metricsFromProvider.flatMap { case (name, value) => + // just allow searching from list cause the list is small enough + supportedCustomMetrics.find(_.name == name).map(_ -> value) + } + return StateStoreMetrics(Math.max(numEntriesInDb, 0), + Math.max(bytesUsedByDb, 0), customMetrics) + } + + /* + * Whether all updates have been committed + */ + override def hasCommitted: Boolean = { + state == COMMITTED + } + + override def toString(): String = { + s"RocksDbStateStore[id=(op=${id.operatorId},part=${id.partitionId}),dir=$baseDir]" + } + + } + + /* + * Initialize the provide with more contextual information from the SQL operator. + * This method will be called first after creating an instance of the StateStoreProvider by + * reflection. + * + * @param stateStoreId Id of the versioned StateStores that this provider will generate + * @param keySchema Schema of keys to be stored + * @param valueSchema Schema of value to be stored + * @param keyIndexOrdinal Optional column (represent as the ordinal of the field in keySchema) by + * which the StateStore implementation could index the data. + * @param storeConfs Configurations used by the StateStores + * @param hadoopConf Hadoop configuration that could be used by StateStore + * to save state data + */ + override def init(stateStoreId: StateStoreId, + keySchema: StructType, + valueSchema: StructType, + keyIndexOrdinal: Option[ Int ], // for sorting the data by their keys + storeConfs: StateStoreConf, + hadoopConf: Configuration): Unit = { + this.stateStoreId_ = stateStoreId + this.keySchema = keySchema + this.valueSchema = valueSchema + this.storeConf = storeConfs + this.hadoopConf = hadoopConf + // TODO add new conf for `maxVersionsToRetainInMemory` + this.numberOfVersionsToRetain = storeConfs.maxVersionsToRetainInMemory + fm.mkdirs(baseDir) + this.rocksDbConf = storeConf.confs. + filter(_._1.startsWith("spark.sql.streaming.stateStore.rocksDb")) + .map { + case (k, v) => (k.toLowerCase(Locale.ROOT), v) + } + } + + /* + * Return the id of the StateStores this provider will generate. + * Should be the same as the one passed in init(). + */ + override def stateStoreId: StateStoreId = stateStoreId_ + + /* + * Called when the provider instance is unloaded from the executor + */ + override def close(): Unit = { + storeMap.values.asScala.foreach(_.close) + storeMap.clear() + } + + private val storeMap = new util.HashMap[Long, RocksDbStateStore]() + + /* + * Optional custom metrics that the implementation may want to report. + * + * @note The StateStore objects created by this provider must report the same custom metrics + * (specifically, same names) through `StateStore.metrics`. + */ + // TODO + override def supportedCustomMetrics: Seq[ StateStoreCustomMetric ] = { + Nil + } + + override def toString(): String = { + s"RocksDbStateStoreProvider[" + + s"id = (op=${stateStoreId.operatorId},part=${stateStoreId.partitionId}),dir = $baseDir]" + } + + def getMetricsForProvider(): Map[String, Long] = synchronized { + Map.empty[String, Long] + } + + /* + * Return an instance of [[StateStore]] representing state data of the given version + */ + override def getStore(version: Long): StateStore = synchronized { + logInfo(s"get Store for version $version") + require(version >= 0, "Version cannot be less than 0") + if (storeMap.containsKey(version)) { + storeMap.get(version) + } else { + val store = createStore(version) + storeMap.put(version, store) + store + } + } + + def createStore(version: Long): RocksDbStateStore = { + val newStore = new RocksDbStateStore(version) + logInfo(s"Creating a new Store for version $version and partition ${stateStoreId_.partitionId}") + if (version > 0 & !checkIfStateExists(version)) { + // load the data in the rocksDB + logInfo(s"Loading state for $version and partition ${stateStoreId_.partitionId}") + loadState(version) + } + newStore + } + + def checkIfStateExists(version: Long): Boolean = { + val dbPath: Path = new Path(rocksDbPath, version.toString.toUpperCase(Locale.ROOT)) + val f: File = new File(dbPath.toString) + f.exists() + } + + def loadState(version: Long): Unit = { + // search for state on snapshot + var rocksDbWriteInstance: OptimisticTransactionDbInstance = null + var lastAvailableVersion = version + var found = false + val (result, elapsedMs) = Utils.timeTakenMs { + try { + if (checkIfStateExists(version - 1) ) { + found = true + lastAvailableVersion = version - 1 + } else { + // TODO check for numberOfVersionsToRetain + // Destroy DB so that we can recontruct it using snapshot and delta files + RocksDbInstance.destroyDB(rocksDbPath) + } + + // Check for snapshot files starting from "version" + while (!found && lastAvailableVersion > 0) { + found = { + try { + loadSnapshotFile(lastAvailableVersion) + } catch { + case e: Exception => + logError(s"$e while reading snapshot file") + throw e + } + } + if (!found) { + lastAvailableVersion = lastAvailableVersion - 1 + } + logInfo(s"Snapshot for $lastAvailableVersion for " + + s"partition ${stateStoreId_.partitionId} found = $found") + } + + rocksDbWriteInstance = new OptimisticTransactionDbInstance( + keySchema, valueSchema, version.toString) + rocksDbWriteInstance.open(rocksDbPath, rocksDbConf) + rocksDbWriteInstance.startTransactions() + + // Load all the deltas from the version after the last available + // one up to the target version. + // The last available version is the one with a full snapshot, so it doesn't need deltas. + for (deltaVersion <- (lastAvailableVersion + 1) to version) { + val fileToRead = deltaFile(baseDir, deltaVersion) + updateFromDeltaFile(fm, fileToRead, keySchema, valueSchema, + rocksDbWriteInstance, sparkConf) + logInfo(s"Read delta file for version $version of $this from $fileToRead") + } + + rocksDbWriteInstance.commit(Some(getBackupPath(version))) + rocksDbWriteInstance.close() + rocksDbWriteInstance = null + } + catch { + case e: IllegalStateException => + logError(s"Exception while loading state ${e.getMessage}") + if (rocksDbWriteInstance != null) { + rocksDbWriteInstance.abort() + rocksDbWriteInstance.close() + } + throw e + } + } + logInfo(s"Loading state for $version takes $elapsedMs ms.") + } + + private def loadSnapshotFile(version: Long): Boolean = { + val fileToRead = snapshotFile(baseDir, version) + val tmpLocDir: File = new File(getTempPath(version)) + val tmpLocFile: File = new File(getTempPath(version).concat(".tar")) + try { + if (!fm.exists(fileToRead)) { + return false + } + logInfo(s"Will download $fileToRead at location ${tmpLocFile.toString()}") + if (downloadFile(fm, fileToRead, new Path(tmpLocFile.getAbsolutePath), sparkConf)) { + FileUtility.unTar(tmpLocFile) + if (!tmpLocDir.list().exists(_.endsWith(".sst"))) { + logWarning("Snaphot files are corrupted") + throw new IOException( + s"Error reading snapshot file $fileToRead of $this:" + + s" No SST files found") + } + FileUtils.moveDirectory(tmpLocDir, new File(rocksDbPath)) + return true + } + } catch { + case e: Exception => + logError(s"Exception while loading snapshot file $e") + throw e + } finally { + if (tmpLocFile.exists()) { + tmpLocFile.delete() + } + FileUtils.deleteDirectory(tmpLocDir) + } + return false + } + + /** Optional method for providers to allow for background maintenance (e.g. compactions) */ + override def doMaintenance(): Unit = { + try { + val (files: Seq[WALUtils.StoreFile], e1) = Utils.timeTakenMs(fetchFiles(fm, baseDir)) + logDebug(s"fetchFiles() took $e1 ms.") + doSnapshot(files) + cleanup(files) + cleanRocksDBBackupInstances(files) + } catch { + case NonFatal(e) => + logWarning(s"Error performing snapshot and cleaning up $this") + } + } + + private def doSnapshot(files: Seq[WALUtils.StoreFile]): Unit = { + if ( files.nonEmpty ) { + val lastVersion = files.last.version + val deltaFilesForLastVersion = + filesForVersion(files, lastVersion).filter(_.isSnapshot == false) + if ( deltaFilesForLastVersion.size > storeConf.minDeltasForSnapshot ) { + val dbPath = getBackupPath(lastVersion) + val snapShotFileName = getTempPath(lastVersion).concat(".snapshot") + val f = new File(snapShotFileName) + f.delete() // delete any existing tarball + try { + val (_, t1) = Utils.timeTakenMs { + FileUtility.createTarFile(dbPath, snapShotFileName) + val targetFile = snapshotFile(baseDir, lastVersion) + uploadFile(fm, new Path(snapShotFileName), targetFile, sparkConf) + } + logInfo(s"Creating snapshot file for" + + s" ${stateStoreId_.partitionId} took $t1 ms.") + } catch { + case e: Exception => + logError(s"Exception while creating snapshot $e} ") + throw e + } finally { + f.delete() // delete the tarball + } + } + } + } + + /* + * Clean up old snapshots and delta files that are not needed any more. It ensures that last + * few versions of the store can be recovered from the files, so re-executed RDD operations + * can re-apply updates on the past versions of the store. + */ + private[state] def cleanup(files: Seq[WALUtils.StoreFile]): Unit = { + try { + if (files.nonEmpty) { + val earliestVersionToRetain = files.last.version - storeConf.minVersionsToRetain + if (earliestVersionToRetain > 0) { + val earliestFileToRetain = filesForVersion(files, earliestVersionToRetain).head + val filesToDelete = files.filter(_.version < earliestFileToRetain.version) + val (_, e2) = Utils.timeTakenMs { + filesToDelete.foreach { f => + fm.delete(f.path) + val file = new File(rocksDbPath, f.version.toString.toUpperCase(Locale.ROOT)) + if (file.exists()) { + file.delete() + } + } + } + logDebug(s"deleting files took $e2 ms.") + logInfo(s"Deleted files older than ${earliestFileToRetain.version} for $this: " + + filesToDelete.mkString(", ")) + } + } + } catch { + case NonFatal(e) => + logWarning(s"Error cleaning up files for $this", e) + } + } + + private def cleanRocksDBBackupInstances(files: Seq[WALUtils.StoreFile]): Unit = { + try { + if (files.nonEmpty) { + val earliestVersionToRetain = files.last.version - storeConf.minVersionsToRetain + if (earliestVersionToRetain > 0) { + for (v <- (earliestVersionToRetain - 1) to 1) { + // Destroy the backup path + logDebug((s"Destroying backup version = $v")) + RocksDbInstance.destroyDB(getBackupPath(v)) + } + } + } + } catch { + case NonFatal(e) => logWarning(s"Error cleaning up files for $this", e) + } + } + + + // Used only for unit tests + private[sql] def latestIterator(): Iterator[UnsafeRowPair] = synchronized { + val versionsInFiles = fetchFiles(fm, baseDir).map(_.version).toSet + var itr = Iterator.empty + if (versionsInFiles.nonEmpty) { + val maxVersion = versionsInFiles.max + if ( maxVersion == 0 ) { + return Iterator.empty + } + // FIXME assuming maxVersion exists in rocksDB + val path = getBackupPath(maxVersion) + val r: RocksDbInstance = new RocksDbInstance(keySchema, valueSchema, maxVersion.toString) + try { + r.open(path, rocksDbConf, readOnly = true) + return r.iterator(false) + } catch { + case e: Exception => + // do nothing + } + } + Iterator.empty + } + + // making it public for unit tests + lazy val rocksDbPath: String = { + val checkpointRootLocationPath = new Path(stateStoreId.checkpointRootLocation) + val basePath = new Path(RocksDbStateStoreProvider.ROCKS_DB_BASE_PATH, new Path("db", + checkpointRootLocationPath.getName + "_" + checkpointRootLocationPath.hashCode())) + + val dir = basePath.toString + Path.SEPARATOR + + stateStoreId_.operatorId + Path.SEPARATOR + + stateStoreId_.partitionId + + val f: File = new File(dir) + + if (!f.exists()) { + logInfo(s"creating rocksDb directory at : $dir") + f.mkdirs() + } + dir + } + + private def getBackupPath(version: Long): String = { + val checkpointRootLocationPath = new Path(stateStoreId.checkpointRootLocation) + + val basePath = new Path(RocksDbStateStoreProvider.ROCKS_DB_BASE_PATH, + new Path("backup", + checkpointRootLocationPath.getName + "_" + checkpointRootLocationPath.hashCode())) + + val dir = basePath.toString + Path.SEPARATOR + + stateStoreId_.operatorId + Path.SEPARATOR + + stateStoreId_.partitionId + + val f: File = new File(dir) + + if (!f.exists()) { + logInfo(s"creating rocksDb directory at : $dir") + f.mkdirs() + } + + dir + Path.SEPARATOR + version + } + + private def getTempPath(version: Long): String = { + val checkpointRootLocationPath = new Path(stateStoreId.checkpointRootLocation) + + val basePath = new Path(RocksDbStateStoreProvider.ROCKS_DB_BASE_PATH, + new Path("tmp", + checkpointRootLocationPath.getName + "_" + checkpointRootLocationPath.hashCode())) + + val dir = basePath.toString + Path.SEPARATOR + + stateStoreId_.operatorId + Path.SEPARATOR + + stateStoreId_.partitionId + + val f: File = new File(dir) + + if (!f.exists()) { + logInfo(s"creating rocksDb directory at : $dir") + f.mkdirs() + } + + dir + Path.SEPARATOR + version + } + +} + +object RocksDbStateStoreProvider { + + val ROCKS_DB_BASE_PATH: String = "/media/ephemeral0/spark/rocksdb" + +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/WALUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/WALUtils.scala new file mode 100644 index 000000000000..f1d861bed66f --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/WALUtils.scala @@ -0,0 +1,286 @@ +/* + * 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.Locale + +import com.google.common.io.ByteStreams +import org.apache.commons.io.IOUtils +import org.apache.hadoop.fs.{FileStatus, FSDataInputStream, FSError, Path} +import scala.collection.mutable + +import org.apache.spark.SparkConf +import org.apache.spark.io.LZ4CompressionCodec +import org.apache.spark.sql.catalyst.expressions.UnsafeRow +import org.apache.spark.sql.execution.streaming.CheckpointFileManager +import org.apache.spark.sql.execution.streaming.CheckpointFileManager.CancellableFSDataOutputStream +import org.apache.spark.sql.types.StructType + + +object WALUtils { + + case class StoreFile(version: Long, path: Path, isSnapshot: Boolean) + + /** Files needed to recover the given version of the store */ + def filesForVersion(allFiles: Seq[StoreFile], version: Long): Seq[StoreFile] = { + require(version >= 0) + require(allFiles.exists(_.version == version)) + + val latestSnapshotFileBeforeVersion = allFiles + .filter(_.isSnapshot == true) + .takeWhile(_.version <= version) + .lastOption + val deltaBatchFiles = latestSnapshotFileBeforeVersion match { + case Some(snapshotFile) => + + val deltaFiles = allFiles.filter { file => + file.version > snapshotFile.version && file.version <= version + }.toList + verify( + deltaFiles.size == version - snapshotFile.version, + s"Unexpected list of delta files for version $version for $this: $deltaFiles" + ) + deltaFiles + + case None => + allFiles.takeWhile(_.version <= version) + } + latestSnapshotFileBeforeVersion.toSeq ++ deltaBatchFiles + } + + /** Fetch all the files that back the store */ + def fetchFiles(fm: CheckpointFileManager, baseDir: Path): Seq[StoreFile] = { + val files: Seq[FileStatus] = try { + fm.list(baseDir) + } catch { + case _: java.io.FileNotFoundException => + Seq.empty + } + val versionToFiles = new mutable.HashMap[Long, StoreFile] + files.foreach { status => + val path = status.getPath + val nameParts = path.getName.split("\\.") + if (nameParts.size == 2) { + val version = nameParts(0).toLong + nameParts(1).toLowerCase(Locale.ROOT) match { + case "delta" => + // ignore the file otherwise, snapshot file already exists for that batch id + if (!versionToFiles.contains(version)) { + versionToFiles.put(version, StoreFile(version, path, isSnapshot = false)) + } + case "snapshot" => + versionToFiles.put(version, StoreFile(version, path, isSnapshot = true)) + case _ => + // logWarning(s"Could not identify file $path for $this") + } + } + } + val storeFiles = versionToFiles.values.toSeq.sortBy(_.version) + // logDebug(s"Current set of files for $this: ${storeFiles.mkString(", ")}") + storeFiles + } + + def compressStream(outputStream: DataOutputStream, sparkConf: SparkConf): DataOutputStream = { + val compressed = new LZ4CompressionCodec(sparkConf).compressedOutputStream(outputStream) + new DataOutputStream(compressed) + } + + def decompressStream(inputStream: DataInputStream, sparkConf: SparkConf): DataInputStream = { + val compressed = new LZ4CompressionCodec(sparkConf).compressedInputStream(inputStream) + new DataInputStream(compressed) + } + + + def writeUpdateToDeltaFile( + output: DataOutputStream, + key: UnsafeRow, + value: UnsafeRow): Unit = { + val keyBytes = key.getBytes() + val valueBytes = value.getBytes() + output.writeInt(keyBytes.size) + output.write(keyBytes) + output.writeInt(valueBytes.size) + output.write(valueBytes) + } + + def writeRemoveToDeltaFile(output: DataOutputStream, key: UnsafeRow): Unit = { + val keyBytes = key.getBytes() + output.writeInt(keyBytes.size) + output.write(keyBytes) + output.writeInt(-1) + } + + def finalizeDeltaFile(output: DataOutputStream): Unit = { + output.writeInt(-1) // Write this magic number to signify end of file + output.close() + } + + def updateFromDeltaFile( fm: CheckpointFileManager, + fileToRead: Path, + keySchema: StructType, + valueSchema: StructType, + newRocksDb: OptimisticTransactionDbInstance, + sparkConf: SparkConf): Unit = { + var input: DataInputStream = null + val sourceStream = try { + fm.open(fileToRead) + } catch { + case f: FileNotFoundException => + throw new IllegalStateException( + s"Error reading delta file $fileToRead of $this: $fileToRead does not exist", f) + } + try { + input = decompressStream(sourceStream, sparkConf) + var eof = false + + while(!eof) { + val keySize = input.readInt() + if (keySize == -1) { + eof = true + } else if (keySize < 0) { + newRocksDb.abort + newRocksDb.close() + throw new IOException( + s"Error reading delta file $fileToRead of $this: key size cannot be $keySize") + } else { + val keyRowBuffer = new Array[Byte](keySize) + ByteStreams.readFully(input, keyRowBuffer, 0, keySize) + + val keyRow = new UnsafeRow(keySchema.fields.length) + keyRow.pointTo(keyRowBuffer, keySize) + + val valueSize = input.readInt() + if (valueSize < 0) { + newRocksDb.remove(key = keyRow) + } else { + val valueRowBuffer = new Array[Byte](valueSize) + ByteStreams.readFully(input, valueRowBuffer, 0, valueSize) + val valueRow = new UnsafeRow(valueSchema.fields.length) + // If valueSize in existing file is not multiple of 8, floor it to multiple of 8. + // This is a workaround for the following: + // Prior to Spark 2.3 mistakenly append 4 bytes to the value row in + // `RowBasedKeyValueBatch`, which gets persisted into the checkpoint data + valueRow.pointTo(valueRowBuffer, (valueSize / 8) * 8) + newRocksDb.put(keyRow, valueRow) + } + } + } + } finally { + if (input != null) input.close() + } + } + + /* + * Try to cancel the underlying stream and safely close the compressed stream. + * + * @param compressedStream the compressed stream. + * @param rawStream the underlying stream which needs to be cancelled. + */ + def cancelDeltaFile( + compressedStream: DataOutputStream, + rawStream: CancellableFSDataOutputStream): Unit = { + try { + if (rawStream != null) rawStream.cancel() + IOUtils.closeQuietly(compressedStream) + } catch { + case e: FSError if e.getCause.isInstanceOf[IOException] => + // Closing the compressedStream causes the stream to write/flush flush data into the + // rawStream. Since the rawStream is already closed, there may be errors. + // Usually its an IOException. However, Hadoop's RawLocalFileSystem wraps + // IOException into FSError. + } + } + + def uploadFile(fm: CheckpointFileManager, + sourceFile: Path, + targetFile: Path, + sparkConf: SparkConf): Unit = { + var output: CancellableFSDataOutputStream = null + var in: BufferedInputStream = null + try { + in = new BufferedInputStream(new FileInputStream(sourceFile.toString)) + output = fm.createAtomic(targetFile, overwriteIfPossible = true) + // output = compressStream(rawOutput, sparkConf) + val buffer = new Array[ Byte ](1024) + var len = in.read(buffer) + while (len > 0) { + output.write(buffer, 0, len) + len = in.read(buffer) + } + output.close() + } catch { + case e: Throwable => + if (output != null) output.cancel() + throw e + } finally { + if ( in != null ) { + in.close() + } + } + } + + def downloadFile(fm: CheckpointFileManager, + sourceFile: Path, + targetFile: Path, + sparkConf: SparkConf): Boolean = { + var in: FSDataInputStream = null + var output: BufferedOutputStream = null + try { + in = fm.open(sourceFile) + // in = decompressStream(fm.open(sourceFile), sparkConf) + output = new BufferedOutputStream(new FileOutputStream(targetFile.toString)) + val buffer = new Array[ Byte ](1024) + var eof = false + while (!eof) { + val len = in.read(buffer) + if (len > 0) { + output.write(buffer, 0, len) + } else { + eof = true + } + } + output.close() + } catch { + case e: Throwable => + new File(targetFile.toString).delete() + throw e + } finally { + output.close() + if ( in != null ) { + in.close() + } + } + return true + } + + def deltaFile(baseDir: Path, version: Long): Path = { + new Path(baseDir, s"$version.delta") + } + + def snapshotFile(baseDir: Path, version: Long): Path = { + new Path(baseDir, s"$version.snapshot") + } + + def verify(condition: => Boolean, msg: String): Unit = { + if (!condition) { + throw new IllegalStateException(msg) + } + } + +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreSuite.scala new file mode 100644 index 000000000000..46d2e9bbfe2a --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreSuite.scala @@ -0,0 +1,516 @@ +/* + * 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.UUID + +import org.apache.commons.io.FileUtils +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.scalatest.{BeforeAndAfter, PrivateMethodTester} +import org.scalatest.concurrent.Eventually.{eventually, timeout} +import org.scalatest.time.SpanSugar._ +import scala.collection.JavaConverters._ +import scala.collection.mutable +import scala.util.Random + +import org.apache.spark.{SparkConf, SparkContext, SparkEnv} +import org.apache.spark.LocalSparkContext.withSpark +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.expressions.UnsafeRow +import org.apache.spark.sql.catalyst.util.quietly +import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.functions.count +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} +import org.apache.spark.util.Utils + +class RocksDbStateStoreSuite extends StateStoreSuiteBase[RocksDbStateStoreProvider] + with BeforeAndAfter with PrivateMethodTester { + type MapType = mutable.HashMap[UnsafeRow, UnsafeRow] + type ProviderMapType = java.util.concurrent.ConcurrentHashMap[UnsafeRow, UnsafeRow] + + import StateStoreCoordinatorSuite._ + import StateStoreTestsHelper._ + + val keySchema = StructType(Seq(StructField("key", StringType, true))) + val valueSchema = StructType(Seq(StructField("value", IntegerType, true))) + + before { + StateStore.stop() + require(!StateStore.isMaintenanceRunning) + } + + after { + StateStore.stop() + require(!StateStore.isMaintenanceRunning) + } + + def updateVersionTo( + provider: StateStoreProvider, + currentVersion: Int, + targetVersion: Int): Int = { + var newCurrentVersion = currentVersion + for (i <- newCurrentVersion until targetVersion) { + newCurrentVersion = incrementVersion(provider, i) + } + require(newCurrentVersion === targetVersion) + newCurrentVersion + } + + def incrementVersion(provider: StateStoreProvider, currentVersion: Int): Int = { + val store = provider.getStore(currentVersion) + put(store, "a", currentVersion + 1) + store.commit() + currentVersion + 1 + } + + def checkLoadedVersions( + rocksDbWriteInstance: RocksDbInstance, + count: Int, + earliestKey: Long, + latestKey: Long): Unit = { + assert(rocksDbWriteInstance.iterator(false).length === count) + } + + def checkVersion( + rocksDbWriteInstance: RocksDbInstance, + version: Long, + expectedData: Map[String, Int]): Unit = { + + val originValueMap = rocksDbWriteInstance.iterator(false).map { row => + rowToString(row.key) -> rowToInt(row.value) + }.toMap[String, Int] + + assert(originValueMap === expectedData) + } + + test("snapshotting") { + val provider = newStoreProvider(opId = Random.nextInt, partition = 0, minDeltasForSnapshot = 5) + + var currentVersion = 0 + + currentVersion = updateVersionTo(provider, currentVersion, 2) + require(getData(provider) === Set("a" -> 2)) + provider.doMaintenance() // should not generate snapshot files + assert(getData(provider) === Set("a" -> 2)) + + for (i <- 1 to currentVersion) { + assert(fileExists(provider, i, isSnapshot = false)) // all delta files present + assert(!fileExists(provider, i, isSnapshot = true)) // no snapshot files present + } + + // After version 6, snapshotting should generate one snapshot file + currentVersion = updateVersionTo(provider, currentVersion, 6) + require(getData(provider) === Set("a" -> 6), "store not updated correctly") + provider.doMaintenance() // should generate snapshot files + + val snapshotVersion = (0 to 6).find(version => fileExists(provider, version, isSnapshot = true)) + assert(snapshotVersion.nonEmpty, "snapshot file not generated") + deleteFilesEarlierThanVersion(provider, snapshotVersion.get) + assert( + getData(provider, snapshotVersion.get) === Set("a" -> snapshotVersion.get), + "snapshotting messed up the data of the snapshotted version") + assert( + getData(provider) === Set("a" -> 6), + "snapshotting messed up the data of the final version") + + // After version 20, snapshotting should generate newer snapshot files + currentVersion = updateVersionTo(provider, currentVersion, 20) + require(getData(provider) === Set("a" -> 20), "store not updated correctly") + provider.doMaintenance() // do snapshot + + val latestSnapshotVersion = (0 to 20).filter(version => + fileExists(provider, version, isSnapshot = true)).lastOption + assert(latestSnapshotVersion.nonEmpty, "no snapshot file found") + assert(latestSnapshotVersion.get > snapshotVersion.get, "newer snapshot not generated") + + deleteFilesEarlierThanVersion(provider, latestSnapshotVersion.get) + assert(getData(provider) === Set("a" -> 20), "snapshotting messed up the data") + } + + test("cleaning") { + val provider = newStoreProvider(opId = Random.nextInt, partition = 0, minDeltasForSnapshot = 5) + + for (i <- 1 to 20) { + val store = provider.getStore(i - 1) + put(store, "a", i) + store.commit() + provider.doMaintenance() // do cleanup + } + require( + rowsToSet(provider.latestIterator()) === Set("a" -> 20), + "store not updated correctly") + + assert(!fileExists(provider, version = 1, isSnapshot = false)) // first file should be deleted + + // last couple of versions should be retrievable + assert(getData(provider, 20) === Set("a" -> 20)) + assert(getData(provider, 19) === Set("a" -> 19)) + } + + testQuietly("SPARK-19677: Committing a delta file atop an existing one should not fail on HDFS") { + val conf = new Configuration() + conf.set("fs.fake.impl", classOf[RenameLikeHDFSFileSystem].getName) + conf.set("fs.defaultFS", "fake:///") + + val provider = newStoreProvider(opId = Random.nextInt, partition = 0, hadoopConf = conf) + provider.getStore(0).commit() + provider.getStore(0).commit() + + // Verify we don't leak temp files + val tempFiles = FileUtils.listFiles(new File(provider.stateStoreId.checkpointRootLocation), + null, true).asScala.filter(_.getName.startsWith("temp-")) + assert(tempFiles.isEmpty) + } + + test("corrupted file handling") { + val provider = newStoreProvider(opId = Random.nextInt, partition = 0, minDeltasForSnapshot = 5) + for (i <- 1 to 6) { + val store = provider.getStore(i - 1) + put(store, "a", i) + store.commit() + provider.doMaintenance() // do cleanup + } + val snapshotVersion = (0 to 10).find( version => + fileExists(provider, version, isSnapshot = true)).getOrElse(fail("snapshot file not found")) + + // Corrupt snapshot file and verify that it throws error + provider.close() + assert(getData(provider, snapshotVersion) === Set("a" -> snapshotVersion)) + RocksDbInstance.destroyDB(provider.rocksDbPath) + + corruptFile(provider, snapshotVersion, isSnapshot = true) + intercept[Exception] { + provider.close() + RocksDbInstance.destroyDB(provider.rocksDbPath) + getData(provider, snapshotVersion) + } + + // Corrupt delta file and verify that it throws error + provider.close() + RocksDbInstance.destroyDB(provider.rocksDbPath) + assert(getData(provider, snapshotVersion - 1) === Set("a" -> (snapshotVersion - 1))) + + corruptFile(provider, snapshotVersion - 1, isSnapshot = false) + intercept[Exception] { + provider.close() + RocksDbInstance.destroyDB(provider.rocksDbPath) + getData(provider, snapshotVersion - 1) + } + + // Delete delta file and verify that it throws error + deleteFilesEarlierThanVersion(provider, snapshotVersion) + intercept[Exception] { + provider.close() + RocksDbInstance.destroyDB(provider.rocksDbPath) + getData(provider, snapshotVersion - 1) + } + } + + test("StateStore.get") { + quietly { + val dir = newDir() + val storeId = StateStoreProviderId(StateStoreId(dir, 0, 0), UUID.randomUUID) + val sqlConf = new SQLConf + sqlConf.setConfString("spark.sql.streaming.stateStore.providerClass", + "org.apache.spark.sql.execution.streaming.state.RocksDbStateStoreProvider") + val storeConf = new StateStoreConf(sqlConf) + assert(storeConf.providerClass === + "org.apache.spark.sql.execution.streaming.state.RocksDbStateStoreProvider") + val hadoopConf = new Configuration() + + // Verify that trying to get incorrect versions throw errors + intercept[IllegalArgumentException] { + StateStore.get( + storeId, keySchema, valueSchema, None, -1, storeConf, hadoopConf) + } + assert(!StateStore.isLoaded(storeId)) // version -1 should not attempt to load the store + + intercept[IllegalStateException] { + StateStore.get( + storeId, keySchema, valueSchema, None, 1, storeConf, hadoopConf) + } + + // Increase version of the store and try to get again + val store0 = StateStore.get( + storeId, keySchema, valueSchema, None, 0, storeConf, hadoopConf) + assert(store0.version === 0) + put(store0, "a", 1) + store0.commit() + + val store1 = StateStore.get( + storeId, keySchema, valueSchema, None, 1, storeConf, hadoopConf) + assert(StateStore.isLoaded(storeId)) + assert(store1.version === 1) + assert(rowsToSet(store1.iterator()) === Set("a" -> 1)) + + // Verify that you can also load older version + val store0reloaded = StateStore.get( + storeId, keySchema, valueSchema, None, 0, storeConf, hadoopConf) + assert(store0reloaded.version === 0) + assert(rowsToSet(store0reloaded.iterator()) === Set.empty) + + // Verify that you can remove the store and still reload and use it + StateStore.unload(storeId) + assert(!StateStore.isLoaded(storeId)) + + val store1reloaded = StateStore.get( + storeId, keySchema, valueSchema, None, 1, storeConf, hadoopConf) + assert(StateStore.isLoaded(storeId)) + assert(store1reloaded.version === 1) + put(store1reloaded, "a", 2) + assert(store1reloaded.commit() === 2) + assert(rowsToSet(store1reloaded.iterator()) === Set("a" -> 2)) + } + } + + test("maintenance") { + val conf = new SparkConf() + .setMaster("local") + .setAppName("test") + // Make maintenance thread do snapshots and cleanups very fast + .set(StateStore.MAINTENANCE_INTERVAL_CONFIG, "10ms") + // Make sure that when SparkContext stops, the StateStore maintenance thread 'quickly' + // fails to talk to the StateStoreCoordinator and unloads all the StateStores + .set("spark.rpc.numRetries", "1") + val opId = 0 + val dir = newDir() + val storeProviderId = StateStoreProviderId(StateStoreId(dir, opId, 0), UUID.randomUUID) + val sqlConf = new SQLConf() + sqlConf.setConfString("spark.sql.streaming.stateStore.providerClass", + "org.apache.spark.sql.execution.streaming.state.RocksDbStateStoreProvider") + sqlConf.setConf(SQLConf.MIN_BATCHES_TO_RETAIN, 2) + val storeConf = StateStoreConf(sqlConf) + val hadoopConf = new Configuration() + val provider = newStoreProvider(storeProviderId.storeId) + + var latestStoreVersion = 0 + + def generateStoreVersions() { + for (i <- 1 to 20) { + val store = StateStore.get(storeProviderId, keySchema, valueSchema, None, + latestStoreVersion, storeConf, hadoopConf) + put(store, "a", i) + store.commit() + latestStoreVersion += 1 + } + } + + val timeoutDuration = 60 seconds + + quietly { + withSpark(new SparkContext(conf)) { sc => + withCoordinatorRef(sc) { coordinatorRef => + require(!StateStore.isMaintenanceRunning, "StateStore is unexpectedly running") + + // Generate sufficient versions of store for snapshots + generateStoreVersions() + + eventually(timeout(timeoutDuration)) { + // Store should have been reported to the coordinator + assert(coordinatorRef.getLocation(storeProviderId).nonEmpty, + "active instance was not reported") + + // Background maintenance should clean up and generate snapshots + assert(StateStore.isMaintenanceRunning, "Maintenance task is not running") + + // Some snapshots should have been generated + val snapshotVersions = (1 to latestStoreVersion).filter { version => + fileExists(provider, version, isSnapshot = true) + } + assert(snapshotVersions.nonEmpty, "no snapshot file found") + } + + // Generate more versions such that there is another snapshot and + // the earliest delta file will be cleaned up + generateStoreVersions() + + // Earliest delta file should get cleaned up + eventually(timeout(timeoutDuration)) { + assert(!fileExists(provider, 1, isSnapshot = false), "earliest file not deleted") + } + + // If driver decides to deactivate all stores related to a query run, + // then this instance should be unloaded + coordinatorRef.deactivateInstances(storeProviderId.queryRunId) + eventually(timeout(timeoutDuration)) { + assert(!StateStore.isLoaded(storeProviderId)) + } + + // Reload the store and verify + StateStore.get(storeProviderId, keySchema, valueSchema, indexOrdinal = None, + latestStoreVersion, storeConf, hadoopConf) + assert(StateStore.isLoaded(storeProviderId)) + + // If some other executor loads the store, then this instance should be unloaded + coordinatorRef.reportActiveInstance(storeProviderId, "other-host", "other-exec") + eventually(timeout(timeoutDuration)) { + assert(!StateStore.isLoaded(storeProviderId)) + } + + // Reload the store and verify + StateStore.get(storeProviderId, keySchema, valueSchema, indexOrdinal = None, + latestStoreVersion, storeConf, hadoopConf) + assert(StateStore.isLoaded(storeProviderId)) + } + } + + // Verify if instance is unloaded if SparkContext is stopped + eventually(timeout(timeoutDuration)) { + require(SparkEnv.get === null) + assert(!StateStore.isLoaded(storeProviderId)) + assert(!StateStore.isMaintenanceRunning) + } + } + } + + test("SPARK-21145: Restarted queries create new provider instances") { + try { + val checkpointLocation = Utils.createTempDir().getAbsoluteFile + val spark = SparkSession.builder().master("local[2]").getOrCreate() + SparkSession.setActiveSession(spark) + implicit val sqlContext = spark.sqlContext + spark.conf.set("spark.sql.shuffle.partitions", "1") + spark.conf.set("spark.sql.streaming.stateStore.providerClass", + "org.apache.spark.sql.execution.streaming.state.RocksDbStateStoreProvider") + import spark.implicits._ + val inputData = MemoryStream[Int] + + def runQueryAndGetLoadedProviders(): Seq[StateStoreProvider] = { + val aggregated = inputData.toDF().groupBy("value").agg(count("*")) + // stateful query + val query = aggregated.writeStream + .format("memory") + .outputMode("complete") + .queryName("query") + .option("checkpointLocation", checkpointLocation.toString) + .start() + inputData.addData(1, 2, 3) + query.processAllAvailable() + require(query.lastProgress != null) // at least one batch processed after start + val loadedProvidersMethod = + PrivateMethod[mutable.HashMap[StateStoreProviderId, StateStoreProvider]]('loadedProviders) + val loadedProvidersMap = StateStore invokePrivate loadedProvidersMethod() + val loadedProviders = loadedProvidersMap.synchronized { loadedProvidersMap.values.toSeq } + query.stop() + loadedProviders + } + + val loadedProvidersAfterRun1 = runQueryAndGetLoadedProviders() + require(loadedProvidersAfterRun1.length === 1) + + val loadedProvidersAfterRun2 = runQueryAndGetLoadedProviders() + assert(loadedProvidersAfterRun2.length === 2) // two providers loaded for 2 runs + + // Both providers should have the same StateStoreId, but the should be different objects + assert(loadedProvidersAfterRun2(0).stateStoreId === loadedProvidersAfterRun2(1).stateStoreId) + assert(loadedProvidersAfterRun2(0) ne loadedProvidersAfterRun2(1)) + + } finally { + SparkSession.getActiveSession.foreach { spark => + spark.streams.active.foreach(_.stop()) + spark.stop() + } + } + } + + override def newStoreProvider(): RocksDbStateStoreProvider = { + newStoreProvider(opId = Random.nextInt(), partition = 0) + } + + override def newStoreProvider(storeId: StateStoreId): RocksDbStateStoreProvider = { + newStoreProvider(storeId.operatorId, storeId.partitionId, dir = storeId.checkpointRootLocation) + } + + override def getLatestData(storeProvider: RocksDbStateStoreProvider): Set[(String, Int)] = { + getData(storeProvider) + } + + override def getData( + provider: RocksDbStateStoreProvider, + version: Int = -1): Set[(String, Int)] = { + val reloadedProvider = newStoreProvider(provider.stateStoreId) + if (version < 0) { + // TODO find out last version from rocksDB + reloadedProvider.latestIterator().map(rowsToStringInt).toSet + } else { + reloadedProvider.getStore(version).iterator().map(rowsToStringInt).toSet + } + } + + def newStoreProvider( + opId: Long, + partition: Int, + 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): RocksDbStateStoreProvider = { + val sqlConf = new SQLConf() + sqlConf.setConf(SQLConf.STATE_STORE_MIN_DELTAS_FOR_SNAPSHOT, minDeltasForSnapshot) + sqlConf.setConf(SQLConf.MAX_BATCHES_TO_RETAIN_IN_MEMORY, numOfVersToRetainInMemory) + sqlConf.setConf(SQLConf.MIN_BATCHES_TO_RETAIN, 2) + val provider = new RocksDbStateStoreProvider + provider.init( + StateStoreId(dir, opId, partition), + keySchema, + valueSchema, + keyIndexOrdinal = None, + new StateStoreConf(sqlConf), + hadoopConf) + provider + } + + + def fileExists( + provider: RocksDbStateStoreProvider, + version: Long, + isSnapshot: Boolean): Boolean = { + val method = PrivateMethod[Path]('baseDir) + val basePath = provider invokePrivate method() + val fileName = if (isSnapshot) s"$version.snapshot" else s"$version.delta" + val filePath = new File(basePath.toString, fileName) + filePath.exists + } + + def deleteFilesEarlierThanVersion(provider: RocksDbStateStoreProvider, version: Long): Unit = { + val method = PrivateMethod[Path]('baseDir) + val basePath = provider invokePrivate method() + for (version <- 0 until version.toInt) { + for (isSnapshot <- Seq(false, true)) { + val fileName = if (isSnapshot) s"$version.snapshot" else s"$version.delta" + val filePath = new File(basePath.toString, fileName) + if (filePath.exists) filePath.delete() + } + } + } + + def corruptFile( + provider: RocksDbStateStoreProvider, + version: Long, + isSnapshot: Boolean): Unit = { + val method = PrivateMethod[Path]('baseDir) + val basePath = provider invokePrivate method() + val fileName = if (isSnapshot) s"$version.snapshot" else s"$version.delta" + val filePath = new File(basePath.toString, fileName) + filePath.delete() + filePath.createNewFile() + } + +} From 292befe19da5484efbd8173f2d8da3fbddd86b22 Mon Sep 17 00:00:00 2001 From: Vikram Agrawal Date: Thu, 20 Jun 2019 14:56:29 +0530 Subject: [PATCH 02/26] Add Apache License --- .../main/java/org/apache/spark/io/FileUtility.java | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/core/src/main/java/org/apache/spark/io/FileUtility.java b/core/src/main/java/org/apache/spark/io/FileUtility.java index a62185ddfb5f..e91882222317 100644 --- a/core/src/main/java/org/apache/spark/io/FileUtility.java +++ b/core/src/main/java/org/apache/spark/io/FileUtility.java @@ -1,3 +1,16 @@ +/* + * Licensed 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.io; import org.apache.commons.compress.archivers.ArchiveException; From 05568eeebbeb14c26889f881dadfee8dc6b7bd94 Mon Sep 17 00:00:00 2001 From: Vikram Agrawal Date: Tue, 25 Jun 2019 10:17:07 +0530 Subject: [PATCH 03/26] Fix indentation and scalastyle errors --- .../streaming/state/RocksDbInstance.scala | 83 ++--- .../state/RocksDbStateStoreProvider.scala | 303 ++++++++++-------- .../execution/streaming/state/WALUtils.scala | 64 ++-- .../state/RocksDbStateStoreSuite.scala | 215 +++++++------ 4 files changed, 364 insertions(+), 301 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbInstance.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbInstance.scala index 5e031960905e..a8dcd671900e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbInstance.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbInstance.scala @@ -30,10 +30,8 @@ import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.types.StructType import org.apache.spark.util.Utils - -class RocksDbInstance(keySchema: StructType, - valueSchema: StructType, - identifier: String) extends Logging { +class RocksDbInstance(keySchema: StructType, valueSchema: StructType, identifier: String) + extends Logging { import RocksDbInstance._ RocksDB.loadLibrary() @@ -54,15 +52,19 @@ class RocksDbInstance(keySchema: StructType, try { setOptions(conf) db = readOnly match { - case true => options.setCreateIfMissing(false) + case true => + options.setCreateIfMissing(false) RocksDB.openReadOnly(options, path) - case false => options.setCreateIfMissing(true) + case false => + options.setCreateIfMissing(true) RocksDB.open(options, path) } dbPath = path } catch { case e: Throwable => - throw new IllegalStateException(s"Error while creating rocksDb instance ${e.getMessage}", e) + throw new IllegalStateException( + s"Error while creating rocksDb instance ${e.getMessage}", + e) } } @@ -123,13 +125,13 @@ class RocksDbInstance(keySchema: StructType, itr.seekToFirst() - new Iterator[ UnsafeRowPair ] { + new Iterator[UnsafeRowPair] { override def hasNext: Boolean = { - if ( itr.isValid ) { + if (itr.isValid) { true } else { itrReadOptions.close() - if ( closeDbOnCompletion ) { + if (closeDbOnCompletion) { close() } logDebug(s"read from DB completed") @@ -150,8 +152,7 @@ class RocksDbInstance(keySchema: StructType, } } - - def printStats: Unit = { + def printStats: Unit = { verify(isOpen(), "Open rocksDb instance before any operation") try { val stats = db.getProperty("rocksdb.stats") @@ -171,13 +172,17 @@ class RocksDbInstance(keySchema: StructType, writeOptions.setSync(false) writeOptions.setDisableWAL(true) - val dataBlockSize = conf.getOrElse( - "spark.sql.streaming.stateStore.rocksDb.blockSizeInKB". - toLowerCase(Locale.ROOT), "64").toInt + val dataBlockSize = conf + .getOrElse( + "spark.sql.streaming.stateStore.rocksDb.blockSizeInKB".toLowerCase(Locale.ROOT), + "64") + .toInt - val metadataBlockSize = conf.getOrElse( - "spark.sql.streaming.stateStore.rocksDb.metadataBlockSizeInKB". - toLowerCase(Locale.ROOT), "4").toInt + val metadataBlockSize = conf + .getOrElse( + "spark.sql.streaming.stateStore.rocksDb.metadataBlockSizeInKB".toLowerCase(Locale.ROOT), + "4") + .toInt // Table configs // https://github.com/facebook/rocksdb/wiki/Partitioned-Index-Filters @@ -196,18 +201,23 @@ class RocksDbInstance(keySchema: StructType, .setFormatVersion(4) // https://rocksdb.org/blog/2019/03/08/format-version-4.html .setIndexBlockRestartInterval(16) - var bufferNumber = conf.getOrElse( - "spark.sql.streaming.stateStore.rocksDb.bufferNumber".toLowerCase(Locale.ROOT), "5").toInt + var bufferNumber = conf + .getOrElse( + "spark.sql.streaming.stateStore.rocksDb.bufferNumber".toLowerCase(Locale.ROOT), + "5") + .toInt bufferNumber = Math.max(bufferNumber, 3) val bufferNumberToMaintain = Math.max(bufferNumber - 2, 3) - logInfo(s"Using Max Buffer Name = $bufferNumber & " + - s"max buffer number to maintain = $bufferNumberToMaintain") - + logInfo( + s"Using Max Buffer Name = $bufferNumber & " + + s"max buffer number to maintain = $bufferNumberToMaintain") + // DB Options - options.setCreateIfMissing(true) + options + .setCreateIfMissing(true) .setMaxWriteBufferNumber(bufferNumber) .setMaxWriteBufferNumberToMaintain(bufferNumberToMaintain) .setMaxBackgroundCompactions(4) @@ -256,9 +266,10 @@ class RocksDbInstance(keySchema: StructType, } class OptimisticTransactionDbInstance( - keySchema: StructType, valueSchema: StructType, identifier: String) - extends RocksDbInstance( - keySchema: StructType, valueSchema: StructType, identifier: String) { + keySchema: StructType, + valueSchema: StructType, + identifier: String) + extends RocksDbInstance(keySchema: StructType, valueSchema: StructType, identifier: String) { import RocksDbInstance._ RocksDB.loadLibrary() @@ -284,8 +295,10 @@ class OptimisticTransactionDbInstance( dbPath = path } catch { case e: Throwable => - throw new IllegalStateException(s"Error while creating OptimisticTransactionDb instance" + - s" ${e.getMessage}", e) + throw new IllegalStateException( + s"Error while creating OptimisticTransactionDb instance" + + s" ${e.getMessage}", + e) } } @@ -332,7 +345,7 @@ class OptimisticTransactionDbInstance( txn.commit() txn.close() txn = null - backupPath.foreach(f => createCheckpoint(otdb.asInstanceOf[ RocksDB ], f)) + backupPath.foreach(f => createCheckpoint(otdb.asInstanceOf[RocksDB], f)) } catch { case e: Exception => log.error(s"Unable to commit the transactions. Error message = ${e.getMessage}") @@ -342,8 +355,7 @@ class OptimisticTransactionDbInstance( def printTrxStats(): Unit = { verify(txn != null, "No open Transaction") - logInfo( - s""" + logInfo(s""" | deletes = ${txn.getNumDeletes} | numKeys = ${txn.getNumKeys} | puts = ${txn.getNumPuts} @@ -351,7 +363,6 @@ class OptimisticTransactionDbInstance( """.stripMargin) } - override def abort(): Unit = { verify(txn != null, "No Transaction to abort") txn.rollbackToSavePoint() @@ -370,7 +381,8 @@ class OptimisticTransactionDbInstance( override def iterator(closeDbOnCompletion: Boolean): Iterator[UnsafeRowPair] = { verify(txn != null, "Transaction is not set") - verify(closeDbOnCompletion == false, + verify( + closeDbOnCompletion == false, "Cannot close a DB without aborting/commiting the transactions") val readOptions = new ReadOptions() val itr: RocksIterator = txn.getIterator(readOptions) @@ -391,8 +403,7 @@ object RocksDbInstance { private val destroyOptions: Options = new Options() - val lRUCache = new LRUCache(1024 * 1024 * 1024, 6, - false, 0.05) + val lRUCache = new LRUCache(1024 * 1024 * 1024, 6, false, 0.05) def destroyDB(path: String): Unit = { val f: File = new File(path) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreProvider.scala index 2c7b43e24800..869543fa8f2e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreProvider.scala @@ -35,7 +35,6 @@ 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 */ @@ -46,10 +45,10 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log @volatile private var hadoopConf: Configuration = _ @volatile private var numberOfVersionsToRetain: Int = _ - /* - * Additional configurations related to rocksDb. This will capture all configs in - * SQLConf that start with `spark.sql.streaming.stateStore.rocksDb` - */ + /* + * 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() @@ -60,8 +59,7 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log import WALUtils._ /** Implementation of [[StateStore]] API which is backed by RocksDb and HDFS */ - class RocksDbStateStore(val version: Long) - extends StateStore with Logging { + class RocksDbStateStore(val version: Long) extends StateStore with Logging { /** Trait and classes representing the internal state of the store */ trait STATE @@ -85,8 +83,8 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log private def initTransaction(): Unit = { if (state == LOADED && rocksDbWriteInstance == null) { logDebug(s"Creating Transactional DB for batch $version") - rocksDbWriteInstance = new OptimisticTransactionDbInstance( - keySchema, valueSchema, newVersion.toString) + rocksDbWriteInstance = + new OptimisticTransactionDbInstance(keySchema, valueSchema, newVersion.toString) rocksDbWriteInstance.open(rocksDbPath, rocksDbConf) state = UPDATING rocksDbWriteInstance.startTransactions() @@ -100,8 +98,10 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log override def put(key: UnsafeRow, value: UnsafeRow): Unit = { initTransaction - verify(state == UPDATING, s"Current state of the store is $state. " + - s"Cannot put after already committed or aborted") + verify( + state == UPDATING, + s"Current state of the store is $state. " + + s"Cannot put after already committed or aborted") val keyCopy = key.copy() val valueCopy = value.copy() rocksDbWriteInstance.put(keyCopy, valueCopy) @@ -116,8 +116,9 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log writeRemoveToDeltaFile(compressedStream, key) } - override def getRange( start: Option[UnsafeRow], - end: Option[UnsafeRow]): Iterator[UnsafeRowPair] = { + override def getRange( + start: Option[UnsafeRow], + end: Option[UnsafeRow]): Iterator[UnsafeRowPair] = { verify(state == UPDATING, "Cannot getRange after already committed or aborted") iterator() } @@ -125,8 +126,10 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log /** Commit all the updates that have been made to the store, and return the new version. */ override def commit(): Long = { initTransaction - verify(state == UPDATING, s"Current state of the store is $state " + - s"Cannot commit after already committed or aborted") + verify( + state == UPDATING, + s"Current state of the store is $state " + + s"Cannot commit after already committed or aborted") try { state = COMMITTED synchronized { @@ -138,11 +141,10 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log newVersion } catch { case NonFatal(e) => - throw new IllegalStateException( - s"Error committing version $newVersion into $this", e) + throw new IllegalStateException(s"Error committing version $newVersion into $this", e) } finally { - storeMap.remove(version) - close() + storeMap.remove(version) + close() } } @@ -168,22 +170,22 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log case NonFatal(e) => throw new IllegalStateException(s"Error aborting version $newVersion into $this", e) } finally { - storeMap.remove(version) - close() + storeMap.remove(version) + close() } } def close(): Unit = { - if ( rocksDbWriteInstance != null ) { + if (rocksDbWriteInstance != null) { rocksDbWriteInstance.close() rocksDbWriteInstance = null } } - /* - * Get an iterator of all the store data. - * This can be called only after committing all the updates made in the current thread. - */ + /* + * Get an iterator of all the store data. + * This can be called only after committing all the updates made in the current thread. + */ override def iterator(): Iterator[UnsafeRowPair] = { state match { case UPDATING => @@ -206,7 +208,8 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log logDebug(s"state = committed using checkpointed DB with version $newVersion") // use checkpointed db for current updated version val path = getBackupPath(newVersion) - val r: RocksDbInstance = new RocksDbInstance(keySchema, valueSchema, newVersion.toString) + val r: RocksDbInstance = + new RocksDbInstance(keySchema, valueSchema, newVersion.toString) r.open(path, rocksDbConf, readOnly = true) r.iterator(closeDbOnCompletion = true) @@ -216,12 +219,15 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log override def metrics: StateStoreMetrics = { val metricsFromProvider: Map[String, Long] = getMetricsForProvider() - val customMetrics = metricsFromProvider.flatMap { case (name, value) => - // just allow searching from list cause the list is small enough - supportedCustomMetrics.find(_.name == name).map(_ -> value) + val customMetrics = metricsFromProvider.flatMap { + case (name, value) => + // just allow searching from list cause the list is small enough + supportedCustomMetrics.find(_.name == name).map(_ -> value) } - return StateStoreMetrics(Math.max(numEntriesInDb, 0), - Math.max(bytesUsedByDb, 0), customMetrics) + return StateStoreMetrics( + Math.max(numEntriesInDb, 0), + Math.max(bytesUsedByDb, 0), + customMetrics) } /* @@ -237,26 +243,27 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log } - /* - * Initialize the provide with more contextual information from the SQL operator. - * This method will be called first after creating an instance of the StateStoreProvider by - * reflection. - * - * @param stateStoreId Id of the versioned StateStores that this provider will generate - * @param keySchema Schema of keys to be stored - * @param valueSchema Schema of value to be stored - * @param keyIndexOrdinal Optional column (represent as the ordinal of the field in keySchema) by - * which the StateStore implementation could index the data. - * @param storeConfs Configurations used by the StateStores - * @param hadoopConf Hadoop configuration that could be used by StateStore - * to save state data - */ - override def init(stateStoreId: StateStoreId, - keySchema: StructType, - valueSchema: StructType, - keyIndexOrdinal: Option[ Int ], // for sorting the data by their keys - storeConfs: StateStoreConf, - hadoopConf: Configuration): Unit = { + /* + * Initialize the provide with more contextual information from the SQL operator. + * This method will be called first after creating an instance of the StateStoreProvider by + * reflection. + * + * @param stateStoreId Id of the versioned StateStores that this provider will generate + * @param keySchema Schema of keys to be stored + * @param valueSchema Schema of value to be stored + * @param keyIndexOrdinal Optional column (represent as the ordinal of the field in keySchema) by + * which the StateStore implementation could index the data. + * @param storeConfs Configurations used by the StateStores + * @param hadoopConf Hadoop configuration that could be used by StateStore + * to save state data + */ + override def init( + stateStoreId: StateStoreId, + keySchema: StructType, + valueSchema: StructType, + keyIndexOrdinal: Option[Int], // for sorting the data by their keys + storeConfs: StateStoreConf, + hadoopConf: Configuration): Unit = { this.stateStoreId_ = stateStoreId this.keySchema = keySchema this.valueSchema = valueSchema @@ -265,17 +272,17 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log // TODO add new conf for `maxVersionsToRetainInMemory` this.numberOfVersionsToRetain = storeConfs.maxVersionsToRetainInMemory fm.mkdirs(baseDir) - this.rocksDbConf = storeConf.confs. - filter(_._1.startsWith("spark.sql.streaming.stateStore.rocksDb")) + this.rocksDbConf = storeConf.confs + .filter(_._1.startsWith("spark.sql.streaming.stateStore.rocksDb")) .map { case (k, v) => (k.toLowerCase(Locale.ROOT), v) } } - /* - * Return the id of the StateStores this provider will generate. - * Should be the same as the one passed in init(). - */ + /* + * Return the id of the StateStores this provider will generate. + * Should be the same as the one passed in init(). + */ override def stateStoreId: StateStoreId = stateStoreId_ /* @@ -288,14 +295,14 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log private val storeMap = new util.HashMap[Long, RocksDbStateStore]() - /* - * Optional custom metrics that the implementation may want to report. - * - * @note The StateStore objects created by this provider must report the same custom metrics - * (specifically, same names) through `StateStore.metrics`. - */ + /* + * Optional custom metrics that the implementation may want to report. + * + * @note The StateStore objects created by this provider must report the same custom metrics + * (specifically, same names) through `StateStore.metrics`. + */ // TODO - override def supportedCustomMetrics: Seq[ StateStoreCustomMetric ] = { + override def supportedCustomMetrics: Seq[StateStoreCustomMetric] = { Nil } @@ -325,7 +332,8 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log def createStore(version: Long): RocksDbStateStore = { val newStore = new RocksDbStateStore(version) - logInfo(s"Creating a new Store for version $version and partition ${stateStoreId_.partitionId}") + logInfo( + s"Creating a new Store for version $version and partition ${stateStoreId_.partitionId}") if (version > 0 & !checkIfStateExists(version)) { // load the data in the rocksDB logInfo(s"Loading state for $version and partition ${stateStoreId_.partitionId}") @@ -346,63 +354,68 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log var lastAvailableVersion = version var found = false val (result, elapsedMs) = Utils.timeTakenMs { - try { - if (checkIfStateExists(version - 1) ) { - found = true - lastAvailableVersion = version - 1 - } else { - // TODO check for numberOfVersionsToRetain - // Destroy DB so that we can recontruct it using snapshot and delta files - RocksDbInstance.destroyDB(rocksDbPath) - } - - // Check for snapshot files starting from "version" - while (!found && lastAvailableVersion > 0) { - found = { - try { - loadSnapshotFile(lastAvailableVersion) - } catch { - case e: Exception => - logError(s"$e while reading snapshot file") - throw e - } - } - if (!found) { - lastAvailableVersion = lastAvailableVersion - 1 - } - logInfo(s"Snapshot for $lastAvailableVersion for " + - s"partition ${stateStoreId_.partitionId} found = $found") - } - - rocksDbWriteInstance = new OptimisticTransactionDbInstance( - keySchema, valueSchema, version.toString) + try { + if (checkIfStateExists(version - 1)) { + found = true + lastAvailableVersion = version - 1 + } else { + // TODO check for numberOfVersionsToRetain + // Destroy DB so that we can recontruct it using snapshot and delta files + RocksDbInstance.destroyDB(rocksDbPath) + } + + // Check for snapshot files starting from "version" + while (!found && lastAvailableVersion > 0) { + found = { + try { + loadSnapshotFile(lastAvailableVersion) + } catch { + case e: Exception => + logError(s"$e while reading snapshot file") + throw e + } + } + if (!found) { + lastAvailableVersion = lastAvailableVersion - 1 + } + logInfo( + s"Snapshot for $lastAvailableVersion for " + + s"partition ${stateStoreId_.partitionId} found = $found") + } + + rocksDbWriteInstance = + new OptimisticTransactionDbInstance(keySchema, valueSchema, version.toString) rocksDbWriteInstance.open(rocksDbPath, rocksDbConf) rocksDbWriteInstance.startTransactions() - // Load all the deltas from the version after the last available - // one up to the target version. - // The last available version is the one with a full snapshot, so it doesn't need deltas. - for (deltaVersion <- (lastAvailableVersion + 1) to version) { - val fileToRead = deltaFile(baseDir, deltaVersion) - updateFromDeltaFile(fm, fileToRead, keySchema, valueSchema, - rocksDbWriteInstance, sparkConf) - logInfo(s"Read delta file for version $version of $this from $fileToRead") - } - - rocksDbWriteInstance.commit(Some(getBackupPath(version))) - rocksDbWriteInstance.close() - rocksDbWriteInstance = null - } - catch { - case e: IllegalStateException => - logError(s"Exception while loading state ${e.getMessage}") - if (rocksDbWriteInstance != null) { - rocksDbWriteInstance.abort() - rocksDbWriteInstance.close() - } - throw e - } - } + // Load all the deltas from the version after the last available + // one up to the target version. + // The last available version is the one with a full snapshot, so it doesn't need deltas. + for (deltaVersion <- (lastAvailableVersion + 1) to version) { + val fileToRead = deltaFile(baseDir, deltaVersion) + updateFromDeltaFile( + fm, + fileToRead, + keySchema, + valueSchema, + rocksDbWriteInstance, + sparkConf) + logInfo(s"Read delta file for version $version of $this from $fileToRead") + } + + rocksDbWriteInstance.commit(Some(getBackupPath(version))) + rocksDbWriteInstance.close() + rocksDbWriteInstance = null + } catch { + case e: IllegalStateException => + logError(s"Exception while loading state ${e.getMessage}") + if (rocksDbWriteInstance != null) { + rocksDbWriteInstance.abort() + rocksDbWriteInstance.close() + } + throw e + } + } logInfo(s"Loading state for $version takes $elapsedMs ms.") } @@ -454,11 +467,11 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log } private def doSnapshot(files: Seq[WALUtils.StoreFile]): Unit = { - if ( files.nonEmpty ) { + if (files.nonEmpty) { val lastVersion = files.last.version val deltaFilesForLastVersion = filesForVersion(files, lastVersion).filter(_.isSnapshot == false) - if ( deltaFilesForLastVersion.size > storeConf.minDeltasForSnapshot ) { + if (deltaFilesForLastVersion.size > storeConf.minDeltasForSnapshot) { val dbPath = getBackupPath(lastVersion) val snapShotFileName = getTempPath(lastVersion).concat(".snapshot") val f = new File(snapShotFileName) @@ -469,8 +482,9 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log val targetFile = snapshotFile(baseDir, lastVersion) uploadFile(fm, new Path(snapShotFileName), targetFile, sparkConf) } - logInfo(s"Creating snapshot file for" + - s" ${stateStoreId_.partitionId} took $t1 ms.") + logInfo( + s"Creating snapshot file for" + + s" ${stateStoreId_.partitionId} took $t1 ms.") } catch { case e: Exception => logError(s"Exception while creating snapshot $e} ") @@ -482,11 +496,11 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log } } - /* - * Clean up old snapshots and delta files that are not needed any more. It ensures that last - * few versions of the store can be recovered from the files, so re-executed RDD operations - * can re-apply updates on the past versions of the store. - */ + /* + * Clean up old snapshots and delta files that are not needed any more. It ensures that last + * few versions of the store can be recovered from the files, so re-executed RDD operations + * can re-apply updates on the past versions of the store. + */ private[state] def cleanup(files: Seq[WALUtils.StoreFile]): Unit = { try { if (files.nonEmpty) { @@ -504,8 +518,9 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log } } logDebug(s"deleting files took $e2 ms.") - logInfo(s"Deleted files older than ${earliestFileToRetain.version} for $this: " + - filesToDelete.mkString(", ")) + logInfo( + s"Deleted files older than ${earliestFileToRetain.version} for $this: " + + filesToDelete.mkString(", ")) } } } catch { @@ -531,14 +546,13 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log } } - // Used only for unit tests private[sql] def latestIterator(): Iterator[UnsafeRowPair] = synchronized { val versionsInFiles = fetchFiles(fm, baseDir).map(_.version).toSet var itr = Iterator.empty if (versionsInFiles.nonEmpty) { val maxVersion = versionsInFiles.max - if ( maxVersion == 0 ) { + if (maxVersion == 0) { return Iterator.empty } // FIXME assuming maxVersion exists in rocksDB @@ -549,7 +563,7 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log return r.iterator(false) } catch { case e: Exception => - // do nothing + // do nothing } } Iterator.empty @@ -558,8 +572,11 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log // making it public for unit tests lazy val rocksDbPath: String = { val checkpointRootLocationPath = new Path(stateStoreId.checkpointRootLocation) - val basePath = new Path(RocksDbStateStoreProvider.ROCKS_DB_BASE_PATH, new Path("db", - checkpointRootLocationPath.getName + "_" + checkpointRootLocationPath.hashCode())) + val basePath = new Path( + RocksDbStateStoreProvider.ROCKS_DB_BASE_PATH, + new Path( + "db", + checkpointRootLocationPath.getName + "_" + checkpointRootLocationPath.hashCode())) val dir = basePath.toString + Path.SEPARATOR + stateStoreId_.operatorId + Path.SEPARATOR + @@ -577,8 +594,10 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log private def getBackupPath(version: Long): String = { val checkpointRootLocationPath = new Path(stateStoreId.checkpointRootLocation) - val basePath = new Path(RocksDbStateStoreProvider.ROCKS_DB_BASE_PATH, - new Path("backup", + val basePath = new Path( + RocksDbStateStoreProvider.ROCKS_DB_BASE_PATH, + new Path( + "backup", checkpointRootLocationPath.getName + "_" + checkpointRootLocationPath.hashCode())) val dir = basePath.toString + Path.SEPARATOR + @@ -598,8 +617,10 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log private def getTempPath(version: Long): String = { val checkpointRootLocationPath = new Path(stateStoreId.checkpointRootLocation) - val basePath = new Path(RocksDbStateStoreProvider.ROCKS_DB_BASE_PATH, - new Path("tmp", + val basePath = new Path( + RocksDbStateStoreProvider.ROCKS_DB_BASE_PATH, + new Path( + "tmp", checkpointRootLocationPath.getName + "_" + checkpointRootLocationPath.hashCode())) val dir = basePath.toString + Path.SEPARATOR + @@ -618,7 +639,7 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log } -object RocksDbStateStoreProvider { +object RocksDbStateStoreProvider { val ROCKS_DB_BASE_PATH: String = "/media/ephemeral0/spark/rocksdb" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/WALUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/WALUtils.scala index f1d861bed66f..e9c52c9e8e47 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/WALUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/WALUtils.scala @@ -32,7 +32,6 @@ import org.apache.spark.sql.execution.streaming.CheckpointFileManager import org.apache.spark.sql.execution.streaming.CheckpointFileManager.CancellableFSDataOutputStream import org.apache.spark.sql.types.StructType - object WALUtils { case class StoreFile(version: Long, path: Path, isSnapshot: Boolean) @@ -48,14 +47,12 @@ object WALUtils { .lastOption val deltaBatchFiles = latestSnapshotFileBeforeVersion match { case Some(snapshotFile) => - val deltaFiles = allFiles.filter { file => file.version > snapshotFile.version && file.version <= version }.toList verify( deltaFiles.size == version - snapshotFile.version, - s"Unexpected list of delta files for version $version for $this: $deltaFiles" - ) + s"Unexpected list of delta files for version $version for $this: $deltaFiles") deltaFiles case None => @@ -87,7 +84,7 @@ object WALUtils { case "snapshot" => versionToFiles.put(version, StoreFile(version, path, isSnapshot = true)) case _ => - // logWarning(s"Could not identify file $path for $this") + // logWarning(s"Could not identify file $path for $this") } } } @@ -106,11 +103,7 @@ object WALUtils { new DataInputStream(compressed) } - - def writeUpdateToDeltaFile( - output: DataOutputStream, - key: UnsafeRow, - value: UnsafeRow): Unit = { + def writeUpdateToDeltaFile(output: DataOutputStream, key: UnsafeRow, value: UnsafeRow): Unit = { val keyBytes = key.getBytes() val valueBytes = value.getBytes() output.writeInt(keyBytes.size) @@ -127,29 +120,31 @@ object WALUtils { } def finalizeDeltaFile(output: DataOutputStream): Unit = { - output.writeInt(-1) // Write this magic number to signify end of file + output.writeInt(-1) // Write this magic number to signify end of file output.close() } - def updateFromDeltaFile( fm: CheckpointFileManager, - fileToRead: Path, - keySchema: StructType, - valueSchema: StructType, - newRocksDb: OptimisticTransactionDbInstance, - sparkConf: SparkConf): Unit = { + def updateFromDeltaFile( + fm: CheckpointFileManager, + fileToRead: Path, + keySchema: StructType, + valueSchema: StructType, + newRocksDb: OptimisticTransactionDbInstance, + sparkConf: SparkConf): Unit = { var input: DataInputStream = null val sourceStream = try { fm.open(fileToRead) } catch { case f: FileNotFoundException => throw new IllegalStateException( - s"Error reading delta file $fileToRead of $this: $fileToRead does not exist", f) + s"Error reading delta file $fileToRead of $this: $fileToRead does not exist", + f) } try { input = decompressStream(sourceStream, sparkConf) var eof = false - while(!eof) { + while (!eof) { val keySize = input.readInt() if (keySize == -1) { eof = true @@ -193,8 +188,8 @@ object WALUtils { * @param rawStream the underlying stream which needs to be cancelled. */ def cancelDeltaFile( - compressedStream: DataOutputStream, - rawStream: CancellableFSDataOutputStream): Unit = { + compressedStream: DataOutputStream, + rawStream: CancellableFSDataOutputStream): Unit = { try { if (rawStream != null) rawStream.cancel() IOUtils.closeQuietly(compressedStream) @@ -207,17 +202,17 @@ object WALUtils { } } - def uploadFile(fm: CheckpointFileManager, - sourceFile: Path, - targetFile: Path, - sparkConf: SparkConf): Unit = { + def uploadFile( + fm: CheckpointFileManager, + sourceFile: Path, + targetFile: Path, + sparkConf: SparkConf): Unit = { var output: CancellableFSDataOutputStream = null var in: BufferedInputStream = null try { in = new BufferedInputStream(new FileInputStream(sourceFile.toString)) output = fm.createAtomic(targetFile, overwriteIfPossible = true) - // output = compressStream(rawOutput, sparkConf) - val buffer = new Array[ Byte ](1024) + val buffer = new Array[Byte](1024) var len = in.read(buffer) while (len > 0) { output.write(buffer, 0, len) @@ -229,23 +224,24 @@ object WALUtils { if (output != null) output.cancel() throw e } finally { - if ( in != null ) { + if (in != null) { in.close() } } } - def downloadFile(fm: CheckpointFileManager, - sourceFile: Path, - targetFile: Path, - sparkConf: SparkConf): Boolean = { + def downloadFile( + fm: CheckpointFileManager, + sourceFile: Path, + targetFile: Path, + sparkConf: SparkConf): Boolean = { var in: FSDataInputStream = null var output: BufferedOutputStream = null try { in = fm.open(sourceFile) // in = decompressStream(fm.open(sourceFile), sparkConf) output = new BufferedOutputStream(new FileOutputStream(targetFile.toString)) - val buffer = new Array[ Byte ](1024) + val buffer = new Array[Byte](1024) var eof = false while (!eof) { val len = in.read(buffer) @@ -262,7 +258,7 @@ object WALUtils { throw e } finally { output.close() - if ( in != null ) { + if (in != null) { in.close() } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreSuite.scala index 46d2e9bbfe2a..83e97fc2a87f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreSuite.scala @@ -41,8 +41,10 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} import org.apache.spark.util.Utils -class RocksDbStateStoreSuite extends StateStoreSuiteBase[RocksDbStateStoreProvider] - with BeforeAndAfter with PrivateMethodTester { +class RocksDbStateStoreSuite + extends StateStoreSuiteBase[RocksDbStateStoreProvider] + with BeforeAndAfter + with PrivateMethodTester { type MapType = mutable.HashMap[UnsafeRow, UnsafeRow] type ProviderMapType = java.util.concurrent.ConcurrentHashMap[UnsafeRow, UnsafeRow] @@ -63,9 +65,9 @@ class RocksDbStateStoreSuite extends StateStoreSuiteBase[RocksDbStateStoreProvid } def updateVersionTo( - provider: StateStoreProvider, - currentVersion: Int, - targetVersion: Int): Int = { + provider: StateStoreProvider, + currentVersion: Int, + targetVersion: Int): Int = { var newCurrentVersion = currentVersion for (i <- newCurrentVersion until targetVersion) { newCurrentVersion = incrementVersion(provider, i) @@ -82,46 +84,51 @@ class RocksDbStateStoreSuite extends StateStoreSuiteBase[RocksDbStateStoreProvid } def checkLoadedVersions( - rocksDbWriteInstance: RocksDbInstance, - count: Int, - earliestKey: Long, - latestKey: Long): Unit = { + rocksDbWriteInstance: RocksDbInstance, + count: Int, + earliestKey: Long, + latestKey: Long): Unit = { assert(rocksDbWriteInstance.iterator(false).length === count) } def checkVersion( - rocksDbWriteInstance: RocksDbInstance, - version: Long, - expectedData: Map[String, Int]): Unit = { - - val originValueMap = rocksDbWriteInstance.iterator(false).map { row => - rowToString(row.key) -> rowToInt(row.value) - }.toMap[String, Int] + rocksDbWriteInstance: RocksDbInstance, + version: Long, + expectedData: Map[String, Int]): Unit = { + + val originValueMap = rocksDbWriteInstance + .iterator(false) + .map { row => + rowToString(row.key) -> rowToInt(row.value) + } + .toMap[String, Int] assert(originValueMap === expectedData) } test("snapshotting") { - val provider = newStoreProvider(opId = Random.nextInt, partition = 0, minDeltasForSnapshot = 5) + val provider = + newStoreProvider(opId = Random.nextInt, partition = 0, minDeltasForSnapshot = 5) var currentVersion = 0 currentVersion = updateVersionTo(provider, currentVersion, 2) require(getData(provider) === Set("a" -> 2)) - provider.doMaintenance() // should not generate snapshot files + provider.doMaintenance() // should not generate snapshot files assert(getData(provider) === Set("a" -> 2)) for (i <- 1 to currentVersion) { - assert(fileExists(provider, i, isSnapshot = false)) // all delta files present - assert(!fileExists(provider, i, isSnapshot = true)) // no snapshot files present + assert(fileExists(provider, i, isSnapshot = false)) // all delta files present + assert(!fileExists(provider, i, isSnapshot = true)) // no snapshot files present } // After version 6, snapshotting should generate one snapshot file currentVersion = updateVersionTo(provider, currentVersion, 6) require(getData(provider) === Set("a" -> 6), "store not updated correctly") - provider.doMaintenance() // should generate snapshot files + provider.doMaintenance() // should generate snapshot files - val snapshotVersion = (0 to 6).find(version => fileExists(provider, version, isSnapshot = true)) + val snapshotVersion = + (0 to 6).find(version => fileExists(provider, version, isSnapshot = true)) assert(snapshotVersion.nonEmpty, "snapshot file not generated") deleteFilesEarlierThanVersion(provider, snapshotVersion.get) assert( @@ -134,10 +141,10 @@ class RocksDbStateStoreSuite extends StateStoreSuiteBase[RocksDbStateStoreProvid // After version 20, snapshotting should generate newer snapshot files currentVersion = updateVersionTo(provider, currentVersion, 20) require(getData(provider) === Set("a" -> 20), "store not updated correctly") - provider.doMaintenance() // do snapshot + provider.doMaintenance() // do snapshot - val latestSnapshotVersion = (0 to 20).filter(version => - fileExists(provider, version, isSnapshot = true)).lastOption + val latestSnapshotVersion = + (0 to 20).filter(version => fileExists(provider, version, isSnapshot = true)).lastOption assert(latestSnapshotVersion.nonEmpty, "no snapshot file found") assert(latestSnapshotVersion.get > snapshotVersion.get, "newer snapshot not generated") @@ -146,7 +153,8 @@ class RocksDbStateStoreSuite extends StateStoreSuiteBase[RocksDbStateStoreProvid } test("cleaning") { - val provider = newStoreProvider(opId = Random.nextInt, partition = 0, minDeltasForSnapshot = 5) + val provider = + newStoreProvider(opId = Random.nextInt, partition = 0, minDeltasForSnapshot = 5) for (i <- 1 to 20) { val store = provider.getStore(i - 1) @@ -175,29 +183,33 @@ class RocksDbStateStoreSuite extends StateStoreSuiteBase[RocksDbStateStoreProvid provider.getStore(0).commit() // Verify we don't leak temp files - val tempFiles = FileUtils.listFiles(new File(provider.stateStoreId.checkpointRootLocation), - null, true).asScala.filter(_.getName.startsWith("temp-")) + val tempFiles = FileUtils + .listFiles(new File(provider.stateStoreId.checkpointRootLocation), null, true) + .asScala + .filter(_.getName.startsWith("temp-")) assert(tempFiles.isEmpty) } test("corrupted file handling") { - val provider = newStoreProvider(opId = Random.nextInt, partition = 0, minDeltasForSnapshot = 5) + val provider = + newStoreProvider(opId = Random.nextInt, partition = 0, minDeltasForSnapshot = 5) for (i <- 1 to 6) { val store = provider.getStore(i - 1) put(store, "a", i) store.commit() provider.doMaintenance() // do cleanup } - val snapshotVersion = (0 to 10).find( version => - fileExists(provider, version, isSnapshot = true)).getOrElse(fail("snapshot file not found")) + val snapshotVersion = (0 to 10) + .find(version => fileExists(provider, version, isSnapshot = true)) + .getOrElse(fail("snapshot file not found")) // Corrupt snapshot file and verify that it throws error - provider.close() + provider.close() assert(getData(provider, snapshotVersion) === Set("a" -> snapshotVersion)) RocksDbInstance.destroyDB(provider.rocksDbPath) corruptFile(provider, snapshotVersion, isSnapshot = true) - intercept[Exception] { + intercept[Exception] { provider.close() RocksDbInstance.destroyDB(provider.rocksDbPath) getData(provider, snapshotVersion) @@ -229,41 +241,39 @@ class RocksDbStateStoreSuite extends StateStoreSuiteBase[RocksDbStateStoreProvid val dir = newDir() val storeId = StateStoreProviderId(StateStoreId(dir, 0, 0), UUID.randomUUID) val sqlConf = new SQLConf - sqlConf.setConfString("spark.sql.streaming.stateStore.providerClass", + sqlConf.setConfString( + "spark.sql.streaming.stateStore.providerClass", "org.apache.spark.sql.execution.streaming.state.RocksDbStateStoreProvider") val storeConf = new StateStoreConf(sqlConf) - assert(storeConf.providerClass === - "org.apache.spark.sql.execution.streaming.state.RocksDbStateStoreProvider") + assert( + storeConf.providerClass === + "org.apache.spark.sql.execution.streaming.state.RocksDbStateStoreProvider") val hadoopConf = new Configuration() // Verify that trying to get incorrect versions throw errors intercept[IllegalArgumentException] { - StateStore.get( - storeId, keySchema, valueSchema, None, -1, storeConf, hadoopConf) + StateStore.get(storeId, keySchema, valueSchema, None, -1, storeConf, hadoopConf) } assert(!StateStore.isLoaded(storeId)) // version -1 should not attempt to load the store intercept[IllegalStateException] { - StateStore.get( - storeId, keySchema, valueSchema, None, 1, storeConf, hadoopConf) + StateStore.get(storeId, keySchema, valueSchema, None, 1, storeConf, hadoopConf) } // Increase version of the store and try to get again - val store0 = StateStore.get( - storeId, keySchema, valueSchema, None, 0, storeConf, hadoopConf) + val store0 = StateStore.get(storeId, keySchema, valueSchema, None, 0, storeConf, hadoopConf) assert(store0.version === 0) put(store0, "a", 1) store0.commit() - val store1 = StateStore.get( - storeId, keySchema, valueSchema, None, 1, storeConf, hadoopConf) + val store1 = StateStore.get(storeId, keySchema, valueSchema, None, 1, storeConf, hadoopConf) assert(StateStore.isLoaded(storeId)) assert(store1.version === 1) assert(rowsToSet(store1.iterator()) === Set("a" -> 1)) // Verify that you can also load older version - val store0reloaded = StateStore.get( - storeId, keySchema, valueSchema, None, 0, storeConf, hadoopConf) + val store0reloaded = + StateStore.get(storeId, keySchema, valueSchema, None, 0, storeConf, hadoopConf) assert(store0reloaded.version === 0) assert(rowsToSet(store0reloaded.iterator()) === Set.empty) @@ -271,8 +281,8 @@ class RocksDbStateStoreSuite extends StateStoreSuiteBase[RocksDbStateStoreProvid StateStore.unload(storeId) assert(!StateStore.isLoaded(storeId)) - val store1reloaded = StateStore.get( - storeId, keySchema, valueSchema, None, 1, storeConf, hadoopConf) + val store1reloaded = + StateStore.get(storeId, keySchema, valueSchema, None, 1, storeConf, hadoopConf) assert(StateStore.isLoaded(storeId)) assert(store1reloaded.version === 1) put(store1reloaded, "a", 2) @@ -294,7 +304,8 @@ class RocksDbStateStoreSuite extends StateStoreSuiteBase[RocksDbStateStoreProvid val dir = newDir() val storeProviderId = StateStoreProviderId(StateStoreId(dir, opId, 0), UUID.randomUUID) val sqlConf = new SQLConf() - sqlConf.setConfString("spark.sql.streaming.stateStore.providerClass", + sqlConf.setConfString( + "spark.sql.streaming.stateStore.providerClass", "org.apache.spark.sql.execution.streaming.state.RocksDbStateStoreProvider") sqlConf.setConf(SQLConf.MIN_BATCHES_TO_RETAIN, 2) val storeConf = StateStoreConf(sqlConf) @@ -305,8 +316,14 @@ class RocksDbStateStoreSuite extends StateStoreSuiteBase[RocksDbStateStoreProvid def generateStoreVersions() { for (i <- 1 to 20) { - val store = StateStore.get(storeProviderId, keySchema, valueSchema, None, - latestStoreVersion, storeConf, hadoopConf) + val store = StateStore.get( + storeProviderId, + keySchema, + valueSchema, + None, + latestStoreVersion, + storeConf, + hadoopConf) put(store, "a", i) store.commit() latestStoreVersion += 1 @@ -325,7 +342,8 @@ class RocksDbStateStoreSuite extends StateStoreSuiteBase[RocksDbStateStoreProvid eventually(timeout(timeoutDuration)) { // Store should have been reported to the coordinator - assert(coordinatorRef.getLocation(storeProviderId).nonEmpty, + assert( + coordinatorRef.getLocation(storeProviderId).nonEmpty, "active instance was not reported") // Background maintenance should clean up and generate snapshots @@ -355,8 +373,14 @@ class RocksDbStateStoreSuite extends StateStoreSuiteBase[RocksDbStateStoreProvid } // Reload the store and verify - StateStore.get(storeProviderId, keySchema, valueSchema, indexOrdinal = None, - latestStoreVersion, storeConf, hadoopConf) + StateStore.get( + storeProviderId, + keySchema, + valueSchema, + indexOrdinal = None, + latestStoreVersion, + storeConf, + hadoopConf) assert(StateStore.isLoaded(storeProviderId)) // If some other executor loads the store, then this instance should be unloaded @@ -366,8 +390,14 @@ class RocksDbStateStoreSuite extends StateStoreSuiteBase[RocksDbStateStoreProvid } // Reload the store and verify - StateStore.get(storeProviderId, keySchema, valueSchema, indexOrdinal = None, - latestStoreVersion, storeConf, hadoopConf) + StateStore.get( + storeProviderId, + keySchema, + valueSchema, + indexOrdinal = None, + latestStoreVersion, + storeConf, + hadoopConf) assert(StateStore.isLoaded(storeProviderId)) } } @@ -388,7 +418,8 @@ class RocksDbStateStoreSuite extends StateStoreSuiteBase[RocksDbStateStoreProvid SparkSession.setActiveSession(spark) implicit val sqlContext = spark.sqlContext spark.conf.set("spark.sql.shuffle.partitions", "1") - spark.conf.set("spark.sql.streaming.stateStore.providerClass", + spark.conf.set( + "spark.sql.streaming.stateStore.providerClass", "org.apache.spark.sql.execution.streaming.state.RocksDbStateStoreProvider") import spark.implicits._ val inputData = MemoryStream[Int] @@ -406,7 +437,8 @@ class RocksDbStateStoreSuite extends StateStoreSuiteBase[RocksDbStateStoreProvid query.processAllAvailable() require(query.lastProgress != null) // at least one batch processed after start val loadedProvidersMethod = - PrivateMethod[mutable.HashMap[StateStoreProviderId, StateStoreProvider]]('loadedProviders) + PrivateMethod[mutable.HashMap[StateStoreProviderId, StateStoreProvider]]( + 'loadedProviders) val loadedProvidersMap = StateStore invokePrivate loadedProvidersMethod() val loadedProviders = loadedProvidersMap.synchronized { loadedProvidersMap.values.toSeq } query.stop() @@ -417,10 +449,11 @@ class RocksDbStateStoreSuite extends StateStoreSuiteBase[RocksDbStateStoreProvid require(loadedProvidersAfterRun1.length === 1) val loadedProvidersAfterRun2 = runQueryAndGetLoadedProviders() - assert(loadedProvidersAfterRun2.length === 2) // two providers loaded for 2 runs + assert(loadedProvidersAfterRun2.length === 2) // two providers loaded for 2 runs // Both providers should have the same StateStoreId, but the should be different objects - assert(loadedProvidersAfterRun2(0).stateStoreId === loadedProvidersAfterRun2(1).stateStoreId) + assert( + loadedProvidersAfterRun2(0).stateStoreId === loadedProvidersAfterRun2(1).stateStoreId) assert(loadedProvidersAfterRun2(0) ne loadedProvidersAfterRun2(1)) } finally { @@ -436,7 +469,10 @@ class RocksDbStateStoreSuite extends StateStoreSuiteBase[RocksDbStateStoreProvid } override def newStoreProvider(storeId: StateStoreId): RocksDbStateStoreProvider = { - newStoreProvider(storeId.operatorId, storeId.partitionId, dir = storeId.checkpointRootLocation) + newStoreProvider( + storeId.operatorId, + storeId.partitionId, + dir = storeId.checkpointRootLocation) } override def getLatestData(storeProvider: RocksDbStateStoreProvider): Set[(String, Int)] = { @@ -444,8 +480,8 @@ class RocksDbStateStoreSuite extends StateStoreSuiteBase[RocksDbStateStoreProvid } override def getData( - provider: RocksDbStateStoreProvider, - version: Int = -1): Set[(String, Int)] = { + provider: RocksDbStateStoreProvider, + version: Int = -1): Set[(String, Int)] = { val reloadedProvider = newStoreProvider(provider.stateStoreId) if (version < 0) { // TODO find out last version from rocksDB @@ -456,32 +492,31 @@ class RocksDbStateStoreSuite extends StateStoreSuiteBase[RocksDbStateStoreProvid } def newStoreProvider( - opId: Long, - partition: Int, - 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): RocksDbStateStoreProvider = { - val sqlConf = new SQLConf() - sqlConf.setConf(SQLConf.STATE_STORE_MIN_DELTAS_FOR_SNAPSHOT, minDeltasForSnapshot) - sqlConf.setConf(SQLConf.MAX_BATCHES_TO_RETAIN_IN_MEMORY, numOfVersToRetainInMemory) - sqlConf.setConf(SQLConf.MIN_BATCHES_TO_RETAIN, 2) - val provider = new RocksDbStateStoreProvider - provider.init( - StateStoreId(dir, opId, partition), - keySchema, - valueSchema, - keyIndexOrdinal = None, - new StateStoreConf(sqlConf), - hadoopConf) - provider - } - + opId: Long, + partition: Int, + 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): RocksDbStateStoreProvider = { + val sqlConf = new SQLConf() + sqlConf.setConf(SQLConf.STATE_STORE_MIN_DELTAS_FOR_SNAPSHOT, minDeltasForSnapshot) + sqlConf.setConf(SQLConf.MAX_BATCHES_TO_RETAIN_IN_MEMORY, numOfVersToRetainInMemory) + sqlConf.setConf(SQLConf.MIN_BATCHES_TO_RETAIN, 2) + val provider = new RocksDbStateStoreProvider + provider.init( + StateStoreId(dir, opId, partition), + keySchema, + valueSchema, + keyIndexOrdinal = None, + new StateStoreConf(sqlConf), + hadoopConf) + provider + } def fileExists( - provider: RocksDbStateStoreProvider, - version: Long, - isSnapshot: Boolean): Boolean = { + provider: RocksDbStateStoreProvider, + version: Long, + isSnapshot: Boolean): Boolean = { val method = PrivateMethod[Path]('baseDir) val basePath = provider invokePrivate method() val fileName = if (isSnapshot) s"$version.snapshot" else s"$version.delta" @@ -502,9 +537,9 @@ class RocksDbStateStoreSuite extends StateStoreSuiteBase[RocksDbStateStoreProvid } def corruptFile( - provider: RocksDbStateStoreProvider, - version: Long, - isSnapshot: Boolean): Unit = { + provider: RocksDbStateStoreProvider, + version: Long, + isSnapshot: Boolean): Unit = { val method = PrivateMethod[Path]('baseDir) val basePath = provider invokePrivate method() val fileName = if (isSnapshot) s"$version.snapshot" else s"$version.delta" From 3f5f6b23d8c3455cf92df02f7f9bf3d850583c3b Mon Sep 17 00:00:00 2001 From: Vikram Agrawal Date: Tue, 25 Jun 2019 10:18:55 +0530 Subject: [PATCH 04/26] remove unwanted lines --- .../apache/spark/sql/execution/streaming/state/WALUtils.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/WALUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/WALUtils.scala index e9c52c9e8e47..b4813f7eb58b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/WALUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/WALUtils.scala @@ -239,7 +239,6 @@ object WALUtils { var output: BufferedOutputStream = null try { in = fm.open(sourceFile) - // in = decompressStream(fm.open(sourceFile), sparkConf) output = new BufferedOutputStream(new FileOutputStream(targetFile.toString)) val buffer = new Array[Byte](1024) var eof = false From f0f2f8d3ce8b0a4aa62134090ba90bcf39d97dc8 Mon Sep 17 00:00:00 2001 From: Vikram Agrawal Date: Tue, 25 Jun 2019 10:38:56 +0530 Subject: [PATCH 05/26] Fix review comments --- core/src/main/java/org/apache/spark/io/FileUtility.java | 3 ++- .../apache/spark/sql/execution/streaming/state/WALUtils.scala | 1 - 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/apache/spark/io/FileUtility.java b/core/src/main/java/org/apache/spark/io/FileUtility.java index e91882222317..4fc6765e20e3 100644 --- a/core/src/main/java/org/apache/spark/io/FileUtility.java +++ b/core/src/main/java/org/apache/spark/io/FileUtility.java @@ -92,7 +92,8 @@ public static void createTarFile(String source, String destFileName) throws Exce tarOs.finish(); tarOs.close(); } catch (IOException e) { - e.printStackTrace(); + throw new IllegalStateException(String.format( + "createTarFile failed with exception %s.", e.getMessage())); } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/WALUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/WALUtils.scala index b4813f7eb58b..771ade3e2a40 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/WALUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/WALUtils.scala @@ -89,7 +89,6 @@ object WALUtils { } } val storeFiles = versionToFiles.values.toSeq.sortBy(_.version) - // logDebug(s"Current set of files for $this: ${storeFiles.mkString(", ")}") storeFiles } From 827ace4d87d30ae293f1f43c814f1ab9d4a96977 Mon Sep 17 00:00:00 2001 From: Vikram Agrawal Date: Tue, 25 Jun 2019 11:14:18 +0530 Subject: [PATCH 06/26] update hadoop manifest file --- dev/deps/spark-deps-hadoop-2.7 | 1 + dev/deps/spark-deps-hadoop-3.2 | 1 + 2 files changed, 2 insertions(+) diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 2650ae59a34a..75d118703c30 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -171,6 +171,7 @@ parquet-jackson-1.10.1.jar protobuf-java-2.5.0.jar py4j-0.10.8.1.jar pyrolite-4.23.jar +rocksdbjni-6.0.1.jar scala-compiler-2.12.8.jar scala-library-2.12.8.jar scala-parser-combinators_2.12-1.1.0.jar diff --git a/dev/deps/spark-deps-hadoop-3.2 b/dev/deps/spark-deps-hadoop-3.2 index 356fdb9fabd9..a1d34013ae37 100644 --- a/dev/deps/spark-deps-hadoop-3.2 +++ b/dev/deps/spark-deps-hadoop-3.2 @@ -191,6 +191,7 @@ protobuf-java-2.5.0.jar py4j-0.10.8.1.jar pyrolite-4.23.jar re2j-1.1.jar +rocksdbjni-6.0.1.jar scala-compiler-2.12.8.jar scala-library-2.12.8.jar scala-parser-combinators_2.12-1.1.0.jar From db1ed2b57d80a18ed8f7228b4af6cc847033febd Mon Sep 17 00:00:00 2001 From: Vikram Agrawal Date: Wed, 26 Jun 2019 14:37:03 +0530 Subject: [PATCH 07/26] make local directory a configurable option --- .../state/RocksDbStateStoreProvider.scala | 22 ++-- .../state/RocksDbStateStoreSuite.scala | 66 ++++++++++- .../streaming/state/StateStoreSuite.scala | 104 +++++++++--------- 3 files changed, 131 insertions(+), 61 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreProvider.scala index 869543fa8f2e..d556cbeae0e3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreProvider.scala @@ -44,6 +44,7 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log @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 @@ -194,8 +195,8 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log rocksDbWriteInstance.iterator(closeDbOnCompletion = false) case LOADED | ABORTED => - // use checkpointed db for previous version - logDebug(s"state = loaded/aborted using checkpointed DB with version $version") + // use check-pointed db for previous version + logDebug(s"state = loaded/aborted using check-pointed DB with version $version") if (version == 0) { Iterator.empty } else { @@ -205,8 +206,8 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log r.iterator(closeDbOnCompletion = true) } case COMMITTED => - logDebug(s"state = committed using checkpointed DB with version $newVersion") - // use checkpointed db for current updated version + logDebug(s"state = committed using check-pointed DB with version $newVersion") + // use check-pointed db for current updated version val path = getBackupPath(newVersion) val r: RocksDbInstance = new RocksDbInstance(keySchema, valueSchema, newVersion.toString) @@ -277,6 +278,10 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log .map { case (k, v) => (k.toLowerCase(Locale.ROOT), v) } + this.localDirectory = this.rocksDbConf + .getOrElse( + "spark.sql.streaming.stateStore.rocksDb.localDirectory".toLowerCase(Locale.ROOT), + RocksDbStateStoreProvider.ROCKS_DB_BASE_PATH) } /* @@ -359,7 +364,6 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log found = true lastAvailableVersion = version - 1 } else { - // TODO check for numberOfVersionsToRetain // Destroy DB so that we can recontruct it using snapshot and delta files RocksDbInstance.destroyDB(rocksDbPath) } @@ -573,7 +577,7 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log lazy val rocksDbPath: String = { val checkpointRootLocationPath = new Path(stateStoreId.checkpointRootLocation) val basePath = new Path( - RocksDbStateStoreProvider.ROCKS_DB_BASE_PATH, + localDirectory, new Path( "db", checkpointRootLocationPath.getName + "_" + checkpointRootLocationPath.hashCode())) @@ -595,7 +599,7 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log val checkpointRootLocationPath = new Path(stateStoreId.checkpointRootLocation) val basePath = new Path( - RocksDbStateStoreProvider.ROCKS_DB_BASE_PATH, + localDirectory, new Path( "backup", checkpointRootLocationPath.getName + "_" + checkpointRootLocationPath.hashCode())) @@ -618,7 +622,7 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log val checkpointRootLocationPath = new Path(stateStoreId.checkpointRootLocation) val basePath = new Path( - RocksDbStateStoreProvider.ROCKS_DB_BASE_PATH, + localDirectory, new Path( "tmp", checkpointRootLocationPath.getName + "_" + checkpointRootLocationPath.hashCode())) @@ -637,6 +641,8 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log dir + Path.SEPARATOR + version } + // making it public for unit tests + def getLocalDirectory: String = localDirectory } object RocksDbStateStoreProvider { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreSuite.scala index 83e97fc2a87f..a430ebf63674 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreSuite.scala @@ -106,6 +106,52 @@ class RocksDbStateStoreSuite assert(originValueMap === expectedData) } + test("get, put, remove, commit, and all data iterator") { + val provider = newStoreProvider() + + // 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") === None) + assert(store.iterator().isEmpty) + + // Verify state after updating + put(store, "a", 1) + assert(get(store, "a") === Some(1)) + + assert(store.iterator().nonEmpty) + assert(getLatestData(provider).isEmpty) + + // Make updates, commit and then verify state + put(store, "b", 2) + put(store, "aa", 3) + remove(store, _.startsWith("a")) + assert(store.commit() === 1) + + assert(store.hasCommitted) + assert(rowsToSet(store.iterator()) === Set("b" -> 2)) + assert(getLatestData(provider) === Set("b" -> 2)) + + // Trying to get newer versions should fail + intercept[Exception] { + provider.getStore(2) + } + intercept[Exception] { + getData(provider, 2) + } + + // New updates to the reloaded store with new version, and does not change old version + val reloadedProvider = newStoreProvider(store.id, provider.getLocalDirectory) + val reloadedStore = reloadedProvider.getStore(1) + put(reloadedStore, "c", 4) + assert(reloadedStore.commit() === 2) + assert(rowsToSet(reloadedStore.iterator()) === Set("b" -> 2, "c" -> 4)) + assert(getLatestData(provider) === Set("b" -> 2, "c" -> 4)) + assert(getData(provider, version = 1) === Set("b" -> 2)) + } + test("snapshotting") { val provider = newStoreProvider(opId = Random.nextInt, partition = 0, minDeltasForSnapshot = 5) @@ -244,6 +290,8 @@ class RocksDbStateStoreSuite sqlConf.setConfString( "spark.sql.streaming.stateStore.providerClass", "org.apache.spark.sql.execution.streaming.state.RocksDbStateStoreProvider") + val localdir = Utils.createTempDir().getAbsoluteFile.toString + sqlConf.setConfString("spark.sql.streaming.stateStore.rocksDb.localDirectory", localdir) val storeConf = new StateStoreConf(sqlConf) assert( storeConf.providerClass === @@ -308,6 +356,9 @@ class RocksDbStateStoreSuite "spark.sql.streaming.stateStore.providerClass", "org.apache.spark.sql.execution.streaming.state.RocksDbStateStoreProvider") sqlConf.setConf(SQLConf.MIN_BATCHES_TO_RETAIN, 2) + sqlConf.setConfString( + "spark.sql.streaming.stateStore.rocksDb.localDirectory", + Utils.createTempDir().getAbsoluteFile.toString) val storeConf = StateStoreConf(sqlConf) val hadoopConf = new Configuration() val provider = newStoreProvider(storeProviderId.storeId) @@ -421,6 +472,9 @@ class RocksDbStateStoreSuite spark.conf.set( "spark.sql.streaming.stateStore.providerClass", "org.apache.spark.sql.execution.streaming.state.RocksDbStateStoreProvider") + spark.conf.set( + "spark.sql.streaming.stateStore.rocksDb.localDirectory", + Utils.createTempDir().getAbsoluteFile.toString) import spark.implicits._ val inputData = MemoryStream[Int] @@ -475,6 +529,14 @@ class RocksDbStateStoreSuite dir = storeId.checkpointRootLocation) } + def newStoreProvider(storeId: StateStoreId, localDir: String): RocksDbStateStoreProvider = { + newStoreProvider( + storeId.operatorId, + storeId.partitionId, + dir = storeId.checkpointRootLocation, + localDir = localDir) + } + override def getLatestData(storeProvider: RocksDbStateStoreProvider): Set[(String, Int)] = { getData(storeProvider) } @@ -482,7 +544,7 @@ class RocksDbStateStoreSuite override def getData( provider: RocksDbStateStoreProvider, version: Int = -1): Set[(String, Int)] = { - val reloadedProvider = newStoreProvider(provider.stateStoreId) + val reloadedProvider = newStoreProvider(provider.stateStoreId, provider.getLocalDirectory) if (version < 0) { // TODO find out last version from rocksDB reloadedProvider.latestIterator().map(rowsToStringInt).toSet @@ -495,6 +557,7 @@ class RocksDbStateStoreSuite opId: Long, partition: Int, dir: String = newDir(), + localDir: 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): RocksDbStateStoreProvider = { @@ -502,6 +565,7 @@ class RocksDbStateStoreSuite sqlConf.setConf(SQLConf.STATE_STORE_MIN_DELTAS_FOR_SNAPSHOT, minDeltasForSnapshot) sqlConf.setConf(SQLConf.MAX_BATCHES_TO_RETAIN_IN_MEMORY, numOfVersToRetainInMemory) sqlConf.setConf(SQLConf.MIN_BATCHES_TO_RETAIN, 2) + sqlConf.setConfString("spark.sql.streaming.stateStore.rocksDb.localDirectory", localDir) val provider = new RocksDbStateStoreProvider provider.init( StateStoreId(dir, opId, partition), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala index af4369de800e..d52338791489 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala @@ -198,6 +198,58 @@ class StateStoreSuite extends StateStoreSuiteBase[HDFSBackedStateStoreProvider] assert(loadedMaps.size() === 0) } + test("get, put, remove, commit, and all data iterator") { + val provider = newStoreProvider() + + // 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") === None) + assert(store.iterator().isEmpty) + assert(store.metrics.numKeys === 0) + + // Verify state after updating + put(store, "a", 1) + assert(get(store, "a") === Some(1)) + assert(store.metrics.numKeys === 1) + + assert(store.iterator().nonEmpty) + assert(getLatestData(provider).isEmpty) + + // Make updates, commit and then verify state + put(store, "b", 2) + put(store, "aa", 3) + assert(store.metrics.numKeys === 3) + remove(store, _.startsWith("a")) + assert(store.metrics.numKeys === 1) + assert(store.commit() === 1) + + assert(store.hasCommitted) + assert(rowsToSet(store.iterator()) === Set("b" -> 2)) + assert(getLatestData(provider) === Set("b" -> 2)) + + // Trying to get newer versions should fail + intercept[Exception] { + provider.getStore(2) + } + intercept[Exception] { + getData(provider, 2) + } + + // New updates to the reloaded store with new version, and does not change old version + val reloadedProvider = newStoreProvider(store.id) + val reloadedStore = reloadedProvider.getStore(1) + assert(reloadedStore.metrics.numKeys === 1) + put(reloadedStore, "c", 4) + assert(reloadedStore.metrics.numKeys === 2) + assert(reloadedStore.commit() === 2) + assert(rowsToSet(reloadedStore.iterator()) === Set("b" -> 2, "c" -> 4)) + assert(getLatestData(provider) === Set("b" -> 2, "c" -> 4)) + assert(getData(provider, version = 1) === Set("b" -> 2)) + } + test("snapshotting") { val provider = newStoreProvider(opId = Random.nextInt, partition = 0, minDeltasForSnapshot = 5) @@ -817,58 +869,6 @@ abstract class StateStoreSuiteBase[ProviderClass <: StateStoreProvider] extends SparkFunSuite { import StateStoreTestsHelper._ - test("get, put, remove, commit, and all data iterator") { - val provider = newStoreProvider() - - // 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") === None) - assert(store.iterator().isEmpty) - assert(store.metrics.numKeys === 0) - - // Verify state after updating - put(store, "a", 1) - assert(get(store, "a") === Some(1)) - assert(store.metrics.numKeys === 1) - - assert(store.iterator().nonEmpty) - assert(getLatestData(provider).isEmpty) - - // Make updates, commit and then verify state - put(store, "b", 2) - put(store, "aa", 3) - assert(store.metrics.numKeys === 3) - remove(store, _.startsWith("a")) - assert(store.metrics.numKeys === 1) - assert(store.commit() === 1) - - assert(store.hasCommitted) - assert(rowsToSet(store.iterator()) === Set("b" -> 2)) - assert(getLatestData(provider) === Set("b" -> 2)) - - // Trying to get newer versions should fail - intercept[Exception] { - provider.getStore(2) - } - intercept[Exception] { - getData(provider, 2) - } - - // New updates to the reloaded store with new version, and does not change old version - val reloadedProvider = newStoreProvider(store.id) - val reloadedStore = reloadedProvider.getStore(1) - assert(reloadedStore.metrics.numKeys === 1) - put(reloadedStore, "c", 4) - assert(reloadedStore.metrics.numKeys === 2) - assert(reloadedStore.commit() === 2) - assert(rowsToSet(reloadedStore.iterator()) === Set("b" -> 2, "c" -> 4)) - assert(getLatestData(provider) === Set("b" -> 2, "c" -> 4)) - assert(getData(provider, version = 1) === Set("b" -> 2)) - } - test("removing while iterating") { val provider = newStoreProvider() From 3ad88eb4154a347cf9cac1ce3943df1cb85191fc Mon Sep 17 00:00:00 2001 From: Vikram Agrawal Date: Mon, 8 Jul 2019 15:06:21 +0530 Subject: [PATCH 08/26] Review Comments Fixes --- .../java/org/apache/spark/io/FileUtility.java | 161 ++++++++++-------- .../streaming/state/RocksDbInstance.scala | 67 ++++---- .../state/RocksDbStateStoreProvider.scala | 83 +++++---- 3 files changed, 162 insertions(+), 149 deletions(-) diff --git a/core/src/main/java/org/apache/spark/io/FileUtility.java b/core/src/main/java/org/apache/spark/io/FileUtility.java index 4fc6765e20e3..3efbdc4e8a13 100644 --- a/core/src/main/java/org/apache/spark/io/FileUtility.java +++ b/core/src/main/java/org/apache/spark/io/FileUtility.java @@ -13,8 +13,6 @@ */ package org.apache.spark.io; -import org.apache.commons.compress.archivers.ArchiveException; -import org.apache.commons.compress.archivers.ArchiveStreamFactory; import org.apache.commons.compress.archivers.tar.TarArchiveEntry; import org.apache.commons.compress.archivers.tar.TarArchiveInputStream; import org.apache.commons.compress.archivers.tar.TarArchiveOutputStream; @@ -24,79 +22,104 @@ public class FileUtility { - /** - * Untar an input file into an output file. - * - * The output file is created in the output folder, having the same name as - * the input file, minus the '.tar' extension. - * - * @param inputFile the input .tar file - * @throws IOException - * - * @throws ArchiveException - */ - public static void unTar(final File inputFile) - throws IOException, ArchiveException { + /** + * Extract an input tar file into an output files and directories. + * @param inputTarFileLoc the input file location for the tar file + * @param destDirLoc destination for the extracted files + * + * @throws IOException + * @throws IllegalStateException + */ + public static final String ENCODING = "utf-8"; - String outputDir = inputFile.getAbsolutePath().split(".tar")[0]; - File outputTarDir = new File(outputDir); - outputTarDir.mkdir(); - final InputStream is = new FileInputStream(inputFile); - final TarArchiveInputStream debInputStream = (TarArchiveInputStream) new ArchiveStreamFactory().createArchiveInputStream( - "tar", is); - TarArchiveEntry entry = null; - while ((entry = (TarArchiveEntry) debInputStream.getNextEntry()) != null) { - final File outputFile = new File(outputDir, entry.getName()); - if (entry.isDirectory()) { - if (!outputFile.exists()) { - if (!outputFile.mkdirs()) { - throw new IllegalStateException(String.format( - "Couldn't create directory %s.", outputFile.getAbsolutePath())); - } - } - } else { - final OutputStream outputFileStream = new FileOutputStream(outputFile); - IOUtils.copy(debInputStream, outputFileStream); - outputFileStream.close(); - } - } - debInputStream.close(); + public static void extractTarFile(String inputTarFileLoc, String destDirLoc) + throws IOException, IllegalStateException { + File inputFile = new File(inputTarFileLoc); + if (!inputTarFileLoc.endsWith("tar")) { + throw new IllegalStateException(String.format( + "Input File %s should end with tar extension.", inputTarFileLoc)); + } + File destDir = new File(destDirLoc); + if (destDir.exists() && !destDir.delete()) { + throw new IllegalStateException(String.format( + "Couldn't delete the existing destination directory %s ", destDirLoc)); + } else if (!destDir.mkdir()) { + throw new IllegalStateException(String.format( + "Couldn't create directory %s ", destDirLoc)); } - public static void createTarFile(String source, String destFileName) throws Exception { - TarArchiveOutputStream tarOs = null; - File f = new File(destFileName); - if (f.exists()) { - f.delete(); - } - try { - FileOutputStream fos = new FileOutputStream(destFileName); - tarOs = (TarArchiveOutputStream) new ArchiveStreamFactory().createArchiveOutputStream("tar", fos); - tarOs = new TarArchiveOutputStream(fos); - File folder = new File(source); - File[] fileNames = folder.listFiles(); - for(File file : fileNames){ - TarArchiveEntry tar_file = new TarArchiveEntry(file.getName()); - tar_file.setSize(file.length()); - tarOs.putArchiveEntry(tar_file); - BufferedInputStream bis = new BufferedInputStream(new FileInputStream(file)); - IOUtils.copy(bis, tarOs); - bis.close(); - tarOs.closeArchiveEntry(); - } - } catch (IOException e) { + final InputStream is = new FileInputStream(inputFile); + final TarArchiveInputStream debInputStream = new TarArchiveInputStream(is, ENCODING); + OutputStream outputFileStream = null; + try { + TarArchiveEntry entry; + while ((entry = (TarArchiveEntry) debInputStream.getNextEntry()) != null) { + final File outputFile = new File(destDirLoc, entry.getName()); + if (entry.isDirectory()) { + if (!outputFile.exists() && !outputFile.mkdirs()) { throw new IllegalStateException(String.format( - "createTarFile failed with exception %s.", e.getMessage())); - } finally { - try { - tarOs.finish(); - tarOs.close(); - } catch (IOException e) { - throw new IllegalStateException(String.format( - "createTarFile failed with exception %s.", e.getMessage())); - } + "Couldn't create directory %s.", outputFile.getAbsolutePath())); + } + } else { + outputFileStream = new FileOutputStream(outputFile); + IOUtils.copy(debInputStream, outputFileStream); + outputFileStream.close(); + outputFileStream = null; } + } + } catch (IOException e){ + throw new IllegalStateException(String.format( + "extractTarFile failed with exception %s.", e.getMessage())); + } finally { + debInputStream.close(); + if (outputFileStream != null) { + outputFileStream.close(); + } } + } + /** + * create a tar file for input source directory location . + * @param source the source directory location + * @param destFileLoc destination of the created tarball + * + * @throws IOException + * @throws IllegalStateException + */ + public static void createTarFile(String source, String destFileLoc) + throws IllegalStateException, IOException { + TarArchiveOutputStream tarOs = null; + File f = new File(destFileLoc); + if (f.exists() && !f.delete()) { + throw new IllegalStateException(String.format( + "Couldn't delete the destination file location %s", destFileLoc)); + } + BufferedInputStream bis = null; + try { + FileOutputStream fos = new FileOutputStream(destFileLoc); + tarOs = new TarArchiveOutputStream(fos, ENCODING); + File folder = new File(source); + File[] fileNames = folder.listFiles(); + for(File file : fileNames){ + TarArchiveEntry tar_file = new TarArchiveEntry(file.getName()); + tar_file.setSize(file.length()); + tarOs.putArchiveEntry(tar_file); + bis = new BufferedInputStream(new FileInputStream(file)); + IOUtils.copy(bis, tarOs); + bis.close(); + bis = null; + tarOs.closeArchiveEntry(); + } + } catch (IOException e) { + throw new IllegalStateException(String.format( + "createTarFile failed with exception %s.", e.getMessage())); + } finally { + tarOs.finish(); + tarOs.close(); + if (bis != null) { + bis.close(); + } + } + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbInstance.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbInstance.scala index a8dcd671900e..f495b99d9b35 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbInstance.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbInstance.scala @@ -48,16 +48,15 @@ class RocksDbInstance(keySchema: StructType, valueSchema: StructType, identifier } def open(path: String, conf: Map[String, String], readOnly: Boolean): Unit = { - verify(db == null, "Another rocksDb instance is already actve") + require(db == null, "Another rocksDb instance is already active") try { setOptions(conf) - db = readOnly match { - case true => - options.setCreateIfMissing(false) - RocksDB.openReadOnly(options, path) - case false => - options.setCreateIfMissing(true) - RocksDB.open(options, path) + db = if (readOnly) { + options.setCreateIfMissing(false) + RocksDB.openReadOnly(options, path) + } else { + options.setCreateIfMissing(true) + RocksDB.open(options, path) } dbPath = path } catch { @@ -69,7 +68,7 @@ class RocksDbInstance(keySchema: StructType, valueSchema: StructType, identifier } def get(key: UnsafeRow): UnsafeRow = { - verify(isOpen(), "Open rocksDb instance before any operation") + require(isOpen(), "Open rocksDb instance before any operation") Option(db.get(readOptions, key.getBytes)) match { case Some(valueInBytes) => val value = new UnsafeRow(valueSchema.fields.length) @@ -80,12 +79,12 @@ class RocksDbInstance(keySchema: StructType, valueSchema: StructType, identifier } def put(key: UnsafeRow, value: UnsafeRow): Unit = { - verify(isOpen(), "Open rocksDb instance before any operation") + require(isOpen(), "Open rocksDb instance before any operation") db.put(key.getBytes, value.getBytes) } def remove(key: UnsafeRow): Unit = { - verify(isOpen(), "Open rocksDb instance before any operation") + require(isOpen(), "Open rocksDb instance before any operation") db.delete(key.getBytes) } @@ -106,7 +105,7 @@ class RocksDbInstance(keySchema: StructType, valueSchema: StructType, identifier } def iterator(closeDbOnCompletion: Boolean): Iterator[UnsafeRowPair] = { - verify(isOpen(), "Open rocksDb instance before any operation") + require(isOpen(), "Open rocksDb instance before any operation") Option(db.getSnapshot) match { case Some(snapshot) => logDebug(s"Inside rockdDB iterator function") @@ -153,7 +152,7 @@ class RocksDbInstance(keySchema: StructType, valueSchema: StructType, identifier } def printStats: Unit = { - verify(isOpen(), "Open rocksDb instance before any operation") + require(isOpen(), "Open rocksDb instance before any operation") try { val stats = db.getProperty("rocksdb.stats") logInfo(s"Stats = $stats") @@ -235,7 +234,7 @@ class RocksDbInstance(keySchema: StructType, valueSchema: StructType, identifier } def createCheckpoint(rocksDb: RocksDB, dir: String): Unit = { - verify(isOpen(), "Open rocksDb instance before any operation") + require(isOpen(), "Open rocksDb instance before any operation") val (result, elapsedMs) = Utils.timeTakenMs { val c = Checkpoint.create(rocksDb) val f: File = new File(dir) @@ -249,7 +248,7 @@ class RocksDbInstance(keySchema: StructType, valueSchema: StructType, identifier } def createBackup(dir: String): Unit = { - verify(isOpen(), "Open rocksDb instance before any operation") + require(isOpen(), "Open rocksDb instance before any operation") val (result, elapsedMs) = Utils.timeTakenMs { val backupableDBOptions = new BackupableDBOptions(dir) backupableDBOptions.setDestroyOldData(true) @@ -286,8 +285,8 @@ class OptimisticTransactionDbInstance( } override def open(path: String, conf: Map[String, String], readOnly: Boolean): Unit = { - verify(otdb == null, "Another OptimisticTransactionDbInstance instance is already actve") - verify(readOnly == false, "Cannot open OptimisticTransactionDbInstance in Readonly mode") + require(otdb == null, "Another OptimisticTransactionDbInstance instance is already actve") + require(readOnly == false, "Cannot open OptimisticTransactionDbInstance in Readonly mode") try { setOptions(conf) options.setCreateIfMissing(true) @@ -303,7 +302,7 @@ class OptimisticTransactionDbInstance( } def startTransactions(): Unit = { - verify(isOpen(), "Open OptimisticTransactionDbInstance before performing any operation") + require(isOpen(), "Open OptimisticTransactionDbInstance before performing any operation") Option(txn) match { case None => val optimisticTransactionOptions = new OptimisticTransactionOptions() @@ -315,17 +314,17 @@ class OptimisticTransactionDbInstance( } override def put(key: UnsafeRow, value: UnsafeRow): Unit = { - verify(txn != null, "Start Transaction before inserting any key") + require(txn != null, "Start Transaction before inserting any key") txn.put(key.getBytes, value.getBytes) } override def remove(key: UnsafeRow): Unit = { - verify(txn != null, "Start Transaction before deleting any key") + require(txn != null, "Start Transaction before deleting any key") txn.delete(key.getBytes) } override def get(key: UnsafeRow): UnsafeRow = { - verify(txn != null, "Start Transaction before fetching any key-value") + require(txn != null, "Start Transaction before fetching any key-value") Option(txn.get(readOptions, key.getBytes)) match { case Some(valueInBytes) => val value = new UnsafeRow(valueSchema.fields.length) @@ -337,7 +336,7 @@ class OptimisticTransactionDbInstance( } override def commit(backupPath: Option[String] = None): Unit = { - verify(txn != null, "Start Transaction before fetching any key-value") + require(txn != null, "Start Transaction before fetching any key-value") // printTrxStats try { val file = new File(dbPath, identifier.toUpperCase(Locale.ROOT)) @@ -354,24 +353,24 @@ class OptimisticTransactionDbInstance( } def printTrxStats(): Unit = { - verify(txn != null, "No open Transaction") + require(txn != null, "No open Transaction") logInfo(s""" - | deletes = ${txn.getNumDeletes} - | numKeys = ${txn.getNumKeys} - | puts = ${txn.getNumPuts} - | time = ${txn.getElapsedTime} + | deletes = ${txn.getNumDeletes} + | numKeys = ${txn.getNumKeys} + | puts = ${txn.getNumPuts} + | time = ${txn.getElapsedTime} """.stripMargin) } override def abort(): Unit = { - verify(txn != null, "No Transaction to abort") + require(txn != null, "No Transaction to abort") txn.rollbackToSavePoint() txn.close() txn = null } override def close(): Unit = { - verify(isOpen(), "No DB to close") + require(isOpen(), "No DB to close") readOptions.close() writeOptions.close() logDebug("Closing the transaction db") @@ -380,8 +379,8 @@ class OptimisticTransactionDbInstance( } override def iterator(closeDbOnCompletion: Boolean): Iterator[UnsafeRowPair] = { - verify(txn != null, "Transaction is not set") - verify( + require(txn != null, "Transaction is not set") + require( closeDbOnCompletion == false, "Cannot close a DB without aborting/commiting the transactions") val readOptions = new ReadOptions() @@ -425,10 +424,4 @@ object RocksDbInstance { } } - def verify(condition: => Boolean, msg: String): Unit = { - if (!condition) { - throw new IllegalStateException(msg) - } - } - } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreProvider.scala index d556cbeae0e3..e96f572db274 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreProvider.scala @@ -21,11 +21,12 @@ 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 scala.collection.JavaConverters._ -import scala.util.control.NonFatal import org.apache.spark.{SparkConf, SparkEnv} import org.apache.spark.internal.Logging @@ -59,7 +60,7 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log import WALUtils._ - /** Implementation of [[StateStore]] API which is backed by RocksDb and HDFS */ + /** 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 */ @@ -78,6 +79,12 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log 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 @@ -93,12 +100,12 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log } override def get(key: UnsafeRow): UnsafeRow = { - initTransaction + initTransaction() rocksDbWriteInstance.get(key) } override def put(key: UnsafeRow, value: UnsafeRow): Unit = { - initTransaction + initTransaction() verify( state == UPDATING, s"Current state of the store is $state. " + @@ -110,7 +117,7 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log } override def remove(key: UnsafeRow): Unit = { - initTransaction + initTransaction() verify(state == UPDATING, "Cannot remove after already committed or aborted") rocksDbWriteInstance.remove(key) // TODO check if removed value is null @@ -126,7 +133,7 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log /** Commit all the updates that have been made to the store, and return the new version. */ override def commit(): Long = { - initTransaction + initTransaction() verify( state == UPDATING, s"Current state of the store is $state " + @@ -225,10 +232,7 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log // just allow searching from list cause the list is small enough supportedCustomMetrics.find(_.name == name).map(_ -> value) } - return StateStoreMetrics( - Math.max(numEntriesInDb, 0), - Math.max(bytesUsedByDb, 0), - customMetrics) + StateStoreMetrics(Math.max(numEntriesInDb, 0), Math.max(bytesUsedByDb, 0), customMetrics) } /* @@ -245,7 +249,7 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log } /* - * Initialize the provide with more contextual information from the SQL operator. + * Initialize the provider with more contextual information from the SQL operator. * This method will be called first after creating an instance of the StateStoreProvider by * reflection. * @@ -270,7 +274,6 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log this.valueSchema = valueSchema this.storeConf = storeConfs this.hadoopConf = hadoopConf - // TODO add new conf for `maxVersionsToRetainInMemory` this.numberOfVersionsToRetain = storeConfs.maxVersionsToRetainInMemory fm.mkdirs(baseDir) this.rocksDbConf = storeConf.confs @@ -335,7 +338,7 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log } } - def createStore(version: Long): RocksDbStateStore = { + private def createStore(version: Long): RocksDbStateStore = { val newStore = new RocksDbStateStore(version) logInfo( s"Creating a new Store for version $version and partition ${stateStoreId_.partitionId}") @@ -348,9 +351,7 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log } def checkIfStateExists(version: Long): Boolean = { - val dbPath: Path = new Path(rocksDbPath, version.toString.toUpperCase(Locale.ROOT)) - val f: File = new File(dbPath.toString) - f.exists() + new File(rocksDbPath, version.toString).exists() } def loadState(version: Long): Unit = { @@ -358,31 +359,29 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log var rocksDbWriteInstance: OptimisticTransactionDbInstance = null var lastAvailableVersion = version var found = false - val (result, elapsedMs) = Utils.timeTakenMs { + val (_, elapsedMs) = Utils.timeTakenMs { try { if (checkIfStateExists(version - 1)) { found = true lastAvailableVersion = version - 1 } else { - // Destroy DB so that we can recontruct it using snapshot and delta files + // Destroy DB so that we can reconstruct it using snapshot and delta files RocksDbInstance.destroyDB(rocksDbPath) } // Check for snapshot files starting from "version" while (!found && lastAvailableVersion > 0) { - found = { - try { - loadSnapshotFile(lastAvailableVersion) - } catch { - case e: Exception => - logError(s"$e while reading snapshot file") - throw e - } + try { + found = loadSnapshotFile(lastAvailableVersion) + } catch { + case e: Exception => + logError(s"$e while reading snapshot file") + throw e } if (!found) { lastAvailableVersion = lastAvailableVersion - 1 } - logInfo( + logDebug( s"Snapshot for $lastAvailableVersion for " + s"partition ${stateStoreId_.partitionId} found = $found") } @@ -425,17 +424,18 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log private def loadSnapshotFile(version: Long): Boolean = { val fileToRead = snapshotFile(baseDir, version) - val tmpLocDir: File = new File(getTempPath(version)) - val tmpLocFile: File = new File(getTempPath(version).concat(".tar")) + if (!fm.exists(fileToRead)) { + return false + } + val versionTempPath = getTempPath(version) + val tmpLocDir: File = new File(versionTempPath) + val tmpLocFile: File = new File(s"${versionTempPath}.tar") try { - if (!fm.exists(fileToRead)) { - return false - } logInfo(s"Will download $fileToRead at location ${tmpLocFile.toString()}") if (downloadFile(fm, fileToRead, new Path(tmpLocFile.getAbsolutePath), sparkConf)) { - FileUtility.unTar(tmpLocFile) + FileUtility.extractTarFile(s"{versionTempPath}.tar", versionTempPath) if (!tmpLocDir.list().exists(_.endsWith(".sst"))) { - logWarning("Snaphot files are corrupted") + logWarning("Snapshot files are corrupted") throw new IOException( s"Error reading snapshot file $fileToRead of $this:" + s" No SST files found") @@ -477,7 +477,7 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log filesForVersion(files, lastVersion).filter(_.isSnapshot == false) if (deltaFilesForLastVersion.size > storeConf.minDeltasForSnapshot) { val dbPath = getBackupPath(lastVersion) - val snapShotFileName = getTempPath(lastVersion).concat(".snapshot") + val snapShotFileName = s"{getTempPath(lastVersion)}.snapshot" val f = new File(snapShotFileName) f.delete() // delete any existing tarball try { @@ -486,12 +486,10 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log val targetFile = snapshotFile(baseDir, lastVersion) uploadFile(fm, new Path(snapShotFileName), targetFile, sparkConf) } - logInfo( - s"Creating snapshot file for" + - s" ${stateStoreId_.partitionId} took $t1 ms.") + logInfo(s"Creating snapshot file for ${stateStoreId_.partitionId} took $t1 ms.") } catch { case e: Exception => - logError(s"Exception while creating snapshot $e} ") + logError(s"Exception while creating snapshot $e") throw e } finally { f.delete() // delete the tarball @@ -515,7 +513,7 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log val (_, e2) = Utils.timeTakenMs { filesToDelete.foreach { f => fm.delete(f.path) - val file = new File(rocksDbPath, f.version.toString.toUpperCase(Locale.ROOT)) + val file = new File(rocksDbPath, f.version.toString) if (file.exists()) { file.delete() } @@ -540,7 +538,7 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log if (earliestVersionToRetain > 0) { for (v <- (earliestVersionToRetain - 1) to 1) { // Destroy the backup path - logDebug((s"Destroying backup version = $v")) + logDebug(s"Destroying backup version = $v") RocksDbInstance.destroyDB(getBackupPath(v)) } } @@ -553,7 +551,6 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log // Used only for unit tests private[sql] def latestIterator(): Iterator[UnsafeRowPair] = synchronized { val versionsInFiles = fetchFiles(fm, baseDir).map(_.version).toSet - var itr = Iterator.empty if (versionsInFiles.nonEmpty) { val maxVersion = versionsInFiles.max if (maxVersion == 0) { From 41a688d3fea4608e8a478b628f79d88f01d3eeff Mon Sep 17 00:00:00 2001 From: Vikram Agrawal Date: Mon, 8 Jul 2019 21:37:12 +0530 Subject: [PATCH 09/26] More Review comments fixes --- .../main/java/org/apache/spark/io/FileUtility.java | 13 ++----------- .../streaming/state/RocksDbStateStoreProvider.scala | 7 +++---- .../sql/execution/streaming/state/WALUtils.scala | 7 ++++--- .../streaming/state/RocksDbStateStoreSuite.scala | 7 +++---- 4 files changed, 12 insertions(+), 22 deletions(-) diff --git a/core/src/main/java/org/apache/spark/io/FileUtility.java b/core/src/main/java/org/apache/spark/io/FileUtility.java index 3efbdc4e8a13..f97d0d084558 100644 --- a/core/src/main/java/org/apache/spark/io/FileUtility.java +++ b/core/src/main/java/org/apache/spark/io/FileUtility.java @@ -21,14 +21,9 @@ import java.io.*; public class FileUtility { - /** * Extract an input tar file into an output files and directories. - * @param inputTarFileLoc the input file location for the tar file - * @param destDirLoc destination for the extracted files - * - * @throws IOException - * @throws IllegalStateException + * throws IllegalStateException on error */ public static final String ENCODING = "utf-8"; @@ -80,11 +75,7 @@ public static void extractTarFile(String inputTarFileLoc, String destDirLoc) /** * create a tar file for input source directory location . - * @param source the source directory location - * @param destFileLoc destination of the created tarball - * - * @throws IOException - * @throws IllegalStateException + * throws IOException on error */ public static void createTarFile(String source, String destFileLoc) throws IllegalStateException, IOException { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreProvider.scala index e96f572db274..283b98683ee4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreProvider.scala @@ -139,11 +139,11 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log s"Current state of the store is $state " + s"Cannot commit after already committed or aborted") try { - state = COMMITTED synchronized { rocksDbWriteInstance.commit(Some(getBackupPath(newVersion))) finalizeDeltaFile(compressedStream) } + state = COMMITTED numEntriesInDb = rocksDbWriteInstance.otdb.getLongProperty("rocksdb.estimate-num-keys") bytesUsedByDb = numEntriesInDb * (keySchema.defaultSize + valueSchema.defaultSize) newVersion @@ -479,7 +479,6 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log val dbPath = getBackupPath(lastVersion) val snapShotFileName = s"{getTempPath(lastVersion)}.snapshot" val f = new File(snapShotFileName) - f.delete() // delete any existing tarball try { val (_, t1) = Utils.timeTakenMs { FileUtility.createTarFile(dbPath, snapShotFileName) @@ -536,7 +535,7 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log if (files.nonEmpty) { val earliestVersionToRetain = files.last.version - storeConf.minVersionsToRetain if (earliestVersionToRetain > 0) { - for (v <- (earliestVersionToRetain - 1) to 1) { + for (v <- (earliestVersionToRetain - 1) to 1 by -1) { // Destroy the backup path logDebug(s"Destroying backup version = $v") RocksDbInstance.destroyDB(getBackupPath(v)) @@ -571,7 +570,7 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log } // making it public for unit tests - lazy val rocksDbPath: String = { + private[sql] lazy val rocksDbPath: String = { val checkpointRootLocationPath = new Path(stateStoreId.checkpointRootLocation) val basePath = new Path( localDirectory, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/WALUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/WALUtils.scala index 771ade3e2a40..ace6a31608d3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/WALUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/WALUtils.scala @@ -35,6 +35,7 @@ import org.apache.spark.sql.types.StructType object WALUtils { case class StoreFile(version: Long, path: Path, isSnapshot: Boolean) + val EOF_MARKER = -1 /** Files needed to recover the given version of the store */ def filesForVersion(allFiles: Seq[StoreFile], version: Long): Seq[StoreFile] = { @@ -115,11 +116,11 @@ object WALUtils { val keyBytes = key.getBytes() output.writeInt(keyBytes.size) output.write(keyBytes) - output.writeInt(-1) + output.writeInt(EOF_MARKER) } def finalizeDeltaFile(output: DataOutputStream): Unit = { - output.writeInt(-1) // Write this magic number to signify end of file + output.writeInt(EOF_MARKER) // Write this magic number to signify end of file output.close() } @@ -145,7 +146,7 @@ object WALUtils { while (!eof) { val keySize = input.readInt() - if (keySize == -1) { + if (keySize == EOF_MARKER) { eof = true } else if (keySize < 0) { newRocksDb.abort diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreSuite.scala index a430ebf63674..d444aff1c017 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreSuite.scala @@ -288,7 +288,7 @@ class RocksDbStateStoreSuite val storeId = StateStoreProviderId(StateStoreId(dir, 0, 0), UUID.randomUUID) val sqlConf = new SQLConf sqlConf.setConfString( - "spark.sql.streaming.stateStore.providerClass", + SQLConf.STATE_STORE_PROVIDER_CLASS.key, "org.apache.spark.sql.execution.streaming.state.RocksDbStateStoreProvider") val localdir = Utils.createTempDir().getAbsoluteFile.toString sqlConf.setConfString("spark.sql.streaming.stateStore.rocksDb.localDirectory", localdir) @@ -353,7 +353,7 @@ class RocksDbStateStoreSuite val storeProviderId = StateStoreProviderId(StateStoreId(dir, opId, 0), UUID.randomUUID) val sqlConf = new SQLConf() sqlConf.setConfString( - "spark.sql.streaming.stateStore.providerClass", + SQLConf.STATE_STORE_PROVIDER_CLASS.key, "org.apache.spark.sql.execution.streaming.state.RocksDbStateStoreProvider") sqlConf.setConf(SQLConf.MIN_BATCHES_TO_RETAIN, 2) sqlConf.setConfString( @@ -470,7 +470,7 @@ class RocksDbStateStoreSuite implicit val sqlContext = spark.sqlContext spark.conf.set("spark.sql.shuffle.partitions", "1") spark.conf.set( - "spark.sql.streaming.stateStore.providerClass", + SQLConf.STATE_STORE_PROVIDER_CLASS.key, "org.apache.spark.sql.execution.streaming.state.RocksDbStateStoreProvider") spark.conf.set( "spark.sql.streaming.stateStore.rocksDb.localDirectory", @@ -546,7 +546,6 @@ class RocksDbStateStoreSuite version: Int = -1): Set[(String, Int)] = { val reloadedProvider = newStoreProvider(provider.stateStoreId, provider.getLocalDirectory) if (version < 0) { - // TODO find out last version from rocksDB reloadedProvider.latestIterator().map(rowsToStringInt).toSet } else { reloadedProvider.getStore(version).iterator().map(rowsToStringInt).toSet From a484d633f709956722e873bafd5df2465078dc24 Mon Sep 17 00:00:00 2001 From: Vikram Agrawal Date: Wed, 31 Jul 2019 15:11:36 +0530 Subject: [PATCH 10/26] Address Review Comments and fix memory leaks --- .../java/org/apache/spark/io/FileUtility.java | 82 +++-- pom.xml | 1 + sql/core/pom.xml | 2 +- .../streaming/state/RocksDbInstance.scala | 304 ++++++++++-------- .../state/RocksDbStateStoreProvider.scala | 289 ++++++++--------- .../execution/streaming/state/WALUtils.scala | 2 +- 6 files changed, 340 insertions(+), 340 deletions(-) diff --git a/core/src/main/java/org/apache/spark/io/FileUtility.java b/core/src/main/java/org/apache/spark/io/FileUtility.java index f97d0d084558..9a07be9c1b4b 100644 --- a/core/src/main/java/org/apache/spark/io/FileUtility.java +++ b/core/src/main/java/org/apache/spark/io/FileUtility.java @@ -21,95 +21,91 @@ import java.io.*; public class FileUtility { + /** * Extract an input tar file into an output files and directories. - * throws IllegalStateException on error + * @param inputTarFileLoc the input file location for the tar file + * @param destDirLoc destination for the extracted files + * + * @throws IllegalStateException */ public static final String ENCODING = "utf-8"; public static void extractTarFile(String inputTarFileLoc, String destDirLoc) - throws IOException, IllegalStateException { + throws IllegalStateException { File inputFile = new File(inputTarFileLoc); - if (!inputTarFileLoc.endsWith("tar")) { + if (!inputTarFileLoc.endsWith(".tar")) { throw new IllegalStateException(String.format( - "Input File %s should end with tar extension.", inputTarFileLoc)); + "Input File[%s] should end with tar extension.", inputTarFileLoc)); } File destDir = new File(destDirLoc); if (destDir.exists() && !destDir.delete()) { throw new IllegalStateException(String.format( - "Couldn't delete the existing destination directory %s ", destDirLoc)); + "Couldn't delete the existing destination directory[%s] ", destDirLoc)); } else if (!destDir.mkdir()) { throw new IllegalStateException(String.format( - "Couldn't create directory %s ", destDirLoc)); + "Couldn't create directory %s ", destDirLoc)); } - final InputStream is = new FileInputStream(inputFile); - final TarArchiveInputStream debInputStream = new TarArchiveInputStream(is, ENCODING); - OutputStream outputFileStream = null; - try { + try (InputStream is = new FileInputStream(inputFile); + TarArchiveInputStream debInputStream = new TarArchiveInputStream(is, ENCODING)) { TarArchiveEntry entry; while ((entry = (TarArchiveEntry) debInputStream.getNextEntry()) != null) { final File outputFile = new File(destDirLoc, entry.getName()); if (entry.isDirectory()) { if (!outputFile.exists() && !outputFile.mkdirs()) { throw new IllegalStateException(String.format( - "Couldn't create directory %s.", outputFile.getAbsolutePath())); + "Couldn't create directory %s.", outputFile.getAbsolutePath())); } } else { - outputFileStream = new FileOutputStream(outputFile); - IOUtils.copy(debInputStream, outputFileStream); - outputFileStream.close(); - outputFileStream = null; + try (OutputStream outputFileStream = new FileOutputStream(outputFile)) { + IOUtils.copy(debInputStream, outputFileStream); + } } } } catch (IOException e){ throw new IllegalStateException(String.format( - "extractTarFile failed with exception %s.", e.getMessage())); - } finally { - debInputStream.close(); - if (outputFileStream != null) { - outputFileStream.close(); - } + "extractTarFile failed with exception %s.", e.getMessage())); } } /** * create a tar file for input source directory location . - * throws IOException on error + * @param source the source directory location + * @param destFileLoc destination of the created tarball + * + * @throws IllegalStateException */ + public static void createTarFile(String source, String destFileLoc) - throws IllegalStateException, IOException { - TarArchiveOutputStream tarOs = null; + throws IllegalStateException { File f = new File(destFileLoc); if (f.exists() && !f.delete()) { throw new IllegalStateException(String.format( - "Couldn't delete the destination file location %s", destFileLoc)); + "Couldn't delete the destination file location[%s]", destFileLoc)); + } + File folder = new File(source); + if (!folder.exists()) { + throw new IllegalStateException(String.format( + "Source folder[%s] does not exist", source)); } - BufferedInputStream bis = null; - try { - FileOutputStream fos = new FileOutputStream(destFileLoc); - tarOs = new TarArchiveOutputStream(fos, ENCODING); - File folder = new File(source); + + try (FileOutputStream fos = new FileOutputStream(destFileLoc); + TarArchiveOutputStream tarOs = new TarArchiveOutputStream(fos, ENCODING)) { File[] fileNames = folder.listFiles(); - for(File file : fileNames){ + for (File file : fileNames) { TarArchiveEntry tar_file = new TarArchiveEntry(file.getName()); tar_file.setSize(file.length()); tarOs.putArchiveEntry(tar_file); - bis = new BufferedInputStream(new FileInputStream(file)); - IOUtils.copy(bis, tarOs); - bis.close(); - bis = null; - tarOs.closeArchiveEntry(); + try (BufferedInputStream bis = new BufferedInputStream(new FileInputStream(file))) { + IOUtils.copy(bis, tarOs); + tarOs.closeArchiveEntry(); + } } + tarOs.finish(); } catch (IOException e) { throw new IllegalStateException(String.format( - "createTarFile failed with exception %s.", e.getMessage())); - } finally { - tarOs.finish(); - tarOs.close(); - if (bis != null) { - bis.close(); - } + "createTarFile failed with exception %s.", e.getMessage())); } } diff --git a/pom.xml b/pom.xml index d970da1186bf..b786831eea8f 100644 --- a/pom.xml +++ b/pom.xml @@ -190,6 +190,7 @@ 1.1 2.52.0 2.22 + 6.0.1 diff --git a/sql/core/pom.xml b/sql/core/pom.xml index af42ebeb12af..9029d4d1ea57 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -151,7 +151,7 @@ org.rocksdb rocksdbjni - 6.0.1 + ${rocksdb.version} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbInstance.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbInstance.scala index f495b99d9b35..e161e0b5ee31 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbInstance.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbInstance.scala @@ -17,20 +17,28 @@ package org.apache.spark.sql.execution.streaming.state +import java.io.BufferedWriter import java.io.File +import java.io.FileWriter import java.util.Locale +import scala.collection.JavaConverters._ + import org.apache.commons.io.FileUtils import org.rocksdb._ import org.rocksdb.RocksDB -import org.rocksdb.util.SizeUnit +import org.apache.spark.SparkEnv import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.types.StructType import org.apache.spark.util.Utils -class RocksDbInstance(keySchema: StructType, valueSchema: StructType, identifier: String) +class RocksDbInstance( + keySchema: StructType, + valueSchema: StructType, + version: String, + conf: Map[String, String] = Map.empty) extends Logging { import RocksDbInstance._ @@ -43,14 +51,14 @@ class RocksDbInstance(keySchema: StructType, valueSchema: StructType, identifier protected val table_options = new BlockBasedTableConfig protected val options: Options = new Options() - def isOpen(): Boolean = { + private def isOpen(): Boolean = { db != null } - def open(path: String, conf: Map[String, String], readOnly: Boolean): Unit = { + def open(path: String, readOnly: Boolean): Unit = { require(db == null, "Another rocksDb instance is already active") try { - setOptions(conf) + setOptions db = if (readOnly) { options.setCreateIfMissing(false) RocksDB.openReadOnly(options, path) @@ -88,8 +96,8 @@ class RocksDbInstance(keySchema: StructType, valueSchema: StructType, identifier db.delete(key.getBytes) } - def commit(backupPath: Option[String] = None): Unit = { - backupPath.foreach(f => createCheckpoint(db, f)) + def commit(checkPointPath: Option[String] = None): Unit = { + checkPointPath.foreach(f => createCheckpoint(db, f)) } def abort: Unit = { @@ -97,21 +105,25 @@ class RocksDbInstance(keySchema: StructType, valueSchema: StructType, identifier } def close(): Unit = { - readOptions.close() - writeOptions.close() logDebug("Closing the db") - db.close() - db = null + try { + db.close() + } finally { + db = null + options.close() + readOptions.close() + writeOptions.close() + } } def iterator(closeDbOnCompletion: Boolean): Iterator[UnsafeRowPair] = { require(isOpen(), "Open rocksDb instance before any operation") Option(db.getSnapshot) match { case Some(snapshot) => - logDebug(s"Inside rockdDB iterator function") - var snapshotReadOptions: ReadOptions = new ReadOptions().setSnapshot(snapshot) + var snapshotReadOptions: ReadOptions = + new ReadOptions().setSnapshot(snapshot).setFillCache(false) val itr = db.newIterator(snapshotReadOptions) - createUnsafeRowPairIterator(itr, snapshotReadOptions, closeDbOnCompletion) + createUnsafeRowPairIterator(itr, snapshotReadOptions, snapshot, closeDbOnCompletion) case None => Iterator.empty } @@ -120,20 +132,27 @@ class RocksDbInstance(keySchema: StructType, valueSchema: StructType, identifier protected def createUnsafeRowPairIterator( itr: RocksIterator, itrReadOptions: ReadOptions, + snapshot: Snapshot, closeDbOnCompletion: Boolean): Iterator[UnsafeRowPair] = { itr.seekToFirst() new Iterator[UnsafeRowPair] { + @volatile var isClosed = false override def hasNext: Boolean = { - if (itr.isValid) { + if (!isClosed && itr.isValid) { true } else { - itrReadOptions.close() - if (closeDbOnCompletion) { - close() + if (!isClosed) { + isClosed = true + itrReadOptions.close() + db.releaseSnapshot(snapshot) + if (closeDbOnCompletion) { + close() + } + itr.close() + logDebug(s"read from DB completed") } - logDebug(s"read from DB completed") false } } @@ -151,7 +170,21 @@ class RocksDbInstance(keySchema: StructType, valueSchema: StructType, identifier } } - def printStats: Unit = { + protected def printMemoryStats(db: RocksDB): Unit = { + require(isOpen(), "Open rocksDb instance before any operation") + val usage = MemoryUtil + .getApproximateMemoryUsageByType( + List(db).asJava, + Set(rocksDbLRUCache.asInstanceOf[Cache]).asJava) + .asScala + val numKeys = db.getProperty(db.getDefaultColumnFamily, "rocksdb.estimate-num-keys") + logDebug(s""" + | rocksdb.estimate-num-keys = $numKeys + | ApproximateMemoryUsageByType = ${usage.toString} + | """.stripMargin) + } + + protected def printStats: Unit = { require(isOpen(), "Open rocksDb instance before any operation") try { val stats = db.getProperty("rocksdb.stats") @@ -162,78 +195,65 @@ class RocksDbInstance(keySchema: StructType, valueSchema: StructType, identifier } } - def setOptions(conf: Map[String, String]): Unit = { + private val dataBlockSize = conf + .getOrElse( + "spark.sql.streaming.stateStore.rocksDb.blockSizeInKB".toLowerCase(Locale.ROOT), + "32") + .toInt + + private val memTableMemoryBudget = conf + .getOrElse( + "spark.sql.streaming.stateStore.rocksDb.memtableBudgetInMB".toLowerCase(Locale.ROOT), + "1024") + .toInt + + private val enableStats = conf + .getOrElse( + "spark.sql.streaming.stateStore.rocksDb.enableDbStats".toLowerCase(Locale.ROOT), + "false") + .toBoolean + + protected def setOptions(): Unit = { // Read options - readOptions.setFillCache(false) + readOptions.setFillCache(true) // Write options writeOptions.setSync(false) writeOptions.setDisableWAL(true) - val dataBlockSize = conf - .getOrElse( - "spark.sql.streaming.stateStore.rocksDb.blockSizeInKB".toLowerCase(Locale.ROOT), - "64") - .toInt - - val metadataBlockSize = conf - .getOrElse( - "spark.sql.streaming.stateStore.rocksDb.metadataBlockSizeInKB".toLowerCase(Locale.ROOT), - "4") - .toInt - - // Table configs - // https://github.com/facebook/rocksdb/wiki/Partitioned-Index-Filters + /* + Table configs + Use Partitioned Index Filters + https://github.com/facebook/rocksdb/wiki/Partitioned-Index-Filters + Use format Verion = 4 + https://rocksdb.org/blog/2019/03/08/format-version-4.html + */ table_options - .setBlockSize(dataBlockSize) - .setBlockSizeDeviation(5) - .setMetadataBlockSize(metadataBlockSize) + .setBlockSize(dataBlockSize * 1024) + .setFormatVersion(4) + .setDataBlockIndexType(DataBlockIndexType.kDataBlockBinaryAndHash) + .setBlockCache(rocksDbLRUCache) .setFilterPolicy(new BloomFilter(10, false)) - .setPartitionFilters(true) + .setPinTopLevelIndexAndFilter(false) // Dont pin anything in cache .setIndexType(IndexType.kTwoLevelIndexSearch) - .setBlockCache(lRUCache) - .setCacheIndexAndFilterBlocks(true) - .setPinTopLevelIndexAndFilter(true) - .setCacheIndexAndFilterBlocksWithHighPriority(true) - .setPinL0FilterAndIndexBlocksInCache(true) - .setFormatVersion(4) // https://rocksdb.org/blog/2019/03/08/format-version-4.html - .setIndexBlockRestartInterval(16) - - var bufferNumber = conf - .getOrElse( - "spark.sql.streaming.stateStore.rocksDb.bufferNumber".toLowerCase(Locale.ROOT), - "5") - .toInt - - bufferNumber = Math.max(bufferNumber, 3) - - val bufferNumberToMaintain = Math.max(bufferNumber - 2, 3) - - logInfo( - s"Using Max Buffer Name = $bufferNumber & " + - s"max buffer number to maintain = $bufferNumberToMaintain") + .setPartitionFilters(true) - // DB Options options - .setCreateIfMissing(true) - .setMaxWriteBufferNumber(bufferNumber) - .setMaxWriteBufferNumberToMaintain(bufferNumberToMaintain) - .setMaxBackgroundCompactions(4) - .setMaxBackgroundFlushes(2) - .setMaxOpenFiles(-1) - .setMaxFileOpeningThreads(4) - .setWriteBufferSize(256 * SizeUnit.MB) - .setTargetFileSizeBase(256 * SizeUnit.MB) - .setLevelZeroFileNumCompactionTrigger(10) - .setLevelZeroSlowdownWritesTrigger(20) - .setLevelZeroStopWritesTrigger(40) - .setMaxBytesForLevelBase(2 * SizeUnit.GB) .setTableFormatConfig(table_options) - + .optimizeLevelStyleCompaction(memTableMemoryBudget * 1024 * 1024) + .setBytesPerSync(1048576) + .setMaxOpenFiles(5000) + .setIncreaseParallelism(4) + + if (enableStats) { + options + .setStatistics(new Statistics()) + .setStatsDumpPeriodSec(30) + } } - def createCheckpoint(rocksDb: RocksDB, dir: String): Unit = { + protected def createCheckpoint(rocksDb: RocksDB, dir: String): Unit = { require(isOpen(), "Open rocksDb instance before any operation") val (result, elapsedMs) = Utils.timeTakenMs { val c = Checkpoint.create(rocksDb) @@ -244,53 +264,42 @@ class RocksDbInstance(keySchema: StructType, valueSchema: StructType, identifier c.createCheckpoint(dir) c.close() } - logDebug(s"Creating createCheckpoint at $dir took $elapsedMs ms.") - } - - def createBackup(dir: String): Unit = { - require(isOpen(), "Open rocksDb instance before any operation") - val (result, elapsedMs) = Utils.timeTakenMs { - val backupableDBOptions = new BackupableDBOptions(dir) - backupableDBOptions.setDestroyOldData(true) - val env: Env = Env.getDefault - env.setBackgroundThreads(2) - val be = BackupEngine.open(env, backupableDBOptions) - be.createNewBackup(db, true) // - backupableDBOptions.close() - env.close() - be.close() - } - logInfo(s"Creating backup at $dir takes $elapsedMs ms.") + logInfo(s"Creating Checkpoint at $dir took $elapsedMs ms.") } } class OptimisticTransactionDbInstance( keySchema: StructType, valueSchema: StructType, - identifier: String) - extends RocksDbInstance(keySchema: StructType, valueSchema: StructType, identifier: String) { - + version: String, + conf: Map[String, String] = Map.empty) + extends RocksDbInstance( + keySchema: StructType, + valueSchema: StructType, + version: String, + conf: Map[String, String]) { import RocksDbInstance._ RocksDB.loadLibrary() - var otdb: OptimisticTransactionDB = null - var txn: Transaction = null + private var otdb: OptimisticTransactionDB = null + private var txn: Transaction = null - override def isOpen(): Boolean = { + private def isOpen(): Boolean = { otdb != null } - def open(path: String, conf: Map[String, String]): Unit = { - open(path, conf, false) + def open(path: String): Unit = { + open(path, false) } - override def open(path: String, conf: Map[String, String], readOnly: Boolean): Unit = { - require(otdb == null, "Another OptimisticTransactionDbInstance instance is already actve") + override def open(path: String, readOnly: Boolean): Unit = { + require(otdb == null, "Another OptimisticTransactionDbInstance instance is already active") require(readOnly == false, "Cannot open OptimisticTransactionDbInstance in Readonly mode") try { - setOptions(conf) + setOptions() options.setCreateIfMissing(true) otdb = OptimisticTransactionDB.open(options, path) + db = otdb.getBaseDB dbPath = path } catch { case e: Throwable => @@ -335,33 +344,22 @@ class OptimisticTransactionDbInstance( } } - override def commit(backupPath: Option[String] = None): Unit = { + override def commit(checkPointPath: Option[String] = None): Unit = { require(txn != null, "Start Transaction before fetching any key-value") - // printTrxStats try { - val file = new File(dbPath, identifier.toUpperCase(Locale.ROOT)) - file.createNewFile() txn.commit() txn.close() - txn = null - backupPath.foreach(f => createCheckpoint(otdb.asInstanceOf[RocksDB], f)) + updateVersionInCommitFile() + checkPointPath.foreach(f => createCheckpoint(otdb.asInstanceOf[RocksDB], f)) } catch { case e: Exception => log.error(s"Unable to commit the transactions. Error message = ${e.getMessage}") throw e + } finally { + txn = null } } - def printTrxStats(): Unit = { - require(txn != null, "No open Transaction") - logInfo(s""" - | deletes = ${txn.getNumDeletes} - | numKeys = ${txn.getNumKeys} - | puts = ${txn.getNumPuts} - | time = ${txn.getElapsedTime} - """.stripMargin) - } - override def abort(): Unit = { require(txn != null, "No Transaction to abort") txn.rollbackToSavePoint() @@ -371,57 +369,79 @@ class OptimisticTransactionDbInstance( override def close(): Unit = { require(isOpen(), "No DB to close") - readOptions.close() - writeOptions.close() + require(txn == null, "Transaction should be closed before closing the DB connection") + printMemoryStats(otdb.asInstanceOf[RocksDB]) logDebug("Closing the transaction db") - otdb.close() - otdb = null + try { + otdb.close() + db.close() + otdb = null + db = null + } finally { + options.close() + readOptions.close() + writeOptions.close() + } } override def iterator(closeDbOnCompletion: Boolean): Iterator[UnsafeRowPair] = { require(txn != null, "Transaction is not set") require( closeDbOnCompletion == false, - "Cannot close a DB without aborting/commiting the transactions") + "Cannot close a DB without aborting/committing the transactions") + val snapshot = db.getSnapshot val readOptions = new ReadOptions() + .setSnapshot(snapshot) + .setFillCache(false) // for range lookup, we should not fill cache val itr: RocksIterator = txn.getIterator(readOptions) Option(itr) match { case Some(i) => - logDebug(s"creating iterator from transaction DB") - createUnsafeRowPairIterator(i, readOptions, false) + logDebug(s"creating iterator from a transactional DB") + createUnsafeRowPairIterator(i, readOptions, snapshot, false) case None => Iterator.empty } } + def getApproxEntriesInDb(): Long = { + require(isOpen(), "No DB to find Database Entries") + otdb.getProperty("rocksdb.estimate-num-keys").toLong + } + + protected def updateVersionInCommitFile(): Unit = { + val file = new File(dbPath, COMMIT_FILE_NAME) + val bw = new BufferedWriter(new FileWriter(file)) + bw.write(version.toString) + bw.close() + } + } object RocksDbInstance { RocksDB.loadLibrary() - private val destroyOptions: Options = new Options() + val COMMIT_FILE_NAME = "commit" + + private val DEFAULT_ROCKSDB_CACHE_SIZE_IN_MB = 512 - val lRUCache = new LRUCache(1024 * 1024 * 1024, 6, false, 0.05) + private val rocksDbCacheSizeInMB: Int = if (SparkEnv.get != null) { + SparkEnv.get.conf.getInt( + "spark.sql.streaming.stateStore.rocksDb.cacheSizeInMB", + DEFAULT_ROCKSDB_CACHE_SIZE_IN_MB) + } else { + DEFAULT_ROCKSDB_CACHE_SIZE_IN_MB + } + + lazy val rocksDbLRUCache = new LRUCache(rocksDbCacheSizeInMB * 1024 * 1024, 6, false) def destroyDB(path: String): Unit = { val f: File = new File(path) + val destroyOptions: Options = new Options() if (f.exists()) { RocksDB.destroyDB(path, destroyOptions) FileUtils.deleteDirectory(f) } } - def restoreFromBackup(backupDir: String, dbDir: String): Unit = { - val (result, elapsedMs) = Utils.timeTakenMs { - val backupableDBOptions = new BackupableDBOptions(backupDir) - val be = BackupEngine.open(Env.getDefault, backupableDBOptions) - val restoreOptions = new RestoreOptions(false) - be.restoreDbFromLatestBackup(dbDir, dbDir, restoreOptions) - restoreOptions.close() - backupableDBOptions.close() - be.close() - } - } - } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreProvider.scala index 283b98683ee4..54b67ff7935b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreProvider.scala @@ -56,6 +56,7 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log 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._ @@ -65,9 +66,13 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log /** 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 @@ -91,9 +96,12 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log private def initTransaction(): Unit = { if (state == LOADED && rocksDbWriteInstance == null) { logDebug(s"Creating Transactional DB for batch $version") - rocksDbWriteInstance = - new OptimisticTransactionDbInstance(keySchema, valueSchema, newVersion.toString) - rocksDbWriteInstance.open(rocksDbPath, rocksDbConf) + rocksDbWriteInstance = new OptimisticTransactionDbInstance( + keySchema, + valueSchema, + newVersion.toString, + rocksDbConf) + rocksDbWriteInstance.open(rocksDbPath) state = UPDATING rocksDbWriteInstance.startTransactions() } @@ -106,10 +114,7 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log override def put(key: UnsafeRow, value: UnsafeRow): Unit = { initTransaction() - verify( - state == UPDATING, - s"Current state of the store is $state. " + - s"Cannot put after already committed or aborted") + verify(state == UPDATING, s"Cannot put after already committed or aborted") val keyCopy = key.copy() val valueCopy = value.copy() rocksDbWriteInstance.put(keyCopy, valueCopy) @@ -120,7 +125,6 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log initTransaction() verify(state == UPDATING, "Cannot remove after already committed or aborted") rocksDbWriteInstance.remove(key) - // TODO check if removed value is null writeRemoveToDeltaFile(compressedStream, key) } @@ -134,17 +138,14 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log /** Commit all the updates that have been made to the store, and return the new version. */ override def commit(): Long = { initTransaction() - verify( - state == UPDATING, - s"Current state of the store is $state " + - s"Cannot commit after already committed or aborted") + verify(state == UPDATING, s"Cannot commit after already committed or aborted") try { synchronized { rocksDbWriteInstance.commit(Some(getBackupPath(newVersion))) finalizeDeltaFile(compressedStream) } state = COMMITTED - numEntriesInDb = rocksDbWriteInstance.otdb.getLongProperty("rocksdb.estimate-num-keys") + numEntriesInDb = rocksDbWriteInstance.getApproxEntriesInDb() bytesUsedByDb = numEntriesInDb * (keySchema.defaultSize + valueSchema.defaultSize) newVersion } catch { @@ -208,8 +209,9 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log Iterator.empty } else { val path = getBackupPath(version) - val r: RocksDbInstance = new RocksDbInstance(keySchema, valueSchema, version.toString) - r.open(path, rocksDbConf, readOnly = true) + val r: RocksDbInstance = + new RocksDbInstance(keySchema, valueSchema, version.toString, rocksDbConf) + r.open(path, readOnly = true) r.iterator(closeDbOnCompletion = true) } case COMMITTED => @@ -217,8 +219,8 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log // use check-pointed db for current updated version val path = getBackupPath(newVersion) val r: RocksDbInstance = - new RocksDbInstance(keySchema, valueSchema, newVersion.toString) - r.open(path, rocksDbConf, readOnly = true) + new RocksDbInstance(keySchema, valueSchema, newVersion.toString, rocksDbConf) + r.open(path, readOnly = true) r.iterator(closeDbOnCompletion = true) case _ => Iterator.empty @@ -284,7 +286,7 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log this.localDirectory = this.rocksDbConf .getOrElse( "spark.sql.streaming.stateStore.rocksDb.localDirectory".toLowerCase(Locale.ROOT), - RocksDbStateStoreProvider.ROCKS_DB_BASE_PATH) + Utils.createTempDir().getAbsoluteFile.toString) } /* @@ -340,91 +342,71 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log private def createStore(version: Long): RocksDbStateStore = { val newStore = new RocksDbStateStore(version) - logInfo( - s"Creating a new Store for version $version and partition ${stateStoreId_.partitionId}") - if (version > 0 & !checkIfStateExists(version)) { - // load the data in the rocksDB - logInfo(s"Loading state for $version and partition ${stateStoreId_.partitionId}") - loadState(version) + if (version > 0) { + // load the data into the rocksDB + logInfo( + s"Loading state into the db for $version and partition ${stateStoreId_.partitionId}") + loadIntoRocksDB(version) } newStore } - def checkIfStateExists(version: Long): Boolean = { - new File(rocksDbPath, version.toString).exists() - } - - def loadState(version: Long): Unit = { - // search for state on snapshot - var rocksDbWriteInstance: OptimisticTransactionDbInstance = null - var lastAvailableVersion = version - var found = false + private def loadIntoRocksDB(version: Long): Unit = { + /* + 1. Get last available/committed Rocksdb version in local folder + 2. If last committed version = version, we already have loaded rocksdb state. + 3. If last committed version = version - 1, + we have to apply delta for version in the existing rocksdb + 4. Otherwise we have to recreate a new rocksDB store by using Snapshots/Delta + */ val (_, elapsedMs) = Utils.timeTakenMs { - try { - if (checkIfStateExists(version - 1)) { - found = true - lastAvailableVersion = version - 1 - } else { - // Destroy DB so that we can reconstruct it using snapshot and delta files - RocksDbInstance.destroyDB(rocksDbPath) - } - - // Check for snapshot files starting from "version" - while (!found && lastAvailableVersion > 0) { + var lastAvailableVersion = getLastCommittedVersion() + if (lastAvailableVersion == -1L || lastAvailableVersion <= version - 2) { + // Destroy existing DB so that we can reconstruct it using snapshot and delta files + RocksDbInstance.destroyDB(rocksDbPath) + var lastAvailableSnapShotVersion: Long = version + 1 + // load from snapshot + var found = false + while (!found && lastAvailableSnapShotVersion > 0) { try { - found = loadSnapshotFile(lastAvailableVersion) + lastAvailableSnapShotVersion = lastAvailableSnapShotVersion - 1 + found = loadSnapshotFile(lastAvailableSnapShotVersion) + logDebug( + s"Snapshot for version $lastAvailableSnapShotVersion " + + "and partition ${stateStoreId_.partitionId}: found = $found") } catch { case e: Exception => logError(s"$e while reading snapshot file") throw e } - if (!found) { - lastAvailableVersion = lastAvailableVersion - 1 - } - logDebug( - s"Snapshot for $lastAvailableVersion for " + - s"partition ${stateStoreId_.partitionId} found = $found") - } - - rocksDbWriteInstance = - new OptimisticTransactionDbInstance(keySchema, valueSchema, version.toString) - rocksDbWriteInstance.open(rocksDbPath, rocksDbConf) - rocksDbWriteInstance.startTransactions() - - // Load all the deltas from the version after the last available - // one up to the target version. - // The last available version is the one with a full snapshot, so it doesn't need deltas. - for (deltaVersion <- (lastAvailableVersion + 1) to version) { - val fileToRead = deltaFile(baseDir, deltaVersion) - updateFromDeltaFile( - fm, - fileToRead, - keySchema, - valueSchema, - rocksDbWriteInstance, - sparkConf) - logInfo(s"Read delta file for version $version of $this from $fileToRead") } + lastAvailableVersion = lastAvailableSnapShotVersion + } + if (lastAvailableVersion < version) { + applyDelta(version, lastAvailableVersion) + } + } + logInfo( + s"Loading state for $version and partition ${stateStoreId_.partitionId} took $elapsedMs ms.") + } - rocksDbWriteInstance.commit(Some(getBackupPath(version))) - rocksDbWriteInstance.close() - rocksDbWriteInstance = null + private def getLastCommittedVersion(): Long = { + val f = new File(rocksDbPath, "commit") + if (f.exists()) { + try { + val fileContents = scala.io.Source.fromFile(f.getAbsolutePath).getLines.mkString + return fileContents.toLong } catch { - case e: IllegalStateException => - logError(s"Exception while loading state ${e.getMessage}") - if (rocksDbWriteInstance != null) { - rocksDbWriteInstance.abort() - rocksDbWriteInstance.close() - } - throw e + case e: Exception => + logWarning("Exception while reading committed file") } } - logInfo(s"Loading state for $version takes $elapsedMs ms.") + return -1L } private def loadSnapshotFile(version: Long): Boolean = { val fileToRead = snapshotFile(baseDir, version) - if (!fm.exists(fileToRead)) { + if (version == 0 || !fm.exists(fileToRead)) { return false } val versionTempPath = getTempPath(version) @@ -433,7 +415,7 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log try { logInfo(s"Will download $fileToRead at location ${tmpLocFile.toString()}") if (downloadFile(fm, fileToRead, new Path(tmpLocFile.getAbsolutePath), sparkConf)) { - FileUtility.extractTarFile(s"{versionTempPath}.tar", versionTempPath) + FileUtility.extractTarFile(tmpLocFile.getAbsolutePath, versionTempPath) if (!tmpLocDir.list().exists(_.endsWith(".sst"))) { logWarning("Snapshot files are corrupted") throw new IOException( @@ -441,7 +423,9 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log s" No SST files found") } FileUtils.moveDirectory(tmpLocDir, new File(rocksDbPath)) - return true + true + } else { + false } } catch { case e: Exception => @@ -453,7 +437,42 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log } FileUtils.deleteDirectory(tmpLocDir) } - return false + } + + private def applyDelta(version: Long, lastAvailableVersion: Long): Unit = { + var rocksDbWriteInstance: OptimisticTransactionDbInstance = null + try { + rocksDbWriteInstance = + new OptimisticTransactionDbInstance(keySchema, valueSchema, version.toString, rocksDbConf) + rocksDbWriteInstance.open(rocksDbPath) + rocksDbWriteInstance.startTransactions() + // Load all the deltas from the version after the last available + // one up to the target version. + // The last available version is the one with a full snapshot, so it doesn't need deltas. + for (deltaVersion <- (lastAvailableVersion + 1) to version) { + val fileToRead = deltaFile(baseDir, deltaVersion) + updateFromDeltaFile( + fm, + fileToRead, + keySchema, + valueSchema, + rocksDbWriteInstance, + sparkConf) + logInfo(s"Read delta file for version $version of $this from $fileToRead") + } + rocksDbWriteInstance.commit(Some(getBackupPath(version))) + } catch { + case e: Exception => + logError(s"Exception while loading state ${e.getMessage}") + if (rocksDbWriteInstance != null) { + rocksDbWriteInstance.abort() + } + throw e + } finally { + if (rocksDbWriteInstance != null) { + rocksDbWriteInstance.close() + } + } } /** Optional method for providers to allow for background maintenance (e.g. compactions) */ @@ -552,97 +571,61 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log val versionsInFiles = fetchFiles(fm, baseDir).map(_.version).toSet if (versionsInFiles.nonEmpty) { val maxVersion = versionsInFiles.max - if (maxVersion == 0) { - return Iterator.empty - } - // FIXME assuming maxVersion exists in rocksDB - val path = getBackupPath(maxVersion) - val r: RocksDbInstance = new RocksDbInstance(keySchema, valueSchema, maxVersion.toString) - try { - r.open(path, rocksDbConf, readOnly = true) - return r.iterator(false) - } catch { - case e: Exception => - // do nothing + if (maxVersion > 0) { + loadIntoRocksDB(maxVersion) + val r: RocksDbInstance = + new RocksDbInstance(keySchema, valueSchema, maxVersion.toString, rocksDbConf) + try { + r.open(rocksDbPath, readOnly = true) + return r.iterator(false) + } catch { + case e: Exception => + logWarning(s"Exception ${e.getMessage} while getting latest Iterator") + } } } Iterator.empty } - // making it public for unit tests - private[sql] lazy val rocksDbPath: String = { - val checkpointRootLocationPath = new Path(stateStoreId.checkpointRootLocation) - val basePath = new Path( - localDirectory, - new Path( - "db", - checkpointRootLocationPath.getName + "_" + checkpointRootLocationPath.hashCode())) - - val dir = basePath.toString + Path.SEPARATOR + - stateStoreId_.operatorId + Path.SEPARATOR + - stateStoreId_.partitionId - - val f: File = new File(dir) + private[sql] def getLocalDirectory: String = localDirectory - if (!f.exists()) { - logInfo(s"creating rocksDb directory at : $dir") - f.mkdirs() - } - dir + private[sql] lazy val rocksDbPath: String = { + getPath("db") } private def getBackupPath(version: Long): String = { - val checkpointRootLocationPath = new Path(stateStoreId.checkpointRootLocation) - - val basePath = new Path( - localDirectory, - new Path( - "backup", - checkpointRootLocationPath.getName + "_" + checkpointRootLocationPath.hashCode())) - - val dir = basePath.toString + Path.SEPARATOR + - stateStoreId_.operatorId + Path.SEPARATOR + - stateStoreId_.partitionId - - val f: File = new File(dir) - - if (!f.exists()) { - logInfo(s"creating rocksDb directory at : $dir") - f.mkdirs() - } - - dir + Path.SEPARATOR + version + getPath("backup", version.toString) } private def getTempPath(version: Long): String = { + getPath("tmp", version.toString) + } + + private def getPath(parentFolderName: String, version: String = null): String = { val checkpointRootLocationPath = new Path(stateStoreId.checkpointRootLocation) val basePath = new Path( localDirectory, new Path( - "tmp", + parentFolderName, checkpointRootLocationPath.getName + "_" + checkpointRootLocationPath.hashCode())) - val dir = basePath.toString + Path.SEPARATOR + - stateStoreId_.operatorId + Path.SEPARATOR + - stateStoreId_.partitionId - - val f: File = new File(dir) + val dirPath = new Path( + basePath, + new Path(stateStoreId_.operatorId.toString, stateStoreId_.partitionId.toString)) + val f: File = new File(dirPath.toString) if (!f.exists()) { - logInfo(s"creating rocksDb directory at : $dir") + logInfo(s"creating rocksDb directory at : ${dirPath.toString}") f.mkdirs() } - dir + Path.SEPARATOR + version + val path = if (version != null) { + new Path(dirPath, version) + } else { + dirPath + } + path.toString } - // making it public for unit tests - def getLocalDirectory: String = localDirectory -} - -object RocksDbStateStoreProvider { - - val ROCKS_DB_BASE_PATH: String = "/media/ephemeral0/spark/rocksdb" - } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/WALUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/WALUtils.scala index ace6a31608d3..589783b9e420 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/WALUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/WALUtils.scala @@ -35,7 +35,7 @@ import org.apache.spark.sql.types.StructType object WALUtils { case class StoreFile(version: Long, path: Path, isSnapshot: Boolean) - val EOF_MARKER = -1 + private val EOF_MARKER = -1 /** Files needed to recover the given version of the store */ def filesForVersion(allFiles: Seq[StoreFile], version: Long): Seq[StoreFile] = { From f983d7831d07f9e0046345d28d645da69b783605 Mon Sep 17 00:00:00 2001 From: Vikram Agrawal Date: Wed, 31 Jul 2019 15:19:53 +0530 Subject: [PATCH 11/26] minor change --- .../execution/streaming/state/RocksDbStateStoreProvider.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreProvider.scala index 54b67ff7935b..1596397407a5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreProvider.scala @@ -391,7 +391,7 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log } private def getLastCommittedVersion(): Long = { - val f = new File(rocksDbPath, "commit") + val f = new File(rocksDbPath, RocksDbInstance.COMMIT_FILE_NAME) if (f.exists()) { try { val fileContents = scala.io.Source.fromFile(f.getAbsolutePath).getLines.mkString From d41560a6daeadd23f168e5f29114e48287f9d16a Mon Sep 17 00:00:00 2001 From: Vikram Agrawal Date: Wed, 31 Jul 2019 15:45:07 +0530 Subject: [PATCH 12/26] fix dependencies issue --- dev/deps/spark-deps-hadoop-2.7 | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 48eea7f108de..d201323a62cf 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -171,8 +171,8 @@ parquet-hadoop-bundle-1.6.0.jar parquet-jackson-1.10.1.jar protobuf-java-2.5.0.jar py4j-0.10.8.1.jar -rocksdbjni-6.0.1.jar pyrolite-4.30.jar +rocksdbjni-6.0.1.jar scala-compiler-2.12.8.jar scala-library-2.12.8.jar scala-parser-combinators_2.12-1.1.0.jar From ebaea37b1681bbbbd40db539f82e5a1a22b45a15 Mon Sep 17 00:00:00 2001 From: Vikram Agrawal Date: Wed, 31 Jul 2019 17:35:00 +0530 Subject: [PATCH 13/26] Fix documentation issues --- .../main/java/org/apache/spark/io/FileUtility.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/main/java/org/apache/spark/io/FileUtility.java b/core/src/main/java/org/apache/spark/io/FileUtility.java index 9a07be9c1b4b..3edf784b6346 100644 --- a/core/src/main/java/org/apache/spark/io/FileUtility.java +++ b/core/src/main/java/org/apache/spark/io/FileUtility.java @@ -24,10 +24,10 @@ public class FileUtility { /** * Extract an input tar file into an output files and directories. - * @param inputTarFileLoc the input file location for the tar file - * @param destDirLoc destination for the extracted files + * inputTarFileLoc: the input file location for the tar file + * destDirLoc: destination for the extracted files * - * @throws IllegalStateException + * throws IllegalStateException */ public static final String ENCODING = "utf-8"; @@ -71,10 +71,10 @@ public static void extractTarFile(String inputTarFileLoc, String destDirLoc) /** * create a tar file for input source directory location . - * @param source the source directory location - * @param destFileLoc destination of the created tarball + * source: the source directory location + * destFileLoc: destination of the created tarball * - * @throws IllegalStateException + * throws IllegalStateException */ public static void createTarFile(String source, String destFileLoc) From 818f716eac8d18e2de8389ddfd4ca7505cb83be5 Mon Sep 17 00:00:00 2001 From: Vikram Agrawal Date: Wed, 31 Jul 2019 17:44:02 +0530 Subject: [PATCH 14/26] minor changes --- .../state/RocksDbStateStoreProvider.scala | 20 +++++++++---------- .../state/RocksDbStateStoreSuite.scala | 8 -------- 2 files changed, 10 insertions(+), 18 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreProvider.scala index 1596397407a5..0275d143cb8b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreProvider.scala @@ -141,7 +141,7 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log verify(state == UPDATING, s"Cannot commit after already committed or aborted") try { synchronized { - rocksDbWriteInstance.commit(Some(getBackupPath(newVersion))) + rocksDbWriteInstance.commit(Some(getCheckpointPath(newVersion))) finalizeDeltaFile(compressedStream) } state = COMMITTED @@ -208,7 +208,7 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log if (version == 0) { Iterator.empty } else { - val path = getBackupPath(version) + val path = getCheckpointPath(version) val r: RocksDbInstance = new RocksDbInstance(keySchema, valueSchema, version.toString, rocksDbConf) r.open(path, readOnly = true) @@ -217,7 +217,7 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log case COMMITTED => logDebug(s"state = committed using check-pointed DB with version $newVersion") // use check-pointed db for current updated version - val path = getBackupPath(newVersion) + val path = getCheckpointPath(newVersion) val r: RocksDbInstance = new RocksDbInstance(keySchema, valueSchema, newVersion.toString, rocksDbConf) r.open(path, readOnly = true) @@ -460,7 +460,7 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log sparkConf) logInfo(s"Read delta file for version $version of $this from $fileToRead") } - rocksDbWriteInstance.commit(Some(getBackupPath(version))) + rocksDbWriteInstance.commit(Some(getCheckpointPath(version))) } catch { case e: Exception => logError(s"Exception while loading state ${e.getMessage}") @@ -495,7 +495,7 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log val deltaFilesForLastVersion = filesForVersion(files, lastVersion).filter(_.isSnapshot == false) if (deltaFilesForLastVersion.size > storeConf.minDeltasForSnapshot) { - val dbPath = getBackupPath(lastVersion) + val dbPath = getCheckpointPath(lastVersion) val snapShotFileName = s"{getTempPath(lastVersion)}.snapshot" val f = new File(snapShotFileName) try { @@ -555,9 +555,9 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log val earliestVersionToRetain = files.last.version - storeConf.minVersionsToRetain if (earliestVersionToRetain > 0) { for (v <- (earliestVersionToRetain - 1) to 1 by -1) { - // Destroy the backup path - logDebug(s"Destroying backup version = $v") - RocksDbInstance.destroyDB(getBackupPath(v)) + // Destroy the checkpointed path + logDebug(s"Destroying checkpoint version = $v") + RocksDbInstance.destroyDB(getCheckpointPath(v)) } } } @@ -593,8 +593,8 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log getPath("db") } - private def getBackupPath(version: Long): String = { - getPath("backup", version.toString) + private def getCheckpointPath(version: Long): String = { + getPath("checkpoint", version.toString) } private def getTempPath(version: Long): String = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreSuite.scala index d444aff1c017..a6b43f9c0c4a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreSuite.scala @@ -290,8 +290,6 @@ class RocksDbStateStoreSuite sqlConf.setConfString( SQLConf.STATE_STORE_PROVIDER_CLASS.key, "org.apache.spark.sql.execution.streaming.state.RocksDbStateStoreProvider") - val localdir = Utils.createTempDir().getAbsoluteFile.toString - sqlConf.setConfString("spark.sql.streaming.stateStore.rocksDb.localDirectory", localdir) val storeConf = new StateStoreConf(sqlConf) assert( storeConf.providerClass === @@ -356,9 +354,6 @@ class RocksDbStateStoreSuite SQLConf.STATE_STORE_PROVIDER_CLASS.key, "org.apache.spark.sql.execution.streaming.state.RocksDbStateStoreProvider") sqlConf.setConf(SQLConf.MIN_BATCHES_TO_RETAIN, 2) - sqlConf.setConfString( - "spark.sql.streaming.stateStore.rocksDb.localDirectory", - Utils.createTempDir().getAbsoluteFile.toString) val storeConf = StateStoreConf(sqlConf) val hadoopConf = new Configuration() val provider = newStoreProvider(storeProviderId.storeId) @@ -472,9 +467,6 @@ class RocksDbStateStoreSuite spark.conf.set( SQLConf.STATE_STORE_PROVIDER_CLASS.key, "org.apache.spark.sql.execution.streaming.state.RocksDbStateStoreProvider") - spark.conf.set( - "spark.sql.streaming.stateStore.rocksDb.localDirectory", - Utils.createTempDir().getAbsoluteFile.toString) import spark.implicits._ val inputData = MemoryStream[Int] From 603958be666a35794c04fee4878b6628f2357ed1 Mon Sep 17 00:00:00 2001 From: Vikram Agrawal Date: Wed, 7 Aug 2019 16:59:34 +0530 Subject: [PATCH 15/26] List directories instead of using a loop to clean up checkpointed rocksdb instances --- .../state/RocksDbStateStoreProvider.scala | 29 +++++++++++++------ 1 file changed, 20 insertions(+), 9 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreProvider.scala index 0275d143cb8b..339bb84c3f0c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreProvider.scala @@ -482,7 +482,7 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log logDebug(s"fetchFiles() took $e1 ms.") doSnapshot(files) cleanup(files) - cleanRocksDBBackupInstances(files) + cleanRocksDBCheckpoints(files) } catch { case NonFatal(e) => logWarning(s"Error performing snapshot and cleaning up $this") @@ -549,18 +549,29 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log } } - private def cleanRocksDBBackupInstances(files: Seq[WALUtils.StoreFile]): Unit = { + private def cleanRocksDBCheckpoints(files: Seq[WALUtils.StoreFile]): Unit = { try { - if (files.nonEmpty) { - val earliestVersionToRetain = files.last.version - storeConf.minVersionsToRetain - if (earliestVersionToRetain > 0) { - for (v <- (earliestVersionToRetain - 1) to 1 by -1) { - // Destroy the checkpointed path - logDebug(s"Destroying checkpoint version = $v") - RocksDbInstance.destroyDB(getCheckpointPath(v)) + val (_, e2) = Utils.timeTakenMs { + if (files.nonEmpty) { + val earliestVersionToRetain = files.last.version - storeConf.minVersionsToRetain + if (earliestVersionToRetain > 0) { + new File(getCheckpointPath(earliestVersionToRetain)).getParentFile + .listFiles(new FileFilter { + def accept(f: File): Boolean = { + try { + f.getName.toLong < earliestVersionToRetain + } catch { + case _: NumberFormatException => false + } + } + }) + .foreach(p => RocksDbInstance.destroyDB(p.getAbsolutePath)) + logInfo( + s"Deleted rocksDB checkpoints older than ${earliestVersionToRetain} for $this: ") } } } + logDebug(s"deleting rocksDB checkpoints took $e2 ms.") } catch { case NonFatal(e) => logWarning(s"Error cleaning up files for $this", e) } From 562f7551425ed5d30545b223089b3a7384d09b46 Mon Sep 17 00:00:00 2001 From: Vikram Agrawal Date: Wed, 28 Aug 2019 20:16:42 +0530 Subject: [PATCH 16/26] Address Review Comments --- .../java/org/apache/spark/io/FileUtility.java | 20 ++--- pom.xml | 8 +- sql/core/pom.xml | 6 -- .../streaming/state/RocksDbInstance.scala | 9 +- .../state/RocksDbStateStoreProvider.scala | 84 ++++++++++++------- .../execution/streaming/state/WALUtils.scala | 8 +- .../state/RocksDbStateStoreSuite.scala | 6 +- 7 files changed, 81 insertions(+), 60 deletions(-) diff --git a/core/src/main/java/org/apache/spark/io/FileUtility.java b/core/src/main/java/org/apache/spark/io/FileUtility.java index 3edf784b6346..5e21c6218b2f 100644 --- a/core/src/main/java/org/apache/spark/io/FileUtility.java +++ b/core/src/main/java/org/apache/spark/io/FileUtility.java @@ -32,19 +32,19 @@ public class FileUtility { public static final String ENCODING = "utf-8"; public static void extractTarFile(String inputTarFileLoc, String destDirLoc) - throws IllegalStateException { + throws IllegalStateException { File inputFile = new File(inputTarFileLoc); if (!inputTarFileLoc.endsWith(".tar")) { throw new IllegalStateException(String.format( - "Input File[%s] should end with tar extension.", inputTarFileLoc)); + "Input File[%s] should end with tar extension.", inputTarFileLoc)); } File destDir = new File(destDirLoc); if (destDir.exists() && !destDir.delete()) { throw new IllegalStateException(String.format( - "Couldn't delete the existing destination directory[%s] ", destDirLoc)); + "Couldn't delete the existing destination directory[%s] ", destDirLoc)); } else if (!destDir.mkdir()) { throw new IllegalStateException(String.format( - "Couldn't create directory %s ", destDirLoc)); + "Couldn't create directory %s ", destDirLoc)); } try (InputStream is = new FileInputStream(inputFile); @@ -55,7 +55,7 @@ public static void extractTarFile(String inputTarFileLoc, String destDirLoc) if (entry.isDirectory()) { if (!outputFile.exists() && !outputFile.mkdirs()) { throw new IllegalStateException(String.format( - "Couldn't create directory %s.", outputFile.getAbsolutePath())); + "Couldn't create directory %s.", outputFile.getAbsolutePath())); } } else { try (OutputStream outputFileStream = new FileOutputStream(outputFile)) { @@ -65,7 +65,7 @@ public static void extractTarFile(String inputTarFileLoc, String destDirLoc) } } catch (IOException e){ throw new IllegalStateException(String.format( - "extractTarFile failed with exception %s.", e.getMessage())); + "extractTarFile failed with exception %s.", e.getMessage())); } } @@ -78,16 +78,16 @@ public static void extractTarFile(String inputTarFileLoc, String destDirLoc) */ public static void createTarFile(String source, String destFileLoc) - throws IllegalStateException { + throws IllegalStateException { File f = new File(destFileLoc); if (f.exists() && !f.delete()) { throw new IllegalStateException(String.format( - "Couldn't delete the destination file location[%s]", destFileLoc)); + "Couldn't delete the destination file location[%s]", destFileLoc)); } File folder = new File(source); if (!folder.exists()) { throw new IllegalStateException(String.format( - "Source folder[%s] does not exist", source)); + "Source folder[%s] does not exist", source)); } try (FileOutputStream fos = new FileOutputStream(destFileLoc); @@ -105,7 +105,7 @@ public static void createTarFile(String source, String destFileLoc) tarOs.finish(); } catch (IOException e) { throw new IllegalStateException(String.format( - "createTarFile failed with exception %s.", e.getMessage())); + "createTarFile failed with exception %s.", e.getMessage())); } } diff --git a/pom.xml b/pom.xml index 5cba3faedb42..18110444ce54 100644 --- a/pom.xml +++ b/pom.xml @@ -190,7 +190,7 @@ 1.1 2.52.0 2.22 - 6.0.1 + 6.2.2 @@ -335,6 +335,12 @@ 4.14 + + org.rocksdb + rocksdbjni + ${rocksdb.version} + + diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 9029d4d1ea57..02ed6f8adaa6 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -147,12 +147,6 @@ mockito-core test - - - org.rocksdb - rocksdbjni - ${rocksdb.version} - target/scala-${scala.binary.version}/classes diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbInstance.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbInstance.scala index e161e0b5ee31..b96da533a2da 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbInstance.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbInstance.scala @@ -31,6 +31,7 @@ import org.rocksdb.RocksDB import org.apache.spark.SparkEnv import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions.UnsafeRow +import org.apache.spark.sql.execution.streaming.state.RocksDbStateStoreProvider.ROCKS_DB_STATE_STORE_CONF_PREFIX import org.apache.spark.sql.types.StructType import org.apache.spark.util.Utils @@ -197,19 +198,19 @@ class RocksDbInstance( private val dataBlockSize = conf .getOrElse( - "spark.sql.streaming.stateStore.rocksDb.blockSizeInKB".toLowerCase(Locale.ROOT), + s"${ROCKS_DB_STATE_STORE_CONF_PREFIX}.blockSizeInKB".toLowerCase(Locale.ROOT), "32") .toInt private val memTableMemoryBudget = conf .getOrElse( - "spark.sql.streaming.stateStore.rocksDb.memtableBudgetInMB".toLowerCase(Locale.ROOT), + s"${ROCKS_DB_STATE_STORE_CONF_PREFIX}.memtableBudgetInMB".toLowerCase(Locale.ROOT), "1024") .toInt private val enableStats = conf .getOrElse( - "spark.sql.streaming.stateStore.rocksDb.enableDbStats".toLowerCase(Locale.ROOT), + s"${ROCKS_DB_STATE_STORE_CONF_PREFIX}.enableDbStats".toLowerCase(Locale.ROOT), "false") .toBoolean @@ -427,7 +428,7 @@ object RocksDbInstance { private val rocksDbCacheSizeInMB: Int = if (SparkEnv.get != null) { SparkEnv.get.conf.getInt( - "spark.sql.streaming.stateStore.rocksDb.cacheSizeInMB", + s"${ROCKS_DB_STATE_STORE_CONF_PREFIX}.cacheSizeInMB", DEFAULT_ROCKSDB_CACHE_SIZE_IN_MB) } else { DEFAULT_ROCKSDB_CACHE_SIZE_IN_MB diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreProvider.scala index 339bb84c3f0c..a64f71db3abf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreProvider.scala @@ -22,6 +22,7 @@ import java.util import java.util.Locale import scala.collection.JavaConverters._ +import scala.io.Source import scala.util.control.NonFatal import org.apache.commons.io.FileUtils @@ -36,6 +37,30 @@ import org.apache.spark.sql.execution.streaming.CheckpointFileManager import org.apache.spark.sql.types.StructType import org.apache.spark.util.Utils +/** + * An implementation of [[StateStoreProvider]] and [[StateStore]] using RocksDB as the storage + * engine. In RocksDB, new writes are inserted into a memtable which is flushed into local storage + * when the memtable fills up. It improves scalability as compared to + * [[HDFSBackedStateStoreProvider]] since now the state data which was large enough to fit in the + * executor memory can be written into the combination of memtable and local storage.The data is + * backed in a HDFS-compatible file system just like [[HDFSBackedStateStoreProvider]] + * + * Fault-tolerance model: + * - Every set of updates is written to a delta file before committing. + * - The state store is responsible for managing, collapsing and cleaning up of delta files. + * - Updates are committed in the db atomically + * + * Backup Model: + * - Delta file is written in a HDFS-compatible file system on batch commit + * - RocksDB state is check-pointed into a separate folder on batch commit + * - Maintenance thread periodically takes a snapshot of the latest check-pointed version of + * rocksDB state which is written to a HDFS-compatible file system. + * + * Isolation Guarantee: + * - writes are committed in the transaction. + * - writer thread which started the transaction can read all un-committed updates + * - any other reader thread cannot read any un-committed updates + */ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Logging { /* Internal fields and methods */ @@ -45,7 +70,7 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log @volatile private var storeConf: StateStoreConf = _ @volatile private var hadoopConf: Configuration = _ @volatile private var numberOfVersionsToRetain: Int = _ - @volatile private var localDirectory: String = _ + @volatile private var localDir: String = _ /* * Additional configurations related to rocksDb. This will capture all configs in @@ -60,6 +85,7 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log private case class StoreFile(version: Long, path: Path, isSnapshot: Boolean) import WALUtils._ + import RocksDbStateStoreProvider._ /** Implementation of [[StateStore]] API which is backed by RocksDB and HDFS */ class RocksDbStateStore(val version: Long) extends StateStore with Logging { @@ -102,8 +128,8 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log newVersion.toString, rocksDbConf) rocksDbWriteInstance.open(rocksDbPath) - state = UPDATING rocksDbWriteInstance.startTransactions() + state = UPDATING } } @@ -114,7 +140,7 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log override def put(key: UnsafeRow, value: UnsafeRow): Unit = { initTransaction() - verify(state == UPDATING, s"Cannot put after already committed or aborted") + require(state == UPDATING, s"Cannot put after already committed or aborted") val keyCopy = key.copy() val valueCopy = value.copy() rocksDbWriteInstance.put(keyCopy, valueCopy) @@ -123,7 +149,7 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log override def remove(key: UnsafeRow): Unit = { initTransaction() - verify(state == UPDATING, "Cannot remove after already committed or aborted") + require(state == UPDATING, "Cannot remove after already committed or aborted") rocksDbWriteInstance.remove(key) writeRemoveToDeltaFile(compressedStream, key) } @@ -131,14 +157,14 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log override def getRange( start: Option[UnsafeRow], end: Option[UnsafeRow]): Iterator[UnsafeRowPair] = { - verify(state == UPDATING, "Cannot getRange after already committed or aborted") + require(state == UPDATING, "Cannot getRange after already committed or aborted") iterator() } /** Commit all the updates that have been made to the store, and return the new version. */ override def commit(): Long = { initTransaction() - verify(state == UPDATING, s"Cannot commit after already committed or aborted") + require(state == UPDATING, s"Cannot commit after already committed or aborted") try { synchronized { rocksDbWriteInstance.commit(Some(getCheckpointPath(newVersion))) @@ -279,13 +305,13 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log this.numberOfVersionsToRetain = storeConfs.maxVersionsToRetainInMemory fm.mkdirs(baseDir) this.rocksDbConf = storeConf.confs - .filter(_._1.startsWith("spark.sql.streaming.stateStore.rocksDb")) + .filter(_._1.startsWith(ROCKS_DB_STATE_STORE_CONF_PREFIX)) .map { case (k, v) => (k.toLowerCase(Locale.ROOT), v) } - this.localDirectory = this.rocksDbConf + this.localDir = this.rocksDbConf .getOrElse( - "spark.sql.streaming.stateStore.rocksDb.localDirectory".toLowerCase(Locale.ROOT), + s"${ROCKS_DB_STATE_STORE_CONF_PREFIX}.localDir".toLowerCase(Locale.ROOT), Utils.createTempDir().getAbsoluteFile.toString) } @@ -394,7 +420,7 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log val f = new File(rocksDbPath, RocksDbInstance.COMMIT_FILE_NAME) if (f.exists()) { try { - val fileContents = scala.io.Source.fromFile(f.getAbsolutePath).getLines.mkString + val fileContents = Source.fromFile(f.getAbsolutePath).getLines.mkString return fileContents.toLong } catch { case e: Exception => @@ -598,45 +624,45 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log Iterator.empty } - private[sql] def getLocalDirectory: String = localDirectory + private[sql] def getLocalDir: String = localDir private[sql] lazy val rocksDbPath: String = { getPath("db") } private def getCheckpointPath(version: Long): String = { - getPath("checkpoint", version.toString) + getPath("checkpoint", Some(version.toString)) } private def getTempPath(version: Long): String = { - getPath("tmp", version.toString) + getPath("tmp", Some(version.toString)) } - private def getPath(parentFolderName: String, version: String = null): String = { + private def getPath(subFolderName: String, version: Option[String] = None): String = { val checkpointRootLocationPath = new Path(stateStoreId.checkpointRootLocation) - val basePath = new Path( - localDirectory, - new Path( - parentFolderName, - checkpointRootLocationPath.getName + "_" + checkpointRootLocationPath.hashCode())) - val dirPath = new Path( - basePath, - new Path(stateStoreId_.operatorId.toString, stateStoreId_.partitionId.toString)) + localDir, + new Path( + new Path( + subFolderName, + checkpointRootLocationPath.getName + "_" + checkpointRootLocationPath.hashCode()), + new Path(stateStoreId_.operatorId.toString, stateStoreId_.partitionId.toString))) val f: File = new File(dirPath.toString) - if (!f.exists()) { - logInfo(s"creating rocksDb directory at : ${dirPath.toString}") - f.mkdirs() + if (!f.exists() && !f.mkdirs()) { + throw new IllegalStateException(s"Couldn't create directory ${dirPath.toString}") } - val path = if (version != null) { - new Path(dirPath, version) + if (version.isEmpty) { + dirPath.toString } else { - dirPath + new Path(dirPath, version.get).toString } - path.toString } } + +object RocksDbStateStoreProvider { + val ROCKS_DB_STATE_STORE_CONF_PREFIX = "spark.sql.streaming.stateStore.rocksDb" +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/WALUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/WALUtils.scala index 589783b9e420..1da7e8d8bbee 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/WALUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/WALUtils.scala @@ -51,7 +51,7 @@ object WALUtils { val deltaFiles = allFiles.filter { file => file.version > snapshotFile.version && file.version <= version }.toList - verify( + require( deltaFiles.size == version - snapshotFile.version, s"Unexpected list of delta files for version $version for $this: $deltaFiles") deltaFiles @@ -272,10 +272,4 @@ object WALUtils { new Path(baseDir, s"$version.snapshot") } - def verify(condition: => Boolean, msg: String): Unit = { - if (!condition) { - throw new IllegalStateException(msg) - } - } - } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreSuite.scala index a6b43f9c0c4a..435b5830c346 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreSuite.scala @@ -143,7 +143,7 @@ class RocksDbStateStoreSuite } // New updates to the reloaded store with new version, and does not change old version - val reloadedProvider = newStoreProvider(store.id, provider.getLocalDirectory) + val reloadedProvider = newStoreProvider(store.id, provider.getLocalDir) val reloadedStore = reloadedProvider.getStore(1) put(reloadedStore, "c", 4) assert(reloadedStore.commit() === 2) @@ -536,7 +536,7 @@ class RocksDbStateStoreSuite override def getData( provider: RocksDbStateStoreProvider, version: Int = -1): Set[(String, Int)] = { - val reloadedProvider = newStoreProvider(provider.stateStoreId, provider.getLocalDirectory) + val reloadedProvider = newStoreProvider(provider.stateStoreId, provider.getLocalDir) if (version < 0) { reloadedProvider.latestIterator().map(rowsToStringInt).toSet } else { @@ -556,7 +556,7 @@ class RocksDbStateStoreSuite sqlConf.setConf(SQLConf.STATE_STORE_MIN_DELTAS_FOR_SNAPSHOT, minDeltasForSnapshot) sqlConf.setConf(SQLConf.MAX_BATCHES_TO_RETAIN_IN_MEMORY, numOfVersToRetainInMemory) sqlConf.setConf(SQLConf.MIN_BATCHES_TO_RETAIN, 2) - sqlConf.setConfString("spark.sql.streaming.stateStore.rocksDb.localDirectory", localDir) + sqlConf.setConfString("spark.sql.streaming.stateStore.rocksDb.localDir", localDir) val provider = new RocksDbStateStoreProvider provider.init( StateStoreId(dir, opId, partition), From 4f420686539c8cb367ced78650b8fc63fb3524d1 Mon Sep 17 00:00:00 2001 From: Vikram Agrawal Date: Wed, 28 Aug 2019 20:22:55 +0530 Subject: [PATCH 17/26] Unit tests for FileUtility --- .../org/apache/spark/io/FileUtilitySuite.java | 85 +++++++++++++++++++ 1 file changed, 85 insertions(+) create mode 100644 core/src/test/java/org/apache/spark/io/FileUtilitySuite.java diff --git a/core/src/test/java/org/apache/spark/io/FileUtilitySuite.java b/core/src/test/java/org/apache/spark/io/FileUtilitySuite.java new file mode 100644 index 000000000000..2cfcbf760ec7 --- /dev/null +++ b/core/src/test/java/org/apache/spark/io/FileUtilitySuite.java @@ -0,0 +1,85 @@ +/* + * 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.io; + +import org.apache.commons.io.FileUtils; +import org.apache.commons.lang3.RandomUtils; +import org.apache.hadoop.fs.Path; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import sun.security.action.GetPropertyAction; + +import java.io.File; +import java.io.IOException; + +/** + * Tests functionality of {@link FileUtility} + */ +public class FileUtilitySuite { + + protected File sourceFolder; + protected File destFile; + protected File destTarLoc; + + @Before + public void setUp() throws IOException { + File tmpLocation = new File( + GetPropertyAction.privilegedGetProperty("java.io.tmpdir")); + Path sourceFolderPath = new Path(tmpLocation.toString(), + "FileUtilTest" + RandomUtils.nextLong()); + sourceFolder = new File(sourceFolderPath.toString()); + sourceFolder.mkdirs(); + destTarLoc = File.createTempFile("dest-tar", ".tar"); + destFile = File.createTempFile("dest-file", ".tmp"); + } + + @After + public void tearDown() { + sourceFolder.delete(); + destTarLoc.delete(); + destFile.delete(); + } + + @Test + public void testCreationAndExtraction() throws IllegalStateException, IOException { + // Create a temp file in the source folder + Assert.assertEquals(sourceFolder.listFiles().length , 0); + File inputFile = File.createTempFile("source-file", ".tmp", sourceFolder); + // Create a byte array of size 1 KB with random bytes + byte[] randomBytes = RandomUtils.nextBytes(1 * 1024); + FileUtils.writeByteArrayToFile(inputFile, randomBytes); + + // Create the tarball + destTarLoc.delete(); + Assert.assertFalse(destTarLoc.exists()); + FileUtility.createTarFile(sourceFolder.toString(), destTarLoc.getAbsolutePath()); + Assert.assertTrue(destTarLoc.exists()); + + // Extract the tarball + String destFilePath = destFile.getAbsolutePath(); + destFile.delete(); + Assert.assertFalse(destFile.exists()); + FileUtility.extractTarFile(destTarLoc.getAbsolutePath(), destFilePath); + + Assert.assertTrue(destFile.exists()); + Assert.assertEquals(destFile.listFiles().length , 1); + Assert.assertArrayEquals(randomBytes, FileUtils.readFileToByteArray(destFile.listFiles()[0])); + } + +} From 7d4d5d15834051842b8c17e246ad2688d67646fd Mon Sep 17 00:00:00 2001 From: Vikram Agrawal Date: Wed, 28 Aug 2019 20:29:57 +0530 Subject: [PATCH 18/26] Update spark dependencies --- dev/deps/spark-deps-hadoop-2.7 | 1 - dev/deps/spark-deps-hadoop-3.2 | 1 - 2 files changed, 2 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index d201323a62cf..82389ea6aa58 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -172,7 +172,6 @@ parquet-jackson-1.10.1.jar protobuf-java-2.5.0.jar py4j-0.10.8.1.jar pyrolite-4.30.jar -rocksdbjni-6.0.1.jar scala-compiler-2.12.8.jar scala-library-2.12.8.jar scala-parser-combinators_2.12-1.1.0.jar diff --git a/dev/deps/spark-deps-hadoop-3.2 b/dev/deps/spark-deps-hadoop-3.2 index 262ebd6af621..a02735d2be89 100644 --- a/dev/deps/spark-deps-hadoop-3.2 +++ b/dev/deps/spark-deps-hadoop-3.2 @@ -191,7 +191,6 @@ protobuf-java-2.5.0.jar py4j-0.10.8.1.jar pyrolite-4.30.jar re2j-1.1.jar -rocksdbjni-6.0.1.jar scala-compiler-2.12.8.jar scala-library-2.12.8.jar scala-parser-combinators_2.12-1.1.0.jar From 0b129f3de008e194a26f664cad1018766ff7d685 Mon Sep 17 00:00:00 2001 From: Vikram Agrawal Date: Wed, 28 Aug 2019 21:20:19 +0530 Subject: [PATCH 19/26] Revert POM dependency change --- pom.xml | 6 ------ sql/core/pom.xml | 5 +++++ 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/pom.xml b/pom.xml index 18110444ce54..d5d8ff93f94d 100644 --- a/pom.xml +++ b/pom.xml @@ -335,12 +335,6 @@ 4.14 - - org.rocksdb - rocksdbjni - ${rocksdb.version} - - diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 02ed6f8adaa6..2112543cc9b8 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -147,6 +147,11 @@ mockito-core test + + org.rocksdb + rocksdbjni + ${rocksdb.version} + target/scala-${scala.binary.version}/classes From c38bd6c031365e9e24d27a643ecbba7b63b2e0e2 Mon Sep 17 00:00:00 2001 From: Vikram Agrawal Date: Wed, 28 Aug 2019 21:34:28 +0530 Subject: [PATCH 20/26] Add dependencies --- dev/deps/spark-deps-hadoop-2.7 | 1 + dev/deps/spark-deps-hadoop-3.2 | 1 + 2 files changed, 2 insertions(+) diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 82389ea6aa58..9412dc650130 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -172,6 +172,7 @@ parquet-jackson-1.10.1.jar protobuf-java-2.5.0.jar py4j-0.10.8.1.jar pyrolite-4.30.jar +rocksdbjni-6.2.2.jar scala-compiler-2.12.8.jar scala-library-2.12.8.jar scala-parser-combinators_2.12-1.1.0.jar diff --git a/dev/deps/spark-deps-hadoop-3.2 b/dev/deps/spark-deps-hadoop-3.2 index a02735d2be89..edf5f2a414f3 100644 --- a/dev/deps/spark-deps-hadoop-3.2 +++ b/dev/deps/spark-deps-hadoop-3.2 @@ -191,6 +191,7 @@ protobuf-java-2.5.0.jar py4j-0.10.8.1.jar pyrolite-4.30.jar re2j-1.1.jar +rocksdbjni-6.2.2.jar scala-compiler-2.12.8.jar scala-library-2.12.8.jar scala-parser-combinators_2.12-1.1.0.jar From fcf2a862535ed36225684c4f40352b95e0fcab16 Mon Sep 17 00:00:00 2001 From: Vikram Agrawal Date: Wed, 28 Aug 2019 22:12:23 +0530 Subject: [PATCH 21/26] Fix fileutility unit test --- .../java/org/apache/spark/io/FileUtilitySuite.java | 10 ++-------- 1 file changed, 2 insertions(+), 8 deletions(-) diff --git a/core/src/test/java/org/apache/spark/io/FileUtilitySuite.java b/core/src/test/java/org/apache/spark/io/FileUtilitySuite.java index 2cfcbf760ec7..99fb5e7107f2 100644 --- a/core/src/test/java/org/apache/spark/io/FileUtilitySuite.java +++ b/core/src/test/java/org/apache/spark/io/FileUtilitySuite.java @@ -18,12 +18,11 @@ import org.apache.commons.io.FileUtils; import org.apache.commons.lang3.RandomUtils; -import org.apache.hadoop.fs.Path; +import org.apache.spark.util.Utils; import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Test; -import sun.security.action.GetPropertyAction; import java.io.File; import java.io.IOException; @@ -39,19 +38,14 @@ public class FileUtilitySuite { @Before public void setUp() throws IOException { - File tmpLocation = new File( - GetPropertyAction.privilegedGetProperty("java.io.tmpdir")); - Path sourceFolderPath = new Path(tmpLocation.toString(), + sourceFolder = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "FileUtilTest" + RandomUtils.nextLong()); - sourceFolder = new File(sourceFolderPath.toString()); - sourceFolder.mkdirs(); destTarLoc = File.createTempFile("dest-tar", ".tar"); destFile = File.createTempFile("dest-file", ".tmp"); } @After public void tearDown() { - sourceFolder.delete(); destTarLoc.delete(); destFile.delete(); } From fb86f0d61eb2881164e8be4b48236dada03b8a5a Mon Sep 17 00:00:00 2001 From: Vikram Agrawal Date: Thu, 29 Aug 2019 10:21:13 +0530 Subject: [PATCH 22/26] Minor Fixes in FileUtilitySuite --- .../org/apache/spark/io/FileUtilitySuite.java | 23 ++++++++----------- 1 file changed, 10 insertions(+), 13 deletions(-) diff --git a/core/src/test/java/org/apache/spark/io/FileUtilitySuite.java b/core/src/test/java/org/apache/spark/io/FileUtilitySuite.java index 99fb5e7107f2..9b904b4ded77 100644 --- a/core/src/test/java/org/apache/spark/io/FileUtilitySuite.java +++ b/core/src/test/java/org/apache/spark/io/FileUtilitySuite.java @@ -33,21 +33,20 @@ public class FileUtilitySuite { protected File sourceFolder; - protected File destFile; protected File destTarLoc; + protected File destFolder; @Before public void setUp() throws IOException { - sourceFolder = Utils.createTempDir(System.getProperty("java.io.tmpdir"), - "FileUtilTest" + RandomUtils.nextLong()); - destTarLoc = File.createTempFile("dest-tar", ".tar"); - destFile = File.createTempFile("dest-file", ".tmp"); + String tmpDir = System.getProperty("java.io.tmpdir"); + sourceFolder = Utils.createTempDir(tmpDir, "FileUtilTest-src-" + RandomUtils.nextLong()); + destFolder = Utils.createTempDir(tmpDir, "FileUtilTest-dest-" + RandomUtils.nextLong()); + destTarLoc= File.createTempFile("dest-tar", ".tar"); } @After public void tearDown() { destTarLoc.delete(); - destFile.delete(); } @Test @@ -66,14 +65,12 @@ public void testCreationAndExtraction() throws IllegalStateException, IOExceptio Assert.assertTrue(destTarLoc.exists()); // Extract the tarball - String destFilePath = destFile.getAbsolutePath(); - destFile.delete(); - Assert.assertFalse(destFile.exists()); - FileUtility.extractTarFile(destTarLoc.getAbsolutePath(), destFilePath); + Assert.assertEquals(destFolder.listFiles().length , 0); + FileUtility.extractTarFile(destTarLoc.getAbsolutePath(), destFolder.getAbsolutePath()); - Assert.assertTrue(destFile.exists()); - Assert.assertEquals(destFile.listFiles().length , 1); - Assert.assertArrayEquals(randomBytes, FileUtils.readFileToByteArray(destFile.listFiles()[0])); + Assert.assertTrue(destFolder.exists()); + Assert.assertEquals(destFolder.listFiles().length , 1); + Assert.assertArrayEquals(randomBytes, FileUtils.readFileToByteArray(destFolder.listFiles()[0])); } } From 4544abcce18ab7becc671a8c4ae55fea9708d087 Mon Sep 17 00:00:00 2001 From: Vikram Agrawal Date: Thu, 29 Aug 2019 15:03:34 +0530 Subject: [PATCH 23/26] Dummy commit to trigger the build --- core/src/test/java/org/apache/spark/io/FileUtilitySuite.java | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/test/java/org/apache/spark/io/FileUtilitySuite.java b/core/src/test/java/org/apache/spark/io/FileUtilitySuite.java index 9b904b4ded77..a9a21a302eca 100644 --- a/core/src/test/java/org/apache/spark/io/FileUtilitySuite.java +++ b/core/src/test/java/org/apache/spark/io/FileUtilitySuite.java @@ -68,6 +68,7 @@ public void testCreationAndExtraction() throws IllegalStateException, IOExceptio Assert.assertEquals(destFolder.listFiles().length , 0); FileUtility.extractTarFile(destTarLoc.getAbsolutePath(), destFolder.getAbsolutePath()); + // Verify that the extraction was successful Assert.assertTrue(destFolder.exists()); Assert.assertEquals(destFolder.listFiles().length , 1); Assert.assertArrayEquals(randomBytes, FileUtils.readFileToByteArray(destFolder.listFiles()[0])); From b3ef8ea6fa5734b28134c6d545c28b0ef01bc2d4 Mon Sep 17 00:00:00 2001 From: Vikram Agrawal Date: Tue, 24 Sep 2019 13:48:26 +0530 Subject: [PATCH 24/26] create RocksDbStateStoreConf for rocksdb related configs --- .../streaming/state/RocksDbInstance.scala | 58 +++------ .../state/RocksDbStateStoreConf.scala | 33 +++-- .../state/RocksDbStateStoreProvider.scala | 113 +++++++++--------- 3 files changed, 88 insertions(+), 116 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbInstance.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbInstance.scala index b96da533a2da..a9e2db2f0380 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbInstance.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbInstance.scala @@ -31,16 +31,15 @@ import org.rocksdb.RocksDB import org.apache.spark.SparkEnv import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions.UnsafeRow -import org.apache.spark.sql.execution.streaming.state.RocksDbStateStoreProvider.ROCKS_DB_STATE_STORE_CONF_PREFIX import org.apache.spark.sql.types.StructType import org.apache.spark.util.Utils class RocksDbInstance( - keySchema: StructType, - valueSchema: StructType, - version: String, - conf: Map[String, String] = Map.empty) - extends Logging { + keySchema: StructType, + valueSchema: StructType, + version: String, + conf: RocksDbStateStoreConf) + extends Logging { import RocksDbInstance._ RocksDB.loadLibrary() @@ -51,6 +50,7 @@ class RocksDbInstance( protected val writeOptions: WriteOptions = new WriteOptions() protected val table_options = new BlockBasedTableConfig protected val options: Options = new Options() + protected val rocksDbLRUCache = new LRUCache(conf.cacheSizeInMB * 1024 * 1024, 6, false) private def isOpen(): Boolean = { db != null @@ -114,6 +114,7 @@ class RocksDbInstance( options.close() readOptions.close() writeOptions.close() + rocksDbLRUCache.close() } } @@ -196,23 +197,9 @@ class RocksDbInstance( } } - private val dataBlockSize = conf - .getOrElse( - s"${ROCKS_DB_STATE_STORE_CONF_PREFIX}.blockSizeInKB".toLowerCase(Locale.ROOT), - "32") - .toInt - - private val memTableMemoryBudget = conf - .getOrElse( - s"${ROCKS_DB_STATE_STORE_CONF_PREFIX}.memtableBudgetInMB".toLowerCase(Locale.ROOT), - "1024") - .toInt - - private val enableStats = conf - .getOrElse( - s"${ROCKS_DB_STATE_STORE_CONF_PREFIX}.enableDbStats".toLowerCase(Locale.ROOT), - "false") - .toBoolean + private val dataBlockSize = conf.blockSizeInKB + private val memTableMemoryBudget = conf.memtableBudgetInMB + private val enableStats = conf.enableStats protected def setOptions(): Unit = { @@ -270,15 +257,15 @@ class RocksDbInstance( } class OptimisticTransactionDbInstance( + keySchema: StructType, + valueSchema: StructType, + version: String, + conf: RocksDbStateStoreConf) + extends RocksDbInstance( keySchema: StructType, valueSchema: StructType, version: String, - conf: Map[String, String] = Map.empty) - extends RocksDbInstance( - keySchema: StructType, - valueSchema: StructType, - version: String, - conf: Map[String, String]) { + conf: RocksDbStateStoreConf) { import RocksDbInstance._ RocksDB.loadLibrary() @@ -382,6 +369,7 @@ class OptimisticTransactionDbInstance( options.close() readOptions.close() writeOptions.close() + rocksDbLRUCache.close() } } @@ -424,18 +412,6 @@ object RocksDbInstance { val COMMIT_FILE_NAME = "commit" - private val DEFAULT_ROCKSDB_CACHE_SIZE_IN_MB = 512 - - private val rocksDbCacheSizeInMB: Int = if (SparkEnv.get != null) { - SparkEnv.get.conf.getInt( - s"${ROCKS_DB_STATE_STORE_CONF_PREFIX}.cacheSizeInMB", - DEFAULT_ROCKSDB_CACHE_SIZE_IN_MB) - } else { - DEFAULT_ROCKSDB_CACHE_SIZE_IN_MB - } - - lazy val rocksDbLRUCache = new LRUCache(rocksDbCacheSizeInMB * 1024 * 1024, 6, false) - def destroyDB(path: String): Unit = { val f: File = new File(path) val destroyOptions: Options = new Options() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreConf.scala index 6c0a649ae613..2089c9f63b89 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreConf.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.execution.streaming.state -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.util.Utils class RocksDbStateStoreConf(@transient private val stateStoreConf: StateStoreConf) @@ -26,26 +25,26 @@ class RocksDbStateStoreConf(@transient private val stateStoreConf: StateStoreCon private val DEFAULT_BLOCKSIZE_IN_KB = 32 private val DEFAULT_MEMTABLE_BUDGET_IN_MB = 1024 private val DEFAULT_CACHE_SIZE_IN_MB = 512 - def this() = this(StateStoreConf.empty) - val blockSizeInKB: Int = stateStoreConf.sqlConf - .getConf(SQLConf.ROCKSDB_STATE_STORE_DATA_BLOCK_SIZE) - .getOrElse(DEFAULT_BLOCKSIZE_IN_KB) + val BLOCK_SIZE_KEY = "spark.sql.streaming.stateStore.rocksDb.blockSizeInKB" + val MEMTABLE_BUDGET_KEY = "spark.sql.streaming.stateStore.rocksDb.memtableBudgetInMB" + val CACHE_SIZE_KEY = "spark.sql.streaming.stateStore.rocksDb.cacheSizeInMB" + val ENABLE_STATS_KEY = "spark.sql.streaming.stateStore.rocksDb.enableDbStats" + val LOCAL_DIR_KEY = "spark.sql.streaming.stateStore.rocksDb.localDir" + + val blockSizeInKB: Int = stateStoreConf.confs + .getOrElse(BLOCK_SIZE_KEY, DEFAULT_BLOCKSIZE_IN_KB.toString).toInt - val memtableBudgetInMB: Int = stateStoreConf.sqlConf - .getConf(SQLConf.ROCKSDB_STATE_STORE_MEMTABLE_BUDGET) - .getOrElse(DEFAULT_MEMTABLE_BUDGET_IN_MB) + val memtableBudgetInMB: Int = stateStoreConf.confs + .getOrElse(MEMTABLE_BUDGET_KEY, DEFAULT_MEMTABLE_BUDGET_IN_MB.toString).toInt - val cacheSizeInMB: Int = stateStoreConf.sqlConf - .getConf(SQLConf.ROCKSDB_STATE_STORE_CACHE_SIZE) - .getOrElse(DEFAULT_CACHE_SIZE_IN_MB) + val cacheSizeInMB: Int = stateStoreConf.confs + .getOrElse(CACHE_SIZE_KEY, DEFAULT_CACHE_SIZE_IN_MB.toString).toInt - val enableStats: Boolean = stateStoreConf.sqlConf - .getConf(SQLConf.ROCKSDB_STATE_STORE_ENABLE_STATS) - .getOrElse(false) + val enableStats: Boolean = stateStoreConf.confs + .getOrElse(ENABLE_STATS_KEY, "false").toBoolean - val localDir: String = stateStoreConf.sqlConf - .getConf(SQLConf.ROCKSDB_STATE_STORE_DATA_LOCAL_DIR) - .getOrElse(Utils.createTempDir().getAbsolutePath) + val localDir: String = stateStoreConf.confs + .getOrElse(LOCAL_DIR_KEY, Utils.createTempDir().getAbsolutePath) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreProvider.scala index a64f71db3abf..6f3ccb9e3b7b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreProvider.scala @@ -37,30 +37,30 @@ import org.apache.spark.sql.execution.streaming.CheckpointFileManager import org.apache.spark.sql.types.StructType import org.apache.spark.util.Utils -/** - * An implementation of [[StateStoreProvider]] and [[StateStore]] using RocksDB as the storage - * engine. In RocksDB, new writes are inserted into a memtable which is flushed into local storage - * when the memtable fills up. It improves scalability as compared to - * [[HDFSBackedStateStoreProvider]] since now the state data which was large enough to fit in the - * executor memory can be written into the combination of memtable and local storage.The data is - * backed in a HDFS-compatible file system just like [[HDFSBackedStateStoreProvider]] - * - * Fault-tolerance model: - * - Every set of updates is written to a delta file before committing. - * - The state store is responsible for managing, collapsing and cleaning up of delta files. - * - Updates are committed in the db atomically - * - * Backup Model: - * - Delta file is written in a HDFS-compatible file system on batch commit - * - RocksDB state is check-pointed into a separate folder on batch commit - * - Maintenance thread periodically takes a snapshot of the latest check-pointed version of - * rocksDB state which is written to a HDFS-compatible file system. - * - * Isolation Guarantee: - * - writes are committed in the transaction. - * - writer thread which started the transaction can read all un-committed updates - * - any other reader thread cannot read any un-committed updates - */ + /* + * An implementation of [[StateStoreProvider]] and [[StateStore]] using RocksDB as the storage + * engine. In RocksDB, new writes are inserted into a memtable which is flushed into local storage + * when the memtable fills up. It improves scalability as compared to + * [[HDFSBackedStateStoreProvider]] since now the state data which was large enough to fit in the + * executor memory can be written into the combination of memtable and local storage.The data is + * backed in a HDFS-compatible file system just like [[HDFSBackedStateStoreProvider]] + * + * Fault-tolerance model: + * - Every set of updates is written to a delta file before committing. + * - The state store is responsible for managing, collapsing and cleaning up of delta files. + * - Updates are committed in the db atomically + * + * Backup Model: + * - Delta file is written in a HDFS-compatible file system on batch commit + * - RocksDB state is check-pointed into a separate folder on batch commit + * - Maintenance thread periodically takes a snapshot of the latest check-pointed version of + * rocksDB state which is written to a HDFS-compatible file system. + * + * Isolation Guarantee: + * - writes are committed in the transaction. + * - writer thread which started the transaction can read all un-committed updates + * - any other reader thread cannot read any un-committed updates + */ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Logging { /* Internal fields and methods */ @@ -71,12 +71,7 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log @volatile private var hadoopConf: Configuration = _ @volatile private var numberOfVersionsToRetain: Int = _ @volatile private var localDir: 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] + @volatile private var rocksDbConf: RocksDbStateStoreConf = _ private lazy val baseDir: Path = stateStoreId.storeCheckpointLocation() private lazy val fm = CheckpointFileManager.create(baseDir, hadoopConf) @@ -85,7 +80,6 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log private case class StoreFile(version: Long, path: Path, isSnapshot: Boolean) import WALUtils._ - import RocksDbStateStoreProvider._ /** Implementation of [[StateStore]] API which is backed by RocksDB and HDFS */ class RocksDbStateStore(val version: Long) extends StateStore with Logging { @@ -155,8 +149,8 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log } override def getRange( - start: Option[UnsafeRow], - end: Option[UnsafeRow]): Iterator[UnsafeRowPair] = { + start: Option[UnsafeRow], + end: Option[UnsafeRow]): Iterator[UnsafeRowPair] = { require(state == UPDATING, "Cannot getRange after already committed or aborted") iterator() } @@ -236,7 +230,11 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log } else { val path = getCheckpointPath(version) val r: RocksDbInstance = - new RocksDbInstance(keySchema, valueSchema, version.toString, rocksDbConf) + new RocksDbInstance( + keySchema, + valueSchema, + version.toString, + rocksDbConf) r.open(path, readOnly = true) r.iterator(closeDbOnCompletion = true) } @@ -245,7 +243,11 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log // use check-pointed db for current updated version val path = getCheckpointPath(newVersion) val r: RocksDbInstance = - new RocksDbInstance(keySchema, valueSchema, newVersion.toString, rocksDbConf) + new RocksDbInstance( + keySchema, + valueSchema, + newVersion.toString, + rocksDbConf) r.open(path, readOnly = true) r.iterator(closeDbOnCompletion = true) @@ -291,12 +293,12 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log * to save state data */ override def init( - stateStoreId: StateStoreId, - keySchema: StructType, - valueSchema: StructType, - keyIndexOrdinal: Option[Int], // for sorting the data by their keys - storeConfs: StateStoreConf, - hadoopConf: Configuration): Unit = { + stateStoreId: StateStoreId, + keySchema: StructType, + valueSchema: StructType, + keyIndexOrdinal: Option[Int], // for sorting the data by their keys + storeConfs: StateStoreConf, + hadoopConf: Configuration): Unit = { this.stateStoreId_ = stateStoreId this.keySchema = keySchema this.valueSchema = valueSchema @@ -304,15 +306,8 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log this.hadoopConf = hadoopConf this.numberOfVersionsToRetain = storeConfs.maxVersionsToRetainInMemory fm.mkdirs(baseDir) - this.rocksDbConf = storeConf.confs - .filter(_._1.startsWith(ROCKS_DB_STATE_STORE_CONF_PREFIX)) - .map { - case (k, v) => (k.toLowerCase(Locale.ROOT), v) - } - this.localDir = this.rocksDbConf - .getOrElse( - s"${ROCKS_DB_STATE_STORE_CONF_PREFIX}.localDir".toLowerCase(Locale.ROOT), - Utils.createTempDir().getAbsoluteFile.toString) + this.rocksDbConf = new RocksDbStateStoreConf(storeConfs) + this.localDir = rocksDbConf.localDir } /* @@ -468,8 +463,11 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log private def applyDelta(version: Long, lastAvailableVersion: Long): Unit = { var rocksDbWriteInstance: OptimisticTransactionDbInstance = null try { - rocksDbWriteInstance = - new OptimisticTransactionDbInstance(keySchema, valueSchema, version.toString, rocksDbConf) + rocksDbWriteInstance = new OptimisticTransactionDbInstance( + keySchema, + valueSchema, + version.toString, + rocksDbConf) rocksDbWriteInstance.open(rocksDbPath) rocksDbWriteInstance.startTransactions() // Load all the deltas from the version after the last available @@ -611,7 +609,11 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log if (maxVersion > 0) { loadIntoRocksDB(maxVersion) val r: RocksDbInstance = - new RocksDbInstance(keySchema, valueSchema, maxVersion.toString, rocksDbConf) + new RocksDbInstance( + keySchema, + valueSchema, + maxVersion.toString, + rocksDbConf) try { r.open(rocksDbPath, readOnly = true) return r.iterator(false) @@ -660,9 +662,4 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log new Path(dirPath, version.get).toString } } - -} - -object RocksDbStateStoreProvider { - val ROCKS_DB_STATE_STORE_CONF_PREFIX = "spark.sql.streaming.stateStore.rocksDb" } From 417de569fd0f395334854395ce4bdf9e00570f1e Mon Sep 17 00:00:00 2001 From: Vikram Agrawal Date: Tue, 24 Sep 2019 14:55:10 +0530 Subject: [PATCH 25/26] cache should be singleton --- .../streaming/state/RocksDbInstance.scala | 13 +++++++--- .../state/RocksDbStateStoreConf.scala | 26 ++++++++++--------- 2 files changed, 24 insertions(+), 15 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbInstance.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbInstance.scala index a9e2db2f0380..39904fa2fed5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbInstance.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbInstance.scala @@ -50,7 +50,6 @@ class RocksDbInstance( protected val writeOptions: WriteOptions = new WriteOptions() protected val table_options = new BlockBasedTableConfig protected val options: Options = new Options() - protected val rocksDbLRUCache = new LRUCache(conf.cacheSizeInMB * 1024 * 1024, 6, false) private def isOpen(): Boolean = { db != null @@ -114,7 +113,6 @@ class RocksDbInstance( options.close() readOptions.close() writeOptions.close() - rocksDbLRUCache.close() } } @@ -369,7 +367,6 @@ class OptimisticTransactionDbInstance( options.close() readOptions.close() writeOptions.close() - rocksDbLRUCache.close() } } @@ -412,6 +409,16 @@ object RocksDbInstance { val COMMIT_FILE_NAME = "commit" + private val rocksDbCacheSizeInMB: Int = if (SparkEnv.get != null) { + SparkEnv.get.conf.getInt( + RocksDbStateStoreConf.CACHE_SIZE_KEY, + RocksDbStateStoreConf.DEFAULT_CACHE_SIZE_IN_MB) + } else { + RocksDbStateStoreConf.DEFAULT_CACHE_SIZE_IN_MB + } + + lazy val rocksDbLRUCache = new LRUCache(rocksDbCacheSizeInMB * 1024 * 1024, 6, false) + def destroyDB(path: String): Unit = { val f: File = new File(path) val destroyOptions: Options = new Options() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreConf.scala index 2089c9f63b89..906b6b166e0d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreConf.scala @@ -22,16 +22,9 @@ import org.apache.spark.util.Utils class RocksDbStateStoreConf(@transient private val stateStoreConf: StateStoreConf) extends Serializable { - private val DEFAULT_BLOCKSIZE_IN_KB = 32 - private val DEFAULT_MEMTABLE_BUDGET_IN_MB = 1024 - private val DEFAULT_CACHE_SIZE_IN_MB = 512 - def this() = this(StateStoreConf.empty) + import RocksDbStateStoreConf._ - val BLOCK_SIZE_KEY = "spark.sql.streaming.stateStore.rocksDb.blockSizeInKB" - val MEMTABLE_BUDGET_KEY = "spark.sql.streaming.stateStore.rocksDb.memtableBudgetInMB" - val CACHE_SIZE_KEY = "spark.sql.streaming.stateStore.rocksDb.cacheSizeInMB" - val ENABLE_STATS_KEY = "spark.sql.streaming.stateStore.rocksDb.enableDbStats" - val LOCAL_DIR_KEY = "spark.sql.streaming.stateStore.rocksDb.localDir" + def this() = this(StateStoreConf.empty) val blockSizeInKB: Int = stateStoreConf.confs .getOrElse(BLOCK_SIZE_KEY, DEFAULT_BLOCKSIZE_IN_KB.toString).toInt @@ -39,12 +32,21 @@ class RocksDbStateStoreConf(@transient private val stateStoreConf: StateStoreCon val memtableBudgetInMB: Int = stateStoreConf.confs .getOrElse(MEMTABLE_BUDGET_KEY, DEFAULT_MEMTABLE_BUDGET_IN_MB.toString).toInt - val cacheSizeInMB: Int = stateStoreConf.confs - .getOrElse(CACHE_SIZE_KEY, DEFAULT_CACHE_SIZE_IN_MB.toString).toInt - val enableStats: Boolean = stateStoreConf.confs .getOrElse(ENABLE_STATS_KEY, "false").toBoolean val localDir: String = stateStoreConf.confs .getOrElse(LOCAL_DIR_KEY, Utils.createTempDir().getAbsolutePath) } + +object RocksDbStateStoreConf { + val DEFAULT_BLOCKSIZE_IN_KB = 32 + val DEFAULT_MEMTABLE_BUDGET_IN_MB = 1024 + val DEFAULT_CACHE_SIZE_IN_MB = 512 + + val BLOCK_SIZE_KEY = "spark.sql.streaming.stateStore.rocksDb.blockSizeInKB" + val MEMTABLE_BUDGET_KEY = "spark.sql.streaming.stateStore.rocksDb.memtableBudgetInMB" + val CACHE_SIZE_KEY = "spark.sql.streaming.stateStore.rocksDb.cacheSizeInMB" + val ENABLE_STATS_KEY = "spark.sql.streaming.stateStore.rocksDb.enableDbStats" + val LOCAL_DIR_KEY = "spark.sql.streaming.stateStore.rocksDb.localDir" +} From 45e0d054a38958ac9e1b7c6a9429a3a3df9b8ff1 Mon Sep 17 00:00:00 2001 From: Vikram Agrawal Date: Thu, 26 Sep 2019 12:05:03 +0530 Subject: [PATCH 26/26] Add ConfigBuilder for Rocksdb configuration --- .../streaming/state/RocksDbInstance.scala | 35 ++---- .../state/RocksDbStateStoreConf.scala | 63 +++++++---- .../state/RocksDbStateStoreProvider.scala | 105 +++++++----------- 3 files changed, 90 insertions(+), 113 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbInstance.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbInstance.scala index 39904fa2fed5..2b5e6751317d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbInstance.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbInstance.scala @@ -20,7 +20,6 @@ package org.apache.spark.sql.execution.streaming.state import java.io.BufferedWriter import java.io.File import java.io.FileWriter -import java.util.Locale import scala.collection.JavaConverters._ @@ -28,18 +27,13 @@ import org.apache.commons.io.FileUtils import org.rocksdb._ import org.rocksdb.RocksDB -import org.apache.spark.SparkEnv import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.types.StructType import org.apache.spark.util.Utils -class RocksDbInstance( - keySchema: StructType, - valueSchema: StructType, - version: String, - conf: RocksDbStateStoreConf) - extends Logging { +class RocksDbInstance(keySchema: StructType, valueSchema: StructType, version: String) + extends Logging { import RocksDbInstance._ RocksDB.loadLibrary() @@ -195,9 +189,9 @@ class RocksDbInstance( } } - private val dataBlockSize = conf.blockSizeInKB - private val memTableMemoryBudget = conf.memtableBudgetInMB - private val enableStats = conf.enableStats + private val dataBlockSize = RocksDbStateStoreConf.blockSizeInKB + private val memTableMemoryBudget = RocksDbStateStoreConf.memtableBudgetInMB + private val enableStats = RocksDbStateStoreConf.enableStats protected def setOptions(): Unit = { @@ -255,15 +249,10 @@ class RocksDbInstance( } class OptimisticTransactionDbInstance( - keySchema: StructType, - valueSchema: StructType, - version: String, - conf: RocksDbStateStoreConf) - extends RocksDbInstance( keySchema: StructType, valueSchema: StructType, - version: String, - conf: RocksDbStateStoreConf) { + version: String) + extends RocksDbInstance(keySchema: StructType, valueSchema: StructType, version: String) { import RocksDbInstance._ RocksDB.loadLibrary() @@ -409,15 +398,7 @@ object RocksDbInstance { val COMMIT_FILE_NAME = "commit" - private val rocksDbCacheSizeInMB: Int = if (SparkEnv.get != null) { - SparkEnv.get.conf.getInt( - RocksDbStateStoreConf.CACHE_SIZE_KEY, - RocksDbStateStoreConf.DEFAULT_CACHE_SIZE_IN_MB) - } else { - RocksDbStateStoreConf.DEFAULT_CACHE_SIZE_IN_MB - } - - lazy val rocksDbLRUCache = new LRUCache(rocksDbCacheSizeInMB * 1024 * 1024, 6, false) + lazy val rocksDbLRUCache = new LRUCache(RocksDbStateStoreConf.cacheSize * 1024 * 1024, 6, false) def destroyDB(path: String): Unit = { val f: File = new File(path) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreConf.scala index 906b6b166e0d..9d55b6c786b5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreConf.scala @@ -17,36 +17,51 @@ package org.apache.spark.sql.execution.streaming.state -import org.apache.spark.util.Utils +import org.apache.spark.SparkEnv +import org.apache.spark.internal.config.ConfigBuilder -class RocksDbStateStoreConf(@transient private val stateStoreConf: StateStoreConf) - extends Serializable { +object RocksDbStateStoreConf { - import RocksDbStateStoreConf._ + private[spark] val ROCKSDB_STATE_STORE_DATA_BLOCK_SIZE = + ConfigBuilder("spark.sql.streaming.stateStore.rocksDb.blockSizeInKB") + .doc( + "The maximum size (in KB) of packed data in a block of a table file. " + + "When reading from a table, an entire block is loaded into memory") + .intConf + .createWithDefault(32) - def this() = this(StateStoreConf.empty) + private[spark] val ROCKSDB_STATE_STORE_MEMTABLE_BUDGET = + ConfigBuilder("spark.sql.streaming.stateStore.rocksDb.memtableBudgetInMB") + .doc("The maximum size (in MB) of memory to be used to optimize level style compaction") + .intConf + .createWithDefault(1024) - val blockSizeInKB: Int = stateStoreConf.confs - .getOrElse(BLOCK_SIZE_KEY, DEFAULT_BLOCKSIZE_IN_KB.toString).toInt + private[spark] val ROCKSDB_STATE_STORE_CACHE_SIZE = + ConfigBuilder("spark.sql.streaming.stateStore.rocksDb.cacheSizeInMB") + .doc("The maximum size (in MB) of in-memory LRU cache for RocksDB operations") + .intConf + .createWithDefault(512) - val memtableBudgetInMB: Int = stateStoreConf.confs - .getOrElse(MEMTABLE_BUDGET_KEY, DEFAULT_MEMTABLE_BUDGET_IN_MB.toString).toInt + private[spark] val ROCKSDB_STATE_STORE_ENABLE_STATS = + ConfigBuilder("spark.sql.streaming.stateStore.rocksDb.enableDbStats") + .doc("Enable statistics for rocksdb for debugging and reporting") + .booleanConf + .createWithDefault(false) - val enableStats: Boolean = stateStoreConf.confs - .getOrElse(ENABLE_STATS_KEY, "false").toBoolean + val blockSizeInKB: Int = Option(SparkEnv.get) + .map(_.conf.get(ROCKSDB_STATE_STORE_DATA_BLOCK_SIZE)) + .getOrElse(32) - val localDir: String = stateStoreConf.confs - .getOrElse(LOCAL_DIR_KEY, Utils.createTempDir().getAbsolutePath) -} + val memtableBudgetInMB: Int = Option(SparkEnv.get) + .map(_.conf.get(ROCKSDB_STATE_STORE_MEMTABLE_BUDGET)) + .getOrElse(1024) + + val cacheSize: Int = Option(SparkEnv.get) + .map(_.conf.get(RocksDbStateStoreConf.ROCKSDB_STATE_STORE_CACHE_SIZE)) + .getOrElse(512) + + val enableStats: Boolean = Option(SparkEnv.get) + .map(_.conf.get(ROCKSDB_STATE_STORE_ENABLE_STATS)) + .getOrElse(false) -object RocksDbStateStoreConf { - val DEFAULT_BLOCKSIZE_IN_KB = 32 - val DEFAULT_MEMTABLE_BUDGET_IN_MB = 1024 - val DEFAULT_CACHE_SIZE_IN_MB = 512 - - val BLOCK_SIZE_KEY = "spark.sql.streaming.stateStore.rocksDb.blockSizeInKB" - val MEMTABLE_BUDGET_KEY = "spark.sql.streaming.stateStore.rocksDb.memtableBudgetInMB" - val CACHE_SIZE_KEY = "spark.sql.streaming.stateStore.rocksDb.cacheSizeInMB" - val ENABLE_STATS_KEY = "spark.sql.streaming.stateStore.rocksDb.enableDbStats" - val LOCAL_DIR_KEY = "spark.sql.streaming.stateStore.rocksDb.localDir" } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreProvider.scala index 6f3ccb9e3b7b..cdc803b860a1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDbStateStoreProvider.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql.execution.streaming.state; import java.io._ import java.util -import java.util.Locale import scala.collection.JavaConverters._ import scala.io.Source @@ -37,30 +36,30 @@ import org.apache.spark.sql.execution.streaming.CheckpointFileManager import org.apache.spark.sql.types.StructType import org.apache.spark.util.Utils - /* - * An implementation of [[StateStoreProvider]] and [[StateStore]] using RocksDB as the storage - * engine. In RocksDB, new writes are inserted into a memtable which is flushed into local storage - * when the memtable fills up. It improves scalability as compared to - * [[HDFSBackedStateStoreProvider]] since now the state data which was large enough to fit in the - * executor memory can be written into the combination of memtable and local storage.The data is - * backed in a HDFS-compatible file system just like [[HDFSBackedStateStoreProvider]] - * - * Fault-tolerance model: - * - Every set of updates is written to a delta file before committing. - * - The state store is responsible for managing, collapsing and cleaning up of delta files. - * - Updates are committed in the db atomically - * - * Backup Model: - * - Delta file is written in a HDFS-compatible file system on batch commit - * - RocksDB state is check-pointed into a separate folder on batch commit - * - Maintenance thread periodically takes a snapshot of the latest check-pointed version of - * rocksDB state which is written to a HDFS-compatible file system. - * - * Isolation Guarantee: - * - writes are committed in the transaction. - * - writer thread which started the transaction can read all un-committed updates - * - any other reader thread cannot read any un-committed updates - */ +/* + * An implementation of [[StateStoreProvider]] and [[StateStore]] using RocksDB as the storage + * engine. In RocksDB, new writes are inserted into a memtable which is flushed into local storage + * when the memtable fills up. It improves scalability as compared to + * [[HDFSBackedStateStoreProvider]] since now the state data which was large enough to fit in the + * executor memory can be written into the combination of memtable and local storage.The data is + * backed in a HDFS-compatible file system just like [[HDFSBackedStateStoreProvider]] + * + * Fault-tolerance model: + * - Every set of updates is written to a delta file before committing. + * - The state store is responsible for managing, collapsing and cleaning up of delta files. + * - Updates are committed in the db atomically + * + * Backup Model: + * - Delta file is written in a HDFS-compatible file system on batch commit + * - RocksDB state is check-pointed into a separate folder on batch commit + * - Maintenance thread periodically takes a snapshot of the latest check-pointed version of + * rocksDB state which is written to a HDFS-compatible file system. + * + * Isolation Guarantee: + * - writes are committed in the transaction. + * - writer thread which started the transaction can read all un-committed updates + * - any other reader thread cannot read any un-committed updates + */ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Logging { /* Internal fields and methods */ @@ -71,7 +70,6 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log @volatile private var hadoopConf: Configuration = _ @volatile private var numberOfVersionsToRetain: Int = _ @volatile private var localDir: String = _ - @volatile private var rocksDbConf: RocksDbStateStoreConf = _ private lazy val baseDir: Path = stateStoreId.storeCheckpointLocation() private lazy val fm = CheckpointFileManager.create(baseDir, hadoopConf) @@ -116,11 +114,8 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log 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 = + new OptimisticTransactionDbInstance(keySchema, valueSchema, newVersion.toString) rocksDbWriteInstance.open(rocksDbPath) rocksDbWriteInstance.startTransactions() state = UPDATING @@ -149,8 +144,8 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log } override def getRange( - start: Option[UnsafeRow], - end: Option[UnsafeRow]): Iterator[UnsafeRowPair] = { + start: Option[UnsafeRow], + end: Option[UnsafeRow]): Iterator[UnsafeRowPair] = { require(state == UPDATING, "Cannot getRange after already committed or aborted") iterator() } @@ -230,11 +225,7 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log } else { val path = getCheckpointPath(version) val r: RocksDbInstance = - new RocksDbInstance( - keySchema, - valueSchema, - version.toString, - rocksDbConf) + new RocksDbInstance(keySchema, valueSchema, version.toString) r.open(path, readOnly = true) r.iterator(closeDbOnCompletion = true) } @@ -243,11 +234,7 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log // use check-pointed db for current updated version val path = getCheckpointPath(newVersion) val r: RocksDbInstance = - new RocksDbInstance( - keySchema, - valueSchema, - newVersion.toString, - rocksDbConf) + new RocksDbInstance(keySchema, valueSchema, newVersion.toString) r.open(path, readOnly = true) r.iterator(closeDbOnCompletion = true) @@ -293,12 +280,12 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log * to save state data */ override def init( - stateStoreId: StateStoreId, - keySchema: StructType, - valueSchema: StructType, - keyIndexOrdinal: Option[Int], // for sorting the data by their keys - storeConfs: StateStoreConf, - hadoopConf: Configuration): Unit = { + stateStoreId: StateStoreId, + keySchema: StructType, + valueSchema: StructType, + keyIndexOrdinal: Option[Int], // for sorting the data by their keys + storeConfs: StateStoreConf, + hadoopConf: Configuration): Unit = { this.stateStoreId_ = stateStoreId this.keySchema = keySchema this.valueSchema = valueSchema @@ -306,8 +293,10 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log this.hadoopConf = hadoopConf this.numberOfVersionsToRetain = storeConfs.maxVersionsToRetainInMemory fm.mkdirs(baseDir) - this.rocksDbConf = new RocksDbStateStoreConf(storeConfs) - this.localDir = rocksDbConf.localDir + this.localDir = storeConfs.confs + .getOrElse( + "spark.sql.streaming.stateStore.rocksDb.localDir", + Utils.createTempDir().getAbsolutePath) } /* @@ -463,11 +452,8 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log private def applyDelta(version: Long, lastAvailableVersion: Long): Unit = { var rocksDbWriteInstance: OptimisticTransactionDbInstance = null try { - rocksDbWriteInstance = new OptimisticTransactionDbInstance( - keySchema, - valueSchema, - version.toString, - rocksDbConf) + rocksDbWriteInstance = + new OptimisticTransactionDbInstance(keySchema, valueSchema, version.toString) rocksDbWriteInstance.open(rocksDbPath) rocksDbWriteInstance.startTransactions() // Load all the deltas from the version after the last available @@ -608,12 +594,7 @@ private[sql] class RocksDbStateStoreProvider extends StateStoreProvider with Log val maxVersion = versionsInFiles.max if (maxVersion > 0) { loadIntoRocksDB(maxVersion) - val r: RocksDbInstance = - new RocksDbInstance( - keySchema, - valueSchema, - maxVersion.toString, - rocksDbConf) + val r: RocksDbInstance = new RocksDbInstance(keySchema, valueSchema, maxVersion.toString) try { r.open(rocksDbPath, readOnly = true) return r.iterator(false)