You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by ge...@apache.org on 2020/06/02 23:43:47 UTC

[spark] branch branch-3.0 updated: [SPARK-31756][WEBUI][3.0] Add real headless browser support for UI test

This is an automated email from the ASF dual-hosted git repository.

gengliang pushed a commit to branch branch-3.0
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.0 by this push:
     new 32cb37c  [SPARK-31756][WEBUI][3.0] Add real headless browser support for UI test
32cb37c is described below

commit 32cb37ccbf8f539f9e8d07f137fce1c4e4db5fb0
Author: Kousuke Saruta <sa...@oss.nttdata.com>
AuthorDate: Tue Jun 2 16:41:19 2020 -0700

    [SPARK-31756][WEBUI][3.0] Add real headless browser support for UI test
    
    ### What changes were proposed in this pull request?
    
    This PR backports the change of #28627 .
    Some fixes for UI like #28690 should be merged to `branch-3.0` including testcases so #28627 is need to be backported.
    
    This PR mainly adds two things.
    
    1. Real headless browser support for UI test
    2. A test suite using headless Chrome as one instance of  those browsers.
    
    Also, for environment where Chrome and Chrome driver is not installed, `ChromeUITest` tag is added to filter out the test suite.
    By default, test suites with `ChromeUITest` is disabled.
    
    ### Why are the changes needed?
    
    In `branch-3.0`, there are two problems for UI test.
    1. Lots of tests especially JavaScript related ones are done manually.
    Appearance is better to be confirmed by our eyes but logic should be tested by test cases ideally.
    
    2. Compared to the real web browsers, HtmlUnit doesn't seem to support JavaScript enough.
    I added a JavaScript related test before for SPARK-31534 using HtmlUnit which is simple library based headless browser for test.
    The test I added works somehow but some JavaScript related error is shown in unit-tests.log.
    
    ```
    ======= EXCEPTION START ========
    Exception class=[net.sourceforge.htmlunit.corejs.javascript.JavaScriptException]
    com.gargoylesoftware.htmlunit.ScriptException: Error: TOOLTIP: Option "sanitizeFn" provided type "window" but expected type "(null|function)". (http://192.168.1.209:60724/static/jquery-3.4.1.min.js#2)
            at com.gargoylesoftware.htmlunit.javascript.JavaScriptEngine$HtmlUnitContextAction.run(JavaScriptEngine.java:904)
            at net.sourceforge.htmlunit.corejs.javascript.Context.call(Context.java:628)
            at net.sourceforge.htmlunit.corejs.javascript.ContextFactory.call(ContextFactory.java:515)
            at com.gargoylesoftware.htmlunit.javascript.JavaScriptEngine.callFunction(JavaScriptEngine.java:835)
            at com.gargoylesoftware.htmlunit.javascript.JavaScriptEngine.callFunction(JavaScriptEngine.java:807)
            at com.gargoylesoftware.htmlunit.InteractivePage.executeJavaScriptFunctionIfPossible(InteractivePage.java:216)
            at com.gargoylesoftware.htmlunit.javascript.background.JavaScriptFunctionJob.runJavaScript(JavaScriptFunctionJob.java:52)
            at com.gargoylesoftware.htmlunit.javascript.background.JavaScriptExecutionJob.run(JavaScriptExecutionJob.java:102)
            at com.gargoylesoftware.htmlunit.javascript.background.JavaScriptJobManagerImpl.runSingleJob(JavaScriptJobManagerImpl.java:426)
            at com.gargoylesoftware.htmlunit.javascript.background.DefaultJavaScriptExecutor.run(DefaultJavaScriptExecutor.java:157)
            at java.lang.Thread.run(Thread.java:748)
    Caused by: net.sourceforge.htmlunit.corejs.javascript.JavaScriptException: Error: TOOLTIP: Option "sanitizeFn" provided type "window" but expected type "(null|function)". (http://192.168.1.209:60724/static/jquery-3.4.1.min.js#2)
            at net.sourceforge.htmlunit.corejs.javascript.Interpreter.interpretLoop(Interpreter.java:1009)
            at net.sourceforge.htmlunit.corejs.javascript.Interpreter.interpret(Interpreter.java:800)
            at net.sourceforge.htmlunit.corejs.javascript.InterpretedFunction.call(InterpretedFunction.java:105)
            at net.sourceforge.htmlunit.corejs.javascript.ContextFactory.doTopCall(ContextFactory.java:413)
            at com.gargoylesoftware.htmlunit.javascript.HtmlUnitContextFactory.doTopCall(HtmlUnitContextFactory.java:252)
            at net.sourceforge.htmlunit.corejs.javascript.ScriptRuntime.doTopCall(ScriptRuntime.java:3264)
            at com.gargoylesoftware.htmlunit.javascript.JavaScriptEngine$4.doRun(JavaScriptEngine.java:828)
            at com.gargoylesoftware.htmlunit.javascript.JavaScriptEngine$HtmlUnitContextAction.run(JavaScriptEngine.java:889)
            ... 10 more
    JavaScriptException value = Error: TOOLTIP: Option "sanitizeFn" provided type "window" but expected type "(null|function)".
    == CALLING JAVASCRIPT ==
      function () {
          throw e;
      }
    ======= EXCEPTION END ========
    ```
    I tried to upgrade HtmlUnit to 2.40.0 but what is worse, the test become not working even though it works on real browsers like Chrome, Safari and Firefox without error.
    ```
    [info] UISeleniumSuite:
    [info] - SPARK-31534: text for tooltip should be escaped *** FAILED *** (17 seconds, 745 milliseconds)
    [info]   The code passed to eventually never returned normally. Attempted 2 times over 12.910785232 seconds. Last failure message: com.gargoylesoftware.htmlunit.ScriptException: ReferenceError: Assignment to undefined "regeneratorRuntime" in strict mode (http://192.168.1.209:62132/static/vis-timeline-graph2d.min.js#52(Function)#1)
    ```
    To resolve those problems, it's better to support headless browser for UI test.
    
    ### Does this PR introduce _any_ user-facing change?
    
    No.
    
    ### How was this patch tested?
    
    I tested with following patterns. Both Chrome and Chrome driver should be installed to test.
    
    1. sbt / with default excluded tags (ChromeUISeleniumSuite is expected to be skipped and SQLQueryTestSuite is expected to succeed)
    `build/sbt -Dspark.test.webdriver.chrome.driver=/path/to/chromedriver "testOnly org.apache.spark.ui.ChromeUISeleniumSuite org.apache.spark.sql.SQLQueryTestSuite"`
    
    2. sbt / overwrite default excluded tags as empty string (Both suites are expected to succeed)
    `build/sbt -Dtest.default.exclude.tags= -Dspark.test.webdriver.chrome.driver=/path/to/chromedriver "testOnly org.apache.spark.ui.ChromeUISeleniumSuite org.apache.spark.sql.SQLQueryTestSuite"`
    
    3. sbt / set `test.exclude.tags` to `org.apache.spark.tags.ExtendedSQLTest` (Both suites are expected to be skipped)
    `build/sbt -Dtest.exclude.tags=org.apache.spark.tags.ExtendedSQLTest -Dspark.test.webdriver.chrome.driver=/path/to/chromedriver "testOnly org.apache.spark.ui.ChromeUISeleniumSuite org.apache.spark.sql.SQLQueryTestSuite"`
    
    4. Maven / with default excluded tags (ChromeUISeleniumSuite is expected to be skipped and SQLQueryTestSuite is expected to succeed)
    `build/mvn -Dspark.test.webdriver.chrome.driver=/path/to/chromedriver -Dtest=none -DwildcardSuites=org.apache.spark.ui.ChromeUISeleniumSuite,org.apache.spark.sql.SQLQueryTestSuite test`
    
    5. Maven / overwrite default excluded tags as empty string (Both suites are expected to succeed)
    `build/mvn -Dtest.default.exclude.tags= -Dspark.test.webdriver.chrome.driver=/path/to/chromedriver -Dtest=none -DwildcardSuites=org.apache.spark.ui.ChromeUISeleniumSuite,org.apache.spark.sql.SQLQueryTestSuite test`
    
    6. Maven / set `test.exclude.tags` to `org.apache.spark.tags.ExtendedSQLTest` (Both suites are expected to be skipped)
    `build/mvn -Dtest.exclude.tags=org.apache.spark.tags.ExtendedSQLTest  -Dspark.test.webdriver.chrome.driver=/path/to/chromedriver -Dtest=none -DwildcardSuites=org.apache.spark.ui.ChromeUISeleniumSuite,org.apache.spark.sql.SQLQueryTestSuite test`
    
    Closes #28702 from sarutak/real-headless-browser-support-3.0.
    
    Authored-by: Kousuke Saruta <sa...@oss.nttdata.com>
    Signed-off-by: Gengliang Wang <ge...@databricks.com>
