You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2022/02/06 03:08:36 UTC

[GitHub] [spark] HeartSaVioR commented on a change in pull request #35356: [SPARK-38056][Web UI] Fix issue of Structured streaming not working in history server when using LevelDB

HeartSaVioR commented on a change in pull request #35356:
URL: https://github.com/apache/spark/pull/35356#discussion_r800120110



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatusListenerSuite.scala
##########
@@ -213,4 +214,34 @@ class StreamingQueryStatusListenerSuite extends StreamTest {
     addQueryProgress()
     checkQueryProcessData(5)
   }
+
+  test("SPARK-38056: test writing StreamingQueryData to an in-memory store") {
+    val store = new ElementTrackingStore(new InMemoryStore(), sparkConf)
+    store.write(testStreamingQueryData)
+  }
+
+  test("SPARK-38056: test writing StreamingQueryData to a LevelDB store") {
+    assume(!Utils.isMacOnAppleSilicon)
+    val testDir = Utils.createTempDir()
+    try {
+      val kvStore = KVUtils.open(testDir, getClass.getName)
+      val store = new ElementTrackingStore(kvStore, sparkConf)
+      store.write(testStreamingQueryData)
+    } finally {
+      Utils.deleteRecursively(testDir)
+    }
+  }

Review comment:
       Let's add a new test against RocksDB KVStore. It's not directly available, but available as one of disk backend options in hybrid KVStore. Instead of initializing hybrid KVStore, we can directly initialize RocksDB KVStore to simplify testing.
   
   ```
     test("SPARK-38056: test writing StreamingQueryData to a RocksDB store") {
       assume(!Utils.isMacOnAppleSilicon)
       import org.apache.spark.util.kvstore.RocksDB
       val testDir = Utils.createTempDir()
       try {
         val kvStore = new RocksDB(testDir)
         val store = new ElementTrackingStore(kvStore, sparkConf)
         store.write(testStreamingQueryData)
       } finally {
         kvStore.close()
         Utils.deleteRecursively(testDir)
       }
     }
   ```
   
   And while we are here, there are lots of redundant code which can be pushed to the method testStreamingQueryData, while initializing and cleaning up KVStore can be done in individual test methods.
   
   ```
   private def testStreamingQueryData(kvStore: KVStore): Unit = {
     val id = UUID.randomUUID()
     val testData = new StreamingQueryData(
         "some-query",
         id,
         id.toString,
         isActive = false,
         None,
         1L,
         None
       )
     val store = new ElementTrackingStore(kvStore, sparkConf)
     store.write(testStreamingQueryData)
   }
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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



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