[SPARK-32462][WEBUI] Reset previous search text for datatable

### What changes were proposed in this pull request?

This PR proposes to change the behavior of DataTable for stage-page and executors-page not to save the previous search text.

### Why are the changes needed?

DataTable is used in stage-page and executors-page for pagination and filter tasks/executors by search text.
In the current implementation, search text is saved so if we visit stage-page for a job, the previous search text is filled in the textbox and the task table is filtered.
I'm sometimes surprised by this behavior as the stage-page lists no tasks because tasks are filtered by the previous search text.
I think, it's not useful.

### Does this PR introduce _any_ user-facing change?

Yes. Search text is no longer saved.

### How was this patch tested?

New testcase with the following command.
```
$ build/sbt -Dguava.version=27.0-jre -Dtest.default.exclude.tags= -Dspark.test.webdriver.chrome.driver=/path/to/chromedriver "testOnly org.apache.spark.ui.ChromeUISeleniumSuite -- -z Search"
```

Closes #29265 from sarutak/fix-search-box.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
diff --git a/core/src/main/resources/org/apache/spark/ui/static/utils.js b/core/src/main/resources/org/apache/spark/ui/static/utils.js
index 2e46111..4571fc1 100644
--- a/core/src/main/resources/org/apache/spark/ui/static/utils.js
+++ b/core/src/main/resources/org/apache/spark/ui/static/utils.js
@@ -158,6 +158,9 @@
 function setDataTableDefaults() {
   $.extend($.fn.dataTable.defaults, {
     stateSave: true,
+    stateSaveParams: function(_, data) {
+        data.search.search = "";
+    },
     lengthMenu: [[20, 40, 60, 100, -1], [20, 40, 60, 100, "All"]],
     pageLength: 20
   });
@@ -166,7 +169,7 @@
 function formatDate(date) {
   if (date <= 0) return "-";
   else {
-     var dt = new Date(date.replace("GMT", "Z"))
+     var dt = new Date(date.replace("GMT", "Z"));
      return formatDateString(dt);
   }
 }
diff --git a/core/src/test/scala/org/apache/spark/ui/ChromeUISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/ChromeUISeleniumSuite.scala
index 9ba705c..459af67 100644
--- a/core/src/test/scala/org/apache/spark/ui/ChromeUISeleniumSuite.scala
+++ b/core/src/test/scala/org/apache/spark/ui/ChromeUISeleniumSuite.scala
@@ -17,7 +17,7 @@
 
 package org.apache.spark.ui
 
-import org.openqa.selenium.WebDriver
+import org.openqa.selenium.{JavascriptExecutor, WebDriver}
 import org.openqa.selenium.chrome.{ChromeDriver, ChromeOptions}
 
 import org.apache.spark.tags.ChromeUITest
@@ -28,7 +28,7 @@
 @ChromeUITest
 class ChromeUISeleniumSuite extends RealBrowserUISeleniumSuite("webdriver.chrome.driver") {
 
-  override var webDriver: WebDriver = _
+  override var webDriver: WebDriver with JavascriptExecutor = _
 
   override def beforeAll(): Unit = {
     super.beforeAll()
diff --git a/core/src/test/scala/org/apache/spark/ui/RealBrowserUISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/RealBrowserUISeleniumSuite.scala
index 2c135850..3f296ac 100644
--- a/core/src/test/scala/org/apache/spark/ui/RealBrowserUISeleniumSuite.scala
+++ b/core/src/test/scala/org/apache/spark/ui/RealBrowserUISeleniumSuite.scala
@@ -17,7 +17,7 @@
 
 package org.apache.spark.ui
 
-import org.openqa.selenium.{By, WebDriver}
+import org.openqa.selenium.{By, JavascriptExecutor, WebDriver}
 import org.scalatest.BeforeAndAfterAll
 import org.scalatest.concurrent.Eventually._
 import org.scalatest.matchers.must.Matchers
@@ -37,7 +37,7 @@
 abstract class RealBrowserUISeleniumSuite(val driverProp: String)
   extends SparkFunSuite with WebBrowser with Matchers with BeforeAndAfterAll {
 
-  implicit var webDriver: WebDriver
+  implicit var webDriver: WebDriver with JavascriptExecutor
   private val driverPropPrefix = "spark.test."
 
   override def beforeAll(): Unit = {
@@ -130,6 +130,58 @@
     }
   }
 
+  test("Search text for paged tables should not be saved") {
+    withSpark(newSparkContext()) { sc =>
+      sc.parallelize(1 to 10).collect
+
+      eventually(timeout(10.seconds), interval(1.seconds)) {
+        val taskSearchBox = "$(\"input[aria-controls='active-tasks-table']\")"
+        goToUi(sc, "/stages/stage/?id=0&attempt=0")
+        // Wait for ajax loading done.
+        Thread.sleep(20)
+        setValueToSearchBox(taskSearchBox, "task1")
+        val taskSearchText = getTextFromSearchBox(taskSearchBox)
+        assert(taskSearchText === "task1")
+
+        val executorSearchBox = "$(\"input[aria-controls='active-executors-table']\")"
+        goToUi(sc, "/executors")
+        Thread.sleep(20)
+        setValueToSearchBox(executorSearchBox, "executor1")
+        val executorSearchText = getTextFromSearchBox(executorSearchBox)
+        assert(executorSearchText === "executor1")
+
+        goToUi(sc, "/stages/stage/?id=0&attempt=0")
+        Thread.sleep(20)
+        val revisitTaskSearchText = getTextFromSearchBox(taskSearchBox)
+        assert(revisitTaskSearchText === "")
+
+        goToUi(sc, "/executors")
+        Thread.sleep(20)
+        val revisitExecutorSearchText = getTextFromSearchBox(executorSearchBox)
+        assert(revisitExecutorSearchText === "")
+      }
+    }
+
+    def setValueToSearchBox(searchBox: String, text: String): Unit = {
+      webDriver.executeScript(s"$searchBox.val('$text');")
+      fireDataTable(searchBox)
+    }
+
+    def getTextFromSearchBox(searchBox: String): String = {
+      webDriver.executeScript(s"return $searchBox.val();").toString
+    }
+
+    def fireDataTable(searchBox: String): Unit = {
+      webDriver.executeScript(
+        s"""
+           |var keyEvent = $$.Event('keyup');
+           |// 13 means enter key.
+           |keyEvent.keyCode = keyEvent.which = 13;
+           |$searchBox.trigger(keyEvent);
+         """.stripMargin)
+    }
+  }
+
   /**
    * Create a test SparkContext with the SparkUI enabled.
    * It is safe to `get` the SparkUI directly from the SparkContext returned here.
diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala
index 6817fcc..d7caeaa 100644
--- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala
+++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala
@@ -46,7 +46,6 @@
 import org.apache.spark.internal.config.UI._
 import org.apache.spark.shuffle.FetchFailedException
 import org.apache.spark.status.api.v1.{JacksonMessageWriter, RDDDataDistribution, StageStatus}
-import org.apache.spark.util.CallSite
 
 private[spark] class SparkUICssErrorHandler extends DefaultCssErrorHandler {