---
 .../java/org/apache/spark/tags/ChromeUITest.java   |  27 +++++
 .../apache/spark/ui/ChromeUISeleniumSuite.scala    |  49 +++++++++
 .../spark/ui/RealBrowserUISeleniumSuite.scala      | 109 +++++++++++++++++++++
 .../org/apache/spark/ui/UISeleniumSuite.scala      |  27 -----
 pom.xml                                            |   7 +-
 project/SparkBuild.scala                           |   7 ++
 6 files changed, 198 insertions(+), 28 deletions(-)

diff --git a/common/tags/src/test/java/org/apache/spark/tags/ChromeUITest.java b/common/tags/src/test/java/org/apache/spark/tags/ChromeUITest.java
new file mode 100644
index 0000000..e3fed3d
--- /dev/null
+++ b/common/tags/src/test/java/org/apache/spark/tags/ChromeUITest.java
@@ -0,0 +1,27 @@
+/*
+ * 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.tags;
+
+import java.lang.annotation.*;
+
+import org.scalatest.TagAnnotation;
+
+@TagAnnotation
+@Retention(RetentionPolicy.RUNTIME)
+@Target({ElementType.METHOD, ElementType.TYPE})
+public @interface ChromeUITest { }
diff --git a/core/src/test/scala/org/apache/spark/ui/ChromeUISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/ChromeUISeleniumSuite.scala
new file mode 100644
index 0000000..9ba705c
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/ui/ChromeUISeleniumSuite.scala
@@ -0,0 +1,49 @@
+/*
+ * 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.ui
+
+import org.openqa.selenium.WebDriver
+import org.openqa.selenium.chrome.{ChromeDriver, ChromeOptions}
+
+import org.apache.spark.tags.ChromeUITest
+
+/**
+ * Selenium tests for the Spark Web UI with Chrome.
+ */
+@ChromeUITest
+class ChromeUISeleniumSuite extends RealBrowserUISeleniumSuite("webdriver.chrome.driver") {
+
+  override var webDriver: WebDriver = _
+
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+    val chromeOptions = new ChromeOptions
+    chromeOptions.addArguments("--headless", "--disable-gpu")
+    webDriver = new ChromeDriver(chromeOptions)
+  }
+
+  override def afterAll(): Unit = {
+    try {
+      if (webDriver != null) {
+        webDriver.quit()
+      }
+    } finally {
+      super.afterAll()
+    }
+  }
+}
diff --git a/core/src/test/scala/org/apache/spark/ui/RealBrowserUISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/RealBrowserUISeleniumSuite.scala
new file mode 100644
index 0000000..84f8882
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/ui/RealBrowserUISeleniumSuite.scala
@@ -0,0 +1,109 @@
+/*
+ * 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.ui
+
+import org.openqa.selenium.{By, WebDriver}
+import org.scalatest._
+import org.scalatest.concurrent.Eventually._
+import org.scalatest.time.SpanSugar._
+import org.scalatestplus.selenium.WebBrowser
+
+import org.apache.spark._
+import org.apache.spark.LocalSparkContext.withSpark
+import org.apache.spark.internal.config.MEMORY_OFFHEAP_SIZE
+import org.apache.spark.internal.config.UI.{UI_ENABLED, UI_KILL_ENABLED, UI_PORT}
+import org.apache.spark.util.CallSite
+
+/**
+ * Selenium tests for the Spark Web UI with real web browsers.
+ */
+abstract class RealBrowserUISeleniumSuite(val driverProp: String)
+  extends SparkFunSuite with WebBrowser with Matchers with BeforeAndAfterAll {
+
+  implicit var webDriver: WebDriver
+  private val driverPropPrefix = "spark.test."
+
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+    assume(
+      sys.props(driverPropPrefix + driverProp) !== null,
+      "System property " + driverPropPrefix + driverProp +
+        " should be set to the corresponding driver path.")
+    sys.props(driverProp) = sys.props(driverPropPrefix + driverProp)
+  }
+
+  override def afterAll(): Unit = {
+    sys.props.remove(driverProp)
+    super.afterAll()
+  }
+
+  test("SPARK-31534: text for tooltip should be escaped") {
+    withSpark(newSparkContext()) { sc =>
+      sc.setLocalProperty(CallSite.LONG_FORM, "collect at <console>:25")
+      sc.setLocalProperty(CallSite.SHORT_FORM, "collect at <console>:25")
+      sc.parallelize(1 to 10).collect
+
+      eventually(timeout(10.seconds), interval(50.milliseconds)) {
+        goToUi(sc, "/jobs")
+
+        val jobDesc =
+          webDriver.findElement(By.cssSelector("div[class='application-timeline-content']"))
+        jobDesc.getAttribute("data-title") should include  ("collect at &lt;console&gt;:25")
+
+        goToUi(sc, "/jobs/job/?id=0")
+        webDriver.get(sc.ui.get.webUrl.stripSuffix("/") + "/jobs/job/?id=0")
+        val stageDesc = webDriver.findElement(By.cssSelector("div[class='job-timeline-content']"))
+        stageDesc.getAttribute("data-title") should include ("collect at &lt;console&gt;:25")
+
+        // Open DAG Viz.
+        webDriver.findElement(By.id("job-dag-viz")).click()
+        val nodeDesc = webDriver.findElement(By.cssSelector("g[class='node_0 node']"))
+        nodeDesc.getAttribute("name") should include ("collect at &lt;console&gt;:25")
+      }
+    }
+  }
+
+  /**
+   * Create a test SparkContext with the SparkUI enabled.
+   * It is safe to `get` the SparkUI directly from the SparkContext returned here.
+   */
+  private def newSparkContext(
+      killEnabled: Boolean = true,
+      master: String = "local",
+      additionalConfs: Map[String, String] = Map.empty): SparkContext = {
+    val conf = new SparkConf()
+      .setMaster(master)
+      .setAppName("test")
+      .set(UI_ENABLED, true)
+      .set(UI_PORT, 0)
+      .set(UI_KILL_ENABLED, killEnabled)
+      .set(MEMORY_OFFHEAP_SIZE.key, "64m")
+    additionalConfs.foreach { case (k, v) => conf.set(k, v) }
+    val sc = new SparkContext(conf)
+    assert(sc.ui.isDefined)
+    sc
+  }
+
+  def goToUi(sc: SparkContext, path: String): Unit = {
+    goToUi(sc.ui.get, path)
+  }
+
+  def goToUi(ui: SparkUI, path: String): Unit = {
+    go to (ui.webUrl.stripSuffix("/") + path)
+  }
+}
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 3ec9385..909056e 100644
--- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala
+++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala
@@ -773,33 +773,6 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser with Matchers with B
     }
   }
 
