xuanyuanking commented on a change in pull request #33187: URL: https://github.com/apache/spark/pull/33187#discussion_r665896729
########## File path: sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreSuite.scala ########## @@ -0,0 +1,152 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.streaming.state + +import java.util.UUID + +import scala.util.Random + +import org.apache.hadoop.conf.Configuration +import org.scalatest.BeforeAndAfter + +import org.apache.spark.SparkConf +import org.apache.spark.sql.LocalSparkSession.withSparkSession +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.execution.streaming.StatefulOperatorStateInfo +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.Platform +import org.apache.spark.util.Utils + +class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvider] + with BeforeAndAfter { + + import StateStoreTestsHelper._ + + test("version encoding") { + import RocksDBStateStoreProvider._ + + val provider = newStoreProvider() + val store = provider.getStore(0) + val keyRow = stringToRow("a") + val valueRow = intToRow(1) + store.put(keyRow, valueRow) + val iter = provider.rocksDB.iterator() + assert(iter.hasNext) + val kv = iter.next() + + // Verify the version encoded in first byte of the key and value byte arrays + assert(Platform.getByte(kv.key, Platform.BYTE_ARRAY_OFFSET) === STATE_ENCODING_VERSION) + assert(Platform.getByte(kv.value, Platform.BYTE_ARRAY_OFFSET) === STATE_ENCODING_VERSION) + } + + test("RocksDB confs are passed correctly from SparkSession to db instance") { + val sparkConf = new SparkConf().setMaster("local").setAppName(this.getClass.getSimpleName) + withSparkSession(SparkSession.builder.config(sparkConf).getOrCreate()) { spark => + // Set the session confs that should be passed into RocksDB + val testConfs = Seq( + ("spark.sql.streaming.stateStore.providerClass", + classOf[RocksDBStateStoreProvider].getName), + (RocksDBConf.ROCKSDB_CONF_NAME_PREFIX + ".compactOnCommit", "true"), + (RocksDBConf.ROCKSDB_CONF_NAME_PREFIX + ".lockAcquireTimeoutMs", "10") + ) + testConfs.foreach { case (k, v) => spark.conf.set(k, v) } + + // Prepare test objects for running task on state store + val testRDD = spark.sparkContext.makeRDD[String](Seq("a"), 1) + val testSchema = StructType(Seq(StructField("key", StringType, true))) + val testStateInfo = StatefulOperatorStateInfo( + checkpointLocation = Utils.createTempDir().getAbsolutePath, + queryRunId = UUID.randomUUID, operatorId = 0, storeVersion = 0, numPartitions = 5) + + // Create state store in a task and get the RocksDBConf from the instantiated RocksDB instance + val rocksDBConfInTask: RocksDBConf = testRDD.mapPartitionsWithStateStore[RocksDBConf]( + spark.sqlContext, testStateInfo, testSchema, testSchema, None) { + (store: StateStore, _: Iterator[String]) => + // Use reflection to get RockDB instance + val dbInstanceMethod = + store.getClass.getMethods.filter(_.getName.contains("dbInstance")).head + Iterator(dbInstanceMethod.invoke(store).asInstanceOf[RocksDB].conf) + }.collect().head + + // Verify the confs are same as those configured in the session conf + assert(rocksDBConfInTask.compactOnCommit == true) + assert(rocksDBConfInTask.lockAcquireTimeoutMs == 10L) + } + } + + test("rocksdb file manager metrics exposed") { + import RocksDBStateStoreProvider._ + def getCustomMetric(metrics: StateStoreMetrics, customMetric: StateStoreCustomMetric): Long = { + val metricPair = metrics.customMetrics.find(_._1.name == customMetric.name) + assert(metricPair.isDefined) + metricPair.get._2 + } + + val provider = newStoreProvider() + val store = provider.getStore(0) + // Verify state after updating + put(store, "a", 1) + assert(get(store, "a") === Some(1)) + assert(store.commit() === 1) + assert(store.hasCommitted) + val storeMetrics = store.metrics + assert(storeMetrics.numKeys === 1) + assert(getCustomMetric(storeMetrics, CUSTOM_METRIC_FILES_COPIED) == 1L) Review comment: Agree, will update in the next commit. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