-  test("SPARK-31534: text for tooltip should be escaped") {
-    withSpark(newSparkContext()) { sc =>
-      sc.setLocalProperty(CallSite.LONG_FORM, "collect at <console>:25")
-      sc.setLocalProperty(CallSite.SHORT_FORM, "collect at <console>:25")
-      sc.parallelize(1 to 10).collect
-
-      val driver = webDriver.asInstanceOf[HtmlUnitDriver]
-      driver.setJavascriptEnabled(true)
-
-      eventually(timeout(10.seconds), interval(50.milliseconds)) {
-        goToUi(sc, "/jobs")
-        val jobDesc =
-          driver.findElement(By.cssSelector("div[class='application-timeline-content']"))
-        jobDesc.getAttribute("data-title") should include  ("collect at &lt;console&gt;:25")
-
-        goToUi(sc, "/jobs/job/?id=0")
-        val stageDesc = driver.findElement(By.cssSelector("div[class='job-timeline-content']"))
-        stageDesc.getAttribute("data-title") should include ("collect at &lt;console&gt;:25")
-
-        // Open DAG Viz.
-        driver.findElement(By.id("job-dag-viz")).click()
-        val nodeDesc = driver.findElement(By.cssSelector("g[class='node_0 node']"))
-        nodeDesc.getAttribute("name") should include ("collect at &lt;console&gt;:25")
-      }
-    }
-  }
-
   def goToUi(sc: SparkContext, path: String): Unit = {
     goToUi(sc.ui.get, path)
   }
diff --git a/pom.xml b/pom.xml
index 0231f8f..d8d5595 100644
--- a/pom.xml
+++ b/pom.xml
@@ -204,6 +204,9 @@
     <leveldbjni.group>org.fusesource.leveldbjni</leveldbjni.group>
 
     <test.java.home>${java.home}</test.java.home>
+
+    <!-- Some UI tests require Chrome and Chrome driver installed so those tests are disabled by default. -->
+    <test.default.exclude.tags>org.apache.spark.tags.ChromeUITest</test.default.exclude.tags>
     <test.exclude.tags></test.exclude.tags>
     <test.include.tags></test.include.tags>
 
@@ -243,6 +246,7 @@
       things breaking.
     -->
     <spark.test.home>${session.executionRootDirectory}</spark.test.home>
+    <spark.test.webdriver.chrome.driver></spark.test.webdriver.chrome.driver>
 
     <CodeCacheSize>1g</CodeCacheSize>
   </properties>
@@ -2504,10 +2508,11 @@
               <spark.ui.enabled>false</spark.ui.enabled>
               <spark.ui.showConsoleProgress>false</spark.ui.showConsoleProgress>
               <spark.unsafe.exceptionOnMemoryLeak>true</spark.unsafe.exceptionOnMemoryLeak>
+              <spark.test.webdriver.chrome.driver>${spark.test.webdriver.chrome.driver}</spark.test.webdriver.chrome.driver>
               <!-- Needed by sql/hive tests. -->
               <test.src.tables>__not_used__</test.src.tables>
             </systemProperties>
-            <tagsToExclude>${test.exclude.tags}</tagsToExclude>
+            <tagsToExclude>${test.exclude.tags},${test.default.exclude.tags}</tagsToExclude>
             <tagsToInclude>${test.include.tags}</tagsToInclude>
           </configuration>
           <executions>
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index 7bd92b3..c9521ea 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -966,6 +966,9 @@ object TestSettings {
       "2.12"
     }
      */
+
+  private val defaultExcludedTags = Seq("org.apache.spark.tags.ChromeUITest")
+
   lazy val settings = Seq (
     // Fork new JVMs for tests and set Java options for those
     fork := true,
@@ -1003,6 +1006,10 @@ object TestSettings {
       sys.props.get("test.exclude.tags").map { tags =>
         tags.split(",").flatMap { tag => Seq("-l", tag) }.toSeq
       }.getOrElse(Nil): _*),
+    testOptions in Test += Tests.Argument(TestFrameworks.ScalaTest,
+      sys.props.get("test.default.exclude.tags").map(tags => tags.split(",").toSeq)
+        .map(tags => tags.filter(!_.trim.isEmpty)).getOrElse(defaultExcludedTags)
+        .flatMap(tag => Seq("-l", tag)): _*),
     testOptions in Test += Tests.Argument(TestFrameworks.JUnit,
       sys.props.get("test.exclude.tags").map { tags =>
         Seq("--exclude-categories=" + tags)


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