diff --git a/core/src/main/scala/com/tribbloids/spookystuff/ConflictDetection.scala b/core/src/main/scala/com/tribbloids/spookystuff/ConflictDetection.scala
index 316d6f052..e7a604d1b 100644
--- a/core/src/main/scala/com/tribbloids/spookystuff/ConflictDetection.scala
+++ b/core/src/main/scala/com/tribbloids/spookystuff/ConflictDetection.scala
@@ -27,7 +27,7 @@ object ConflictDetection {
def conflicts: Seq[Try[Unit]] = {
- val allObj = Cleanable.getTyped[ConflictDetection]
+ val allObj = Cleanable.All.typed[ConflictDetection]
val allResourceIDs: Map[String, Seq[Any]] = allObj
.map {
diff --git a/core/src/main/scala/com/tribbloids/spookystuff/SpookyKryoRegistrator.scala b/core/src/main/scala/com/tribbloids/spookystuff/SpookyKryoRegistrator.scala
index e087e2137..edf929877 100644
--- a/core/src/main/scala/com/tribbloids/spookystuff/SpookyKryoRegistrator.scala
+++ b/core/src/main/scala/com/tribbloids/spookystuff/SpookyKryoRegistrator.scala
@@ -1,17 +1,16 @@
package com.tribbloids.spookystuff
-import java.util.UUID
-import java.util.concurrent.TimeUnit
import com.esotericsoftware.kryo.Kryo
import com.tribbloids.spookystuff.conf.{Dir, SpookyConf}
-import com.tribbloids.spookystuff.dsl._
import com.tribbloids.spookystuff.doc._
+import com.tribbloids.spookystuff.dsl._
import com.tribbloids.spookystuff.metrics.SpookyMetrics
import com.tribbloids.spookystuff.row.FetchedRow
-import org.apache.spark.SerializableWritable
import org.apache.spark.serializer.KryoRegistrator
import org.apache.spark.sql.catalyst.ScalaReflection.universe.TypeTag
+import java.util.UUID
+import java.util.concurrent.TimeUnit
import scala.collection.immutable.ListMap
import scala.concurrent.duration.FiniteDuration
@@ -37,7 +36,6 @@ class SpookyKryoRegistrator extends KryoRegistrator {
//used by broadcast & accumulator
classOf[SpookyConf],
classOf[Dir.Conf],
- classOf[SerializableWritable[_]],
classOf[SpookyContext],
classOf[SpookyMetrics],
//used by Expressions
diff --git a/core/src/main/scala/com/tribbloids/spookystuff/actions/Action.scala b/core/src/main/scala/com/tribbloids/spookystuff/actions/Action.scala
index 26c8f4595..7f8147d1b 100644
--- a/core/src/main/scala/com/tribbloids/spookystuff/actions/Action.scala
+++ b/core/src/main/scala/com/tribbloids/spookystuff/actions/Action.scala
@@ -67,7 +67,6 @@ trait Action extends ActionLike with TraceAPI {
case ae: ActionException => ae
case _ => new ActionException(message, e)
}
-// ex.setStackTrace(e.getStackTrace)
throw ex
}
@@ -99,7 +98,7 @@ trait Action extends ActionLike with TraceAPI {
}
}
- protected[actions] def withDriversDuring[T](session: Session)(f: => T): T = {
+ protected[actions] def withTimeoutDuring[T](session: Session)(f: => T): T = {
var baseStr = s"[${session.taskContextOpt.map(_.partitionId()).getOrElse(0)}]+> ${this.toString}"
this match {
@@ -121,7 +120,7 @@ trait Action extends ActionLike with TraceAPI {
}
final def exe(session: Session): Seq[DocOption] = {
- withDriversDuring(session) {
+ withTimeoutDuring(session) {
doExe(session)
}
}
diff --git a/core/src/main/scala/com/tribbloids/spookystuff/actions/Block.scala b/core/src/main/scala/com/tribbloids/spookystuff/actions/Block.scala
index 448156596..bf2152333 100644
--- a/core/src/main/scala/com/tribbloids/spookystuff/actions/Block.scala
+++ b/core/src/main/scala/com/tribbloids/spookystuff/actions/Block.scala
@@ -120,7 +120,7 @@ object ClusterRetry {
e
)
} else logger.warn(s"Failover on ${e.getClass.getSimpleName}: Cluster-wise retries has depleted")
- logger.info("\t\\-->", e)
+ logger.debug("\t\\-->", e)
}
pages
diff --git a/core/src/main/scala/com/tribbloids/spookystuff/actions/Timed.scala b/core/src/main/scala/com/tribbloids/spookystuff/actions/Timed.scala
index 4669fcb4b..82f318693 100644
--- a/core/src/main/scala/com/tribbloids/spookystuff/actions/Timed.scala
+++ b/core/src/main/scala/com/tribbloids/spookystuff/actions/Timed.scala
@@ -2,19 +2,18 @@ package com.tribbloids.spookystuff.actions
import com.tribbloids.spookystuff.Const
import com.tribbloids.spookystuff.session.Session
-import com.tribbloids.spookystuff.utils.TimeoutConf
-
+import com.tribbloids.spookystuff.utils.Timeout
trait Timed extends Action {
- var _timeout: TimeoutConf = _
+ var _timeout: Timeout = _
- def in(timeout: TimeoutConf): this.type = {
+ def in(timeout: Timeout): this.type = {
this._timeout = timeout
this
}
- def timeout(session: Session): TimeoutConf = {
+ def timeout(session: Session): Timeout = {
val base =
if (this._timeout == null) session.spooky.spookyConf.remoteResourceTimeout
else this._timeout
@@ -22,7 +21,7 @@ trait Timed extends Action {
base
}
- def hardTerminateTimeout(session: Session): TimeoutConf = {
+ def hardTerminateTimeout(session: Session): Timeout = {
val original = timeout(session)
original.copy(max = original.max + Const.hardTerminateOverhead)
}
diff --git a/core/src/main/scala/com/tribbloids/spookystuff/conf/DriverFactory.scala b/core/src/main/scala/com/tribbloids/spookystuff/conf/DriverFactory.scala
index c599d3aff..54a805649 100644
--- a/core/src/main/scala/com/tribbloids/spookystuff/conf/DriverFactory.scala
+++ b/core/src/main/scala/com/tribbloids/spookystuff/conf/DriverFactory.scala
@@ -17,7 +17,8 @@ package com.tribbloids.spookystuff.conf
import com.tribbloids.spookystuff.session.{DriverLike, Session}
import com.tribbloids.spookystuff.utils.CachingUtils.ConcurrentMap
-import com.tribbloids.spookystuff.utils.lifespan.{Cleanable, Lifespan}
+import com.tribbloids.spookystuff.utils.lifespan.Cleanable.Lifespan
+import com.tribbloids.spookystuff.utils.lifespan.Cleanable
import com.tribbloids.spookystuff.{DriverStatus, SpookyContext}
import org.apache.spark.TaskContext
@@ -38,7 +39,8 @@ sealed abstract class DriverFactory[D <: DriverLike] extends Serializable {
// release all Drivers that belong to a session
def release(session: Session): Unit
- def driverLifespan(session: Session): Lifespan = Lifespan.TaskOrJVM(ctxFactory = () => session.lifespan.ctx)
+ def driverLifespan(session: Session): Lifespan =
+ Lifespan.TaskOrJVM(ctxFactory = () => session.lifespan.ctx).forShipping
def deployGlobally(spooky: SpookyContext): Unit = {}
}
@@ -80,7 +82,7 @@ object DriverFactory {
final def destroy(driver: D, tcOpt: Option[TaskContext]): Unit = {
driver match {
- case v: Cleanable => v.tryClean()
+ case v: Cleanable => v.clean()
case _ =>
}
}
@@ -107,11 +109,11 @@ object DriverFactory {
override def dispatch(session: Session): D = {
val ls = driverLifespan(session)
- val taskLocalOpt = taskLocals.get(ls.batchIDs)
+ val taskLocalOpt = taskLocals.get(ls.registeredID)
def newDriver: D = {
val fresh = delegate.create(session)
- taskLocals.put(ls.batchIDs, new DriverStatus(fresh))
+ taskLocals.put(ls.registeredID, new DriverStatus(fresh))
fresh
}
@@ -144,7 +146,7 @@ object DriverFactory {
override def release(session: Session): Unit = {
val ls = driverLifespan(session)
- val statusOpt = taskLocals.get(ls.batchIDs)
+ val statusOpt = taskLocals.get(ls.registeredID)
statusOpt.foreach { status =>
status.isBusy = false
}
@@ -152,26 +154,4 @@ object DriverFactory {
override def deployGlobally(spooky: SpookyContext): Unit = delegate.deployGlobally(spooky)
}
-
- ////just for debugging
- ////a bug in this driver has caused it unusable in Firefox 32
- //object FirefoxDriverFactory extends DriverFactory {
- //
- // val baseCaps = new DesiredCapabilities
- // // baseCaps.setJavascriptEnabled(true); //< not really needed: JS enabled by default
- // // baseCaps.setCapability(CapabilityType.SUPPORTS_FINDING_BY_CSS,true)
- //
- // // val FirefoxRootPath = "/usr/lib/phantomjs/"
- // // baseCaps.setCapability("webdriver.firefox.bin", "firefox");
- // // baseCaps.setCapability("webdriver.firefox.profile", "WebDriver");
- //
- // override def newInstance(capabilities: Capabilities, spooky: SpookyContext): WebDriver = {
- // val newCap = baseCaps.merge(capabilities)
- //
- // Utils.retry(Const.DFSInPartitionRetry) {
- // Utils.withDeadline(spooky.distributedResourceTimeout) {new FirefoxDriver(newCap)}
- // }
- // }
- //}
-
}
diff --git a/core/src/main/scala/com/tribbloids/spookystuff/conf/PythonDriverFactory.scala b/core/src/main/scala/com/tribbloids/spookystuff/conf/PythonDriverFactory.scala
index 90f966580..d58aafbf4 100644
--- a/core/src/main/scala/com/tribbloids/spookystuff/conf/PythonDriverFactory.scala
+++ b/core/src/main/scala/com/tribbloids/spookystuff/conf/PythonDriverFactory.scala
@@ -2,7 +2,7 @@ package com.tribbloids.spookystuff.conf
import com.tribbloids.spookystuff.SpookyContext
import com.tribbloids.spookystuff.session.{PythonDriver, Session}
-import com.tribbloids.spookystuff.utils.lifespan.Lifespan
+import com.tribbloids.spookystuff.utils.lifespan.Cleanable.Lifespan
case class PythonDriverFactory(
getExecutable: SpookyContext => String
@@ -18,5 +18,7 @@ case class PythonDriverFactory(
object PythonDriverFactory {
- object _3 extends PythonDriverFactory((_: SpookyContext) => "python3")
+ lazy val python3 = "python3"
+
+ object _3 extends PythonDriverFactory((_: SpookyContext) => python3)
}
diff --git a/core/src/main/scala/com/tribbloids/spookystuff/conf/SpookyConf.scala b/core/src/main/scala/com/tribbloids/spookystuff/conf/SpookyConf.scala
index d2d4d6117..0d82ae148 100644
--- a/core/src/main/scala/com/tribbloids/spookystuff/conf/SpookyConf.scala
+++ b/core/src/main/scala/com/tribbloids/spookystuff/conf/SpookyConf.scala
@@ -3,7 +3,7 @@ package com.tribbloids.spookystuff.conf
import com.tribbloids.spookystuff.dsl._
import com.tribbloids.spookystuff.row.Sampler
import com.tribbloids.spookystuff.session._
-import com.tribbloids.spookystuff.utils.TimeoutConf
+import com.tribbloids.spookystuff.utils.Timeout
import org.apache.spark.SparkConf
import org.apache.spark.storage.StorageLevel
@@ -46,8 +46,8 @@ case class SpookyConf(
var errorDump: Boolean = true,
var errorScreenshot: Boolean = true,
var errorDumpFilePath: ByDoc[String] = FilePaths.UUIDName(FilePaths.Hierarchical),
- var remoteResourceTimeout: TimeoutConf = TimeoutConf(60.seconds),
- var DFSTimeout: TimeoutConf = TimeoutConf(40.seconds),
+ var remoteResourceTimeout: Timeout = Timeout(60.seconds),
+ var DFSTimeout: Timeout = Timeout(40.seconds),
var failOnDFSRead: Boolean = false,
var defaultJoinType: JoinType = JoinType.Inner,
var defaultFlattenSampler: Sampler[Any] = identity,
diff --git a/core/src/main/scala/com/tribbloids/spookystuff/python/ref/BindedRef.scala b/core/src/main/scala/com/tribbloids/spookystuff/python/ref/BindedRef.scala
index f0a66b615..2c77b4cdb 100644
--- a/core/src/main/scala/com/tribbloids/spookystuff/python/ref/BindedRef.scala
+++ b/core/src/main/scala/com/tribbloids/spookystuff/python/ref/BindedRef.scala
@@ -2,19 +2,20 @@ package com.tribbloids.spookystuff.python.ref
import com.tribbloids.spookystuff.SpookyContext
import com.tribbloids.spookystuff.session.PythonDriver
-import com.tribbloids.spookystuff.utils.lifespan.{Lifespan, LocalCleanable}
+import com.tribbloids.spookystuff.utils.lifespan.Cleanable.Lifespan
+import com.tribbloids.spookystuff.utils.lifespan.LocalCleanable
trait BindedRef extends PyRef with LocalCleanable {
def driverTemplate: PythonDriver
@transient var _driver: PythonDriver = _
- def driver = this.synchronized {
+ def driver: PythonDriver = this.synchronized {
Option(_driver).getOrElse {
val v = new PythonDriver(
driverTemplate.pythonExe,
driverTemplate.autoImports,
- _lifespan = new Lifespan.JVM(nameOpt = Some(this.getClass.getSimpleName))
+ _lifespan = Lifespan.JVM.apply(nameOpt = Some(this.getClass.getSimpleName))
)
_driver = v
v
diff --git a/core/src/main/scala/com/tribbloids/spookystuff/python/ref/PyRef.scala b/core/src/main/scala/com/tribbloids/spookystuff/python/ref/PyRef.scala
index 54ef65fd8..57c5bbcc9 100644
--- a/core/src/main/scala/com/tribbloids/spookystuff/python/ref/PyRef.scala
+++ b/core/src/main/scala/com/tribbloids/spookystuff/python/ref/PyRef.scala
@@ -4,9 +4,11 @@ import com.tribbloids.spookystuff.SpookyContext
import com.tribbloids.spookystuff.python.PyConverter
import com.tribbloids.spookystuff.session.{PythonDriver, Session}
import com.tribbloids.spookystuff.utils.CachingUtils.ConcurrentMap
+import com.tribbloids.spookystuff.utils.TreeThrowable
import com.tribbloids.spookystuff.utils.lifespan.Cleanable
import org.apache.spark.ml.dsl.utils._
+import scala.util.Try
trait PyRef extends Cleanable {
@@ -103,9 +105,11 @@ trait PyRef extends Cleanable {
override protected def cleanImpl(): Unit = {
- bindings.foreach { binding =>
- binding.clean(true)
- }
+ TreeThrowable.&&&(
+ bindings.map { binding =>
+ Try(binding.clean(true))
+ }
+ )
}
}
@@ -114,8 +118,8 @@ object PyRef {
object ROOT extends PyRef {}
def sanityCheck(): Unit = {
- val subs = Cleanable.getTyped[PyBinding]
- val refSubs = Cleanable.getTyped[PyRef].map(_.bindings)
+ val subs = Cleanable.All.typed[PyBinding]
+ val refSubs = Cleanable.All.typed[PyRef].map(_.bindings)
assert(
subs.intersect(refSubs).size <= refSubs.size, {
"INTERNAL ERROR: dangling tree!"
diff --git a/core/src/main/scala/com/tribbloids/spookystuff/session/AbstractSession.scala b/core/src/main/scala/com/tribbloids/spookystuff/session/AbstractSession.scala
deleted file mode 100644
index c65f0f18c..000000000
--- a/core/src/main/scala/com/tribbloids/spookystuff/session/AbstractSession.scala
+++ /dev/null
@@ -1,21 +0,0 @@
-package com.tribbloids.spookystuff.session
-
-import com.tribbloids.spookystuff.SpookyContext
-import com.tribbloids.spookystuff.actions._
-import com.tribbloids.spookystuff.utils.io.Progress
-import com.tribbloids.spookystuff.utils.lifespan.LocalCleanable
-import org.apache.spark.TaskContext
-
-import java.util.Date
-import scala.collection.mutable.ArrayBuffer
-
-abstract class AbstractSession(val spooky: SpookyContext) extends LocalCleanable {
-
- spooky.spookyMetrics.sessionInitialized += 1
- val startTime: Long = new Date().getTime
- val backtrace: ArrayBuffer[Action] = ArrayBuffer()
-
- def taskContextOpt: Option[TaskContext] = lifespan.ctx.taskOpt
-
- lazy val progress: Progress = Progress()
-}
diff --git a/core/src/main/scala/com/tribbloids/spookystuff/session/ConflictDetection.scala b/core/src/main/scala/com/tribbloids/spookystuff/session/ConflictDetection.scala
index d67d9a587..369ecc1fc 100644
--- a/core/src/main/scala/com/tribbloids/spookystuff/session/ConflictDetection.scala
+++ b/core/src/main/scala/com/tribbloids/spookystuff/session/ConflictDetection.scala
@@ -27,7 +27,7 @@ object ConflictDetection {
def conflicts: Seq[Try[Unit]] = {
- val allObj = Cleanable.getTyped[ConflictDetection]
+ val allObj = Cleanable.All.typed[ConflictDetection]
val allResourceIDs: Map[String, Seq[Any]] = allObj
.map {
diff --git a/core/src/main/scala/com/tribbloids/spookystuff/session/PythonDriver.scala b/core/src/main/scala/com/tribbloids/spookystuff/session/PythonDriver.scala
index b4208f64f..29c91ba3f 100644
--- a/core/src/main/scala/com/tribbloids/spookystuff/session/PythonDriver.scala
+++ b/core/src/main/scala/com/tribbloids/spookystuff/session/PythonDriver.scala
@@ -2,8 +2,8 @@ package com.tribbloids.spookystuff.session
import com.tribbloids.spookystuff.conf.Python
import com.tribbloids.spookystuff.driver.PythonProcess
-import com.tribbloids.spookystuff.utils.lifespan.Lifespan
-import com.tribbloids.spookystuff.utils.{BypassingRule, CommonUtils, SpookyUtils}
+import com.tribbloids.spookystuff.utils.lifespan.Cleanable.Lifespan
+import com.tribbloids.spookystuff.utils.{BypassingRule, CommonConst, CommonUtils, SpookyUtils}
import com.tribbloids.spookystuff.{PyException, PyInterpretationException, SpookyContext}
import org.apache.commons.io.FileUtils
import org.apache.spark.ml.dsl.utils.DSLUtils
@@ -94,12 +94,10 @@ class PythonDriver(
|import os
|from __future__ import print_function
""".trim.stripMargin,
- override val _lifespan: Lifespan = Lifespan.TaskOrJVM()
+ override val _lifespan: Lifespan = Lifespan.TaskOrJVM().forShipping
) extends PythonProcess(pythonExe)
with DriverLike {
- import scala.concurrent.duration._
-
/**
* NOT thread safe
*/
@@ -143,10 +141,10 @@ class PythonDriver(
override def cleanImpl(): Unit = {
Try {
- CommonUtils.retry(5) {
+ CommonUtils.retry(CommonConst.driverClosingRetries) {
try {
if (process.isAlive) {
- CommonUtils.withTimeout(3.seconds) {
+ CommonUtils.withTimeout(CommonConst.driverClosingTimeout) {
try {
this._interpret("exit()")
} catch {
diff --git a/core/src/main/scala/com/tribbloids/spookystuff/session/Session.scala b/core/src/main/scala/com/tribbloids/spookystuff/session/Session.scala
index df3d829c4..be0aa6231 100644
--- a/core/src/main/scala/com/tribbloids/spookystuff/session/Session.scala
+++ b/core/src/main/scala/com/tribbloids/spookystuff/session/Session.scala
@@ -1,10 +1,16 @@
package com.tribbloids.spookystuff.session
import com.tribbloids.spookystuff.SpookyContext
+import com.tribbloids.spookystuff.actions.Action
import com.tribbloids.spookystuff.conf.{Core, PluginRegistry, PluginSystem, Python}
import com.tribbloids.spookystuff.utils.TreeThrowable
-import com.tribbloids.spookystuff.utils.lifespan.Lifespan
+import com.tribbloids.spookystuff.utils.io.Progress
+import com.tribbloids.spookystuff.utils.lifespan.Cleanable.Lifespan
+import com.tribbloids.spookystuff.utils.lifespan.LocalCleanable
+import org.apache.spark.TaskContext
+import java.util.Date
+import scala.collection.mutable.ArrayBuffer
import scala.reflect.ClassTag
import scala.util.Try
@@ -13,9 +19,17 @@ import scala.util.Try
* should be manually cleaned By ActionLike, so don't set lifespan unless absolutely necessary
*/
class Session(
- override val spooky: SpookyContext,
- override val _lifespan: Lifespan = new Lifespan.JVM()
-) extends AbstractSession(spooky) {
+ val spooky: SpookyContext,
+ override val _lifespan: Lifespan = Lifespan.TaskOrJVM().forShipping
+) extends LocalCleanable {
+
+ spooky.spookyMetrics.sessionInitialized += 1
+ val startTime: Long = new Date().getTime
+ val backtrace: ArrayBuffer[Action] = ArrayBuffer()
+
+ def taskContextOpt: Option[TaskContext] = lifespan.ctx.taskOpt
+
+ lazy val progress: Progress = Progress()
object Drivers extends PluginRegistry.Factory {
@@ -27,7 +41,9 @@ class Session(
override def compute[V <: UB](v: V): Out[V] = {
val plugin: V#Plugin = spooky.Plugins.apply(v)
+ progress.ping()
val result = plugin.driverFactory.dispatch(Session.this)
+ progress.ping()
spooky.getMetric(Core).driverDispatched.add(plugin.driverFactory.toString -> 1L)
@@ -45,12 +61,12 @@ class Session(
override def cleanImpl(): Unit = {
val plugins = spooky.Plugins.cache.values.toList
- val withDrivers = plugins.collect {
- case p: PluginSystem.WithDrivers#PluginLike =>
+ val filtered = plugins.collect {
+ case p: PluginSystem.WithDrivers#Plugin =>
p
}
- val trials = withDrivers.map { p =>
+ val trials = filtered.map { p =>
Try {
p.driverFactoryOpt.foreach { v =>
v.release(this)
@@ -62,14 +78,6 @@ class Session(
TreeThrowable.&&&(trials)
-// Option(spooky.spookyConf.webDriverFactory).foreach { factory =>
-// factory.release(this)
-// spooky.spookyMetrics.webDriverReleased += 1
-// }
-// Option(spooky.spookyConf.pythonDriverFactory).foreach { factory =>
-// factory.release(this)
-// spooky.spookyMetrics.pythonDriverReleased += 1
-// }
spooky.spookyMetrics.sessionReclaimed += 1
}
}
diff --git a/core/src/main/scala/com/tribbloids/spookystuff/utils/BroadcastWrapper.scala b/core/src/main/scala/com/tribbloids/spookystuff/utils/BroadcastWrapper.scala
index 59ef20f0f..dcb88f9c9 100644
--- a/core/src/main/scala/com/tribbloids/spookystuff/utils/BroadcastWrapper.scala
+++ b/core/src/main/scala/com/tribbloids/spookystuff/utils/BroadcastWrapper.scala
@@ -1,7 +1,7 @@
package com.tribbloids.spookystuff.utils
import com.tribbloids.spookystuff.utils.lifespan.Cleanable
-import org.apache.spark.{SparkContext, SparkException}
+import org.apache.spark.SparkContext
import org.apache.spark.broadcast.Broadcast
import org.slf4j.LoggerFactory
@@ -36,13 +36,17 @@ case class BroadcastWrapper[T](
}
private def destroy(): Unit = this.synchronized {
- try {
- Option(wrapped).foreach(_.destroy())
- } catch {
- case _: NullPointerException | _: SparkException =>
- // Spark master is gone, no need to destroy
- case e: Exception =>
- LoggerFactory.getLogger(this.getClass).error("broadcast cannot be destroyed", e)
+ if (sc == null || sc.isStopped) {
+ // do nothing
+ } else {
+ try {
+ Option(wrapped).foreach(_.destroy())
+ } catch {
+// case _: NullPointerException | _: SparkException =>
+ // Spark master is gone, no need to destroy
+ case e: Exception =>
+ LoggerFactory.getLogger(this.getClass).error("broadcast cannot be destroyed", e)
+ }
}
}
diff --git a/core/src/main/scala/org/apache/spark/rdd/spookystuff/ExternalAppendOnlyArray.scala b/core/src/main/scala/org/apache/spark/rdd/spookystuff/ExternalAppendOnlyArray.scala
index 8401a777f..ad03d7c49 100644
--- a/core/src/main/scala/org/apache/spark/rdd/spookystuff/ExternalAppendOnlyArray.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/spookystuff/ExternalAppendOnlyArray.scala
@@ -1,9 +1,11 @@
package org.apache.spark.rdd.spookystuff
+import com.tribbloids.spookystuff.utils.lifespan.Cleanable.Lifespan
+
import java.io._
import java.util.UUID
import java.util.concurrent.atomic.AtomicInteger
-import com.tribbloids.spookystuff.utils.lifespan.{Lifespan, LocalCleanable}
+import com.tribbloids.spookystuff.utils.lifespan.LocalCleanable
import com.tribbloids.spookystuff.utils.serialization.NOTSerializable
import com.tribbloids.spookystuff.utils.{CachingUtils, CommonConst, CommonUtils, ThreadLocal}
import org.apache.spark.serializer
@@ -25,7 +27,7 @@ class ExternalAppendOnlyArray[T] private[spookystuff] (
id: String,
storageLevel: StorageLevel,
serializerFactory: () => serializer.Serializer,
- override val _lifespan: Lifespan = Lifespan.JVM()
+ override val _lifespan: Lifespan = Lifespan.JVM.apply()
)(
implicit val ctag: ClassTag[T]
) extends LocalCleanable {
diff --git a/core/src/main/scala/org/apache/spark/rdd/spookystuff/IncrementallyCachedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/spookystuff/IncrementallyCachedRDD.scala
index 4aba1734e..c300a1dfc 100644
--- a/core/src/main/scala/org/apache/spark/rdd/spookystuff/IncrementallyCachedRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/spookystuff/IncrementallyCachedRDD.scala
@@ -2,7 +2,8 @@ package org.apache.spark.rdd.spookystuff
import com.tribbloids.spookystuff.utils.CachingUtils.ConcurrentMap
import com.tribbloids.spookystuff.utils.accumulator.MapAccumulator
-import com.tribbloids.spookystuff.utils.lifespan.{Cleanable, Lifespan, LocalCleanable}
+import com.tribbloids.spookystuff.utils.lifespan.Cleanable.Lifespan
+import com.tribbloids.spookystuff.utils.lifespan.{Cleanable, LocalCleanable}
import com.tribbloids.spookystuff.utils.{CachingUtils, IDMixin, Retry, SCFunctions}
import org.apache.spark
import org.apache.spark.broadcast.Broadcast
@@ -254,7 +255,7 @@ case class IncrementallyCachedRDD[T: ClassTag](
}
}
- override def _lifespan: Lifespan = Lifespan.JVM()
+ override def _lifespan: Lifespan = Lifespan.JVM.apply()
/**
* can only be called once
@@ -369,7 +370,7 @@ case class IncrementallyCachedRDD[T: ClassTag](
val result = SCFunctions(sparkContext).withJob(info) {
this
- .mapOncePerWorker { v =>
+ .mapOncePerWorker { _ =>
logInfo(info + s" - executor ${SparkHelper.taskLocationStrOpt.getOrElse("??")}")
val result = depCache.cleanUp()
diff --git a/core/src/test/scala/com/tribbloids/spookystuff/Python3DriverSuite.scala b/core/src/test/scala/com/tribbloids/spookystuff/Python3DriverSuite.scala
index ac0bf3e12..2eeffbef1 100644
--- a/core/src/test/scala/com/tribbloids/spookystuff/Python3DriverSuite.scala
+++ b/core/src/test/scala/com/tribbloids/spookystuff/Python3DriverSuite.scala
@@ -3,8 +3,8 @@ package com.tribbloids.spookystuff
import com.tribbloids.spookystuff.python.PyConverter
import Python3DriverSuite.Runner
import com.tribbloids.spookystuff.session.PythonDriver
-import com.tribbloids.spookystuff.utils.CommonUtils
-import com.tribbloids.spookystuff.utils.lifespan.Lifespan
+import com.tribbloids.spookystuff.utils.lifespan.Cleanable.Lifespan
+import com.tribbloids.spookystuff.utils.{CommonConst, CommonUtils}
import org.slf4j.LoggerFactory
import scala.concurrent.Future
@@ -22,10 +22,14 @@ object Python3DriverSuite {
def runIterable[T, R](xs: Iterable[T])(f: (T, PythonDriver) => R): Iterable[R] = {
val pythonExec = this.pythonExec
- val proc = new PythonDriver(pythonExec,
- _lifespan = Lifespan.TaskOrJVM(
- nameOpt = Some("testPython")
- ))
+ val proc = new PythonDriver(
+ pythonExec,
+ _lifespan = Lifespan
+ .TaskOrJVM(
+ nameOpt = Some("testPython")
+ )
+ .forShipping
+ )
try {
val result = xs.map {
f(_, proc)
@@ -205,7 +209,10 @@ class Python3DriverSuite extends SpookyEnvFixture {
}
LoggerFactory.getLogger(this.getClass).info("========= START CLEANING =========")
- CommonUtils.withTimeout(20.seconds, 1.second) { // 3 * 15 << 20
+ CommonUtils.withTimeout(
+ (CommonConst.driverClosingTimeout * CommonConst.driverClosingRetries) + 5.seconds,
+ 1.second
+ ) {
proc.clean()
}
}
diff --git a/core/src/test/scala/com/tribbloids/spookystuff/SpookyEnvFixture.scala b/core/src/test/scala/com/tribbloids/spookystuff/SpookyEnvFixture.scala
index d7bd036ef..51d4e8120 100644
--- a/core/src/test/scala/com/tribbloids/spookystuff/SpookyEnvFixture.scala
+++ b/core/src/test/scala/com/tribbloids/spookystuff/SpookyEnvFixture.scala
@@ -7,7 +7,8 @@ import com.tribbloids.spookystuff.extractors.{Alias, GenExtractor, GenResolved}
import com.tribbloids.spookystuff.row.{SpookySchema, SquashedFetchedRow, TypedField}
import com.tribbloids.spookystuff.session.DriverLike
import com.tribbloids.spookystuff.testutils.{FunSpecx, RemoteDocsFixture, TestHelper}
-import com.tribbloids.spookystuff.utils.lifespan.{Cleanable, Lifespan}
+import com.tribbloids.spookystuff.utils.lifespan.Cleanable
+import com.tribbloids.spookystuff.utils.lifespan.Cleanable.Lifespan
import com.tribbloids.spookystuff.utils.{CommonConst, CommonUtils, Retry, SparkUISupport}
import org.apache.spark.SparkContext
import org.apache.spark.sql.SQLContext
@@ -44,13 +45,15 @@ object SpookyEnvFixture {
Cleanable.uncleaned
.foreach { tuple =>
- val nonLocalDrivers = tuple._2.values
+ val taskCleanable = tuple._2.values
.filter { v =>
- v.lifespan.isInstanceOf[Lifespan.Task]
+ v.lifespan.leaves.exists { ll =>
+ ll._type == Lifespan.Task
+ }
}
Predef.assert(
- nonLocalDrivers.isEmpty,
- s": ${tuple._1} is unclean! ${nonLocalDrivers.size} left:\n" + nonLocalDrivers.mkString("\n")
+ taskCleanable.isEmpty,
+ s": ${tuple._1} is unclean! ${taskCleanable.size} left:\n" + taskCleanable.mkString("\n")
)
}
}
@@ -69,7 +72,7 @@ object SpookyEnvFixture {
if (cleanSweepDrivers) {
//this is necessary as each suite won't automatically cleanup drivers NOT in task when finished
- Cleanable.cleanSweepAll(
+ Cleanable.All.cleanSweep(
condition = {
case _: DriverLike => true
case _ => false
@@ -152,7 +155,7 @@ abstract class SpookyEnvFixture
import com.tribbloids.spookystuff.utils.SpookyViews._
- def _processNames: Seq[String] = Seq("phantomjs", "python")
+ def _processNames: Seq[String] = Seq("phantomjs", s"${PythonDriverFactory.python3} -iu")
final lazy val conditions = {
val _processNames = this._processNames
val exitingPIDs = this.exitingPIDs
@@ -165,34 +168,33 @@ abstract class SpookyEnvFixture
}
}
- override def beforeAll(): Unit = if (SpookyEnvFixture.firstRun) {
+ def validateBeforeAndAfterAll(): Unit = {
- super.beforeAll()
+ TestHelper.cleanTempDirs()
val spooky = this.spooky
val conditions = this.conditions
sc.runEverywhere() { _ =>
SpookyEnvFixture.shouldBeClean(spooky, conditions)
}
+
SpookyEnvFixture.firstRun = false
}
- override def afterAll(): Unit = {
+ override def beforeAll(): Unit = {
- val spooky = this.spooky
- val conditions = this.conditions
- TestHelper.cleanTempDirs()
+ super.beforeAll()
- //unpersist all RDDs, disabled to better detect memory leak
- // sc.getPersistentRDDs.values.toList.foreach {
- // _.unpersist()
- // }
+ if (SpookyEnvFixture.firstRun)
+ validateBeforeAndAfterAll()
+ }
- sc.runEverywhere() { _ =>
- SpookyEnvFixture.shouldBeClean(spooky, conditions)
- }
+ override def afterAll(): Unit = {
+
+ validateBeforeAndAfterAll()
super.afterAll()
+
}
override def beforeEach(): Unit = CommonUtils.retry(3, 1000) {
diff --git a/core/src/test/scala/com/tribbloids/spookystuff/actions/ActionSuite.scala b/core/src/test/scala/com/tribbloids/spookystuff/actions/ActionSuite.scala
index 91a078ad6..d8b7f323f 100644
--- a/core/src/test/scala/com/tribbloids/spookystuff/actions/ActionSuite.scala
+++ b/core/src/test/scala/com/tribbloids/spookystuff/actions/ActionSuite.scala
@@ -5,7 +5,7 @@ import com.tribbloids.spookystuff.dsl._
import com.tribbloids.spookystuff.extractors.impl.Lit
import com.tribbloids.spookystuff.row.{DataRow, FetchedRow, Field}
import com.tribbloids.spookystuff.session.Session
-import com.tribbloids.spookystuff.utils.{CommonUtils, TimeoutConf}
+import com.tribbloids.spookystuff.utils.{CommonUtils, Timeout}
import com.tribbloids.spookystuff.{ActionException, Const, SpookyEnvFixture}
import scala.collection.immutable.ListMap
@@ -20,7 +20,7 @@ class ActionSuite extends SpookyEnvFixture {
it("interpolate should not change timeout") {
import scala.concurrent.duration._
- val randomTimeout = TimeoutConf(Random.nextInt().seconds)
+ val randomTimeout = Timeout(Random.nextInt().seconds)
val action = Wget(Const.keyDelimiter + "{~}").in(randomTimeout)
val rewritten = action
diff --git a/core/src/test/scala/com/tribbloids/spookystuff/execution/TestExplorePlan.scala b/core/src/test/scala/com/tribbloids/spookystuff/execution/TestExplorePlan.scala
index bba0492b7..21c096756 100644
--- a/core/src/test/scala/com/tribbloids/spookystuff/execution/TestExplorePlan.scala
+++ b/core/src/test/scala/com/tribbloids/spookystuff/execution/TestExplorePlan.scala
@@ -75,24 +75,23 @@ class TestExplorePlan extends SpookyEnvFixture with LocalPathDocsFixture {
it("ExplorePlan should work recursively on directory") {
- val resourcePath = DIR_URL
+ val resourcePath = DEEP_DIR_URL
val df = spooky
- .create(Seq(resourcePath.toString))
+ .create(Seq(resourcePath))
.fetch {
Wget('_)
}
- .explore(S"root directory".attr("path"))(
+ .explore(S"root directory URI".text)(
Wget('A)
)()
.flatExtract(S"root file")(
- 'A.ownText ~ 'leaf,
- 'A.attr("path") ~ 'fullPath,
- 'A.allAttr ~ 'metadata
+ A"name".text ~ 'leaf,
+ A"URI".text ~ 'fullPath
)
.toDF(sort = true)
- df.show(truncate = false)
+ assert(df.collectAsList().size() == 6)
}
it("ExplorePlan will throw an exception if OrdinalField == DepthField") {
diff --git a/core/src/test/scala/com/tribbloids/spookystuff/CleanableSuite.scala b/core/src/test/scala/com/tribbloids/spookystuff/lifespan/CleanableSuite.scala
similarity index 58%
rename from core/src/test/scala/com/tribbloids/spookystuff/CleanableSuite.scala
rename to core/src/test/scala/com/tribbloids/spookystuff/lifespan/CleanableSuite.scala
index 5c36be755..6b84aa59f 100644
--- a/core/src/test/scala/com/tribbloids/spookystuff/CleanableSuite.scala
+++ b/core/src/test/scala/com/tribbloids/spookystuff/lifespan/CleanableSuite.scala
@@ -1,36 +1,48 @@
-package com.tribbloids.spookystuff
+package com.tribbloids.spookystuff.lifespan
-import com.tribbloids.spookystuff.CleanableSuite.DummyCleanable
+import com.tribbloids.spookystuff.SpookyEnvFixture
import com.tribbloids.spookystuff.utils.CommonUtils
-import com.tribbloids.spookystuff.utils.lifespan.{Cleanable, Lifespan, LocalCleanable, SparkLifespan}
+import com.tribbloids.spookystuff.utils.lifespan.Cleanable.Lifespan
+import com.tribbloids.spookystuff.utils.lifespan.{Cleanable, LeafType, LocalCleanable}
import com.tribbloids.spookystuff.utils.serialization.AssertSerializable
import org.apache.spark.{HashPartitioner, SparkException, TaskContext}
import scala.util.Random
+// TODO: move to previous module
/**
* Created by peng on 16/11/16.
*/
class CleanableSuite extends SpookyEnvFixture {
+ import com.tribbloids.spookystuff.lifespan.CleanableSuite._
import com.tribbloids.spookystuff.utils.SpookyViews._
override def beforeEach(): Unit = {
super.beforeEach()
sc.runEverywhere(alsoOnDriver = false) { _ =>
- Cleanable.cleanSweepAll {
+ Cleanable.All.cleanSweep {
case _: DummyCleanable => true
case _ => false
}
}
}
+ it("Lifespan.JVM.batchID is serializable") {
+ val v1 = Lifespan.JVM().batchID
+ val v2 = v1.copy(v1.id)
+ assert(v1 == v2)
+
+ AssertSerializable(v1)
+ }
+
it("Lifespan.JVM is serializable") {
- AssertSerializable(Lifespan.JVM())
+ lifespanIsSerializable(Lifespan.JVM())
+
val rdd = sc.uuidSeed().map { _ =>
val lifespan = Lifespan.JVM()
- AssertSerializable(lifespan)
+ lifespanIsSerializable(lifespan)
lifespan
}
rdd.count()
@@ -41,33 +53,21 @@ class CleanableSuite extends SpookyEnvFixture {
val rdd = sc.uuidSeed().map { _ =>
val lifespan = Lifespan.Task()
- AssertSerializable(lifespan)
+ lifespanIsSerializable(lifespan)
lifespan
}
rdd.count()
intercept[SparkException] { //cannot be re-initialized outside Task
- rdd.collect()
+ val vv = rdd.collect()
+ vv
}
}
- it("Lifespan.SparkApp is serializable") {
-
- AssertSerializable(SparkLifespan.App())
-
- //TODO: the following doesn't work in local mode
-// intercept[SparkException] {
-// sc.uuidSeed().foreach { _ =>
-// val lifespan = Lifespan.SparkApp()
-// AssertSerializable(lifespan)
-// }
-// }
- }
-
it("Lifespan.batchIDs should be updated after being shipped to a different executor") {
val rdd = sc.uuidSeed().mapOncePerCore { _ =>
val lifespan = Lifespan.Task()
- val oldID = lifespan.batchIDs.head.asInstanceOf[Lifespan.Task.ID].id
+ val oldID = lifespan.registeredID.head.asInstanceOf[Lifespan.Task.ID].id
lifespan -> oldID
}
@@ -78,7 +78,7 @@ class CleanableSuite extends SpookyEnvFixture {
val old_new = repartitioned.map { tuple =>
val v = tuple._1
val newID = TaskContext.get().taskAttemptId()
- Predef.assert(v.batchIDs.head.asInstanceOf[Lifespan.Task.ID].id == newID)
+ Predef.assert(v.registeredID.head.asInstanceOf[Lifespan.Task.ID].id == newID)
tuple._2 -> newID
}.collectPerPartition
@@ -90,7 +90,7 @@ class CleanableSuite extends SpookyEnvFixture {
val rdd = sc.uuidSeed().mapOncePerWorker { _ =>
val lifespan = Lifespan.TaskOrJVM()
- val oldID = lifespan.batchIDs.head.asInstanceOf[Lifespan.Task.ID].id
+ val oldID = lifespan.registeredID.head.asInstanceOf[Lifespan.Task.ID].id
lifespan -> oldID
}
@@ -99,16 +99,16 @@ class CleanableSuite extends SpookyEnvFixture {
collected
.foreach { tuple =>
val v = tuple._1
- Predef.assert(v.batchIDs.head.isInstanceOf[Lifespan.JVM.ID])
+ Predef.assert(v.registeredID.head.isInstanceOf[Lifespan.JVM.ID])
}
}
- it("Lifespan._id should be updated after being shipped to a new thread created by a different executor") {
+ it("Lifespan.batchIDs should be updated after being shipped to a new thread created by a different executor") {
import scala.concurrent.duration._
val rdd = sc.uuidSeed().mapOncePerCore { _ =>
val lifespan = Lifespan.Task()
- val oldID = lifespan.batchIDs.head.asInstanceOf[Lifespan.Task.ID].id
+ val oldID = lifespan.registeredID.head.asInstanceOf[Lifespan.Task.ID].id
lifespan -> oldID
}
@@ -116,23 +116,22 @@ class CleanableSuite extends SpookyEnvFixture {
.partitionBy(new HashPartitioner(4))
assert(repartitioned.partitions.length == 4)
- repartitioned
- .map { tuple =>
- val v: Lifespan = tuple._1
- val newID = TaskContext.get().taskAttemptId()
- // val newID2 = v._id
- val newID3 = CommonUtils.withTimeout(10.seconds) {
- val result = v.batchIDs.head
- // Predef.assert(v._id == newID2)
- result
- }
- Predef.assert(newID3.asInstanceOf[Lifespan.Task.ID].id == newID)
- tuple._2 -> newID
- }
- .collectPerPartition
- .foreach {
- println
+ val old_new = repartitioned.map { tuple =>
+ val v: Lifespan = tuple._1
+ val newID = TaskContext.get().taskAttemptId()
+ // val newID2 = v._id
+ val newID3 = CommonUtils.withTimeout(10.seconds) {
+ val result = v.registeredID.head
+ // Predef.assert(v._id == newID2)
+ result
}
+ Predef.assert(newID3.asInstanceOf[Lifespan.Task.ID].id == newID)
+ tuple._2 -> newID
+ }.collectPerPartition
+
+ val flatten = old_new.toList.flatten
+
+ assert(flatten.count(v => v._1 == v._2) < flatten.size)
}
it("can get all created Cleanables") {
@@ -151,7 +150,7 @@ class CleanableSuite extends SpookyEnvFixture {
val i2 = sc
.uuidSeed()
.mapOncePerWorker { _ =>
- Cleanable.getTyped[DummyCleanable].map(_.index)
+ Cleanable.All.typed[DummyCleanable].map(_.index)
}
.flatMap(identity)
.collect()
@@ -176,4 +175,19 @@ object CleanableSuite {
override protected def cleanImpl(): Unit = {}
}
+
+ def lifespanIsSerializable(v: Lifespan): Unit = {
+
+ AssertSerializable[Lifespan](
+ v,
+ condition = { (v1, v2) =>
+ AssertSerializable.serializableCondition(v1, v2)
+ Seq(v1, v2).foreach {
+ case v: LeafType#Internal =>
+ v.requireUsable()
+ case _ =>
+ }
+ }
+ )
+ }
}
diff --git a/core/src/test/scala/com/tribbloids/spookystuff/utils/io/HDFSResolverSuite.scala b/core/src/test/scala/com/tribbloids/spookystuff/utils/io/HDFSResolverSuite.scala
index d110a97d0..d7ccf9fa1 100755
--- a/core/src/test/scala/com/tribbloids/spookystuff/utils/io/HDFSResolverSuite.scala
+++ b/core/src/test/scala/com/tribbloids/spookystuff/utils/io/HDFSResolverSuite.scala
@@ -36,7 +36,7 @@ class HDFSResolverSuite extends AbstractURIResolverSuite {
}
it("can override login UGI") {
- val user: String = resolverWithUGI.input(HTML_URL) { is =>
+ val user: String = resolverWithUGI.input(HTML_URL) { _ =>
UserGroupInformation.getCurrentUser.getUserName
}
user.shouldBe("dummy")
@@ -47,8 +47,8 @@ class HDFSResolverSuite extends AbstractURIResolverSuite {
val HTML_URL = this.HTML_URL
val users = sc
.parallelize(1 to (sc.defaultParallelism * 2))
- .mapPartitions { itr =>
- val str: String = resolver.input(HTML_URL) { is =>
+ .mapPartitions { _ =>
+ val str: String = resolver.input(HTML_URL) { _ =>
UserGroupInformation.getCurrentUser.getUserName
}
Iterator(str)
diff --git a/dev/CI/stage1-core.sh b/dev/CI/stage1-core.sh
index 635d7101f..0680afabf 100755
--- a/dev/CI/stage1-core.sh
+++ b/dev/CI/stage1-core.sh
@@ -2,11 +2,9 @@
FWDIR="$(cd "`dirname "$0"`"/..; pwd)"
-set -e
-
# shellcheck source=profiles/apache-stable/.common.sh
source "${FWDIR}/profiles/${1}/.common.sh"
-"$FWDIR"/mvn-install.sh "${MVN_PROFILES[@]}" -Pbenchmark && \
-"$FWDIR"/test.sh "${MVN_PROFILES[@]}" -Pbenchmark && \
-"$FWDIR"/test-reports.sh
+("$FWDIR"/mvn-install.sh "${MVN_PROFILES[@]}" -Pbenchmark && \
+"$FWDIR"/test.sh "${MVN_PROFILES[@]}" -Pbenchmark) || \
+("$FWDIR"/test-reports.sh && exit 1)
diff --git a/dev/CI/stage1-uav.sh b/dev/CI/stage1-uav.sh
index 03e1175ac..3485612e3 100755
--- a/dev/CI/stage1-uav.sh
+++ b/dev/CI/stage1-uav.sh
@@ -2,8 +2,6 @@
FWDIR="$(cd "`dirname "$0"`"/..; pwd)"
-set -e
-
# shellcheck source=profiles/apache-stable/.common.sh
source "${FWDIR}/profiles/${1}/.common.sh"
diff --git a/dev/test-install.sh b/dev/test-install.sh
index 292eb8cf4..998f09dee 100755
--- a/dev/test-install.sh
+++ b/dev/test-install.sh
@@ -12,7 +12,4 @@ mvn --version
mvn clean install --errors -f "$FWDIR"/repackaged/selenium-bundle/pom.xml "$@"
"${CRDIR}/tree.sh" "$@"
-#see https://intoli.com/blog/exit-on-errors-in-bash-scripts/
-set -e
-
mvn clean install --errors -f "$FWDIR"/pom.xml -Pdist "$@"
diff --git a/dev/test-reports.sh b/dev/test-reports.sh
index 3ff12aeba..e8c962dca 100755
--- a/dev/test-reports.sh
+++ b/dev/test-reports.sh
@@ -2,4 +2,27 @@
FWDIR="$(cd "`dirname "$0"`"/..; pwd)"
-find "${FWDIR}" -wholename "**/scalatest-report.txt" -exec cat {} \;
\ No newline at end of file
+echo ""
+echo "###############"
+echo "# TEST REPORT #"
+echo "###############"
+echo ""
+
+cd "${FWDIR}"
+
+#find cannot use symbolic link, so cd is used
+#see https://unix.stackexchange.com/questions/93857/find-does-not-work-on-symlinked-path
+find . -wholename "**/scalatest/scalatest-report.txt" -print -exec cat {} \;
+
+
+echo ""
+echo "###############"
+echo "# FAILED #"
+echo "###############"
+echo ""
+
+cd "${FWDIR}"
+
+#find cannot use symbolic link, so cd is used
+#see https://unix.stackexchange.com/questions/93857/find-does-not-work-on-symlinked-path
+find . -wholename "**/scalatest/scalatest-failed.txt" -print -exec cat {} \;
\ No newline at end of file
diff --git a/dev/test.sh b/dev/test.sh
index 1d059dc30..0507de007 100755
--- a/dev/test.sh
+++ b/dev/test.sh
@@ -5,7 +5,5 @@ FWDIR="$(cd "`dirname "$0"`"/..; pwd)"
source "${CRDIR}/.mvn-common.sh"
-set -e
-
#see http://www.bloggure.info/MavenFailTestAtEnd/
-mvn test -f "$FWDIR"/pom.xml --fail-at-end -Pdist "$@"
+mvn test -f "$FWDIR"/pom.xml -Pdist "$@"
diff --git a/integration/src/test/scala/com/tribbloids/spookystuff/integration/IntegrationFixture.scala b/integration/src/test/scala/com/tribbloids/spookystuff/integration/IntegrationFixture.scala
index 26e394bce..a9c5f44c3 100644
--- a/integration/src/test/scala/com/tribbloids/spookystuff/integration/IntegrationFixture.scala
+++ b/integration/src/test/scala/com/tribbloids/spookystuff/integration/IntegrationFixture.scala
@@ -9,13 +9,13 @@ import com.tribbloids.spookystuff.testutils.{LocalURIDocsFixture, TestHelper}
import com.tribbloids.spookystuff.utils.{CommonConst, CommonUtils}
import com.tribbloids.spookystuff.web.conf.{Web, WebDriverFactory}
import com.tribbloids.spookystuff.web.session.CleanWebDriver
-import org.scalatest.BeforeAndAfterAll
+import org.slf4j.LoggerFactory
import java.util.Date
import scala.concurrent.duration
import scala.util.Random
-abstract class IntegrationFixture extends SpookyEnvFixture with BeforeAndAfterAll with LocalURIDocsFixture {
+abstract class IntegrationFixture extends SpookyEnvFixture with LocalURIDocsFixture {
val phantomJS: WebDriverFactory.PhantomJS = WebDriverFactory.PhantomJS()
val htmlUnit: WebDriverFactory.HtmlUnit = WebDriverFactory.HtmlUnit()
@@ -70,7 +70,7 @@ abstract class IntegrationFixture extends SpookyEnvFixture with BeforeAndAfterAl
def assertBeforeCache(): Unit = {
val metrics: SpookyMetrics = spooky.spookyMetrics
val metricsJSON: String = metrics.toNestedMap.toJSON()
- println(metricsJSON)
+ LoggerFactory.getLogger(this.getClass).info(metricsJSON)
val pagesFetched = metrics.pagesFetched.value
remotePagesFetched = metrics.pagesFetchedFromRemote.value
@@ -87,7 +87,7 @@ abstract class IntegrationFixture extends SpookyEnvFixture with BeforeAndAfterAl
def assertAfterCache(): Unit = {
val metrics: SpookyMetrics = spooky.spookyMetrics
val metricsJSON: String = metrics.toNestedMap.toJSON()
- println(metricsJSON)
+ LoggerFactory.getLogger(this.getClass).info(metricsJSON)
val pagesFetched = metrics.pagesFetched.value
assert(pagesFetched >= numPages)
diff --git a/integration/src/test/scala/com/tribbloids/spookystuff/integration/explore/ExploreClickNextPageIT.scala b/integration/src/test/scala/com/tribbloids/spookystuff/integration/explore/ExploreClickNextPageIT.scala
index 9d270c03d..a1f6ab9c6 100644
--- a/integration/src/test/scala/com/tribbloids/spookystuff/integration/explore/ExploreClickNextPageIT.scala
+++ b/integration/src/test/scala/com/tribbloids/spookystuff/integration/explore/ExploreClickNextPageIT.scala
@@ -11,10 +11,6 @@ class ExploreClickNextPageIT extends IntegrationFixture {
import com.tribbloids.spookystuff.dsl._
-// override lazy val driverFactories = Seq(
-// phantomJS //TODO: HtmlUnit does not support Backbone.js
-// )
-
override def doMain(): Unit = {
val snapshotAllPages = (Snapshot()
diff --git a/integration/src/test/scala/com/tribbloids/spookystuff/integration/select/SelectIT.scala b/integration/src/test/scala/com/tribbloids/spookystuff/integration/select/SelectIT.scala
index 64f605052..bf2597fef 100644
--- a/integration/src/test/scala/com/tribbloids/spookystuff/integration/select/SelectIT.scala
+++ b/integration/src/test/scala/com/tribbloids/spookystuff/integration/select/SelectIT.scala
@@ -72,7 +72,18 @@ class SelectIT extends IntegrationFixture {
.toDF(sort = true)
df2.schema.treeString.shouldBe(
- )
+ """
+ |root
+ | |-- _c1: string (nullable = true)
+ | |-- _c2: timestamp (nullable = true)
+ | |-- title: array (nullable = true)
+ | | |-- element: string (containsNull = true)
+ | |-- langs: array (nullable = true)
+ | | |-- element: string (containsNull = true)
+ | |-- expanded: array (nullable = true)
+ | | |-- element: string (containsNull = true)
+ |""".stripMargin
+ )
val rows2 = df2.collect()
val titles = rows2.head.getAs[Iterable[String]]("title")
diff --git a/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/AwaitWithHeartbeat.scala b/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/AwaitWithHeartbeat.scala
index 808e867d4..11e58420b 100644
--- a/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/AwaitWithHeartbeat.scala
+++ b/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/AwaitWithHeartbeat.scala
@@ -71,7 +71,7 @@ case class AwaitWithHeartbeat(
result
}
- def result[T](future: Future[T], timeout: TimeoutConf): T = {
+ def result[T](future: Future[T], timeout: Timeout): T = {
val maxTimeMil = timeout.max.toMillis
val maxNoProgressMil = timeout.noProgress.toMillis
diff --git a/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/CommonConst.scala b/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/CommonConst.scala
index c13d09f3b..bfa73edb0 100644
--- a/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/CommonConst.scala
+++ b/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/CommonConst.scala
@@ -9,11 +9,8 @@ class CommonConst {
val defaultTextCharset = "ISO-8859-1"
val defaultApplicationCharset = "UTF-8"
- // val webClientOptions = new WebClientOptions
- // webClientOptions.setUseInsecureSSL(true)
-
- //TODO: move to SpookyConf as much as possible
- val sessionInitializationTimeout: FiniteDuration = 40.seconds
+ val driverClosingTimeout: FiniteDuration = 3.seconds
+ val driverClosingRetries: Int = 5
val localResourceLocalRetries = 3 //In-node/partition retries
val remoteResourceLocalRetries = 2 //In-node/partition retries
diff --git a/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/CommonUtils.scala b/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/CommonUtils.scala
index 755d75709..be627c0b5 100644
--- a/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/CommonUtils.scala
+++ b/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/CommonUtils.scala
@@ -66,7 +66,7 @@ abstract class CommonUtils {
}
def withTimeout[T](
- timeout: TimeoutConf,
+ timeout: Timeout,
interval: Duration = 10.seconds
)(
fn: => T,
diff --git a/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/HasID.scala b/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/HasID.scala
deleted file mode 100644
index cae1446d1..000000000
--- a/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/HasID.scala
+++ /dev/null
@@ -1,18 +0,0 @@
-package com.tribbloids.spookystuff.utils
-
-trait HasID {
-
- protected def _id: Any
-
- protected def _idCompose(id: Any): Any = {
- val result = _id match {
- case aa: Array[_] => aa.toList
- case _ => id
- }
- result
- }
-
- @transient final lazy val id: Any = {
- _idCompose(_id)
- }
-}
diff --git a/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/IDMixin.scala b/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/IDMixin.scala
index 7a4fe0438..e1d70b2d7 100644
--- a/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/IDMixin.scala
+++ b/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/IDMixin.scala
@@ -1,19 +1,42 @@
package com.tribbloids.spookystuff.utils
-trait IDMixin extends HasID {
+// TODO: renamed to EqualityFix
+trait IDMixin {
+
+ import IDMixin._
+
+ // TODO: renamed to equalityID
+ protected def _id: Any
+ @transient final private lazy val id: Any = {
+ idCompose(_id)
+ }
override def hashCode: Int = id.##
override def equals(v: Any): Boolean = {
if (v == null) false
else if (v.isInstanceOf[AnyRef] && this.eq(v.asInstanceOf[AnyRef])) true
else if (v.isInstanceOf[IDMixin]) { //TODO: should subclass be allowed to == this?
- v.asInstanceOf[IDMixin].id == this.id
+ (v.getClass == this.getClass) &&
+ (v.asInstanceOf[IDMixin].id == this.id)
} else false
}
}
object IDMixin {
+ trait ForProduct extends IDMixin with Product {
+
+ @transient override protected lazy val _id: (String, List[Any]) = productPrefix -> productIterator.toList
+ }
+
+ def idCompose(id: Any): Any = {
+ val result = id match {
+ case aa: Array[_] => aa.toList
+ case _ => id
+ }
+ result
+ }
+
def product2ID(v: Product): (String, List[Any]) = {
v.productPrefix -> v.productIterator.toList
diff --git a/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/ThreadLocal.scala b/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/ThreadLocal.scala
index 66f0adae7..abf6efd0e 100644
--- a/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/ThreadLocal.scala
+++ b/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/ThreadLocal.scala
@@ -2,12 +2,12 @@ package com.tribbloids.spookystuff.utils
import com.tribbloids.spookystuff.utils.lifespan.LifespanContext
-case class ThreadLocal[A](init: LifespanContext => A) extends java.lang.ThreadLocal[A] with (() => A) {
+case class ThreadLocal[A](init: LifespanContext => A) extends java.lang.ThreadLocal[A] {
override def initialValue: A = {
val ctx = LifespanContext()
init(ctx)
}
- def apply: A = get
+// def apply: A = get
}
diff --git a/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/Timeout.scala b/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/Timeout.scala
new file mode 100644
index 000000000..9899ea455
--- /dev/null
+++ b/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/Timeout.scala
@@ -0,0 +1,18 @@
+package com.tribbloids.spookystuff.utils
+
+import scala.concurrent.duration._
+import scala.language.implicitConversions
+
+case class Timeout(
+ max: Duration,
+ noProgress: Duration = 30.seconds
+) {
+
+ override lazy val toString =
+ s"[$max / ${noProgress} if no progress]"
+}
+
+object Timeout {
+
+ implicit def fromDuration(v: Duration): Timeout = Timeout(v)
+}
diff --git a/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/TimeoutConf.scala b/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/TimeoutConf.scala
deleted file mode 100644
index 246ea9d30..000000000
--- a/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/TimeoutConf.scala
+++ /dev/null
@@ -1,14 +0,0 @@
-package com.tribbloids.spookystuff.utils
-
-import scala.concurrent.duration._
-import scala.language.implicitConversions
-
-case class TimeoutConf(
- max: Duration,
- noProgress: Duration = 30.seconds
-) {}
-
-object TimeoutConf {
-
- implicit def fromDuration(v: Duration): TimeoutConf = TimeoutConf(v)
-}
diff --git a/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/WaitBeforeAppExit.scala b/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/WaitBeforeAppExit.scala
index 2ed4f0bf9..2a3d47e2f 100644
--- a/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/WaitBeforeAppExit.scala
+++ b/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/WaitBeforeAppExit.scala
@@ -1,10 +1,11 @@
package com.tribbloids.spookystuff.utils
-import com.tribbloids.spookystuff.utils.lifespan.{Lifespan, LocalCleanable, SparkLifespan}
+import com.tribbloids.spookystuff.utils.lifespan.Cleanable.Lifespan
+import com.tribbloids.spookystuff.utils.lifespan.LocalCleanable
object WaitBeforeAppExit extends LocalCleanable {
- override def _lifespan: Lifespan = SparkLifespan.App()
+ override def _lifespan = Lifespan.ActiveSparkApp()
@volatile var _waitBeforeExitDuration: Long = -1
diff --git a/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/io/lock/Lock.scala b/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/io/lock/Lock.scala
index abebc9843..50ec0daa2 100644
--- a/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/io/lock/Lock.scala
+++ b/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/io/lock/Lock.scala
@@ -2,7 +2,8 @@ package com.tribbloids.spookystuff.utils.io.lock
import com.tribbloids.spookystuff.utils.BypassingRule
import com.tribbloids.spookystuff.utils.io.{URIExecution, URIResolver}
-import com.tribbloids.spookystuff.utils.lifespan.{Lifespan, LocalCleanable}
+import com.tribbloids.spookystuff.utils.lifespan.Cleanable.Lifespan
+import com.tribbloids.spookystuff.utils.lifespan.LocalCleanable
import java.io.FileNotFoundException
import java.nio.file.NoSuchFileException
@@ -10,7 +11,7 @@ import java.nio.file.NoSuchFileException
case class Lock(
source: URIExecution,
expired: LockExpired = URIResolver.default.expired, // TODO: use it!
- override val _lifespan: Lifespan = Lifespan.TaskOrJVM()
+ override val _lifespan: Lifespan = Lifespan.TaskOrJVM().forShipping
) extends LockLike
with LocalCleanable {
diff --git a/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/lifespan/BasicTypes.scala b/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/lifespan/BasicTypes.scala
new file mode 100644
index 000000000..0342afce9
--- /dev/null
+++ b/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/lifespan/BasicTypes.scala
@@ -0,0 +1,81 @@
+package com.tribbloids.spookystuff.utils.lifespan
+
+import com.tribbloids.spookystuff.utils.lifespan.Cleanable.BatchID
+import com.tribbloids.spookystuff.utils.{CommonUtils, IDMixin}
+
+import scala.util.Try
+
+// Java Deserialization only runs constructor of superclass
+// CAUTION: keep the empty constructor in subclasses!
+// Without it Kryo deserializer will bypass the hook registration steps in init() when deserializing
+trait BasicTypes {
+
+ case object Task extends LeafType {
+
+ case class ID(id: Long) extends IDMixin.ForProduct {
+ override def toString: String = s"Task-$id"
+ }
+
+ override protected def _batchID(ctx: LifespanContext): ID =
+ ID(ctx.task.taskAttemptId())
+
+ override protected def _registerHook(ctx: LifespanContext, fn: () => Unit): Unit = {
+ ctx.task.addTaskCompletionListener[Unit] { _ =>
+ fn()
+ }
+ }
+
+ }
+
+ case object JVM extends LeafType {
+
+ val MAX_NUMBER_OF_SHUTDOWN_HOOKS: Int = CommonUtils.numLocalCores
+
+ case class ID(id: Int) extends IDMixin.ForProduct {
+ override def toString: String = s"JVM-$id"
+ }
+
+ override protected def _batchID(ctx: LifespanContext): ID =
+ ID((ctx.thread.getId % MAX_NUMBER_OF_SHUTDOWN_HOOKS).toInt)
+
+ override protected def _registerHook(ctx: LifespanContext, fn: () => Unit): Unit = {
+ try {
+ sys.addShutdownHook {
+ fn()
+ }
+ } catch {
+ case e: IllegalStateException if e.getMessage.contains("Shutdown") =>
+ }
+ }
+ }
+
+ trait Compound extends LifespanInternal {
+
+ def delegateTypes: List[LeafType]
+
+ @transient lazy val delegateInstances: List[LeafType#Internal#ForShipping] = {
+
+ delegateTypes.flatMap { v =>
+ Try {
+ v.apply(nameOpt, ctxFactory)
+ }.toOption
+ }
+ }
+
+ override def children: List[LeafType#Internal] = delegateInstances.map(v => v.value)
+
+ override def _register: Seq[BatchID] = {
+
+ delegateInstances.flatMap(_.registeredID)
+ }
+ }
+
+ case class TaskOrJVM(
+ nameOpt: Option[String] = None,
+ ctxFactory: () => LifespanContext = () => LifespanContext()
+ ) extends Compound {
+ def this() = this(None)
+
+ override lazy val delegateTypes: List[LeafType] = List(Task, JVM)
+ }
+}
diff --git a/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/lifespan/Cleanable.scala b/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/lifespan/Cleanable.scala
index 57f866133..400594ac2 100644
--- a/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/lifespan/Cleanable.scala
+++ b/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/lifespan/Cleanable.scala
@@ -10,68 +10,70 @@ object Cleanable {
import com.tribbloids.spookystuff.utils.CommonViews._
- type TrackingN = Long
- type InBatchMap = ConcurrentCache[Long, Cleanable]
+ type Lifespan = LifespanInternal#ForShipping
+ type Leaf = LeafType#Internal
- val uncleaned: ConcurrentMap[Any, InBatchMap] = ConcurrentMap()
+ // Java Deserialization only runs constructor of superclass
+ object Lifespan extends BasicTypes with HadoopTypes with SparkTypes
- def getOrNew(id: Any): InBatchMap = {
+ type BatchID = LeafType#ID
+ type Batch = ConcurrentCache[Long, Cleanable]
+ val uncleaned: ConcurrentMap[BatchID, Batch] = ConcurrentMap()
- uncleaned.getOrElseUpdateSynchronously(id) {
+ trait Selection {
- ConcurrentMap()
+ def ids: Seq[BatchID]
+
+ final def batches: Seq[Batch] = ids.map { id =>
+ Select(id).getOrCreate
}
- }
- def getByLifespan(
- id: Any,
- condition: Cleanable => Boolean
- ): (InBatchMap, List[Cleanable]) = {
- val batch = uncleaned.getOrElse(id, ConcurrentMap())
- val filtered = batch.values.toList //create deep copy to avoid in-place deletion
- .filter(condition)
- (batch, filtered)
- }
- def getAll(
- condition: Cleanable => Boolean = _ => true
- ): Seq[Cleanable] = {
- uncleaned.values.toList.flatten
- .map(_._2)
- .filter(condition)
- }
- def getTyped[T <: Cleanable: ClassTag]: Seq[T] = {
- val result = getAll {
- case _: T => true
- case _ => false
- }.map { v =>
- v.asInstanceOf[T]
+ def filter(condition: Cleanable => Boolean = _ => true): Seq[Cleanable] = {
+
+ batches.flatMap(batch => batch.values).filter(condition)
+ }
+
+ def typed[T <: Cleanable: ClassTag]: Seq[T] = {
+ val result = filter {
+ case _: T => true
+ case _ => false
+ }.map { v =>
+ v.asInstanceOf[T]
+ }
+
+ result
}
- result
- }
- // cannot execute concurrent
- def cleanSweep(
- id: Any,
- condition: Cleanable => Boolean = _ => true
- ): Unit = {
+ def cleanSweep(condition: Cleanable => Boolean = _ => true): Unit = {
- val (map, filtered) = getByLifespan(id, condition)
- filtered
- .foreach { instance =>
- instance.tryClean()
+ ids.foreach { id =>
+ val batch = Select(id).getOrCreate
+ val filtered = batch.values.filter(condition)
+
+ filtered
+ .foreach { instance =>
+ instance.tryClean()
+ }
+ if (batch.isEmpty) uncleaned.remove(id)
}
- map --= filtered.map(_.trackingNumber)
- if (map.isEmpty) uncleaned.remove(id)
+ }
}
- def cleanSweepAll(
- condition: Cleanable => Boolean = _ => true
- ): Unit = {
+ case class Select(id: BatchID) extends Selection {
+ override def ids: Seq[BatchID] = Seq(id)
- uncleaned.keys.toList
- .foreach { tt =>
- cleanSweep(tt, condition)
- }
+ def getOrExecute(exe: () => Batch): Batch = uncleaned.getOrElseUpdateSynchronously(id) {
+
+ exe()
+ }
+
+ def getOrCreate: Batch = getOrExecute { () =>
+ ConcurrentMap()
+ }
+ }
+
+ case object All extends Selection {
+ override def ids: Seq[BatchID] = uncleaned.keys.toSeq
}
}
@@ -86,7 +88,7 @@ trait Cleanable extends Closeable {
import Cleanable._
- @transient object CleanStateLock
+// @transient object CleanStateLock
/**
* taskOrThreadOnCreation is incorrect in withDeadline or threads not created by Spark
@@ -98,24 +100,24 @@ trait Cleanable extends Closeable {
//each can only be cleaned once
@volatile protected var _isCleaned: Boolean = false
- def isCleaned: Boolean = CleanStateLock.synchronized {
+ def isCleaned: Boolean = {
_isCleaned
}
@volatile var stacktraceAtCleaning: Option[Array[StackTraceElement]] = None
- @transient lazy val uncleanedInBatchs: Seq[InBatchMap] = {
+ @transient lazy val uncleanedInBatches: Seq[Batch] = {
// This weird implementation is to mitigate thread-unsafe competition:
// 2 empty collections being inserted simultaneously
- lifespan.batchIDs.map { id =>
- Cleanable.getOrNew(id)
+ lifespan.registeredID.map { id =>
+ Cleanable.Select(id).getOrCreate
}
}
{
logPrefixed("Created")
- uncleanedInBatchs.foreach { inBatch =>
+ uncleanedInBatches.foreach { inBatch =>
inBatch += this.trackingNumber -> this
}
}
@@ -146,14 +148,14 @@ trait Cleanable extends Closeable {
)
}
- lazy val doCleanOnce: Unit = CleanStateLock.synchronized {
+ lazy val doCleanOnce: Unit = {
stacktraceAtCleaning = Some(Thread.currentThread().getStackTrace)
try {
cleanImpl()
_isCleaned = true
- uncleanedInBatchs.foreach { inBatch =>
+ uncleanedInBatches.foreach { inBatch =>
inBatch -= this.trackingNumber
}
} catch {
@@ -182,7 +184,7 @@ trait Cleanable extends Closeable {
if (!silentOnError(ee))
LoggerFactory
.getLogger(this.getClass)
- .warn(
+ .error(
s"$logPrefix !!! FAIL TO CLEAN UP !!!\n",
ee
)
diff --git a/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/lifespan/HadoopTypes.scala b/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/lifespan/HadoopTypes.scala
new file mode 100644
index 000000000..e014e210a
--- /dev/null
+++ b/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/lifespan/HadoopTypes.scala
@@ -0,0 +1,45 @@
+package com.tribbloids.spookystuff.utils.lifespan
+
+import com.tribbloids.spookystuff.utils.IDMixin
+import com.tribbloids.spookystuff.utils.lifespan.Cleanable.Lifespan
+import org.apache.hadoop.fs.FileSystem
+import org.apache.hadoop.util.ShutdownHookManager
+
+trait HadoopTypes {
+
+ trait HadoopType extends LeafType {}
+
+ case class HadoopShutdown(priority: Int) extends HadoopType {
+
+ case class ID(id: Int) extends IDMixin.ForProduct {
+ override def toString: String = s"JVM-$id"
+ }
+
+ override protected def _batchID(ctx: LifespanContext): ID =
+ ID((ctx.thread.getId % Lifespan.JVM.MAX_NUMBER_OF_SHUTDOWN_HOOKS).toInt)
+
+ override protected def _registerHook(ctx: LifespanContext, fn: () => Unit): Unit = {
+ val hookTask = new Runnable() {
+ override def run(): Unit = fn()
+ }
+
+ try {
+ ShutdownHookManager
+ .get()
+ .addShutdownHook(
+ hookTask,
+ priority
+ )
+ } catch {
+ case e: IllegalStateException if e.getMessage.contains("Shutdown") =>
+ // DO NOTHING
+ }
+ }
+
+ }
+
+ object HadoopShutdown {
+
+ object BeforeSpark extends HadoopShutdown(FileSystem.SHUTDOWN_HOOK_PRIORITY + 60)
+ }
+}
diff --git a/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/lifespan/LeafType.scala b/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/lifespan/LeafType.scala
new file mode 100644
index 000000000..f9ff17a85
--- /dev/null
+++ b/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/lifespan/LeafType.scala
@@ -0,0 +1,53 @@
+package com.tribbloids.spookystuff.utils.lifespan
+
+import com.tribbloids.spookystuff.utils.CachingUtils.ConcurrentMap
+import com.tribbloids.spookystuff.utils.lifespan.Cleanable.BatchID
+import com.tribbloids.spookystuff.utils.serialization.BeforeAndAfterShipping
+
+abstract class LifespanType extends Serializable with Product {
+
+ type ID
+}
+
+abstract class LeafType extends LifespanType {
+
+ protected def _batchID(ctx: LifespanContext): ID
+ protected def _registerHook(ctx: LifespanContext, fn: () => Unit): Unit
+
+ class Internal(
+ val nameOpt: Option[String] = None,
+ val ctxFactory: () => LifespanContext = () => LifespanContext()
+ ) extends LifespanInternal
+ with BeforeAndAfterShipping {
+
+ def _type: LeafType = LeafType.this
+
+ @transient lazy val batchID: ID = _type._batchID(ctx).asInstanceOf[ID]
+
+ def registerHook(fn: () => Unit): Unit = {
+ _type._registerHook(ctx, fn)
+ }
+
+ final override protected def _register: Seq[BatchID] = {
+
+ val batchID = this.batchID
+
+ Cleanable.Select(batchID).getOrExecute { () =>
+ registerHook { () =>
+ Cleanable.Select(batchID).cleanSweep()
+ }
+ ConcurrentMap()
+ }
+
+ Seq(batchID)
+ }
+ }
+
+ def apply(
+ nameOpt: Option[String] = None,
+ ctxFactory: () => LifespanContext = () => LifespanContext()
+ ): Internal#ForShipping = {
+ val i = new Internal(nameOpt, ctxFactory)
+ i.forShipping
+ }
+}
diff --git a/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/lifespan/Lifespan.scala b/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/lifespan/Lifespan.scala
deleted file mode 100644
index 1c5d066d7..000000000
--- a/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/lifespan/Lifespan.scala
+++ /dev/null
@@ -1,151 +0,0 @@
-package com.tribbloids.spookystuff.utils.lifespan
-
-import com.tribbloids.spookystuff.utils.{CommonUtils, IDMixin}
-import org.apache.spark.TaskContext
-
-import scala.util.Try
-
-/**
- * Java Deserialization only runs constructor of superclass
- */
-//CAUTION: keep the empty constructor in subclasses!
-// Without it Kryo deserializer will bypass the hook registration steps in init() when deserializing
-abstract class Lifespan extends IDMixin with Serializable {
-
- {
- init()
- }
-
- /**
- * should be triggerd on both creation and deserialization
- */
- protected def init(): Unit = {
- ctx
- batchIDs
- //always generate on construction
-
- batchIDs.foreach { batchID =>
- if (!Cleanable.uncleaned.contains(batchID)) {
- registerHook { () =>
- Cleanable.cleanSweep(batchID)
- }
- }
- }
- }
-
- def readObject(in: java.io.ObjectInputStream): Unit = {
- in.defaultReadObject()
- init() //redundant?
- }
-
- val ctxFactory: () => LifespanContext
- @transient lazy val ctx: LifespanContext = ctxFactory()
-
- def getBatchIDs: Seq[Any]
- @transient final lazy val batchIDs = getBatchIDs
- final protected def _id: Seq[Any] = batchIDs
-
- def registerHook(
- fn: () => Unit
- ): Unit
-
- def nameOpt: Option[String]
- override def toString: String = {
- val idStr = Try(batchIDs.mkString("/")).getOrElse("[Error]")
- (nameOpt.toSeq ++ Seq(idStr)).mkString(":")
- }
-
-}
-
-object Lifespan {
-
- abstract class LifespanType extends Serializable with Product {
-
- // default companion class constructor
- def apply(nameOpt: Option[String] = None, ctxFactory: () => LifespanContext = () => LifespanContext()): Lifespan
- }
-
- case object Task extends LifespanType {
-
- case class ID(id: Long) extends AnyVal {
- override def toString: String = s"Task-$id"
- }
- }
- case class Task(
- override val nameOpt: Option[String] = None,
- ctxFactory: () => LifespanContext = () => LifespanContext()
- ) extends Lifespan {
- def this() = this(None)
-
- import Task._
-
- def task: TaskContext = ctx.task
-
-// override def tpe: LifespanType = Task
- override def getBatchIDs: Seq[ID] = Seq(ID(task.taskAttemptId()))
-
- override def registerHook(fn: () => Unit): Unit = {
- task.addTaskCompletionListener[Unit] { _ =>
- fn()
- }
- }
- }
-
- case object JVM extends LifespanType {
-
- val MAX_NUMBER_OF_SHUTDOWN_HOOKS: Int = CommonUtils.numLocalCores
-
- case class ID(id: Int) extends AnyVal {
- override def toString: String = s"JVM-$id"
- }
- }
- case class JVM(
- override val nameOpt: Option[String] = None,
- ctxFactory: () => LifespanContext = () => LifespanContext()
- ) extends Lifespan {
- def this() = this(None)
-
- import JVM._
-
- override def getBatchIDs: Seq[ID] = Seq(ID((ctx.thread.getId % JVM.MAX_NUMBER_OF_SHUTDOWN_HOOKS).toInt))
-
- override def registerHook(fn: () => Unit): Unit =
- try {
- sys.addShutdownHook {
- fn()
- }
- } catch {
- case e: IllegalStateException if e.getMessage.contains("Shutdown") =>
- }
- }
-
- trait Compound extends Lifespan {
-
- def delegates: List[LifespanType]
-
- @transient lazy val delegateInstances: List[Lifespan] = {
-
- delegates.flatMap { v =>
- Try {
- v.apply(nameOpt, ctxFactory)
- }.toOption
- }
- }
-
- override def getBatchIDs: Seq[Any] = {
-
- delegateInstances.flatMap(_.batchIDs)
- }
-
- override def registerHook(fn: () => Unit): Unit = {}
- }
-
- case class TaskOrJVM(
- nameOpt: Option[String] = None,
- ctxFactory: () => LifespanContext = () => LifespanContext()
- ) extends Compound {
- def this() = this(None)
-
- override lazy val delegates: List[LifespanType] = List(Task, JVM)
- }
-}
diff --git a/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/lifespan/LifespanInternal.scala b/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/lifespan/LifespanInternal.scala
new file mode 100644
index 000000000..9af6a2f2d
--- /dev/null
+++ b/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/lifespan/LifespanInternal.scala
@@ -0,0 +1,57 @@
+package com.tribbloids.spookystuff.utils.lifespan
+
+import com.tribbloids.spookystuff.utils.IDMixin
+import com.tribbloids.spookystuff.utils.lifespan.Cleanable.BatchID
+import com.tribbloids.spookystuff.utils.serialization.BeforeAndAfterShipping
+
+import scala.util.Try
+
+abstract class LifespanInternal extends BeforeAndAfterShipping with IDMixin {
+
+ {
+ initOnce
+ }
+
+ override def afterArrival(): Unit = {
+ initOnce
+ }
+
+ @transient private final var isInitialised: LifespanInternal = _
+ @transient protected final lazy val initOnce: Unit = {
+ //always generate on construction or deserialization
+
+ doInit()
+ isInitialised = this
+ }
+
+ protected def doInit(): Unit = {
+ ctx
+ registeredID
+ }
+
+ def requireUsable(): Unit = {
+ require(isInitialised != null, s"$this not initialised")
+ }
+
+ def ctxFactory: () => LifespanContext
+ @transient lazy val ctx: LifespanContext = ctxFactory()
+
+ def children: List[LeafType#Internal] = Nil
+
+ @transient final lazy val leaves: Seq[LeafType#Internal] = this match {
+ case v: LeafType#Internal =>
+ Seq(v) ++ children
+ case _ =>
+ children
+ }
+
+ protected def _register: Seq[BatchID]
+ @transient final lazy val registeredID = _register
+ final def _id: Seq[BatchID] = registeredID
+
+ def nameOpt: Option[String]
+ override def toString: String = {
+ val idStr = Try(registeredID.mkString("/")).getOrElse("[Error]")
+ (nameOpt.toSeq ++ Seq(idStr)).mkString(":")
+ }
+}
diff --git a/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/lifespan/SparkLifespan.scala b/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/lifespan/SparkLifespan.scala
deleted file mode 100644
index f2e5d1fc3..000000000
--- a/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/lifespan/SparkLifespan.scala
+++ /dev/null
@@ -1,49 +0,0 @@
-package com.tribbloids.spookystuff.utils.lifespan
-
-import com.tribbloids.spookystuff.utils.lifespan.Lifespan.LifespanType
-import org.apache.spark.SparkContext
-import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd}
-import org.apache.spark.sql.SparkSession
-
-trait SparkLifespan extends LifespanType {
-
- @transient lazy val sc: SparkContext = SparkSession.active.sparkContext
-
- {
- sc
- }
-
- case class ID(id: String) {
- override def toString: String = s"Spark.$productPrefix-$id"
- }
-}
-object SparkLifespan {
-
- case object App extends SparkLifespan {
-
- class Listener(fn: () => Unit) extends SparkListener with Serializable {
-
- override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd): Unit = {
-
- fn()
- }
- }
- }
- case class App(
- override val nameOpt: Option[String] = None,
- ctxFactory: () => LifespanContext = () => LifespanContext()
- ) extends Lifespan {
- def this() = this(None)
-
- import App._
-
- override def getBatchIDs = Seq(ID(sc.applicationId))
-
- override def registerHook(fn: () => Unit): Unit = {
-
- sc.addSparkListener(new Listener(fn))
- }
- }
-
- // TODO: add Job impl
-}
diff --git a/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/lifespan/SparkTypes.scala b/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/lifespan/SparkTypes.scala
new file mode 100644
index 000000000..e734074b9
--- /dev/null
+++ b/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/lifespan/SparkTypes.scala
@@ -0,0 +1,45 @@
+package com.tribbloids.spookystuff.utils.lifespan
+
+import com.tribbloids.spookystuff.utils.IDMixin
+import com.tribbloids.spookystuff.utils.serialization.NOTSerializable
+import org.apache.spark.SparkContext
+import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd}
+import org.apache.spark.sql.SparkSession
+
+trait SparkTypes {
+
+ trait SparkType extends LeafType with NOTSerializable {
+
+ def sparkContext: SparkContext
+
+ {
+ sparkContext
+ }
+
+ case class ID(id: String) extends IDMixin.ForProduct {
+ override def toString: String = s"Spark.$productPrefix-$id"
+ }
+ }
+
+ case class SparkApp(
+ override val sparkContext: SparkContext = SparkSession.active.sparkContext
+ ) extends SparkType {
+
+ class Listener(fn: () => Unit) extends SparkListener with Serializable {
+
+ override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd): Unit = {
+
+ fn()
+ }
+ }
+
+ override protected def _batchID(ctx: LifespanContext): ID = ID(sparkContext.applicationId)
+
+ override protected def _registerHook(ctx: LifespanContext, fn: () => Unit): Unit =
+ sparkContext.addSparkListener(new Listener(fn))
+ }
+
+ object ActiveSparkApp extends SparkApp(SparkSession.active.sparkContext) {}
+
+ // TODO: add Job impl
+}
diff --git a/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/serialization/AssertSerializable.scala b/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/serialization/AssertSerializable.scala
index e37abe710..631984fae 100644
--- a/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/serialization/AssertSerializable.scala
+++ b/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/serialization/AssertSerializable.scala
@@ -1,50 +1,25 @@
package com.tribbloids.spookystuff.utils.serialization
-import java.nio.ByteBuffer
-
-import com.tribbloids.spookystuff.utils.TreeThrowable
import org.apache.spark.serializer.Serializer
import scala.reflect.ClassTag
-import scala.util.Try
object AssertSerializable {
+ def serializableCondition[T <: AnyRef]: (T, T) => Any = { (v1: T, v2: T) =>
+ assert(v1.hashCode() == v2.hashCode(), s"hash code after deserialization is different: $v1 != $v2")
+ assert((v1: T) == (v2: T), s"value after deserialization is different: $v1 != $v2")
+ assert(v1.toString == v2.toString, s"value.toString after deserialization is different: $v1 != $v2")
+ if (!v1.getClass.getName.endsWith("$"))
+ assert(!(v1 eq v2))
+ }
+
def apply[T <: AnyRef: ClassTag](
element: T,
serializers: Seq[Serializer] = SerDeOverride.Default.allSerializers,
- condition: (T, T) => Any = { (v1: T, v2: T) =>
- assert((v1: T) == (v2: T), s"value after deserialization is different: $v1 != $v2")
- assert(v1.toString == v2.toString, s"value.toString after deserialization is different: $v1 != $v2")
- if (!v1.getClass.getCanonicalName.endsWith("$"))
- assert(!(v1 eq v2))
- }
+ condition: (T, T) => Any = serializableCondition[T]
): Unit = {
AssertWeaklySerializable(element, serializers, condition)
}
}
-
-case class AssertWeaklySerializable[T <: AnyRef: ClassTag](
- element: T,
- serializers: Seq[Serializer] = SerDeOverride.Default.allSerializers,
- condition: (T, T) => Any = { (v1: T, v2: T) =>
- true
- }
-) {
-
- val trials = serializers.map { ser =>
- Try {
- val serInstance = ser.newInstance()
- val binary: ByteBuffer = serInstance.serialize(element)
- assert(binary.array().length > 8) //min object overhead length
- val element2 = serInstance.deserialize[T](binary)
- // assert(!element.eq(element2))
- condition(element, element2)
- }
- }
-
- TreeThrowable.&&&(
- trials
- )
-}
diff --git a/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/serialization/AssertWeaklySerializable.scala b/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/serialization/AssertWeaklySerializable.scala
new file mode 100644
index 000000000..c43b694f0
--- /dev/null
+++ b/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/serialization/AssertWeaklySerializable.scala
@@ -0,0 +1,33 @@
+package com.tribbloids.spookystuff.utils.serialization
+
+import java.nio.ByteBuffer
+
+import com.tribbloids.spookystuff.utils.TreeThrowable
+import org.apache.spark.serializer.Serializer
+
+import scala.reflect.ClassTag
+import scala.util.Try
+
+case class AssertWeaklySerializable[T <: AnyRef: ClassTag](
+ element: T,
+ serializers: Seq[Serializer] = SerDeOverride.Default.allSerializers,
+ condition: (T, T) => Any = { (_: T, _: T) =>
+ true
+ }
+) {
+
+ val trials: Seq[Try[Any]] = serializers.map { ser =>
+ Try {
+ val serInstance = ser.newInstance()
+ val binary: ByteBuffer = serInstance.serialize(element)
+ assert(binary.array().length > 8) //min object overhead length
+ val element2 = serInstance.deserialize[T](binary)
+ // assert(!element.eq(element2))
+ condition(element, element2)
+ }
+ }
+
+ TreeThrowable.&&&(
+ trials
+ )
+}
diff --git a/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/serialization/BeforeAndAfterShipping.scala b/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/serialization/BeforeAndAfterShipping.scala
new file mode 100644
index 000000000..08e57f685
--- /dev/null
+++ b/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/serialization/BeforeAndAfterShipping.scala
@@ -0,0 +1,84 @@
+package com.tribbloids.spookystuff.utils.serialization
+
+import com.esotericsoftware.kryo.io.{Input, Output}
+import com.esotericsoftware.kryo.{Kryo, KryoSerializable}
+import org.slf4j.LoggerFactory
+
+import java.io.{InputStream, ObjectInputStream, ObjectOutputStream}
+
+import scala.language.implicitConversions
+
+trait BeforeAndAfterShipping extends Serializable {
+
+ {
+ _container
+ }
+
+ import BeforeAndAfterShipping._
+
+ def beforeDeparture(): Unit = {}
+
+ def afterArrival(): Unit = {}
+
+ @transient private lazy val _container: Container[this.type] = Container(this)
+
+ type ForShipping = Container[this.type]
+ def forShipping: ForShipping = _container
+}
+
+object BeforeAndAfterShipping {
+
+ @inline
+ def logMsg(v: String): Unit = {
+// LoggerFactory.getLogger(this.getClass).debug(v: String)
+ }
+
+ object Container {
+
+ implicit def unbox[T <: BeforeAndAfterShipping](v: Container[T]): T = v.value
+ }
+
+ case class Container[+T <: BeforeAndAfterShipping](
+ @transient private var _value: BeforeAndAfterShipping
+ ) extends Serializable
+ with KryoSerializable {
+
+ final def value: T = _value.asInstanceOf[T]
+
+ def this() = this(null.asInstanceOf[T]) //TODO: useless?
+
+ private def writeObject(aOutputStream: ObjectOutputStream): Unit = {
+ _value.beforeDeparture()
+ logMsg(s"JavaW: ${_value}")
+ aOutputStream.writeObject(_value)
+ }
+
+ private def readObject(aInputStream: ObjectInputStream): Unit = {
+ _value = aInputStream.readObject().asInstanceOf[T]
+// assert(isEOF(aInputStream), "not EOF!")
+
+ logMsg(s"JavaR: ${_value}")
+
+ _value.afterArrival()
+ }
+
+ def isEOF(v: InputStream): Boolean = {
+ val vv = v.read()
+ vv == -1
+ }
+
+ override def write(kryo: Kryo, output: Output): Unit = {
+ _value.beforeDeparture()
+ logMsg(s"KryoW: ${_value}")
+ kryo.writeClassAndObject(output, _value)
+ }
+
+ override def read(kryo: Kryo, input: Input): Unit = {
+ _value = kryo.readClassAndObject(input).asInstanceOf[BeforeAndAfterShipping]
+// assert(isEOF(input), "not EOF!")
+
+ logMsg(s"KryoR: ${_value}")
+ _value.afterArrival()
+ }
+ }
+}
diff --git a/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/serialization/NOTSerializable.scala b/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/serialization/NOTSerializable.scala
index 1e0b20e0a..445b2ff61 100644
--- a/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/serialization/NOTSerializable.scala
+++ b/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/serialization/NOTSerializable.scala
@@ -1,25 +1,28 @@
package com.tribbloids.spookystuff.utils.serialization
-import java.io.NotSerializableException
-import com.esotericsoftware.kryo.io.{Input, Output}
-import com.esotericsoftware.kryo.{Kryo, KryoSerializable}
+import com.tribbloids.spookystuff.utils.serialization.BeforeAndAfterShipping.Container
+
+import java.io.NotSerializableException
/**
* Any subclass in the closure cleaned by Spark ClosureCleaner will trigger a runtime error.
*/
-trait NOTSerializable extends KryoSerializable with Serializable {
+trait NOTSerializable extends BeforeAndAfterShipping {
- private lazy val error = new NotSerializableException(s"${this.getClass.getCanonicalName} is NOT serializable")
+ {
+ trigger
+ }
- private val _serDeHook = SerDeHook({ _ =>
- throw error
- }, { _ =>
- throw error
- })
+ private lazy val error =
+ new NotSerializableException(s"${this.getClass.getCanonicalName} is NOT serializable")
- override def write(kryo: Kryo, output: Output): Unit =
+ override def beforeDeparture(): Unit = {
throw error
+ }
- override def read(kryo: Kryo, input: Input): Unit =
+ override def afterArrival(): Unit = {
throw error
+ }
+
+ private lazy val trigger: Container[this.type] = Container(this)
}
diff --git a/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/serialization/SerDeHook.scala b/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/serialization/SerDeHook.scala
deleted file mode 100644
index 34b61da31..000000000
--- a/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/serialization/SerDeHook.scala
+++ /dev/null
@@ -1,50 +0,0 @@
-package com.tribbloids.spookystuff.utils.serialization
-
-import java.io._
-
-import com.esotericsoftware.kryo.io.{Input, Output}
-import com.esotericsoftware.kryo.{Kryo, KryoSerializable}
-
-/**
- * Attach as a member/property of another class to trigger when the other class is processed by SerDe
- * CAUTION: writeObject & readObject won't be effective unless declared private
- * as a result this class is declared final
- */
-final case class SerDeHook(
- var beforeWrite: OutputStream => Any = _ => {},
- var afterRead: InputStream => Any = _ => {}
- //TODO: afterWrite?
-) extends Serializable
- with KryoSerializable {
-
- @throws(classOf[IOException])
- private def writeObject(out: ObjectOutputStream): Unit = {
- beforeWrite(out)
- out.defaultWriteObject()
- }
-
- @throws(classOf[IOException])
- private def readObject(in: ObjectInputStream): Unit = {
- in.defaultReadObject()
- afterRead(in)
- }
- @throws(classOf[IOException])
- private def readObjectNoData(): Unit = {
- afterRead(new ByteArrayInputStream(Array.empty[Byte]))
- }
-
- override def write(kryo: Kryo, output: Output): Unit = {
- beforeWrite(output)
-
- kryo.writeClassAndObject(output, beforeWrite)
- kryo.writeClassAndObject(output, afterRead)
- }
-
- override def read(kryo: Kryo, input: Input): Unit = {
-
- this.beforeWrite = kryo.readClassAndObject(input).asInstanceOf[OutputStream => Any]
- this.afterRead = kryo.readClassAndObject(input).asInstanceOf[InputStream => Any]
-
- afterRead(input)
- }
-}
diff --git a/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/serialization/SerDeHooks.scala b/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/serialization/SerDeHooks.scala
deleted file mode 100644
index 8b1378917..000000000
--- a/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/serialization/SerDeHooks.scala
+++ /dev/null
@@ -1 +0,0 @@
-
diff --git a/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/serialization/SerDeOverride.scala b/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/serialization/SerDeOverride.scala
index 709a2186b..3732bb8cd 100644
--- a/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/serialization/SerDeOverride.scala
+++ b/mldsl/src/main/scala/com/tribbloids/spookystuff/utils/serialization/SerDeOverride.scala
@@ -2,10 +2,9 @@ package com.tribbloids.spookystuff.utils.serialization
import java.io
import java.nio.ByteBuffer
-
import com.tribbloids.spookystuff.utils.IDMixin
import org.apache.hadoop.io.Writable
-import org.apache.spark.serializer.{JavaSerializer, KryoSerializer, SerializerInstance}
+import org.apache.spark.serializer.{JavaSerializer, KryoSerializer, Serializer, SerializerInstance}
import org.apache.spark.sql.catalyst.ScalaReflection.universe.TypeTag
import org.apache.spark.{SerializableWritable, SparkConf}
@@ -31,7 +30,7 @@ object SerDeOverride {
Some(kryoSerializer.newInstance())
}
- @transient lazy val allSerializers = List(javaSerializer, kryoSerializer)
+ @transient lazy val allSerializers: List[Serializer] = List(javaSerializer, kryoSerializer)
}
object Default extends WithConf
@@ -47,6 +46,7 @@ object SerDeOverride {
* wrapping & unwrapping is lazy
*/
case class SerDeOverride[T: ClassTag](
+ // TODO: replace with twitter MeatLocker?
@transient private val _original: T,
overrideImpl: () => Option[SerializerInstance] = () => None // no override by default
) extends Serializable
@@ -92,34 +92,3 @@ case class SerDeOverride[T: ClassTag](
override def _id: Any = value
}
-
-//TODO: cleanup, useless, can be completely superceded by SerializableWritable?
-//class BinaryWritable[T <: Writable](
-// @transient val obj: T,
-// val serFactory: () => SerializerInstance = BinaryWritable.javaSerFactory
-// ) extends Serializable {
-//
-// val delegate: BinarySerializable[SerializableWritable[T]] = new BinarySerializable(
-// new SerializableWritable(obj),
-// serFactory
-// )
-//
-// @transient lazy val value: T = Option(obj).getOrElse {
-// delegate.value.value
-// }
-//}
-
-//class SerializableUGI(
-// @transient val _ugi: UserGroupInformation,
-// val serFactory: () => SerializerInstance = BinaryWritable.javaSerFactory
-// ) extends Serializable {
-//
-// val name = _ugi.getUserName
-// val credentials: BinaryWritable[Credentials] = new BinaryWritable(_ugi.getCredentials, serFactory)
-//
-// @transient lazy val value: UserGroupInformation = Option(_ugi).getOrElse {
-// val result = UserGroupInformation.createRemoteUser(name)
-// result.addCredentials(credentials.value)
-// result
-// }
-//}
diff --git a/mldsl/src/main/scala/org/apache/spark/ml/dsl/utils/messaging/FallbackSerializer.scala b/mldsl/src/main/scala/org/apache/spark/ml/dsl/utils/messaging/FallbackSerializer.scala
index c86ef5afb..3068f175c 100644
--- a/mldsl/src/main/scala/org/apache/spark/ml/dsl/utils/messaging/FallbackSerializer.scala
+++ b/mldsl/src/main/scala/org/apache/spark/ml/dsl/utils/messaging/FallbackSerializer.scala
@@ -20,7 +20,7 @@ abstract class FallbackSerializer(
case (ti, JString(str)) =>
LoggerFactory
.getLogger(this.getClass)
- .info(
+ .debug(
s"JSON === [${this.getClass.getSimpleName}] ==> Object"
)
try {
@@ -47,7 +47,7 @@ abstract class FallbackSerializer(
case v: Serializable =>
LoggerFactory
.getLogger(this.getClass)
- .info(
+ .debug(
s"Object === [${this.getClass.getSimpleName}] ==> JSON"
)
// try {
diff --git a/mldsl/src/main/scala/org/apache/spark/ml/dsl/utils/messaging/xml/XMLFormats.scala b/mldsl/src/main/scala/org/apache/spark/ml/dsl/utils/messaging/xml/XMLFormats.scala
index 18847edc3..8d00802f0 100644
--- a/mldsl/src/main/scala/org/apache/spark/ml/dsl/utils/messaging/xml/XMLFormats.scala
+++ b/mldsl/src/main/scala/org/apache/spark/ml/dsl/utils/messaging/xml/XMLFormats.scala
@@ -43,7 +43,7 @@ object XMLFormats extends DefaultFormats {
def timezone: TimeZone = dateFormats.head.getTimeZone
- def dateFormats: Seq[SimpleDateFormat] = baseDataFormatsFactory()
+ def dateFormats: Seq[SimpleDateFormat] = baseDataFormatsFactory.get()
}
override val wantsBigDecimal = true
diff --git a/mldsl/src/test/scala/com/tribbloids/spookystuff/testutils/TestHelper.scala b/mldsl/src/test/scala/com/tribbloids/spookystuff/testutils/TestHelper.scala
index e8d484cf6..9de9e61dd 100644
--- a/mldsl/src/test/scala/com/tribbloids/spookystuff/testutils/TestHelper.scala
+++ b/mldsl/src/test/scala/com/tribbloids/spookystuff/testutils/TestHelper.scala
@@ -1,8 +1,6 @@
package com.tribbloids.spookystuff.testutils
-import java.io.File
-import java.util.{Date, Properties}
-
+import com.tribbloids.spookystuff.utils.lifespan.Cleanable.Lifespan
import com.tribbloids.spookystuff.utils.lifespan.LocalCleanable
import com.tribbloids.spookystuff.utils.{CommonConst, CommonUtils, ConfUtils}
import org.apache.hadoop.fs.FileUtil
@@ -11,6 +9,8 @@ import org.apache.spark.sql.{SQLContext, SparkSession}
import org.apache.spark.{SparkConf, SparkContext, SparkEnv, SparkException}
import org.slf4j.LoggerFactory
+import java.io.File
+import java.util.{Date, Properties}
import scala.reflect.ClassTag
import scala.util.{Failure, Success, Try}
@@ -55,7 +55,7 @@ abstract class TestHelper extends LocalCleanable {
def SPARK_HOME: String = System.getenv("SPARK_HOME")
- final val MAX_TOTAL_MEMORY = 16 * 1024
+ final val MAX_TOTAL_MEMORY = 8 * 1024
final val MEMORY_PER_CORE = 1024
// final val EXECUTOR_JVM_MEMORY_OVERHEAD = 256 //TODO: remove, too complex
@@ -64,7 +64,7 @@ abstract class TestHelper extends LocalCleanable {
val METASTORE_PATH: String = CommonUtils.\\\(CommonConst.USER_DIR, "metastore_db")
val WAREHOUSE_PATH: String = CommonUtils.\\\(CommonConst.USER_DIR, "warehouse")
- var sparkSessionInitialised: Boolean = false
+ @transient var sparkSessionInitialised: Boolean = false
{
CommonUtils.debugCPResource()
@@ -78,38 +78,28 @@ abstract class TestHelper extends LocalCleanable {
System.setProperty("fs.s3.awsSecretAccessKey", _)
}
- cleanBeforeAndAfterLifespan()
+ cleanBeforeAndAfter()
}
- override def cleanImpl(): Unit = {
+ override def _lifespan: Lifespan = Lifespan.HadoopShutdown.BeforeSpark()
- if (sparkSessionInitialised) {
+ override def cleanImpl(): Unit = {
- println("=============== Stopping Test Spark Context ==============")
- // Suppress the following log error when shutting down in local-cluster mode:
- // Remote RPC client disassociated. Likely due to containers exceeding thresholds, or network issues.
- // Check driver logs for WARN messages.
- // java.lang.IllegalStateException: Shutdown hooks cannot be modified during shutdown
- val logger = org.apache.log4j.Logger.getRootLogger
- val oldLevel = logger.getLevel
- logger.setLevel(org.apache.log4j.Level.toLevel("OFF"))
- try {
- TestSC.stop()
- } catch {
- case e: Throwable =>
- logger.setLevel(oldLevel)
- logger.error("cannot stop Test SparkContext", e)
- } finally {
- logger.setLevel(oldLevel)
- }
-// println("=============== Test Spark Context has stopped ==============")
- }
+ println("=============== Stopping Test Spark Context ==============")
+ // Suppress the following log error when shutting down in local-cluster mode:
+ // Remote RPC client disassociated. Likely due to containers exceeding thresholds, or network issues.
+ // Check driver logs for WARN messages.
+ // java.lang.IllegalStateException: Shutdown hooks cannot be modified during shutdown
+ val logger = org.apache.log4j.Logger.getRootLogger
+ logger.setLevel(org.apache.log4j.Level.toLevel("OFF"))
- cleanBeforeAndAfterLifespan()
+ TestSC.stop()
+ // println("=============== Test Spark Context has stopped ==============")
+ cleanBeforeAndAfter()
}
- def cleanBeforeAndAfterLifespan(): Unit = {
+ def cleanBeforeAndAfter(): Unit = {
cleanTempDirs(
Seq(
WAREHOUSE_PATH,
@@ -140,7 +130,7 @@ abstract class TestHelper extends LocalCleanable {
)
Option(SPARK_HOME)
- .flatMap { h =>
+ .flatMap { _ =>
tuple match {
case (None, None) =>
None
@@ -182,7 +172,7 @@ abstract class TestHelper extends LocalCleanable {
@transient lazy val envOverrides = Map(
"SPARK_SCALA_VERSION" -> CommonUtils.scalaBinaryVersion
-// "SPARK_LOCAL_HOSTNAME" -> "localhost"
+ // "SPARK_LOCAL_HOSTNAME" -> "localhost"
)
/**
@@ -200,8 +190,7 @@ abstract class TestHelper extends LocalCleanable {
masterStr
} else {
- {
- //TODO: Unstable! remove?
+ if (envOverrides.nonEmpty) {
LoggerFactory
.getLogger(this.getClass)
.warn(
@@ -403,10 +392,10 @@ abstract class TestHelper extends LocalCleanable {
}
val expectedErrorName = implicitly[ClassTag[EE]].runtimeClass.getSimpleName
trial match {
- case Failure(e: EE) =>
+ case Failure(_: EE) =>
case Failure(e) =>
throw new AssertionError(s"Expecting $expectedErrorName, but get ${e.getClass.getSimpleName}", e)
- case Success(n) =>
+ case Success(_) =>
throw new AssertionError(s"expecting $expectedErrorName, but no exception was thrown")
}
}
diff --git a/mldsl/src/test/scala/com/tribbloids/spookystuff/utils/serialization/BeforeAndAfterShippingSuite.scala b/mldsl/src/test/scala/com/tribbloids/spookystuff/utils/serialization/BeforeAndAfterShippingSuite.scala
new file mode 100644
index 000000000..d006c2407
--- /dev/null
+++ b/mldsl/src/test/scala/com/tribbloids/spookystuff/utils/serialization/BeforeAndAfterShippingSuite.scala
@@ -0,0 +1,62 @@
+package com.tribbloids.spookystuff.utils.serialization
+
+import com.tribbloids.spookystuff.testutils.FunSpecx
+import org.scalatest.BeforeAndAfterEach
+
+import java.util.concurrent.atomic.AtomicInteger
+
+class BeforeAndAfterShippingSuite extends FunSpecx with BeforeAndAfterEach {
+
+ import BeforeAndAfterShippingSuite._
+
+ override def beforeEach(): Unit = {
+ beforeCounter.set(0)
+ afterCounter.set(0)
+ }
+
+ it("can serialize container") {
+ val dummy = Dummy()
+
+ AssertSerializable(dummy.forShipping)
+
+ assert(beforeCounter.get() == 2)
+ assert(afterCounter.get() == 2)
+ }
+
+ ignore("can serialize self") {
+ // TODO: this doesn't work, why?
+// it("can serialize self") {
+
+ val dummy = Dummy()
+
+ AssertSerializable(dummy)
+
+ assert(beforeCounter.get() == 2)
+ assert(afterCounter.get() == 2)
+ }
+}
+
+object BeforeAndAfterShippingSuite {
+
+ val beforeCounter = new AtomicInteger(0)
+ val afterCounter = new AtomicInteger(0)
+
+ case class Dummy(
+ i: Int = 1,
+ j: Double = 2.375,
+ s: String = "a"
+ ) extends BeforeAndAfterShipping {
+
+ val s2: String = "b"
+
+ override def beforeDeparture(): Unit = {
+ assert(s2 == "b")
+ beforeCounter.incrementAndGet()
+ }
+
+ override def afterArrival(): Unit = {
+ assert(s2 == "b") // ensure that Dummy is fully initialised when doAfter is called
+ afterCounter.incrementAndGet()
+ }
+ }
+}
diff --git a/mldsl/src/test/scala/com/tribbloids/spookystuff/utils/serialization/SerDeHookSuite.scala b/mldsl/src/test/scala/com/tribbloids/spookystuff/utils/serialization/SerDeHookSuite.scala
deleted file mode 100644
index 4df0a0516..000000000
--- a/mldsl/src/test/scala/com/tribbloids/spookystuff/utils/serialization/SerDeHookSuite.scala
+++ /dev/null
@@ -1,34 +0,0 @@
-package com.tribbloids.spookystuff.utils.serialization
-import com.tribbloids.spookystuff.testutils.FunSpecx
-import org.scalatest.BeforeAndAfter
-
-object SerDeHookSuite {
-
- var readCounter = 0
- var writeCounter = 0
-
- val example = SerDeHook(
- _ => readCounter += 1,
- _ => writeCounter += 1
- )
-}
-
-class SerDeHookSuite extends FunSpecx with BeforeAndAfter {
-
- import SerDeHookSuite._
-
- before {
- readCounter = 0
- writeCounter = 0
- }
-
- it("can be triggered without affecting default SerDe") {
-
- AssertWeaklySerializable(
- example
- )
-
- assert(readCounter == 2)
- assert(writeCounter == 2)
- }
-}
diff --git a/pom.xml b/pom.xml
index 0685b2fe6..c5cd4d9e4 100644
--- a/pom.xml
+++ b/pom.xml
@@ -839,9 +839,12 @@
scalatest-maven-plugin
2.0.2
+ ${project.build.directory}/test-results/scalatest/scalatest-failed.txt
+
${project.build.directory}/test-results/scalatest
.
scalatest-report.txt
+
${project.build.directory}/site/scalatest
diff --git a/repackaged/selenium-bundle/pom.xml b/repackaged/selenium-bundle/pom.xml
index 57620479b..14466b766 100644
--- a/repackaged/selenium-bundle/pom.xml
+++ b/repackaged/selenium-bundle/pom.xml
@@ -12,8 +12,14 @@
${revision}
+
3.141.59
-
+ 1.4.4
+ 2.52.0
+
+
+
+
${selenium.version}
@@ -32,16 +38,22 @@
selenium-support
${selenium.version}
+
+
+
+
+
com.codeborne
phantomjsdriver
- 1.4.4
+ ${phantomjs.version}
+
org.seleniumhq.selenium
htmlunit-driver
- 2.45.0
+ ${htmlunit.version}
diff --git a/test-resources/src/main/resources/log4j.properties b/test-resources/src/main/resources/log4j.properties
index 598f80ce0..56cfb6177 100644
--- a/test-resources/src/main/resources/log4j.properties
+++ b/test-resources/src/main/resources/log4j.properties
@@ -2,13 +2,13 @@
log4j.rootLogger=ERROR, console, file
# Categories
-log4j.logger.com.tribbloids.spookystuff=INFO
-log4j.logger.org.apache.spark.ml.dsl=INFO
-log4j.logger.com.graphhopper.jsprit=INFO
-log4j.logger.org.apache.spark.rdd.spookystuff=INFO
+log4j.logger.com.tribbloids.spookystuff=DEBUG
+log4j.logger.org.apache.spark.ml.dsl=DEBUG
+#log4j.logger.com.graphhopper.jsprit=INFO
+log4j.logger.org.apache.spark.rdd.spookystuff=DEBUG
log4j.appender.console=org.apache.log4j.ConsoleAppender
-log4j.appender.console.threshold=DEBUG
+log4j.appender.console.threshold=WARN
log4j.appender.console.target=System.err
log4j.appender.console.layout=org.apache.log4j.PatternLayout
log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n
diff --git a/web/src/main/java/com/tribbloids/spookystuff/selenium/BySizzleCssSelector.java b/web/src/main/java/com/tribbloids/spookystuff/selenium/BySizzleSelector.java
similarity index 80%
rename from web/src/main/java/com/tribbloids/spookystuff/selenium/BySizzleCssSelector.java
rename to web/src/main/java/com/tribbloids/spookystuff/selenium/BySizzleSelector.java
index 0c98fae9d..da555253f 100644
--- a/web/src/main/java/com/tribbloids/spookystuff/selenium/BySizzleCssSelector.java
+++ b/web/src/main/java/com/tribbloids/spookystuff/selenium/BySizzleSelector.java
@@ -4,9 +4,11 @@
import org.openqa.selenium.*;
import java.io.Serializable;
+import java.nio.charset.Charset;
import java.util.List;
+import java.util.Objects;
-public class BySizzleCssSelector extends By implements Serializable {
+public class BySizzleSelector extends By implements Serializable {
private static final long serialVersionUID = -584931842702178943L;
@@ -14,7 +16,7 @@ public class BySizzleCssSelector extends By implements Serializable {
private static String sizzleSource = null;
- public BySizzleCssSelector(String selector) {
+ public BySizzleSelector(String selector) {
this.selector = selector;
}
@@ -26,7 +28,7 @@ public List findElements(SearchContext context) {
}
else {
throw new WebDriverException(
- "Driver does not support finding an element by selector: " + selector);
+ "Driver does not support finding an element by selector: " + selector);
}
}
@@ -62,7 +64,10 @@ protected Boolean sizzleLoaded(JavascriptExecutor context) {
protected synchronized void injectSizzle(JavascriptExecutor context) {
if (sizzleSource == null) {
try {
- sizzleSource = IOUtils.toString(this.getClass().getResource("/com/tribbloids/spookystuff/lib/js/sizzle.js"));
+ sizzleSource = IOUtils.toString(
+ Objects.requireNonNull(this.getClass().getResource("/com/tribbloids/spookystuff/lib/js/sizzle.js")),
+ Charset.defaultCharset()
+ );
} catch (Throwable e) {
throw new RuntimeException("Cannot load sizzle.js from classpath", e);
}
diff --git a/web/src/main/resources/com/tribbloids/spookystuff/lib/js/sizzle.js b/web/src/main/resources/com/tribbloids/spookystuff/lib/js/sizzle.js
index 2a1a395c1..0c6cf687c 100644
--- a/web/src/main/resources/com/tribbloids/spookystuff/lib/js/sizzle.js
+++ b/web/src/main/resources/com/tribbloids/spookystuff/lib/js/sizzle.js
@@ -1,2143 +1,2478 @@
/*!
- * Sizzle CSS Selector Engine v2.2.1
- * http://sizzlejs.com/
+ * Sizzle CSS Selector Engine v@VERSION
+ * https://sizzlejs.com/
*
- * Copyright jQuery Foundation and other contributors
+ * Copyright JS Foundation and other contributors
* Released under the MIT license
- * http://jquery.org/license
+ * https://js.foundation/
*
- * Date: 2015-10-17
+ * Date: @DATE
*/
-(function( window ) {
-
- var i,
- support,
- Expr,
- getText,
- isXML,
- tokenize,
- compile,
- select,
- outermostContext,
- sortInput,
- hasDuplicate,
-
- // Local document vars
- setDocument,
- document,
- docElem,
- documentIsHTML,
- rbuggyQSA,
- rbuggyMatches,
- matches,
- contains,
-
- // Instance-specific data
- expando = "sizzle" + 1 * new Date(),
- preferredDoc = window.document,
- dirruns = 0,
- done = 0,
- classCache = createCache(),
- tokenCache = createCache(),
- compilerCache = createCache(),
- sortOrder = function( a, b ) {
- if ( a === b ) {
- hasDuplicate = true;
- }
- return 0;
- },
-
- // General-purpose constants
- MAX_NEGATIVE = 1 << 31,
-
- // Instance methods
- hasOwn = ({}).hasOwnProperty,
- arr = [],
- pop = arr.pop,
- push_native = arr.push,
- push = arr.push,
- slice = arr.slice,
- // Use a stripped-down indexOf as it's faster than native
- // http://jsperf.com/thor-indexof-vs-for/5
- indexOf = function( list, elem ) {
- var i = 0,
- len = list.length;
- for ( ; i < len; i++ ) {
- if ( list[i] === elem ) {
- return i;
- }
- }
- return -1;
- },
-
- booleans = "checked|selected|async|autofocus|autoplay|controls|defer|disabled|hidden|ismap|loop|multiple|open|readonly|required|scoped",
-
- // Regular expressions
-
- // http://www.w3.org/TR/css3-selectors/#whitespace
- whitespace = "[\\x20\\t\\r\\n\\f]",
-
- // http://www.w3.org/TR/CSS21/syndata.html#value-def-identifier
- identifier = "(?:\\\\.|[\\w-]|[^\\x00-\\xa0])+",
-
- // Attribute selectors: http://www.w3.org/TR/selectors/#attribute-selectors
- attributes = "\\[" + whitespace + "*(" + identifier + ")(?:" + whitespace +
- // Operator (capture 2)
- "*([*^$|!~]?=)" + whitespace +
- // "Attribute values must be CSS identifiers [capture 5] or strings [capture 3 or capture 4]"
- "*(?:'((?:\\\\.|[^\\\\'])*)'|\"((?:\\\\.|[^\\\\\"])*)\"|(" + identifier + "))|)" + whitespace +
- "*\\]",
-
- pseudos = ":(" + identifier + ")(?:\\((" +
- // To reduce the number of selectors needing tokenize in the preFilter, prefer arguments:
- // 1. quoted (capture 3; capture 4 or capture 5)
- "('((?:\\\\.|[^\\\\'])*)'|\"((?:\\\\.|[^\\\\\"])*)\")|" +
- // 2. simple (capture 6)
- "((?:\\\\.|[^\\\\()[\\]]|" + attributes + ")*)|" +
- // 3. anything else (capture 2)
- ".*" +
- ")\\)|)",
-
- // Leading and non-escaped trailing whitespace, capturing some non-whitespace characters preceding the latter
- rwhitespace = new RegExp( whitespace + "+", "g" ),
- rtrim = new RegExp( "^" + whitespace + "+|((?:^|[^\\\\])(?:\\\\.)*)" + whitespace + "+$", "g" ),
-
- rcomma = new RegExp( "^" + whitespace + "*," + whitespace + "*" ),
- rcombinators = new RegExp( "^" + whitespace + "*([>+~]|" + whitespace + ")" + whitespace + "*" ),
-
- rattributeQuotes = new RegExp( "=" + whitespace + "*([^\\]'\"]*?)" + whitespace + "*\\]", "g" ),
-
- rpseudo = new RegExp( pseudos ),
- ridentifier = new RegExp( "^" + identifier + "$" ),
-
- matchExpr = {
- "ID": new RegExp( "^#(" + identifier + ")" ),
- "CLASS": new RegExp( "^\\.(" + identifier + ")" ),
- "TAG": new RegExp( "^(" + identifier + "|[*])" ),
- "ATTR": new RegExp( "^" + attributes ),
- "PSEUDO": new RegExp( "^" + pseudos ),
- "CHILD": new RegExp( "^:(only|first|last|nth|nth-last)-(child|of-type)(?:\\(" + whitespace +
- "*(even|odd|(([+-]|)(\\d*)n|)" + whitespace + "*(?:([+-]|)" + whitespace +
- "*(\\d+)|))" + whitespace + "*\\)|)", "i" ),
- "bool": new RegExp( "^(?:" + booleans + ")$", "i" ),
- // For use in libraries implementing .is()
- // We use this for POS matching in `select`
- "needsContext": new RegExp( "^" + whitespace + "*[>+~]|:(even|odd|eq|gt|lt|nth|first|last)(?:\\(" +
- whitespace + "*((?:-\\d)?\\d*)" + whitespace + "*\\)|)(?=[^-]|$)", "i" )
- },
-
- rinputs = /^(?:input|select|textarea|button)$/i,
- rheader = /^h\d$/i,
-
- rnative = /^[^{]+\{\s*\[native \w/,
-
- // Easily-parseable/retrievable ID or TAG or CLASS selectors
- rquickExpr = /^(?:#([\w-]+)|(\w+)|\.([\w-]+))$/,
-
- rsibling = /[+~]/,
- rescape = /'|\\/g,
-
- // CSS escapes http://www.w3.org/TR/CSS21/syndata.html#escaped-characters
- runescape = new RegExp( "\\\\([\\da-f]{1,6}" + whitespace + "?|(" + whitespace + ")|.)", "ig" ),
- funescape = function( _, escaped, escapedWhitespace ) {
- var high = "0x" + escaped - 0x10000;
- // NaN means non-codepoint
- // Support: Firefox<24
- // Workaround erroneous numeric interpretation of +"0x"
- return high !== high || escapedWhitespace ?
- escaped :
- high < 0 ?
- // BMP codepoint
- String.fromCharCode( high + 0x10000 ) :
- // Supplemental Plane codepoint (surrogate pair)
- String.fromCharCode( high >> 10 | 0xD800, high & 0x3FF | 0xDC00 );
- },
-
- // Used for iframes
- // See setDocument()
- // Removing the function wrapper causes a "Permission Denied"
- // error in IE
- unloadHandler = function() {
- setDocument();
- };
+( function( window ) {
+var i,
+ support,
+ Expr,
+ getText,
+ isXML,
+ tokenize,
+ compile,
+ select,
+ outermostContext,
+ sortInput,
+ hasDuplicate,
+
+ // Local document vars
+ setDocument,
+ document,
+ docElem,
+ documentIsHTML,
+ rbuggyQSA,
+ rbuggyMatches,
+ matches,
+ contains,
+
+ // Instance-specific data
+ expando = "sizzle" + 1 * new Date(),
+ preferredDoc = window.document,
+ dirruns = 0,
+ done = 0,
+ classCache = createCache(),
+ tokenCache = createCache(),
+ compilerCache = createCache(),
+ nonnativeSelectorCache = createCache(),
+ sortOrder = function( a, b ) {
+ if ( a === b ) {
+ hasDuplicate = true;
+ }
+ return 0;
+ },
+
+ // Instance methods
+ hasOwn = ( {} ).hasOwnProperty,
+ arr = [],
+ pop = arr.pop,
+ pushNative = arr.push,
+ push = arr.push,
+ slice = arr.slice,
+
+ // Use a stripped-down indexOf as it's faster than native
+ // https://jsperf.com/thor-indexof-vs-for/5
+ indexOf = function( list, elem ) {
+ var i = 0,
+ len = list.length;
+ for ( ; i < len; i++ ) {
+ if ( list[ i ] === elem ) {
+ return i;
+ }
+ }
+ return -1;
+ },
+
+ booleans = "checked|selected|async|autofocus|autoplay|controls|defer|disabled|hidden|" +
+ "ismap|loop|multiple|open|readonly|required|scoped",
+
+ // Regular expressions
+
+ // http://www.w3.org/TR/css3-selectors/#whitespace
+ whitespace = "[\\x20\\t\\r\\n\\f]",
+
+ // https://www.w3.org/TR/css-syntax-3/#ident-token-diagram
+ identifier = "(?:\\\\[\\da-fA-F]{1,6}" + whitespace +
+ "?|\\\\[^\\r\\n\\f]|[\\w-]|[^\0-\\x7f])+",
+
+ // Attribute selectors: http://www.w3.org/TR/selectors/#attribute-selectors
+ attributes = "\\[" + whitespace + "*(" + identifier + ")(?:" + whitespace +
+
+ // Operator (capture 2)
+ "*([*^$|!~]?=)" + whitespace +
+
+ // "Attribute values must be CSS identifiers [capture 5]
+ // or strings [capture 3 or capture 4]"
+ "*(?:'((?:\\\\.|[^\\\\'])*)'|\"((?:\\\\.|[^\\\\\"])*)\"|(" + identifier + "))|)" +
+ whitespace + "*\\]",
+
+ pseudos = ":(" + identifier + ")(?:\\((" +
+
+ // To reduce the number of selectors needing tokenize in the preFilter, prefer arguments:
+ // 1. quoted (capture 3; capture 4 or capture 5)
+ "('((?:\\\\.|[^\\\\'])*)'|\"((?:\\\\.|[^\\\\\"])*)\")|" +
+
+ // 2. simple (capture 6)
+ "((?:\\\\.|[^\\\\()[\\]]|" + attributes + ")*)|" +
+
+ // 3. anything else (capture 2)
+ ".*" +
+ ")\\)|)",
+
+ // Leading and non-escaped trailing whitespace, capturing some non-whitespace characters preceding the latter
+ rwhitespace = new RegExp( whitespace + "+", "g" ),
+ rtrim = new RegExp( "^" + whitespace + "+|((?:^|[^\\\\])(?:\\\\.)*)" +
+ whitespace + "+$", "g" ),
+
+ rcomma = new RegExp( "^" + whitespace + "*," + whitespace + "*" ),
+ rcombinators = new RegExp( "^" + whitespace + "*([>+~]|" + whitespace + ")" + whitespace +
+ "*" ),
+ rdescend = new RegExp( whitespace + "|>" ),
+
+ rpseudo = new RegExp( pseudos ),
+ ridentifier = new RegExp( "^" + identifier + "$" ),
+
+ matchExpr = {
+ "ID": new RegExp( "^#(" + identifier + ")" ),
+ "CLASS": new RegExp( "^\\.(" + identifier + ")" ),
+ "TAG": new RegExp( "^(" + identifier + "|[*])" ),
+ "ATTR": new RegExp( "^" + attributes ),
+ "PSEUDO": new RegExp( "^" + pseudos ),
+ "CHILD": new RegExp( "^:(only|first|last|nth|nth-last)-(child|of-type)(?:\\(" +
+ whitespace + "*(even|odd|(([+-]|)(\\d*)n|)" + whitespace + "*(?:([+-]|)" +
+ whitespace + "*(\\d+)|))" + whitespace + "*\\)|)", "i" ),
+ "bool": new RegExp( "^(?:" + booleans + ")$", "i" ),
+
+ // For use in libraries implementing .is()
+ // We use this for POS matching in `select`
+ "needsContext": new RegExp( "^" + whitespace +
+ "*[>+~]|:(even|odd|eq|gt|lt|nth|first|last)(?:\\(" + whitespace +
+ "*((?:-\\d)?\\d*)" + whitespace + "*\\)|)(?=[^-]|$)", "i" )
+ },
+
+ rhtml = /HTML$/i,
+ rinputs = /^(?:input|select|textarea|button)$/i,
+ rheader = /^h\d$/i,
+
+ rnative = /^[^{]+\{\s*\[native \w/,
+
+ // Easily-parseable/retrievable ID or TAG or CLASS selectors
+ rquickExpr = /^(?:#([\w-]+)|(\w+)|\.([\w-]+))$/,
+
+ rsibling = /[+~]/,
+
+ // CSS escapes
+ // http://www.w3.org/TR/CSS21/syndata.html#escaped-characters
+ runescape = new RegExp( "\\\\[\\da-fA-F]{1,6}" + whitespace + "?|\\\\([^\\r\\n\\f])", "g" ),
+ funescape = function( escape, nonHex ) {
+ var high = "0x" + escape.slice( 1 ) - 0x10000;
+
+ return nonHex ?
+
+ // Strip the backslash prefix from a non-hex escape sequence
+ nonHex :
+
+ // Replace a hexadecimal escape sequence with the encoded Unicode code point
+ // Support: IE <=11+
+ // For values outside the Basic Multilingual Plane (BMP), manually construct a
+ // surrogate pair
+ high < 0 ?
+ String.fromCharCode( high + 0x10000 ) :
+ String.fromCharCode( high >> 10 | 0xD800, high & 0x3FF | 0xDC00 );
+ },
+
+ // CSS string/identifier serialization
+ // https://drafts.csswg.org/cssom/#common-serializing-idioms
+ rcssescape = /([\0-\x1f\x7f]|^-?\d)|^-$|[^\0-\x1f\x7f-\uFFFF\w-]/g,
+ fcssescape = function( ch, asCodePoint ) {
+ if ( asCodePoint ) {
+
+ // U+0000 NULL becomes U+FFFD REPLACEMENT CHARACTER
+ if ( ch === "\0" ) {
+ return "\uFFFD";
+ }
+
+ // Control characters and (dependent upon position) numbers get escaped as code points
+ return ch.slice( 0, -1 ) + "\\" +
+ ch.charCodeAt( ch.length - 1 ).toString( 16 ) + " ";
+ }
+
+ // Other potentially-special ASCII characters get backslash-escaped
+ return "\\" + ch;
+ },
+
+ // Used for iframes
+ // See setDocument()
+ // Removing the function wrapper causes a "Permission Denied"
+ // error in IE
+ unloadHandler = function() {
+ setDocument();
+ },
+
+ inDisabledFieldset = addCombinator(
+ function( elem ) {
+ return elem.disabled === true && elem.nodeName.toLowerCase() === "fieldset";
+ },
+ { dir: "parentNode", next: "legend" }
+ );
// Optimize for push.apply( _, NodeList )
- try {
- push.apply(
- (arr = slice.call( preferredDoc.childNodes )),
- preferredDoc.childNodes
- );
- // Support: Android<4.0
- // Detect silently failing push.apply
- arr[ preferredDoc.childNodes.length ].nodeType;
- } catch ( e ) {
- push = { apply: arr.length ?
-
- // Leverage slice if possible
- function( target, els ) {
- push_native.apply( target, slice.call(els) );
- } :
-
- // Support: IE<9
- // Otherwise append directly
- function( target, els ) {
- var j = target.length,
- i = 0;
- // Can't trust NodeList.length
- while ( (target[j++] = els[i++]) ) {}
- target.length = j - 1;
- }
- };
- }
-
- function Sizzle( selector, context, results, seed ) {
- var m, i, elem, nid, nidselect, match, groups, newSelector,
- newContext = context && context.ownerDocument,
-
- // nodeType defaults to 9, since context defaults to document
- nodeType = context ? context.nodeType : 9;
-
- results = results || [];
-
- // Return early from calls with invalid selector or context
- if ( typeof selector !== "string" || !selector ||
- nodeType !== 1 && nodeType !== 9 && nodeType !== 11 ) {
-
- return results;
- }
-
- // Try to shortcut find operations (as opposed to filters) in HTML documents
- if ( !seed ) {
-
- if ( ( context ? context.ownerDocument || context : preferredDoc ) !== document ) {
- setDocument( context );
- }
- context = context || document;
-
- if ( documentIsHTML ) {
-
- // If the selector is sufficiently simple, try using a "get*By*" DOM method
- // (excepting DocumentFragment context, where the methods don't exist)
- if ( nodeType !== 11 && (match = rquickExpr.exec( selector )) ) {
-
- // ID selector
- if ( (m = match[1]) ) {
-
- // Document context
- if ( nodeType === 9 ) {
- if ( (elem = context.getElementById( m )) ) {
-
- // Support: IE, Opera, Webkit
- // TODO: identify versions
- // getElementById can match elements by name instead of ID
- if ( elem.id === m ) {
- results.push( elem );
- return results;
- }
- } else {
- return results;
- }
-
- // Element context
- } else {
-
- // Support: IE, Opera, Webkit
- // TODO: identify versions
- // getElementById can match elements by name instead of ID
- if ( newContext && (elem = newContext.getElementById( m )) &&
- contains( context, elem ) &&
- elem.id === m ) {
-
- results.push( elem );
- return results;
- }
- }
-
- // Type selector
- } else if ( match[2] ) {
- push.apply( results, context.getElementsByTagName( selector ) );
- return results;
-
- // Class selector
- } else if ( (m = match[3]) && support.getElementsByClassName &&
- context.getElementsByClassName ) {
-
- push.apply( results, context.getElementsByClassName( m ) );
- return results;
- }
- }
-
- // Take advantage of querySelectorAll
- if ( support.qsa &&
- !compilerCache[ selector + " " ] &&
- (!rbuggyQSA || !rbuggyQSA.test( selector )) ) {
-
- if ( nodeType !== 1 ) {
- newContext = context;
- newSelector = selector;
-
- // qSA looks outside Element context, which is not what we want
- // Thanks to Andrew Dupont for this workaround technique
- // Support: IE <=8
- // Exclude object elements
- } else if ( context.nodeName.toLowerCase() !== "object" ) {
-
- // Capture the context ID, setting it first if necessary
- if ( (nid = context.getAttribute( "id" )) ) {
- nid = nid.replace( rescape, "\\$&" );
- } else {
- context.setAttribute( "id", (nid = expando) );
- }
-
- // Prefix every selector in the list
- groups = tokenize( selector );
- i = groups.length;
- nidselect = ridentifier.test( nid ) ? "#" + nid : "[id='" + nid + "']";
- while ( i-- ) {
- groups[i] = nidselect + " " + toSelector( groups[i] );
- }
- newSelector = groups.join( "," );
-
- // Expand context for sibling selectors
- newContext = rsibling.test( selector ) && testContext( context.parentNode ) ||
- context;
- }
-
- if ( newSelector ) {
- try {
- push.apply( results,
- newContext.querySelectorAll( newSelector )
- );
- return results;
- } catch ( qsaError ) {
- } finally {
- if ( nid === expando ) {
- context.removeAttribute( "id" );
- }
- }
- }
- }
- }
- }
-
- // All others
- return select( selector.replace( rtrim, "$1" ), context, results, seed );
- }
-
- /**
- * Create key-value caches of limited size
- * @returns {function(string, object)} Returns the Object data after storing it on itself with
- * property name the (space-suffixed) string and (if the cache is larger than Expr.cacheLength)
- * deleting the oldest entry
- */
- function createCache() {
- var keys = [];
-
- function cache( key, value ) {
- // Use (key + " ") to avoid collision with native prototype properties (see Issue #157)
- if ( keys.push( key + " " ) > Expr.cacheLength ) {
- // Only keep the most recent entries
- delete cache[ keys.shift() ];
- }
- return (cache[ key + " " ] = value);
- }
- return cache;
- }
-
- /**
- * Mark a function for special use by Sizzle
- * @param {Function} fn The function to mark
- */
- function markFunction( fn ) {
- fn[ expando ] = true;
- return fn;
- }
-
- /**
- * Support testing using an element
- * @param {Function} fn Passed the created div and expects a boolean result
- */
- function assert( fn ) {
- var div = document.createElement("div");
-
- try {
- return !!fn( div );
- } catch (e) {
- return false;
- } finally {
- // Remove from its parent by default
- if ( div.parentNode ) {
- div.parentNode.removeChild( div );
- }
- // release memory in IE
- div = null;
- }
- }
-
- /**
- * Adds the same handler for all of the specified attrs
- * @param {String} attrs Pipe-separated list of attributes
- * @param {Function} handler The method that will be applied
- */
- function addHandle( attrs, handler ) {
- var arr = attrs.split("|"),
- i = arr.length;
-
- while ( i-- ) {
- Expr.attrHandle[ arr[i] ] = handler;
- }
- }
-
- /**
- * Checks document order of two siblings
- * @param {Element} a
- * @param {Element} b
- * @returns {Number} Returns less than 0 if a precedes b, greater than 0 if a follows b
- */
- function siblingCheck( a, b ) {
- var cur = b && a,
- diff = cur && a.nodeType === 1 && b.nodeType === 1 &&
- ( ~b.sourceIndex || MAX_NEGATIVE ) -
- ( ~a.sourceIndex || MAX_NEGATIVE );
-
- // Use IE sourceIndex if available on both nodes
- if ( diff ) {
- return diff;
- }
-
- // Check if b follows a
- if ( cur ) {
- while ( (cur = cur.nextSibling) ) {
- if ( cur === b ) {
- return -1;
- }
- }
- }
-
- return a ? 1 : -1;
- }
-
- /**
- * Returns a function to use in pseudos for input types
- * @param {String} type
- */
- function createInputPseudo( type ) {
- return function( elem ) {
- var name = elem.nodeName.toLowerCase();
- return name === "input" && elem.type === type;
- };
- }
-
- /**
- * Returns a function to use in pseudos for buttons
- * @param {String} type
- */
- function createButtonPseudo( type ) {
- return function( elem ) {
- var name = elem.nodeName.toLowerCase();
- return (name === "input" || name === "button") && elem.type === type;
- };
- }
-
- /**
- * Returns a function to use in pseudos for positionals
- * @param {Function} fn
- */
- function createPositionalPseudo( fn ) {
- return markFunction(function( argument ) {
- argument = +argument;
- return markFunction(function( seed, matches ) {
- var j,
- matchIndexes = fn( [], seed.length, argument ),
- i = matchIndexes.length;
-
- // Match elements found at the specified indexes
- while ( i-- ) {
- if ( seed[ (j = matchIndexes[i]) ] ) {
- seed[j] = !(matches[j] = seed[j]);
- }
- }
- });
- });
- }
-
- /**
- * Checks a node for validity as a Sizzle context
- * @param {Element|Object=} context
- * @returns {Element|Object|Boolean} The input node if acceptable, otherwise a falsy value
- */
- function testContext( context ) {
- return context && typeof context.getElementsByTagName !== "undefined" && context;
- }
+try {
+ push.apply(
+ ( arr = slice.call( preferredDoc.childNodes ) ),
+ preferredDoc.childNodes
+ );
+
+ // Support: Android<4.0
+ // Detect silently failing push.apply
+ // eslint-disable-next-line no-unused-expressions
+ arr[ preferredDoc.childNodes.length ].nodeType;
+} catch ( e ) {
+ push = { apply: arr.length ?
+
+ // Leverage slice if possible
+ function( target, els ) {
+ pushNative.apply( target, slice.call( els ) );
+ } :
+
+ // Support: IE<9
+ // Otherwise append directly
+ function( target, els ) {
+ var j = target.length,
+ i = 0;
+
+ // Can't trust NodeList.length
+ while ( ( target[ j++ ] = els[ i++ ] ) ) {}
+ target.length = j - 1;
+ }
+ };
+}
+
+function Sizzle( selector, context, results, seed ) {
+ var m, i, elem, nid, match, groups, newSelector,
+ newContext = context && context.ownerDocument,
+
+ // nodeType defaults to 9, since context defaults to document
+ nodeType = context ? context.nodeType : 9;
+
+ results = results || [];
+
+ // Return early from calls with invalid selector or context
+ if ( typeof selector !== "string" || !selector ||
+ nodeType !== 1 && nodeType !== 9 && nodeType !== 11 ) {
+
+ return results;
+ }
+
+ // Try to shortcut find operations (as opposed to filters) in HTML documents
+ if ( !seed ) {
+ setDocument( context );
+ context = context || document;
+
+ if ( documentIsHTML ) {
+
+ // If the selector is sufficiently simple, try using a "get*By*" DOM method
+ // (excepting DocumentFragment context, where the methods don't exist)
+ if ( nodeType !== 11 && ( match = rquickExpr.exec( selector ) ) ) {
+
+ // ID selector
+ if ( ( m = match[ 1 ] ) ) {
+
+ // Document context
+ if ( nodeType === 9 ) {
+ if ( ( elem = context.getElementById( m ) ) ) {
+
+ // Support: IE, Opera, Webkit
+ // TODO: identify versions
+ // getElementById can match elements by name instead of ID
+ if ( elem.id === m ) {
+ results.push( elem );
+ return results;
+ }
+ } else {
+ return results;
+ }
+
+ // Element context
+ } else {
+
+ // Support: IE, Opera, Webkit
+ // TODO: identify versions
+ // getElementById can match elements by name instead of ID
+ if ( newContext && ( elem = newContext.getElementById( m ) ) &&
+ contains( context, elem ) &&
+ elem.id === m ) {
+
+ results.push( elem );
+ return results;
+ }
+ }
+
+ // Type selector
+ } else if ( match[ 2 ] ) {
+ push.apply( results, context.getElementsByTagName( selector ) );
+ return results;
+
+ // Class selector
+ } else if ( ( m = match[ 3 ] ) && support.getElementsByClassName &&
+ context.getElementsByClassName ) {
+
+ push.apply( results, context.getElementsByClassName( m ) );
+ return results;
+ }
+ }
+
+ // Take advantage of querySelectorAll
+ if ( support.qsa &&
+ !nonnativeSelectorCache[ selector + " " ] &&
+ ( !rbuggyQSA || !rbuggyQSA.test( selector ) ) &&
+
+ // Support: IE 8 only
+ // Exclude object elements
+ ( nodeType !== 1 || context.nodeName.toLowerCase() !== "object" ) ) {
+
+ newSelector = selector;
+ newContext = context;
+
+ // qSA considers elements outside a scoping root when evaluating child or
+ // descendant combinators, which is not what we want.
+ // In such cases, we work around the behavior by prefixing every selector in the
+ // list with an ID selector referencing the scope context.
+ // The technique has to be used as well when a leading combinator is used
+ // as such selectors are not recognized by querySelectorAll.
+ // Thanks to Andrew Dupont for this technique.
+ if ( nodeType === 1 &&
+ ( rdescend.test( selector ) || rcombinators.test( selector ) ) ) {
+
+ // Expand context for sibling selectors
+ newContext = rsibling.test( selector ) && testContext( context.parentNode ) ||
+ context;
+
+ // We can use :scope instead of the ID hack if the browser
+ // supports it & if we're not changing the context.
+ if ( newContext !== context || !support.scope ) {
+
+ // Capture the context ID, setting it first if necessary
+ if ( ( nid = context.getAttribute( "id" ) ) ) {
+ nid = nid.replace( rcssescape, fcssescape );
+ } else {
+ context.setAttribute( "id", ( nid = expando ) );
+ }
+ }
+
+ // Prefix every selector in the list
+ groups = tokenize( selector );
+ i = groups.length;
+ while ( i-- ) {
+ groups[ i ] = ( nid ? "#" + nid : ":scope" ) + " " +
+ toSelector( groups[ i ] );
+ }
+ newSelector = groups.join( "," );
+ }
+
+ try {
+ push.apply( results,
+ newContext.querySelectorAll( newSelector )
+ );
+ return results;
+ } catch ( qsaError ) {
+ nonnativeSelectorCache( selector, true );
+ } finally {
+ if ( nid === expando ) {
+ context.removeAttribute( "id" );
+ }
+ }
+ }
+ }
+ }
+
+ // All others
+ return select( selector.replace( rtrim, "$1" ), context, results, seed );
+}
+
+/**
+ * Create key-value caches of limited size
+ * @returns {function(string, object)} Returns the Object data after storing it on itself with
+ * property name the (space-suffixed) string and (if the cache is larger than Expr.cacheLength)
+ * deleting the oldest entry
+ */
+function createCache() {
+ var keys = [];
+
+ function cache( key, value ) {
+
+ // Use (key + " ") to avoid collision with native prototype properties (see Issue #157)
+ if ( keys.push( key + " " ) > Expr.cacheLength ) {
+
+ // Only keep the most recent entries
+ delete cache[ keys.shift() ];
+ }
+ return ( cache[ key + " " ] = value );
+ }
+ return cache;
+}
+
+/**
+ * Mark a function for special use by Sizzle
+ * @param {Function} fn The function to mark
+ */
+function markFunction( fn ) {
+ fn[ expando ] = true;
+ return fn;
+}
+
+/**
+ * Support testing using an element
+ * @param {Function} fn Passed the created element and returns a boolean result
+ */
+function assert( fn ) {
+ var el = document.createElement( "fieldset" );
+
+ try {
+ return !!fn( el );
+ } catch ( e ) {
+ return false;
+ } finally {
+
+ // Remove from its parent by default
+ if ( el.parentNode ) {
+ el.parentNode.removeChild( el );
+ }
+
+ // release memory in IE
+ el = null;
+ }
+}
+
+/**
+ * Adds the same handler for all of the specified attrs
+ * @param {String} attrs Pipe-separated list of attributes
+ * @param {Function} handler The method that will be applied
+ */
+function addHandle( attrs, handler ) {
+ var arr = attrs.split( "|" ),
+ i = arr.length;
+
+ while ( i-- ) {
+ Expr.attrHandle[ arr[ i ] ] = handler;
+ }
+}
+
+/**
+ * Checks document order of two siblings
+ * @param {Element} a
+ * @param {Element} b
+ * @returns {Number} Returns less than 0 if a precedes b, greater than 0 if a follows b
+ */
+function siblingCheck( a, b ) {
+ var cur = b && a,
+ diff = cur && a.nodeType === 1 && b.nodeType === 1 &&
+ a.sourceIndex - b.sourceIndex;
+
+ // Use IE sourceIndex if available on both nodes
+ if ( diff ) {
+ return diff;
+ }
+
+ // Check if b follows a
+ if ( cur ) {
+ while ( ( cur = cur.nextSibling ) ) {
+ if ( cur === b ) {
+ return -1;
+ }
+ }
+ }
+
+ return a ? 1 : -1;
+}
+
+/**
+ * Returns a function to use in pseudos for input types
+ * @param {String} type
+ */
+function createInputPseudo( type ) {
+ return function( elem ) {
+ var name = elem.nodeName.toLowerCase();
+ return name === "input" && elem.type === type;
+ };
+}
+
+/**
+ * Returns a function to use in pseudos for buttons
+ * @param {String} type
+ */
+function createButtonPseudo( type ) {
+ return function( elem ) {
+ var name = elem.nodeName.toLowerCase();
+ return ( name === "input" || name === "button" ) && elem.type === type;
+ };
+}
+
+/**
+ * Returns a function to use in pseudos for :enabled/:disabled
+ * @param {Boolean} disabled true for :disabled; false for :enabled
+ */
+function createDisabledPseudo( disabled ) {
+
+ // Known :disabled false positives: fieldset[disabled] > legend:nth-of-type(n+2) :can-disable
+ return function( elem ) {
+
+ // Only certain elements can match :enabled or :disabled
+ // https://html.spec.whatwg.org/multipage/scripting.html#selector-enabled
+ // https://html.spec.whatwg.org/multipage/scripting.html#selector-disabled
+ if ( "form" in elem ) {
+
+ // Check for inherited disabledness on relevant non-disabled elements:
+ // * listed form-associated elements in a disabled fieldset
+ // https://html.spec.whatwg.org/multipage/forms.html#category-listed
+ // https://html.spec.whatwg.org/multipage/forms.html#concept-fe-disabled
+ // * option elements in a disabled optgroup
+ // https://html.spec.whatwg.org/multipage/forms.html#concept-option-disabled
+ // All such elements have a "form" property.
+ if ( elem.parentNode && elem.disabled === false ) {
+
+ // Option elements defer to a parent optgroup if present
+ if ( "label" in elem ) {
+ if ( "label" in elem.parentNode ) {
+ return elem.parentNode.disabled === disabled;
+ } else {
+ return elem.disabled === disabled;
+ }
+ }
+
+ // Support: IE 6 - 11
+ // Use the isDisabled shortcut property to check for disabled fieldset ancestors
+ return elem.isDisabled === disabled ||
+
+ // Where there is no isDisabled, check manually
+ /* jshint -W018 */
+ elem.isDisabled !== !disabled &&
+ inDisabledFieldset( elem ) === disabled;
+ }
+
+ return elem.disabled === disabled;
+
+ // Try to winnow out elements that can't be disabled before trusting the disabled property.
+ // Some victims get caught in our net (label, legend, menu, track), but it shouldn't
+ // even exist on them, let alone have a boolean value.
+ } else if ( "label" in elem ) {
+ return elem.disabled === disabled;
+ }
+
+ // Remaining elements are neither :enabled nor :disabled
+ return false;
+ };
+}
+
+/**
+ * Returns a function to use in pseudos for positionals
+ * @param {Function} fn
+ */
+function createPositionalPseudo( fn ) {
+ return markFunction( function( argument ) {
+ argument = +argument;
+ return markFunction( function( seed, matches ) {
+ var j,
+ matchIndexes = fn( [], seed.length, argument ),
+ i = matchIndexes.length;
+
+ // Match elements found at the specified indexes
+ while ( i-- ) {
+ if ( seed[ ( j = matchIndexes[ i ] ) ] ) {
+ seed[ j ] = !( matches[ j ] = seed[ j ] );
+ }
+ }
+ } );
+ } );
+}
+
+/**
+ * Checks a node for validity as a Sizzle context
+ * @param {Element|Object=} context
+ * @returns {Element|Object|Boolean} The input node if acceptable, otherwise a falsy value
+ */
+function testContext( context ) {
+ return context && typeof context.getElementsByTagName !== "undefined" && context;
+}
// Expose support vars for convenience
- support = Sizzle.support = {};
-
- /**
- * Detects XML nodes
- * @param {Element|Object} elem An element or a document
- * @returns {Boolean} True iff elem is a non-HTML XML node
- */
- isXML = Sizzle.isXML = function( elem ) {
- // documentElement is verified for cases where it doesn't yet exist
- // (such as loading iframes in IE - #4833)
- var documentElement = elem && (elem.ownerDocument || elem).documentElement;
- return documentElement ? documentElement.nodeName !== "HTML" : false;
- };
-
- /**
- * Sets document-related variables once based on the current document
- * @param {Element|Object} [doc] An element or document object to use to set the document
- * @returns {Object} Returns the current document
- */
- setDocument = Sizzle.setDocument = function( node ) {
- var hasCompare, parent,
- doc = node ? node.ownerDocument || node : preferredDoc;
-
- // Return early if doc is invalid or already selected
- if ( doc === document || doc.nodeType !== 9 || !doc.documentElement ) {
- return document;
- }
-
- // Update global variables
- document = doc;
- docElem = document.documentElement;
- documentIsHTML = !isXML( document );
-
- // Support: IE 9-11, Edge
- // Accessing iframe documents after unload throws "permission denied" errors (jQuery #13936)
- if ( (parent = document.defaultView) && parent.top !== parent ) {
- // Support: IE 11
- if ( parent.addEventListener ) {
- parent.addEventListener( "unload", unloadHandler, false );
-
- // Support: IE 9 - 10 only
- } else if ( parent.attachEvent ) {
- parent.attachEvent( "onunload", unloadHandler );
- }
- }
-
- /* Attributes
- ---------------------------------------------------------------------- */
-
- // Support: IE<8
- // Verify that getAttribute really returns attributes and not properties
- // (excepting IE8 booleans)
- support.attributes = assert(function( div ) {
- div.className = "i";
- return !div.getAttribute("className");
- });
-
- /* getElement(s)By*
- ---------------------------------------------------------------------- */
-
- // Check if getElementsByTagName("*") returns only elements
- support.getElementsByTagName = assert(function( div ) {
- div.appendChild( document.createComment("") );
- return !div.getElementsByTagName("*").length;
- });
-
- // Support: IE<9
- support.getElementsByClassName = rnative.test( document.getElementsByClassName );
-
- // Support: IE<10
- // Check if getElementById returns elements by name
- // The broken getElementById methods don't pick up programatically-set names,
- // so use a roundabout getElementsByName test
- support.getById = assert(function( div ) {
- docElem.appendChild( div ).id = expando;
- return !document.getElementsByName || !document.getElementsByName( expando ).length;
- });
-
- // ID find and filter
- if ( support.getById ) {
- Expr.find["ID"] = function( id, context ) {
- if ( typeof context.getElementById !== "undefined" && documentIsHTML ) {
- var m = context.getElementById( id );
- return m ? [ m ] : [];
- }
- };
- Expr.filter["ID"] = function( id ) {
- var attrId = id.replace( runescape, funescape );
- return function( elem ) {
- return elem.getAttribute("id") === attrId;
- };
- };
- } else {
- // Support: IE6/7
- // getElementById is not reliable as a find shortcut
- delete Expr.find["ID"];
-
- Expr.filter["ID"] = function( id ) {
- var attrId = id.replace( runescape, funescape );
- return function( elem ) {
- var node = typeof elem.getAttributeNode !== "undefined" &&
- elem.getAttributeNode("id");
- return node && node.value === attrId;
- };
- };
- }
-
- // Tag
- Expr.find["TAG"] = support.getElementsByTagName ?
- function( tag, context ) {
- if ( typeof context.getElementsByTagName !== "undefined" ) {
- return context.getElementsByTagName( tag );
-
- // DocumentFragment nodes don't have gEBTN
- } else if ( support.qsa ) {
- return context.querySelectorAll( tag );
- }
- } :
-
- function( tag, context ) {
- var elem,
- tmp = [],
- i = 0,
- // By happy coincidence, a (broken) gEBTN appears on DocumentFragment nodes too
- results = context.getElementsByTagName( tag );
-
- // Filter out possible comments
- if ( tag === "*" ) {
- while ( (elem = results[i++]) ) {
- if ( elem.nodeType === 1 ) {
- tmp.push( elem );
- }
- }
-
- return tmp;
- }
- return results;
- };
-
- // Class
- Expr.find["CLASS"] = support.getElementsByClassName && function( className, context ) {
- if ( typeof context.getElementsByClassName !== "undefined" && documentIsHTML ) {
- return context.getElementsByClassName( className );
- }
- };
-
- /* QSA/matchesSelector
- ---------------------------------------------------------------------- */
-
- // QSA and matchesSelector support
-
- // matchesSelector(:active) reports false when true (IE9/Opera 11.5)
- rbuggyMatches = [];
-
- // qSa(:focus) reports false when true (Chrome 21)
- // We allow this because of a bug in IE8/9 that throws an error
- // whenever `document.activeElement` is accessed on an iframe
- // So, we allow :focus to pass through QSA all the time to avoid the IE error
- // See http://bugs.jquery.com/ticket/13378
- rbuggyQSA = [];
-
- if ( (support.qsa = rnative.test( document.querySelectorAll )) ) {
- // Build QSA regex
- // Regex strategy adopted from Diego Perini
- assert(function( div ) {
- // Select is set to empty string on purpose
- // This is to test IE's treatment of not explicitly
- // setting a boolean content attribute,
- // since its presence should be enough
- // http://bugs.jquery.com/ticket/12359
- docElem.appendChild( div ).innerHTML = "" +
- "";
-
- // Support: IE8, Opera 11-12.16
- // Nothing should be selected when empty strings follow ^= or $= or *=
- // The test attribute must be unknown in Opera but "safe" for WinRT
- // http://msdn.microsoft.com/en-us/library/ie/hh465388.aspx#attribute_section
- if ( div.querySelectorAll("[msallowcapture^='']").length ) {
- rbuggyQSA.push( "[*^$]=" + whitespace + "*(?:''|\"\")" );
- }
-
- // Support: IE8
- // Boolean attributes and "value" are not treated correctly
- if ( !div.querySelectorAll("[selected]").length ) {
- rbuggyQSA.push( "\\[" + whitespace + "*(?:value|" + booleans + ")" );
- }
-
- // Support: Chrome<29, Android<4.4, Safari<7.0+, iOS<7.0+, PhantomJS<1.9.8+
- if ( !div.querySelectorAll( "[id~=" + expando + "-]" ).length ) {
- rbuggyQSA.push("~=");
- }
-
- // Webkit/Opera - :checked should return selected option elements
- // http://www.w3.org/TR/2011/REC-css3-selectors-20110929/#checked
- // IE8 throws error here and will not see later tests
- if ( !div.querySelectorAll(":checked").length ) {
- rbuggyQSA.push(":checked");
- }
-
- // Support: Safari 8+, iOS 8+
- // https://bugs.webkit.org/show_bug.cgi?id=136851
- // In-page `selector#id sibing-combinator selector` fails
- if ( !div.querySelectorAll( "a#" + expando + "+*" ).length ) {
- rbuggyQSA.push(".#.+[+~]");
- }
- });
-
- assert(function( div ) {
- // Support: Windows 8 Native Apps
- // The type and name attributes are restricted during .innerHTML assignment
- var input = document.createElement("input");
- input.setAttribute( "type", "hidden" );
- div.appendChild( input ).setAttribute( "name", "D" );
-
- // Support: IE8
- // Enforce case-sensitivity of name attribute
- if ( div.querySelectorAll("[name=d]").length ) {
- rbuggyQSA.push( "name" + whitespace + "*[*^$|!~]?=" );
- }
-
- // FF 3.5 - :enabled/:disabled and hidden elements (hidden elements are still enabled)
- // IE8 throws error here and will not see later tests
- if ( !div.querySelectorAll(":enabled").length ) {
- rbuggyQSA.push( ":enabled", ":disabled" );
- }
-
- // Opera 10-11 does not throw on post-comma invalid pseudos
- div.querySelectorAll("*,:x");
- rbuggyQSA.push(",.*:");
- });
- }
-
- if ( (support.matchesSelector = rnative.test( (matches = docElem.matches ||
- docElem.webkitMatchesSelector ||
- docElem.mozMatchesSelector ||
- docElem.oMatchesSelector ||
- docElem.msMatchesSelector) )) ) {
-
- assert(function( div ) {
- // Check to see if it's possible to do matchesSelector
- // on a disconnected node (IE 9)
- support.disconnectedMatch = matches.call( div, "div" );
-
- // This should fail with an exception
- // Gecko does not error, returns false instead
- matches.call( div, "[s!='']:x" );
- rbuggyMatches.push( "!=", pseudos );
- });
- }
-
- rbuggyQSA = rbuggyQSA.length && new RegExp( rbuggyQSA.join("|") );
- rbuggyMatches = rbuggyMatches.length && new RegExp( rbuggyMatches.join("|") );
-
- /* Contains
- ---------------------------------------------------------------------- */
- hasCompare = rnative.test( docElem.compareDocumentPosition );
-
- // Element contains another
- // Purposefully self-exclusive
- // As in, an element does not contain itself
- contains = hasCompare || rnative.test( docElem.contains ) ?
- function( a, b ) {
- var adown = a.nodeType === 9 ? a.documentElement : a,
- bup = b && b.parentNode;
- return a === bup || !!( bup && bup.nodeType === 1 && (
- adown.contains ?
- adown.contains( bup ) :
- a.compareDocumentPosition && a.compareDocumentPosition( bup ) & 16
- ));
- } :
- function( a, b ) {
- if ( b ) {
- while ( (b = b.parentNode) ) {
- if ( b === a ) {
- return true;
- }
- }
- }
- return false;
- };
-
- /* Sorting
- ---------------------------------------------------------------------- */
-
- // Document order sorting
- sortOrder = hasCompare ?
- function( a, b ) {
-
- // Flag for duplicate removal
- if ( a === b ) {
- hasDuplicate = true;
- return 0;
- }
-
- // Sort on method existence if only one input has compareDocumentPosition
- var compare = !a.compareDocumentPosition - !b.compareDocumentPosition;
- if ( compare ) {
- return compare;
- }
-
- // Calculate position if both inputs belong to the same document
- compare = ( a.ownerDocument || a ) === ( b.ownerDocument || b ) ?
- a.compareDocumentPosition( b ) :
-
- // Otherwise we know they are disconnected
- 1;
-
- // Disconnected nodes
- if ( compare & 1 ||
- (!support.sortDetached && b.compareDocumentPosition( a ) === compare) ) {
-
- // Choose the first element that is related to our preferred document
- if ( a === document || a.ownerDocument === preferredDoc && contains(preferredDoc, a) ) {
- return -1;
- }
- if ( b === document || b.ownerDocument === preferredDoc && contains(preferredDoc, b) ) {
- return 1;
- }
-
- // Maintain original order
- return sortInput ?
- ( indexOf( sortInput, a ) - indexOf( sortInput, b ) ) :
- 0;
- }
-
- return compare & 4 ? -1 : 1;
- } :
- function( a, b ) {
- // Exit early if the nodes are identical
- if ( a === b ) {
- hasDuplicate = true;
- return 0;
- }
-
- var cur,
- i = 0,
- aup = a.parentNode,
- bup = b.parentNode,
- ap = [ a ],
- bp = [ b ];
-
- // Parentless nodes are either documents or disconnected
- if ( !aup || !bup ) {
- return a === document ? -1 :
- b === document ? 1 :
- aup ? -1 :
- bup ? 1 :
- sortInput ?
- ( indexOf( sortInput, a ) - indexOf( sortInput, b ) ) :
- 0;
-
- // If the nodes are siblings, we can do a quick check
- } else if ( aup === bup ) {
- return siblingCheck( a, b );
- }
-
- // Otherwise we need full lists of their ancestors for comparison
- cur = a;
- while ( (cur = cur.parentNode) ) {
- ap.unshift( cur );
- }
- cur = b;
- while ( (cur = cur.parentNode) ) {
- bp.unshift( cur );
- }
-
- // Walk down the tree looking for a discrepancy
- while ( ap[i] === bp[i] ) {
- i++;
- }
-
- return i ?
- // Do a sibling check if the nodes have a common ancestor
- siblingCheck( ap[i], bp[i] ) :
-
- // Otherwise nodes in our document sort first
- ap[i] === preferredDoc ? -1 :
- bp[i] === preferredDoc ? 1 :
- 0;
- };
-
- return document;
- };
-
- Sizzle.matches = function( expr, elements ) {
- return Sizzle( expr, null, null, elements );
- };
-
- Sizzle.matchesSelector = function( elem, expr ) {
- // Set document vars if needed
- if ( ( elem.ownerDocument || elem ) !== document ) {
- setDocument( elem );
- }
-
- // Make sure that attribute selectors are quoted
- expr = expr.replace( rattributeQuotes, "='$1']" );
-
- if ( support.matchesSelector && documentIsHTML &&
- !compilerCache[ expr + " " ] &&
- ( !rbuggyMatches || !rbuggyMatches.test( expr ) ) &&
- ( !rbuggyQSA || !rbuggyQSA.test( expr ) ) ) {
-
- try {
- var ret = matches.call( elem, expr );
-
- // IE 9's matchesSelector returns false on disconnected nodes
- if ( ret || support.disconnectedMatch ||
- // As well, disconnected nodes are said to be in a document
- // fragment in IE 9
- elem.document && elem.document.nodeType !== 11 ) {
- return ret;
- }
- } catch (e) {}
- }
-
- return Sizzle( expr, document, null, [ elem ] ).length > 0;
- };
-
- Sizzle.contains = function( context, elem ) {
- // Set document vars if needed
- if ( ( context.ownerDocument || context ) !== document ) {
- setDocument( context );
- }
- return contains( context, elem );
- };
-
- Sizzle.attr = function( elem, name ) {
- // Set document vars if needed
- if ( ( elem.ownerDocument || elem ) !== document ) {
- setDocument( elem );
- }
-
- var fn = Expr.attrHandle[ name.toLowerCase() ],
- // Don't get fooled by Object.prototype properties (jQuery #13807)
- val = fn && hasOwn.call( Expr.attrHandle, name.toLowerCase() ) ?
- fn( elem, name, !documentIsHTML ) :
- undefined;
-
- return val !== undefined ?
- val :
- support.attributes || !documentIsHTML ?
- elem.getAttribute( name ) :
- (val = elem.getAttributeNode(name)) && val.specified ?
- val.value :
- null;
- };
-
- Sizzle.error = function( msg ) {
- throw new Error( "Syntax error, unrecognized expression: " + msg );
- };
-
- /**
- * Document sorting and removing duplicates
- * @param {ArrayLike} results
- */
- Sizzle.uniqueSort = function( results ) {
- var elem,
- duplicates = [],
- j = 0,
- i = 0;
-
- // Unless we *know* we can detect duplicates, assume their presence
- hasDuplicate = !support.detectDuplicates;
- sortInput = !support.sortStable && results.slice( 0 );
- results.sort( sortOrder );
-
- if ( hasDuplicate ) {
- while ( (elem = results[i++]) ) {
- if ( elem === results[ i ] ) {
- j = duplicates.push( i );
- }
- }
- while ( j-- ) {
- results.splice( duplicates[ j ], 1 );
- }
- }
-
- // Clear input after sorting to release objects
- // See https://github.com/jquery/sizzle/pull/225
- sortInput = null;
-
- return results;
- };
-
- /**
- * Utility function for retrieving the text value of an array of DOM nodes
- * @param {Array|Element} elem
- */
- getText = Sizzle.getText = function( elem ) {
- var node,
- ret = "",
- i = 0,
- nodeType = elem.nodeType;
-
- if ( !nodeType ) {
- // If no nodeType, this is expected to be an array
- while ( (node = elem[i++]) ) {
- // Do not traverse comment nodes
- ret += getText( node );
- }
- } else if ( nodeType === 1 || nodeType === 9 || nodeType === 11 ) {
- // Use textContent for elements
- // innerText usage removed for consistency of new lines (jQuery #11153)
- if ( typeof elem.textContent === "string" ) {
- return elem.textContent;
- } else {
- // Traverse its children
- for ( elem = elem.firstChild; elem; elem = elem.nextSibling ) {
- ret += getText( elem );
- }
- }
- } else if ( nodeType === 3 || nodeType === 4 ) {
- return elem.nodeValue;
- }
- // Do not include comment or processing instruction nodes
-
- return ret;
- };
-
- Expr = Sizzle.selectors = {
-
- // Can be adjusted by the user
- cacheLength: 50,
-
- createPseudo: markFunction,
-
- match: matchExpr,
-
- attrHandle: {},
-
- find: {},
-
- relative: {
- ">": { dir: "parentNode", first: true },
- " ": { dir: "parentNode" },
- "+": { dir: "previousSibling", first: true },
- "~": { dir: "previousSibling" }
- },
-
- preFilter: {
- "ATTR": function( match ) {
- match[1] = match[1].replace( runescape, funescape );
-
- // Move the given value to match[3] whether quoted or unquoted
- match[3] = ( match[3] || match[4] || match[5] || "" ).replace( runescape, funescape );
-
- if ( match[2] === "~=" ) {
- match[3] = " " + match[3] + " ";
- }
-
- return match.slice( 0, 4 );
- },
-
- "CHILD": function( match ) {
- /* matches from matchExpr["CHILD"]
- 1 type (only|nth|...)
- 2 what (child|of-type)
- 3 argument (even|odd|\d*|\d*n([+-]\d+)?|...)
- 4 xn-component of xn+y argument ([+-]?\d*n|)
- 5 sign of xn-component
- 6 x of xn-component
- 7 sign of y-component
- 8 y of y-component
- */
- match[1] = match[1].toLowerCase();
-
- if ( match[1].slice( 0, 3 ) === "nth" ) {
- // nth-* requires argument
- if ( !match[3] ) {
- Sizzle.error( match[0] );
- }
-
- // numeric x and y parameters for Expr.filter.CHILD
- // remember that false/true cast respectively to 0/1
- match[4] = +( match[4] ? match[5] + (match[6] || 1) : 2 * ( match[3] === "even" || match[3] === "odd" ) );
- match[5] = +( ( match[7] + match[8] ) || match[3] === "odd" );
-
- // other types prohibit arguments
- } else if ( match[3] ) {
- Sizzle.error( match[0] );
- }
-
- return match;
- },
-
- "PSEUDO": function( match ) {
- var excess,
- unquoted = !match[6] && match[2];
-
- if ( matchExpr["CHILD"].test( match[0] ) ) {
- return null;
- }
-
- // Accept quoted arguments as-is
- if ( match[3] ) {
- match[2] = match[4] || match[5] || "";
-
- // Strip excess characters from unquoted arguments
- } else if ( unquoted && rpseudo.test( unquoted ) &&
- // Get excess from tokenize (recursively)
- (excess = tokenize( unquoted, true )) &&
- // advance to the next closing parenthesis
- (excess = unquoted.indexOf( ")", unquoted.length - excess ) - unquoted.length) ) {
-
- // excess is a negative index
- match[0] = match[0].slice( 0, excess );
- match[2] = unquoted.slice( 0, excess );
- }
-
- // Return only captures needed by the pseudo filter method (type and argument)
- return match.slice( 0, 3 );
- }
- },
-
- filter: {
-
- "TAG": function( nodeNameSelector ) {
- var nodeName = nodeNameSelector.replace( runescape, funescape ).toLowerCase();
- return nodeNameSelector === "*" ?
- function() { return true; } :
- function( elem ) {
- return elem.nodeName && elem.nodeName.toLowerCase() === nodeName;
- };
- },
-
- "CLASS": function( className ) {
- var pattern = classCache[ className + " " ];
-
- return pattern ||
- (pattern = new RegExp( "(^|" + whitespace + ")" + className + "(" + whitespace + "|$)" )) &&
- classCache( className, function( elem ) {
- return pattern.test( typeof elem.className === "string" && elem.className || typeof elem.getAttribute !== "undefined" && elem.getAttribute("class") || "" );
- });
- },
-
- "ATTR": function( name, operator, check ) {
- return function( elem ) {
- var result = Sizzle.attr( elem, name );
-
- if ( result == null ) {
- return operator === "!=";
- }
- if ( !operator ) {
- return true;
- }
-
- result += "";
-
- return operator === "=" ? result === check :
- operator === "!=" ? result !== check :
- operator === "^=" ? check && result.indexOf( check ) === 0 :
- operator === "*=" ? check && result.indexOf( check ) > -1 :
- operator === "$=" ? check && result.slice( -check.length ) === check :
- operator === "~=" ? ( " " + result.replace( rwhitespace, " " ) + " " ).indexOf( check ) > -1 :
- operator === "|=" ? result === check || result.slice( 0, check.length + 1 ) === check + "-" :
- false;
- };
- },
-
- "CHILD": function( type, what, argument, first, last ) {
- var simple = type.slice( 0, 3 ) !== "nth",
- forward = type.slice( -4 ) !== "last",
- ofType = what === "of-type";
-
- return first === 1 && last === 0 ?
-
- // Shortcut for :nth-*(n)
- function( elem ) {
- return !!elem.parentNode;
- } :
-
- function( elem, context, xml ) {
- var cache, uniqueCache, outerCache, node, nodeIndex, start,
- dir = simple !== forward ? "nextSibling" : "previousSibling",
- parent = elem.parentNode,
- name = ofType && elem.nodeName.toLowerCase(),
- useCache = !xml && !ofType,
- diff = false;
-
- if ( parent ) {
-
- // :(first|last|only)-(child|of-type)
- if ( simple ) {
- while ( dir ) {
- node = elem;
- while ( (node = node[ dir ]) ) {
- if ( ofType ?
- node.nodeName.toLowerCase() === name :
- node.nodeType === 1 ) {
-
- return false;
- }
- }
- // Reverse direction for :only-* (if we haven't yet done so)
- start = dir = type === "only" && !start && "nextSibling";
- }
- return true;
- }
-
- start = [ forward ? parent.firstChild : parent.lastChild ];
-
- // non-xml :nth-child(...) stores cache data on `parent`
- if ( forward && useCache ) {
-
- // Seek `elem` from a previously-cached index
-
- // ...in a gzip-friendly way
- node = parent;
- outerCache = node[ expando ] || (node[ expando ] = {});
-
- // Support: IE <9 only
- // Defend against cloned attroperties (jQuery gh-1709)
- uniqueCache = outerCache[ node.uniqueID ] ||
- (outerCache[ node.uniqueID ] = {});
-
- cache = uniqueCache[ type ] || [];
- nodeIndex = cache[ 0 ] === dirruns && cache[ 1 ];
- diff = nodeIndex && cache[ 2 ];
- node = nodeIndex && parent.childNodes[ nodeIndex ];
-
- while ( (node = ++nodeIndex && node && node[ dir ] ||
-
- // Fallback to seeking `elem` from the start
- (diff = nodeIndex = 0) || start.pop()) ) {
-
- // When found, cache indexes on `parent` and break
- if ( node.nodeType === 1 && ++diff && node === elem ) {
- uniqueCache[ type ] = [ dirruns, nodeIndex, diff ];
- break;
- }
- }
-
- } else {
- // Use previously-cached element index if available
- if ( useCache ) {
- // ...in a gzip-friendly way
- node = elem;
- outerCache = node[ expando ] || (node[ expando ] = {});
-
- // Support: IE <9 only
- // Defend against cloned attroperties (jQuery gh-1709)
- uniqueCache = outerCache[ node.uniqueID ] ||
- (outerCache[ node.uniqueID ] = {});
-
- cache = uniqueCache[ type ] || [];
- nodeIndex = cache[ 0 ] === dirruns && cache[ 1 ];
- diff = nodeIndex;
- }
-
- // xml :nth-child(...)
- // or :nth-last-child(...) or :nth(-last)?-of-type(...)
- if ( diff === false ) {
- // Use the same loop as above to seek `elem` from the start
- while ( (node = ++nodeIndex && node && node[ dir ] ||
- (diff = nodeIndex = 0) || start.pop()) ) {
-
- if ( ( ofType ?
- node.nodeName.toLowerCase() === name :
- node.nodeType === 1 ) &&
- ++diff ) {
-
- // Cache the index of each encountered element
- if ( useCache ) {
- outerCache = node[ expando ] || (node[ expando ] = {});
-
- // Support: IE <9 only
- // Defend against cloned attroperties (jQuery gh-1709)
- uniqueCache = outerCache[ node.uniqueID ] ||
- (outerCache[ node.uniqueID ] = {});
-
- uniqueCache[ type ] = [ dirruns, diff ];
- }
-
- if ( node === elem ) {
- break;
- }
- }
- }
- }
- }
-
- // Incorporate the offset, then check against cycle size
- diff -= last;
- return diff === first || ( diff % first === 0 && diff / first >= 0 );
- }
- };
- },
-
- "PSEUDO": function( pseudo, argument ) {
- // pseudo-class names are case-insensitive
- // http://www.w3.org/TR/selectors/#pseudo-classes
- // Prioritize by case sensitivity in case custom pseudos are added with uppercase letters
- // Remember that setFilters inherits from pseudos
- var args,
- fn = Expr.pseudos[ pseudo ] || Expr.setFilters[ pseudo.toLowerCase() ] ||
- Sizzle.error( "unsupported pseudo: " + pseudo );
-
- // The user may use createPseudo to indicate that
- // arguments are needed to create the filter function
- // just as Sizzle does
- if ( fn[ expando ] ) {
- return fn( argument );
- }
-
- // But maintain support for old signatures
- if ( fn.length > 1 ) {
- args = [ pseudo, pseudo, "", argument ];
- return Expr.setFilters.hasOwnProperty( pseudo.toLowerCase() ) ?
- markFunction(function( seed, matches ) {
- var idx,
- matched = fn( seed, argument ),
- i = matched.length;
- while ( i-- ) {
- idx = indexOf( seed, matched[i] );
- seed[ idx ] = !( matches[ idx ] = matched[i] );
- }
- }) :
- function( elem ) {
- return fn( elem, 0, args );
- };
- }
-
- return fn;
- }
- },
-
- pseudos: {
- // Potentially complex pseudos
- "not": markFunction(function( selector ) {
- // Trim the selector passed to compile
- // to avoid treating leading and trailing
- // spaces as combinators
- var input = [],
- results = [],
- matcher = compile( selector.replace( rtrim, "$1" ) );
-
- return matcher[ expando ] ?
- markFunction(function( seed, matches, context, xml ) {
- var elem,
- unmatched = matcher( seed, null, xml, [] ),
- i = seed.length;
-
- // Match elements unmatched by `matcher`
- while ( i-- ) {
- if ( (elem = unmatched[i]) ) {
- seed[i] = !(matches[i] = elem);
- }
- }
- }) :
- function( elem, context, xml ) {
- input[0] = elem;
- matcher( input, null, xml, results );
- // Don't keep the element (issue #299)
- input[0] = null;
- return !results.pop();
- };
- }),
-
- "has": markFunction(function( selector ) {
- return function( elem ) {
- return Sizzle( selector, elem ).length > 0;
- };
- }),
-
- "contains": markFunction(function( text ) {
- text = text.replace( runescape, funescape );
- return function( elem ) {
- return ( elem.textContent || elem.innerText || getText( elem ) ).indexOf( text ) > -1;
- };
- }),
-
- // "Whether an element is represented by a :lang() selector
- // is based solely on the element's language value
- // being equal to the identifier C,
- // or beginning with the identifier C immediately followed by "-".
- // The matching of C against the element's language value is performed case-insensitively.
- // The identifier C does not have to be a valid language name."
- // http://www.w3.org/TR/selectors/#lang-pseudo
- "lang": markFunction( function( lang ) {
- // lang value must be a valid identifier
- if ( !ridentifier.test(lang || "") ) {
- Sizzle.error( "unsupported lang: " + lang );
- }
- lang = lang.replace( runescape, funescape ).toLowerCase();
- return function( elem ) {
- var elemLang;
- do {
- if ( (elemLang = documentIsHTML ?
- elem.lang :
- elem.getAttribute("xml:lang") || elem.getAttribute("lang")) ) {
-
- elemLang = elemLang.toLowerCase();
- return elemLang === lang || elemLang.indexOf( lang + "-" ) === 0;
- }
- } while ( (elem = elem.parentNode) && elem.nodeType === 1 );
- return false;
- };
- }),
-
- // Miscellaneous
- "target": function( elem ) {
- var hash = window.location && window.location.hash;
- return hash && hash.slice( 1 ) === elem.id;
- },
-
- "root": function( elem ) {
- return elem === docElem;
- },
-
- "focus": function( elem ) {
- return elem === document.activeElement && (!document.hasFocus || document.hasFocus()) && !!(elem.type || elem.href || ~elem.tabIndex);
- },
-
- // Boolean properties
- "enabled": function( elem ) {
- return elem.disabled === false;
- },
-
- "disabled": function( elem ) {
- return elem.disabled === true;
- },
-
- "checked": function( elem ) {
- // In CSS3, :checked should return both checked and selected elements
- // http://www.w3.org/TR/2011/REC-css3-selectors-20110929/#checked
- var nodeName = elem.nodeName.toLowerCase();
- return (nodeName === "input" && !!elem.checked) || (nodeName === "option" && !!elem.selected);
- },
-
- "selected": function( elem ) {
- // Accessing this property makes selected-by-default
- // options in Safari work properly
- if ( elem.parentNode ) {
- elem.parentNode.selectedIndex;
- }
-
- return elem.selected === true;
- },
-
- // Contents
- "empty": function( elem ) {
- // http://www.w3.org/TR/selectors/#empty-pseudo
- // :empty is negated by element (1) or content nodes (text: 3; cdata: 4; entity ref: 5),
- // but not by others (comment: 8; processing instruction: 7; etc.)
- // nodeType < 6 works because attributes (2) do not appear as children
- for ( elem = elem.firstChild; elem; elem = elem.nextSibling ) {
- if ( elem.nodeType < 6 ) {
- return false;
- }
- }
- return true;
- },
-
- "parent": function( elem ) {
- return !Expr.pseudos["empty"]( elem );
- },
-
- // Element/input types
- "header": function( elem ) {
- return rheader.test( elem.nodeName );
- },
-
- "input": function( elem ) {
- return rinputs.test( elem.nodeName );
- },
-
- "button": function( elem ) {
- var name = elem.nodeName.toLowerCase();
- return name === "input" && elem.type === "button" || name === "button";
- },
-
- "text": function( elem ) {
- var attr;
- return elem.nodeName.toLowerCase() === "input" &&
- elem.type === "text" &&
-
- // Support: IE<8
- // New HTML5 attribute values (e.g., "search") appear with elem.type === "text"
- ( (attr = elem.getAttribute("type")) == null || attr.toLowerCase() === "text" );
- },
-
- // Position-in-collection
- "first": createPositionalPseudo(function() {
- return [ 0 ];
- }),
-
- "last": createPositionalPseudo(function( matchIndexes, length ) {
- return [ length - 1 ];
- }),
-
- "eq": createPositionalPseudo(function( matchIndexes, length, argument ) {
- return [ argument < 0 ? argument + length : argument ];
- }),
-
- "even": createPositionalPseudo(function( matchIndexes, length ) {
- var i = 0;
- for ( ; i < length; i += 2 ) {
- matchIndexes.push( i );
- }
- return matchIndexes;
- }),
-
- "odd": createPositionalPseudo(function( matchIndexes, length ) {
- var i = 1;
- for ( ; i < length; i += 2 ) {
- matchIndexes.push( i );
- }
- return matchIndexes;
- }),
-
- "lt": createPositionalPseudo(function( matchIndexes, length, argument ) {
- var i = argument < 0 ? argument + length : argument;
- for ( ; --i >= 0; ) {
- matchIndexes.push( i );
- }
- return matchIndexes;
- }),
-
- "gt": createPositionalPseudo(function( matchIndexes, length, argument ) {
- var i = argument < 0 ? argument + length : argument;
- for ( ; ++i < length; ) {
- matchIndexes.push( i );
- }
- return matchIndexes;
- })
- }
- };
-
- Expr.pseudos["nth"] = Expr.pseudos["eq"];
+support = Sizzle.support = {};
+
+/**
+ * Detects XML nodes
+ * @param {Element|Object} elem An element or a document
+ * @returns {Boolean} True iff elem is a non-HTML XML node
+ */
+isXML = Sizzle.isXML = function( elem ) {
+ var namespace = elem && elem.namespaceURI,
+ docElem = elem && ( elem.ownerDocument || elem ).documentElement;
+
+ // Support: IE <=8
+ // Assume HTML when documentElement doesn't yet exist, such as inside loading iframes
+ // https://bugs.jquery.com/ticket/4833
+ return !rhtml.test( namespace || docElem && docElem.nodeName || "HTML" );
+};
+
+/**
+ * Sets document-related variables once based on the current document
+ * @param {Element|Object} [doc] An element or document object to use to set the document
+ * @returns {Object} Returns the current document
+ */
+setDocument = Sizzle.setDocument = function( node ) {
+ var hasCompare, subWindow,
+ doc = node ? node.ownerDocument || node : preferredDoc;
+
+ // Return early if doc is invalid or already selected
+ // Support: IE 11+, Edge 17 - 18+
+ // IE/Edge sometimes throw a "Permission denied" error when strict-comparing
+ // two documents; shallow comparisons work.
+ // eslint-disable-next-line eqeqeq
+ if ( doc == document || doc.nodeType !== 9 || !doc.documentElement ) {
+ return document;
+ }
+
+ // Update global variables
+ document = doc;
+ docElem = document.documentElement;
+ documentIsHTML = !isXML( document );
+
+ // Support: IE 9 - 11+, Edge 12 - 18+
+ // Accessing iframe documents after unload throws "permission denied" errors (jQuery #13936)
+ // Support: IE 11+, Edge 17 - 18+
+ // IE/Edge sometimes throw a "Permission denied" error when strict-comparing
+ // two documents; shallow comparisons work.
+ // eslint-disable-next-line eqeqeq
+ if ( preferredDoc != document &&
+ ( subWindow = document.defaultView ) && subWindow.top !== subWindow ) {
+
+ // Support: IE 11, Edge
+ if ( subWindow.addEventListener ) {
+ subWindow.addEventListener( "unload", unloadHandler, false );
+
+ // Support: IE 9 - 10 only
+ } else if ( subWindow.attachEvent ) {
+ subWindow.attachEvent( "onunload", unloadHandler );
+ }
+ }
+
+ // Support: IE 8 - 11+, Edge 12 - 18+, Chrome <=16 - 25 only, Firefox <=3.6 - 31 only,
+ // Safari 4 - 5 only, Opera <=11.6 - 12.x only
+ // IE/Edge & older browsers don't support the :scope pseudo-class.
+ // Support: Safari 6.0 only
+ // Safari 6.0 supports :scope but it's an alias of :root there.
+ support.scope = assert( function( el ) {
+ docElem.appendChild( el ).appendChild( document.createElement( "div" ) );
+ return typeof el.querySelectorAll !== "undefined" &&
+ !el.querySelectorAll( ":scope fieldset div" ).length;
+ } );
+
+ /* Attributes
+ ---------------------------------------------------------------------- */
+
+ // Support: IE<8
+ // Verify that getAttribute really returns attributes and not properties
+ // (excepting IE8 booleans)
+ support.attributes = assert( function( el ) {
+ el.className = "i";
+ return !el.getAttribute( "className" );
+ } );
+
+ /* getElement(s)By*
+ ---------------------------------------------------------------------- */
+
+ // Check if getElementsByTagName("*") returns only elements
+ support.getElementsByTagName = assert( function( el ) {
+ el.appendChild( document.createComment( "" ) );
+ return !el.getElementsByTagName( "*" ).length;
+ } );
+
+ // Support: IE<9
+ support.getElementsByClassName = rnative.test( document.getElementsByClassName );
+
+ // Support: IE<10
+ // Check if getElementById returns elements by name
+ // The broken getElementById methods don't pick up programmatically-set names,
+ // so use a roundabout getElementsByName test
+ support.getById = assert( function( el ) {
+ docElem.appendChild( el ).id = expando;
+ return !document.getElementsByName || !document.getElementsByName( expando ).length;
+ } );
+
+ // ID filter and find
+ if ( support.getById ) {
+ Expr.filter[ "ID" ] = function( id ) {
+ var attrId = id.replace( runescape, funescape );
+ return function( elem ) {
+ return elem.getAttribute( "id" ) === attrId;
+ };
+ };
+ Expr.find[ "ID" ] = function( id, context ) {
+ if ( typeof context.getElementById !== "undefined" && documentIsHTML ) {
+ var elem = context.getElementById( id );
+ return elem ? [ elem ] : [];
+ }
+ };
+ } else {
+ Expr.filter[ "ID" ] = function( id ) {
+ var attrId = id.replace( runescape, funescape );
+ return function( elem ) {
+ var node = typeof elem.getAttributeNode !== "undefined" &&
+ elem.getAttributeNode( "id" );
+ return node && node.value === attrId;
+ };
+ };
+
+ // Support: IE 6 - 7 only
+ // getElementById is not reliable as a find shortcut
+ Expr.find[ "ID" ] = function( id, context ) {
+ if ( typeof context.getElementById !== "undefined" && documentIsHTML ) {
+ var node, i, elems,
+ elem = context.getElementById( id );
+
+ if ( elem ) {
+
+ // Verify the id attribute
+ node = elem.getAttributeNode( "id" );
+ if ( node && node.value === id ) {
+ return [ elem ];
+ }
+
+ // Fall back on getElementsByName
+ elems = context.getElementsByName( id );
+ i = 0;
+ while ( ( elem = elems[ i++ ] ) ) {
+ node = elem.getAttributeNode( "id" );
+ if ( node && node.value === id ) {
+ return [ elem ];
+ }
+ }
+ }
+
+ return [];
+ }
+ };
+ }
+
+ // Tag
+ Expr.find[ "TAG" ] = support.getElementsByTagName ?
+ function( tag, context ) {
+ if ( typeof context.getElementsByTagName !== "undefined" ) {
+ return context.getElementsByTagName( tag );
+
+ // DocumentFragment nodes don't have gEBTN
+ } else if ( support.qsa ) {
+ return context.querySelectorAll( tag );
+ }
+ } :
+
+ function( tag, context ) {
+ var elem,
+ tmp = [],
+ i = 0,
+
+ // By happy coincidence, a (broken) gEBTN appears on DocumentFragment nodes too
+ results = context.getElementsByTagName( tag );
+
+ // Filter out possible comments
+ if ( tag === "*" ) {
+ while ( ( elem = results[ i++ ] ) ) {
+ if ( elem.nodeType === 1 ) {
+ tmp.push( elem );
+ }
+ }
+
+ return tmp;
+ }
+ return results;
+ };
+
+ // Class
+ Expr.find[ "CLASS" ] = support.getElementsByClassName && function( className, context ) {
+ if ( typeof context.getElementsByClassName !== "undefined" && documentIsHTML ) {
+ return context.getElementsByClassName( className );
+ }
+ };
+
+ /* QSA/matchesSelector
+ ---------------------------------------------------------------------- */
+
+ // QSA and matchesSelector support
+
+ // matchesSelector(:active) reports false when true (IE9/Opera 11.5)
+ rbuggyMatches = [];
+
+ // qSa(:focus) reports false when true (Chrome 21)
+ // We allow this because of a bug in IE8/9 that throws an error
+ // whenever `document.activeElement` is accessed on an iframe
+ // So, we allow :focus to pass through QSA all the time to avoid the IE error
+ // See https://bugs.jquery.com/ticket/13378
+ rbuggyQSA = [];
+
+ if ( ( support.qsa = rnative.test( document.querySelectorAll ) ) ) {
+
+ // Build QSA regex
+ // Regex strategy adopted from Diego Perini
+ assert( function( el ) {
+
+ var input;
+
+ // Select is set to empty string on purpose
+ // This is to test IE's treatment of not explicitly
+ // setting a boolean content attribute,
+ // since its presence should be enough
+ // https://bugs.jquery.com/ticket/12359
+ docElem.appendChild( el ).innerHTML = "" +
+ "";
+
+ // Support: IE8, Opera 11-12.16
+ // Nothing should be selected when empty strings follow ^= or $= or *=
+ // The test attribute must be unknown in Opera but "safe" for WinRT
+ // https://msdn.microsoft.com/en-us/library/ie/hh465388.aspx#attribute_section
+ if ( el.querySelectorAll( "[msallowcapture^='']" ).length ) {
+ rbuggyQSA.push( "[*^$]=" + whitespace + "*(?:''|\"\")" );
+ }
+
+ // Support: IE8
+ // Boolean attributes and "value" are not treated correctly
+ if ( !el.querySelectorAll( "[selected]" ).length ) {
+ rbuggyQSA.push( "\\[" + whitespace + "*(?:value|" + booleans + ")" );
+ }
+
+ // Support: Chrome<29, Android<4.4, Safari<7.0+, iOS<7.0+, PhantomJS<1.9.8+
+ if ( !el.querySelectorAll( "[id~=" + expando + "-]" ).length ) {
+ rbuggyQSA.push( "~=" );
+ }
+
+ // Support: IE 11+, Edge 15 - 18+
+ // IE 11/Edge don't find elements on a `[name='']` query in some cases.
+ // Adding a temporary attribute to the document before the selection works
+ // around the issue.
+ // Interestingly, IE 10 & older don't seem to have the issue.
+ input = document.createElement( "input" );
+ input.setAttribute( "name", "" );
+ el.appendChild( input );
+ if ( !el.querySelectorAll( "[name='']" ).length ) {
+ rbuggyQSA.push( "\\[" + whitespace + "*name" + whitespace + "*=" +
+ whitespace + "*(?:''|\"\")" );
+ }
+
+ // Webkit/Opera - :checked should return selected option elements
+ // http://www.w3.org/TR/2011/REC-css3-selectors-20110929/#checked
+ // IE8 throws error here and will not see later tests
+ if ( !el.querySelectorAll( ":checked" ).length ) {
+ rbuggyQSA.push( ":checked" );
+ }
+
+ // Support: Safari 8+, iOS 8+
+ // https://bugs.webkit.org/show_bug.cgi?id=136851
+ // In-page `selector#id sibling-combinator selector` fails
+ if ( !el.querySelectorAll( "a#" + expando + "+*" ).length ) {
+ rbuggyQSA.push( ".#.+[+~]" );
+ }
+
+ // Support: Firefox <=3.6 - 5 only
+ // Old Firefox doesn't throw on a badly-escaped identifier.
+ el.querySelectorAll( "\\\f" );
+ rbuggyQSA.push( "[\\r\\n\\f]" );
+ } );
+
+ assert( function( el ) {
+ el.innerHTML = "" +
+ "";
+
+ // Support: Windows 8 Native Apps
+ // The type and name attributes are restricted during .innerHTML assignment
+ var input = document.createElement( "input" );
+ input.setAttribute( "type", "hidden" );
+ el.appendChild( input ).setAttribute( "name", "D" );
+
+ // Support: IE8
+ // Enforce case-sensitivity of name attribute
+ if ( el.querySelectorAll( "[name=d]" ).length ) {
+ rbuggyQSA.push( "name" + whitespace + "*[*^$|!~]?=" );
+ }
+
+ // FF 3.5 - :enabled/:disabled and hidden elements (hidden elements are still enabled)
+ // IE8 throws error here and will not see later tests
+ if ( el.querySelectorAll( ":enabled" ).length !== 2 ) {
+ rbuggyQSA.push( ":enabled", ":disabled" );
+ }
+
+ // Support: IE9-11+
+ // IE's :disabled selector does not pick up the children of disabled fieldsets
+ docElem.appendChild( el ).disabled = true;
+ if ( el.querySelectorAll( ":disabled" ).length !== 2 ) {
+ rbuggyQSA.push( ":enabled", ":disabled" );
+ }
+
+ // Support: Opera 10 - 11 only
+ // Opera 10-11 does not throw on post-comma invalid pseudos
+ el.querySelectorAll( "*,:x" );
+ rbuggyQSA.push( ",.*:" );
+ } );
+ }
+
+ if ( ( support.matchesSelector = rnative.test( ( matches = docElem.matches ||
+ docElem.webkitMatchesSelector ||
+ docElem.mozMatchesSelector ||
+ docElem.oMatchesSelector ||
+ docElem.msMatchesSelector ) ) ) ) {
+
+ assert( function( el ) {
+
+ // Check to see if it's possible to do matchesSelector
+ // on a disconnected node (IE 9)
+ support.disconnectedMatch = matches.call( el, "*" );
+
+ // This should fail with an exception
+ // Gecko does not error, returns false instead
+ matches.call( el, "[s!='']:x" );
+ rbuggyMatches.push( "!=", pseudos );
+ } );
+ }
+
+ rbuggyQSA = rbuggyQSA.length && new RegExp( rbuggyQSA.join( "|" ) );
+ rbuggyMatches = rbuggyMatches.length && new RegExp( rbuggyMatches.join( "|" ) );
+
+ /* Contains
+ ---------------------------------------------------------------------- */
+ hasCompare = rnative.test( docElem.compareDocumentPosition );
+
+ // Element contains another
+ // Purposefully self-exclusive
+ // As in, an element does not contain itself
+ contains = hasCompare || rnative.test( docElem.contains ) ?
+ function( a, b ) {
+ var adown = a.nodeType === 9 ? a.documentElement : a,
+ bup = b && b.parentNode;
+ return a === bup || !!( bup && bup.nodeType === 1 && (
+ adown.contains ?
+ adown.contains( bup ) :
+ a.compareDocumentPosition && a.compareDocumentPosition( bup ) & 16
+ ) );
+ } :
+ function( a, b ) {
+ if ( b ) {
+ while ( ( b = b.parentNode ) ) {
+ if ( b === a ) {
+ return true;
+ }
+ }
+ }
+ return false;
+ };
+
+ /* Sorting
+ ---------------------------------------------------------------------- */
+
+ // Document order sorting
+ sortOrder = hasCompare ?
+ function( a, b ) {
+
+ // Flag for duplicate removal
+ if ( a === b ) {
+ hasDuplicate = true;
+ return 0;
+ }
+
+ // Sort on method existence if only one input has compareDocumentPosition
+ var compare = !a.compareDocumentPosition - !b.compareDocumentPosition;
+ if ( compare ) {
+ return compare;
+ }
+
+ // Calculate position if both inputs belong to the same document
+ // Support: IE 11+, Edge 17 - 18+
+ // IE/Edge sometimes throw a "Permission denied" error when strict-comparing
+ // two documents; shallow comparisons work.
+ // eslint-disable-next-line eqeqeq
+ compare = ( a.ownerDocument || a ) == ( b.ownerDocument || b ) ?
+ a.compareDocumentPosition( b ) :
+
+ // Otherwise we know they are disconnected
+ 1;
+
+ // Disconnected nodes
+ if ( compare & 1 ||
+ ( !support.sortDetached && b.compareDocumentPosition( a ) === compare ) ) {
+
+ // Choose the first element that is related to our preferred document
+ // Support: IE 11+, Edge 17 - 18+
+ // IE/Edge sometimes throw a "Permission denied" error when strict-comparing
+ // two documents; shallow comparisons work.
+ // eslint-disable-next-line eqeqeq
+ if ( a == document || a.ownerDocument == preferredDoc &&
+ contains( preferredDoc, a ) ) {
+ return -1;
+ }
+
+ // Support: IE 11+, Edge 17 - 18+
+ // IE/Edge sometimes throw a "Permission denied" error when strict-comparing
+ // two documents; shallow comparisons work.
+ // eslint-disable-next-line eqeqeq
+ if ( b == document || b.ownerDocument == preferredDoc &&
+ contains( preferredDoc, b ) ) {
+ return 1;
+ }
+
+ // Maintain original order
+ return sortInput ?
+ ( indexOf( sortInput, a ) - indexOf( sortInput, b ) ) :
+ 0;
+ }
+
+ return compare & 4 ? -1 : 1;
+ } :
+ function( a, b ) {
+
+ // Exit early if the nodes are identical
+ if ( a === b ) {
+ hasDuplicate = true;
+ return 0;
+ }
+
+ var cur,
+ i = 0,
+ aup = a.parentNode,
+ bup = b.parentNode,
+ ap = [ a ],
+ bp = [ b ];
+
+ // Parentless nodes are either documents or disconnected
+ if ( !aup || !bup ) {
+
+ // Support: IE 11+, Edge 17 - 18+
+ // IE/Edge sometimes throw a "Permission denied" error when strict-comparing
+ // two documents; shallow comparisons work.
+ /* eslint-disable eqeqeq */
+ return a == document ? -1 :
+ b == document ? 1 :
+ /* eslint-enable eqeqeq */
+ aup ? -1 :
+ bup ? 1 :
+ sortInput ?
+ ( indexOf( sortInput, a ) - indexOf( sortInput, b ) ) :
+ 0;
+
+ // If the nodes are siblings, we can do a quick check
+ } else if ( aup === bup ) {
+ return siblingCheck( a, b );
+ }
+
+ // Otherwise we need full lists of their ancestors for comparison
+ cur = a;
+ while ( ( cur = cur.parentNode ) ) {
+ ap.unshift( cur );
+ }
+ cur = b;
+ while ( ( cur = cur.parentNode ) ) {
+ bp.unshift( cur );
+ }
+
+ // Walk down the tree looking for a discrepancy
+ while ( ap[ i ] === bp[ i ] ) {
+ i++;
+ }
+
+ return i ?
+
+ // Do a sibling check if the nodes have a common ancestor
+ siblingCheck( ap[ i ], bp[ i ] ) :
+
+ // Otherwise nodes in our document sort first
+ // Support: IE 11+, Edge 17 - 18+
+ // IE/Edge sometimes throw a "Permission denied" error when strict-comparing
+ // two documents; shallow comparisons work.
+ /* eslint-disable eqeqeq */
+ ap[ i ] == preferredDoc ? -1 :
+ bp[ i ] == preferredDoc ? 1 :
+ /* eslint-enable eqeqeq */
+ 0;
+ };
+
+ return document;
+};
+
+Sizzle.matches = function( expr, elements ) {
+ return Sizzle( expr, null, null, elements );
+};
+
+Sizzle.matchesSelector = function( elem, expr ) {
+ setDocument( elem );
+
+ if ( support.matchesSelector && documentIsHTML &&
+ !nonnativeSelectorCache[ expr + " " ] &&
+ ( !rbuggyMatches || !rbuggyMatches.test( expr ) ) &&
+ ( !rbuggyQSA || !rbuggyQSA.test( expr ) ) ) {
+
+ try {
+ var ret = matches.call( elem, expr );
+
+ // IE 9's matchesSelector returns false on disconnected nodes
+ if ( ret || support.disconnectedMatch ||
+
+ // As well, disconnected nodes are said to be in a document
+ // fragment in IE 9
+ elem.document && elem.document.nodeType !== 11 ) {
+ return ret;
+ }
+ } catch ( e ) {
+ nonnativeSelectorCache( expr, true );
+ }
+ }
+
+ return Sizzle( expr, document, null, [ elem ] ).length > 0;
+};
+
+Sizzle.contains = function( context, elem ) {
+
+ // Set document vars if needed
+ // Support: IE 11+, Edge 17 - 18+
+ // IE/Edge sometimes throw a "Permission denied" error when strict-comparing
+ // two documents; shallow comparisons work.
+ // eslint-disable-next-line eqeqeq
+ if ( ( context.ownerDocument || context ) != document ) {
+ setDocument( context );
+ }
+ return contains( context, elem );
+};
+
+Sizzle.attr = function( elem, name ) {
+
+ // Set document vars if needed
+ // Support: IE 11+, Edge 17 - 18+
+ // IE/Edge sometimes throw a "Permission denied" error when strict-comparing
+ // two documents; shallow comparisons work.
+ // eslint-disable-next-line eqeqeq
+ if ( ( elem.ownerDocument || elem ) != document ) {
+ setDocument( elem );
+ }
+
+ var fn = Expr.attrHandle[ name.toLowerCase() ],
+
+ // Don't get fooled by Object.prototype properties (jQuery #13807)
+ val = fn && hasOwn.call( Expr.attrHandle, name.toLowerCase() ) ?
+ fn( elem, name, !documentIsHTML ) :
+ undefined;
+
+ return val !== undefined ?
+ val :
+ support.attributes || !documentIsHTML ?
+ elem.getAttribute( name ) :
+ ( val = elem.getAttributeNode( name ) ) && val.specified ?
+ val.value :
+ null;
+};
+
+Sizzle.escape = function( sel ) {
+ return ( sel + "" ).replace( rcssescape, fcssescape );
+};
+
+Sizzle.error = function( msg ) {
+ throw new Error( "Syntax error, unrecognized expression: " + msg );
+};
+
+/**
+ * Document sorting and removing duplicates
+ * @param {ArrayLike} results
+ */
+Sizzle.uniqueSort = function( results ) {
+ var elem,
+ duplicates = [],
+ j = 0,
+ i = 0;
+
+ // Unless we *know* we can detect duplicates, assume their presence
+ hasDuplicate = !support.detectDuplicates;
+ sortInput = !support.sortStable && results.slice( 0 );
+ results.sort( sortOrder );
+
+ if ( hasDuplicate ) {
+ while ( ( elem = results[ i++ ] ) ) {
+ if ( elem === results[ i ] ) {
+ j = duplicates.push( i );
+ }
+ }
+ while ( j-- ) {
+ results.splice( duplicates[ j ], 1 );
+ }
+ }
+
+ // Clear input after sorting to release objects
+ // See https://github.com/jquery/sizzle/pull/225
+ sortInput = null;
+
+ return results;
+};
+
+/**
+ * Utility function for retrieving the text value of an array of DOM nodes
+ * @param {Array|Element} elem
+ */
+getText = Sizzle.getText = function( elem ) {
+ var node,
+ ret = "",
+ i = 0,
+ nodeType = elem.nodeType;
+
+ if ( !nodeType ) {
+
+ // If no nodeType, this is expected to be an array
+ while ( ( node = elem[ i++ ] ) ) {
+
+ // Do not traverse comment nodes
+ ret += getText( node );
+ }
+ } else if ( nodeType === 1 || nodeType === 9 || nodeType === 11 ) {
+
+ // Use textContent for elements
+ // innerText usage removed for consistency of new lines (jQuery #11153)
+ if ( typeof elem.textContent === "string" ) {
+ return elem.textContent;
+ } else {
+
+ // Traverse its children
+ for ( elem = elem.firstChild; elem; elem = elem.nextSibling ) {
+ ret += getText( elem );
+ }
+ }
+ } else if ( nodeType === 3 || nodeType === 4 ) {
+ return elem.nodeValue;
+ }
+
+ // Do not include comment or processing instruction nodes
+
+ return ret;
+};
+
+Expr = Sizzle.selectors = {
+
+ // Can be adjusted by the user
+ cacheLength: 50,
+
+ createPseudo: markFunction,
+
+ match: matchExpr,
+
+ attrHandle: {},
+
+ find: {},
+
+ relative: {
+ ">": { dir: "parentNode", first: true },
+ " ": { dir: "parentNode" },
+ "+": { dir: "previousSibling", first: true },
+ "~": { dir: "previousSibling" }
+ },
+
+ preFilter: {
+ "ATTR": function( match ) {
+ match[ 1 ] = match[ 1 ].replace( runescape, funescape );
+
+ // Move the given value to match[3] whether quoted or unquoted
+ match[ 3 ] = ( match[ 3 ] || match[ 4 ] ||
+ match[ 5 ] || "" ).replace( runescape, funescape );
+
+ if ( match[ 2 ] === "~=" ) {
+ match[ 3 ] = " " + match[ 3 ] + " ";
+ }
+
+ return match.slice( 0, 4 );
+ },
+
+ "CHILD": function( match ) {
+
+ /* matches from matchExpr["CHILD"]
+ 1 type (only|nth|...)
+ 2 what (child|of-type)
+ 3 argument (even|odd|\d*|\d*n([+-]\d+)?|...)
+ 4 xn-component of xn+y argument ([+-]?\d*n|)
+ 5 sign of xn-component
+ 6 x of xn-component
+ 7 sign of y-component
+ 8 y of y-component
+ */
+ match[ 1 ] = match[ 1 ].toLowerCase();
+
+ if ( match[ 1 ].slice( 0, 3 ) === "nth" ) {
+
+ // nth-* requires argument
+ if ( !match[ 3 ] ) {
+ Sizzle.error( match[ 0 ] );
+ }
+
+ // numeric x and y parameters for Expr.filter.CHILD
+ // remember that false/true cast respectively to 0/1
+ match[ 4 ] = +( match[ 4 ] ?
+ match[ 5 ] + ( match[ 6 ] || 1 ) :
+ 2 * ( match[ 3 ] === "even" || match[ 3 ] === "odd" ) );
+ match[ 5 ] = +( ( match[ 7 ] + match[ 8 ] ) || match[ 3 ] === "odd" );
+
+ // other types prohibit arguments
+ } else if ( match[ 3 ] ) {
+ Sizzle.error( match[ 0 ] );
+ }
+
+ return match;
+ },
+
+ "PSEUDO": function( match ) {
+ var excess,
+ unquoted = !match[ 6 ] && match[ 2 ];
+
+ if ( matchExpr[ "CHILD" ].test( match[ 0 ] ) ) {
+ return null;
+ }
+
+ // Accept quoted arguments as-is
+ if ( match[ 3 ] ) {
+ match[ 2 ] = match[ 4 ] || match[ 5 ] || "";
+
+ // Strip excess characters from unquoted arguments
+ } else if ( unquoted && rpseudo.test( unquoted ) &&
+
+ // Get excess from tokenize (recursively)
+ ( excess = tokenize( unquoted, true ) ) &&
+
+ // advance to the next closing parenthesis
+ ( excess = unquoted.indexOf( ")", unquoted.length - excess ) - unquoted.length ) ) {
+
+ // excess is a negative index
+ match[ 0 ] = match[ 0 ].slice( 0, excess );
+ match[ 2 ] = unquoted.slice( 0, excess );
+ }
+
+ // Return only captures needed by the pseudo filter method (type and argument)
+ return match.slice( 0, 3 );
+ }
+ },
+
+ filter: {
+
+ "TAG": function( nodeNameSelector ) {
+ var nodeName = nodeNameSelector.replace( runescape, funescape ).toLowerCase();
+ return nodeNameSelector === "*" ?
+ function() {
+ return true;
+ } :
+ function( elem ) {
+ return elem.nodeName && elem.nodeName.toLowerCase() === nodeName;
+ };
+ },
+
+ "CLASS": function( className ) {
+ var pattern = classCache[ className + " " ];
+
+ return pattern ||
+ ( pattern = new RegExp( "(^|" + whitespace +
+ ")" + className + "(" + whitespace + "|$)" ) ) && classCache(
+ className, function( elem ) {
+ return pattern.test(
+ typeof elem.className === "string" && elem.className ||
+ typeof elem.getAttribute !== "undefined" &&
+ elem.getAttribute( "class" ) ||
+ ""
+ );
+ } );
+ },
+
+ "ATTR": function( name, operator, check ) {
+ return function( elem ) {
+ var result = Sizzle.attr( elem, name );
+
+ if ( result == null ) {
+ return operator === "!=";
+ }
+ if ( !operator ) {
+ return true;
+ }
+
+ result += "";
+
+ /* eslint-disable max-len */
+
+ return operator === "=" ? result === check :
+ operator === "!=" ? result !== check :
+ operator === "^=" ? check && result.indexOf( check ) === 0 :
+ operator === "*=" ? check && result.indexOf( check ) > -1 :
+ operator === "$=" ? check && result.slice( -check.length ) === check :
+ operator === "~=" ? ( " " + result.replace( rwhitespace, " " ) + " " ).indexOf( check ) > -1 :
+ operator === "|=" ? result === check || result.slice( 0, check.length + 1 ) === check + "-" :
+ false;
+ /* eslint-enable max-len */
+
+ };
+ },
+
+ "CHILD": function( type, what, _argument, first, last ) {
+ var simple = type.slice( 0, 3 ) !== "nth",
+ forward = type.slice( -4 ) !== "last",
+ ofType = what === "of-type";
+
+ return first === 1 && last === 0 ?
+
+ // Shortcut for :nth-*(n)
+ function( elem ) {
+ return !!elem.parentNode;
+ } :
+
+ function( elem, _context, xml ) {
+ var cache, uniqueCache, outerCache, node, nodeIndex, start,
+ dir = simple !== forward ? "nextSibling" : "previousSibling",
+ parent = elem.parentNode,
+ name = ofType && elem.nodeName.toLowerCase(),
+ useCache = !xml && !ofType,
+ diff = false;
+
+ if ( parent ) {
+
+ // :(first|last|only)-(child|of-type)
+ if ( simple ) {
+ while ( dir ) {
+ node = elem;
+ while ( ( node = node[ dir ] ) ) {
+ if ( ofType ?
+ node.nodeName.toLowerCase() === name :
+ node.nodeType === 1 ) {
+
+ return false;
+ }
+ }
+
+ // Reverse direction for :only-* (if we haven't yet done so)
+ start = dir = type === "only" && !start && "nextSibling";
+ }
+ return true;
+ }
+
+ start = [ forward ? parent.firstChild : parent.lastChild ];
+
+ // non-xml :nth-child(...) stores cache data on `parent`
+ if ( forward && useCache ) {
+
+ // Seek `elem` from a previously-cached index
+
+ // ...in a gzip-friendly way
+ node = parent;
+ outerCache = node[ expando ] || ( node[ expando ] = {} );
+
+ // Support: IE <9 only
+ // Defend against cloned attroperties (jQuery gh-1709)
+ uniqueCache = outerCache[ node.uniqueID ] ||
+ ( outerCache[ node.uniqueID ] = {} );
+
+ cache = uniqueCache[ type ] || [];
+ nodeIndex = cache[ 0 ] === dirruns && cache[ 1 ];
+ diff = nodeIndex && cache[ 2 ];
+ node = nodeIndex && parent.childNodes[ nodeIndex ];
+
+ while ( ( node = ++nodeIndex && node && node[ dir ] ||
+
+ // Fallback to seeking `elem` from the start
+ ( diff = nodeIndex = 0 ) || start.pop() ) ) {
+
+ // When found, cache indexes on `parent` and break
+ if ( node.nodeType === 1 && ++diff && node === elem ) {
+ uniqueCache[ type ] = [ dirruns, nodeIndex, diff ];
+ break;
+ }
+ }
+
+ } else {
+
+ // Use previously-cached element index if available
+ if ( useCache ) {
+
+ // ...in a gzip-friendly way
+ node = elem;
+ outerCache = node[ expando ] || ( node[ expando ] = {} );
+
+ // Support: IE <9 only
+ // Defend against cloned attroperties (jQuery gh-1709)
+ uniqueCache = outerCache[ node.uniqueID ] ||
+ ( outerCache[ node.uniqueID ] = {} );
+
+ cache = uniqueCache[ type ] || [];
+ nodeIndex = cache[ 0 ] === dirruns && cache[ 1 ];
+ diff = nodeIndex;
+ }
+
+ // xml :nth-child(...)
+ // or :nth-last-child(...) or :nth(-last)?-of-type(...)
+ if ( diff === false ) {
+
+ // Use the same loop as above to seek `elem` from the start
+ while ( ( node = ++nodeIndex && node && node[ dir ] ||
+ ( diff = nodeIndex = 0 ) || start.pop() ) ) {
+
+ if ( ( ofType ?
+ node.nodeName.toLowerCase() === name :
+ node.nodeType === 1 ) &&
+ ++diff ) {
+
+ // Cache the index of each encountered element
+ if ( useCache ) {
+ outerCache = node[ expando ] ||
+ ( node[ expando ] = {} );
+
+ // Support: IE <9 only
+ // Defend against cloned attroperties (jQuery gh-1709)
+ uniqueCache = outerCache[ node.uniqueID ] ||
+ ( outerCache[ node.uniqueID ] = {} );
+
+ uniqueCache[ type ] = [ dirruns, diff ];
+ }
+
+ if ( node === elem ) {
+ break;
+ }
+ }
+ }
+ }
+ }
+
+ // Incorporate the offset, then check against cycle size
+ diff -= last;
+ return diff === first || ( diff % first === 0 && diff / first >= 0 );
+ }
+ };
+ },
+
+ "PSEUDO": function( pseudo, argument ) {
+
+ // pseudo-class names are case-insensitive
+ // http://www.w3.org/TR/selectors/#pseudo-classes
+ // Prioritize by case sensitivity in case custom pseudos are added with uppercase letters
+ // Remember that setFilters inherits from pseudos
+ var args,
+ fn = Expr.pseudos[ pseudo ] || Expr.setFilters[ pseudo.toLowerCase() ] ||
+ Sizzle.error( "unsupported pseudo: " + pseudo );
+
+ // The user may use createPseudo to indicate that
+ // arguments are needed to create the filter function
+ // just as Sizzle does
+ if ( fn[ expando ] ) {
+ return fn( argument );
+ }
+
+ // But maintain support for old signatures
+ if ( fn.length > 1 ) {
+ args = [ pseudo, pseudo, "", argument ];
+ return Expr.setFilters.hasOwnProperty( pseudo.toLowerCase() ) ?
+ markFunction( function( seed, matches ) {
+ var idx,
+ matched = fn( seed, argument ),
+ i = matched.length;
+ while ( i-- ) {
+ idx = indexOf( seed, matched[ i ] );
+ seed[ idx ] = !( matches[ idx ] = matched[ i ] );
+ }
+ } ) :
+ function( elem ) {
+ return fn( elem, 0, args );
+ };
+ }
+
+ return fn;
+ }
+ },
+
+ pseudos: {
+
+ // Potentially complex pseudos
+ "not": markFunction( function( selector ) {
+
+ // Trim the selector passed to compile
+ // to avoid treating leading and trailing
+ // spaces as combinators
+ var input = [],
+ results = [],
+ matcher = compile( selector.replace( rtrim, "$1" ) );
+
+ return matcher[ expando ] ?
+ markFunction( function( seed, matches, _context, xml ) {
+ var elem,
+ unmatched = matcher( seed, null, xml, [] ),
+ i = seed.length;
+
+ // Match elements unmatched by `matcher`
+ while ( i-- ) {
+ if ( ( elem = unmatched[ i ] ) ) {
+ seed[ i ] = !( matches[ i ] = elem );
+ }
+ }
+ } ) :
+ function( elem, _context, xml ) {
+ input[ 0 ] = elem;
+ matcher( input, null, xml, results );
+
+ // Don't keep the element (issue #299)
+ input[ 0 ] = null;
+ return !results.pop();
+ };
+ } ),
+
+ "has": markFunction( function( selector ) {
+ return function( elem ) {
+ return Sizzle( selector, elem ).length > 0;
+ };
+ } ),
+
+ "contains": markFunction( function( text ) {
+ text = text.replace( runescape, funescape );
+ return function( elem ) {
+ return ( elem.textContent || getText( elem ) ).indexOf( text ) > -1;
+ };
+ } ),
+
+ // "Whether an element is represented by a :lang() selector
+ // is based solely on the element's language value
+ // being equal to the identifier C,
+ // or beginning with the identifier C immediately followed by "-".
+ // The matching of C against the element's language value is performed case-insensitively.
+ // The identifier C does not have to be a valid language name."
+ // http://www.w3.org/TR/selectors/#lang-pseudo
+ "lang": markFunction( function( lang ) {
+
+ // lang value must be a valid identifier
+ if ( !ridentifier.test( lang || "" ) ) {
+ Sizzle.error( "unsupported lang: " + lang );
+ }
+ lang = lang.replace( runescape, funescape ).toLowerCase();
+ return function( elem ) {
+ var elemLang;
+ do {
+ if ( ( elemLang = documentIsHTML ?
+ elem.lang :
+ elem.getAttribute( "xml:lang" ) || elem.getAttribute( "lang" ) ) ) {
+
+ elemLang = elemLang.toLowerCase();
+ return elemLang === lang || elemLang.indexOf( lang + "-" ) === 0;
+ }
+ } while ( ( elem = elem.parentNode ) && elem.nodeType === 1 );
+ return false;
+ };
+ } ),
+
+ // Miscellaneous
+ "target": function( elem ) {
+ var hash = window.location && window.location.hash;
+ return hash && hash.slice( 1 ) === elem.id;
+ },
+
+ "root": function( elem ) {
+ return elem === docElem;
+ },
+
+ "focus": function( elem ) {
+ return elem === document.activeElement &&
+ ( !document.hasFocus || document.hasFocus() ) &&
+ !!( elem.type || elem.href || ~elem.tabIndex );
+ },
+
+ // Boolean properties
+ "enabled": createDisabledPseudo( false ),
+ "disabled": createDisabledPseudo( true ),
+
+ "checked": function( elem ) {
+
+ // In CSS3, :checked should return both checked and selected elements
+ // http://www.w3.org/TR/2011/REC-css3-selectors-20110929/#checked
+ var nodeName = elem.nodeName.toLowerCase();
+ return ( nodeName === "input" && !!elem.checked ) ||
+ ( nodeName === "option" && !!elem.selected );
+ },
+
+ "selected": function( elem ) {
+
+ // Accessing this property makes selected-by-default
+ // options in Safari work properly
+ if ( elem.parentNode ) {
+ // eslint-disable-next-line no-unused-expressions
+ elem.parentNode.selectedIndex;
+ }
+
+ return elem.selected === true;
+ },
+
+ // Contents
+ "empty": function( elem ) {
+
+ // http://www.w3.org/TR/selectors/#empty-pseudo
+ // :empty is negated by element (1) or content nodes (text: 3; cdata: 4; entity ref: 5),
+ // but not by others (comment: 8; processing instruction: 7; etc.)
+ // nodeType < 6 works because attributes (2) do not appear as children
+ for ( elem = elem.firstChild; elem; elem = elem.nextSibling ) {
+ if ( elem.nodeType < 6 ) {
+ return false;
+ }
+ }
+ return true;
+ },
+
+ "parent": function( elem ) {
+ return !Expr.pseudos[ "empty" ]( elem );
+ },
+
+ // Element/input types
+ "header": function( elem ) {
+ return rheader.test( elem.nodeName );
+ },
+
+ "input": function( elem ) {
+ return rinputs.test( elem.nodeName );
+ },
+
+ "button": function( elem ) {
+ var name = elem.nodeName.toLowerCase();
+ return name === "input" && elem.type === "button" || name === "button";
+ },
+
+ "text": function( elem ) {
+ var attr;
+ return elem.nodeName.toLowerCase() === "input" &&
+ elem.type === "text" &&
+
+ // Support: IE<8
+ // New HTML5 attribute values (e.g., "search") appear with elem.type === "text"
+ ( ( attr = elem.getAttribute( "type" ) ) == null ||
+ attr.toLowerCase() === "text" );
+ },
+
+ // Position-in-collection
+ "first": createPositionalPseudo( function() {
+ return [ 0 ];
+ } ),
+
+ "last": createPositionalPseudo( function( _matchIndexes, length ) {
+ return [ length - 1 ];
+ } ),
+
+ "eq": createPositionalPseudo( function( _matchIndexes, length, argument ) {
+ return [ argument < 0 ? argument + length : argument ];
+ } ),
+
+ "even": createPositionalPseudo( function( matchIndexes, length ) {
+ var i = 0;
+ for ( ; i < length; i += 2 ) {
+ matchIndexes.push( i );
+ }
+ return matchIndexes;
+ } ),
+
+ "odd": createPositionalPseudo( function( matchIndexes, length ) {
+ var i = 1;
+ for ( ; i < length; i += 2 ) {
+ matchIndexes.push( i );
+ }
+ return matchIndexes;
+ } ),
+
+ "lt": createPositionalPseudo( function( matchIndexes, length, argument ) {
+ var i = argument < 0 ?
+ argument + length :
+ argument > length ?
+ length :
+ argument;
+ for ( ; --i >= 0; ) {
+ matchIndexes.push( i );
+ }
+ return matchIndexes;
+ } ),
+
+ "gt": createPositionalPseudo( function( matchIndexes, length, argument ) {
+ var i = argument < 0 ? argument + length : argument;
+ for ( ; ++i < length; ) {
+ matchIndexes.push( i );
+ }
+ return matchIndexes;
+ } )
+ }
+};
+
+Expr.pseudos[ "nth" ] = Expr.pseudos[ "eq" ];
// Add button/input type pseudos
- for ( i in { radio: true, checkbox: true, file: true, password: true, image: true } ) {
- Expr.pseudos[ i ] = createInputPseudo( i );
- }
- for ( i in { submit: true, reset: true } ) {
- Expr.pseudos[ i ] = createButtonPseudo( i );
- }
+for ( i in { radio: true, checkbox: true, file: true, password: true, image: true } ) {
+ Expr.pseudos[ i ] = createInputPseudo( i );
+}
+for ( i in { submit: true, reset: true } ) {
+ Expr.pseudos[ i ] = createButtonPseudo( i );
+}
// Easy API for creating new setFilters
- function setFilters() {}
- setFilters.prototype = Expr.filters = Expr.pseudos;
- Expr.setFilters = new setFilters();
-
- tokenize = Sizzle.tokenize = function( selector, parseOnly ) {
- var matched, match, tokens, type,
- soFar, groups, preFilters,
- cached = tokenCache[ selector + " " ];
-
- if ( cached ) {
- return parseOnly ? 0 : cached.slice( 0 );
- }
-
- soFar = selector;
- groups = [];
- preFilters = Expr.preFilter;
-
- while ( soFar ) {
-
- // Comma and first run
- if ( !matched || (match = rcomma.exec( soFar )) ) {
- if ( match ) {
- // Don't consume trailing commas as valid
- soFar = soFar.slice( match[0].length ) || soFar;
- }
- groups.push( (tokens = []) );
- }
-
- matched = false;
-
- // Combinators
- if ( (match = rcombinators.exec( soFar )) ) {
- matched = match.shift();
- tokens.push({
- value: matched,
- // Cast descendant combinators to space
- type: match[0].replace( rtrim, " " )
- });
- soFar = soFar.slice( matched.length );
- }
-
- // Filters
- for ( type in Expr.filter ) {
- if ( (match = matchExpr[ type ].exec( soFar )) && (!preFilters[ type ] ||
- (match = preFilters[ type ]( match ))) ) {
- matched = match.shift();
- tokens.push({
- value: matched,
- type: type,
- matches: match
- });
- soFar = soFar.slice( matched.length );
- }
- }
-
- if ( !matched ) {
- break;
- }
- }
-
- // Return the length of the invalid excess
- // if we're just parsing
- // Otherwise, throw an error or return tokens
- return parseOnly ?
- soFar.length :
- soFar ?
- Sizzle.error( selector ) :
- // Cache the tokens
- tokenCache( selector, groups ).slice( 0 );
- };
-
- function toSelector( tokens ) {
- var i = 0,
- len = tokens.length,
- selector = "";
- for ( ; i < len; i++ ) {
- selector += tokens[i].value;
- }
- return selector;
- }
-
- function addCombinator( matcher, combinator, base ) {
- var dir = combinator.dir,
- checkNonElements = base && dir === "parentNode",
- doneName = done++;
-
- return combinator.first ?
- // Check against closest ancestor/preceding element
- function( elem, context, xml ) {
- while ( (elem = elem[ dir ]) ) {
- if ( elem.nodeType === 1 || checkNonElements ) {
- return matcher( elem, context, xml );
- }
- }
- } :
-
- // Check against all ancestor/preceding elements
- function( elem, context, xml ) {
- var oldCache, uniqueCache, outerCache,
- newCache = [ dirruns, doneName ];
-
- // We can't set arbitrary data on XML nodes, so they don't benefit from combinator caching
- if ( xml ) {
- while ( (elem = elem[ dir ]) ) {
- if ( elem.nodeType === 1 || checkNonElements ) {
- if ( matcher( elem, context, xml ) ) {
- return true;
- }
- }
- }
- } else {
- while ( (elem = elem[ dir ]) ) {
- if ( elem.nodeType === 1 || checkNonElements ) {
- outerCache = elem[ expando ] || (elem[ expando ] = {});
-
- // Support: IE <9 only
- // Defend against cloned attroperties (jQuery gh-1709)
- uniqueCache = outerCache[ elem.uniqueID ] || (outerCache[ elem.uniqueID ] = {});
-
- if ( (oldCache = uniqueCache[ dir ]) &&
- oldCache[ 0 ] === dirruns && oldCache[ 1 ] === doneName ) {
-
- // Assign to newCache so results back-propagate to previous elements
- return (newCache[ 2 ] = oldCache[ 2 ]);
- } else {
- // Reuse newcache so results back-propagate to previous elements
- uniqueCache[ dir ] = newCache;
-
- // A match means we're done; a fail means we have to keep checking
- if ( (newCache[ 2 ] = matcher( elem, context, xml )) ) {
- return true;
- }
- }
- }
- }
- }
- };
- }
-
- function elementMatcher( matchers ) {
- return matchers.length > 1 ?
- function( elem, context, xml ) {
- var i = matchers.length;
- while ( i-- ) {
- if ( !matchers[i]( elem, context, xml ) ) {
- return false;
- }
- }
- return true;
- } :
- matchers[0];
- }
-
- function multipleContexts( selector, contexts, results ) {
- var i = 0,
- len = contexts.length;
- for ( ; i < len; i++ ) {
- Sizzle( selector, contexts[i], results );
- }
- return results;
- }
-
- function condense( unmatched, map, filter, context, xml ) {
- var elem,
- newUnmatched = [],
- i = 0,
- len = unmatched.length,
- mapped = map != null;
-
- for ( ; i < len; i++ ) {
- if ( (elem = unmatched[i]) ) {
- if ( !filter || filter( elem, context, xml ) ) {
- newUnmatched.push( elem );
- if ( mapped ) {
- map.push( i );
- }
- }
- }
- }
-
- return newUnmatched;
- }
-
- function setMatcher( preFilter, selector, matcher, postFilter, postFinder, postSelector ) {
- if ( postFilter && !postFilter[ expando ] ) {
- postFilter = setMatcher( postFilter );
- }
- if ( postFinder && !postFinder[ expando ] ) {
- postFinder = setMatcher( postFinder, postSelector );
- }
- return markFunction(function( seed, results, context, xml ) {
- var temp, i, elem,
- preMap = [],
- postMap = [],
- preexisting = results.length,
-
- // Get initial elements from seed or context
- elems = seed || multipleContexts( selector || "*", context.nodeType ? [ context ] : context, [] ),
-
- // Prefilter to get matcher input, preserving a map for seed-results synchronization
- matcherIn = preFilter && ( seed || !selector ) ?
- condense( elems, preMap, preFilter, context, xml ) :
- elems,
-
- matcherOut = matcher ?
- // If we have a postFinder, or filtered seed, or non-seed postFilter or preexisting results,
- postFinder || ( seed ? preFilter : preexisting || postFilter ) ?
-
- // ...intermediate processing is necessary
- [] :
-
- // ...otherwise use results directly
- results :
- matcherIn;
-
- // Find primary matches
- if ( matcher ) {
- matcher( matcherIn, matcherOut, context, xml );
- }
-
- // Apply postFilter
- if ( postFilter ) {
- temp = condense( matcherOut, postMap );
- postFilter( temp, [], context, xml );
-
- // Un-match failing elements by moving them back to matcherIn
- i = temp.length;
- while ( i-- ) {
- if ( (elem = temp[i]) ) {
- matcherOut[ postMap[i] ] = !(matcherIn[ postMap[i] ] = elem);
- }
- }
- }
-
- if ( seed ) {
- if ( postFinder || preFilter ) {
- if ( postFinder ) {
- // Get the final matcherOut by condensing this intermediate into postFinder contexts
- temp = [];
- i = matcherOut.length;
- while ( i-- ) {
- if ( (elem = matcherOut[i]) ) {
- // Restore matcherIn since elem is not yet a final match
- temp.push( (matcherIn[i] = elem) );
- }
- }
- postFinder( null, (matcherOut = []), temp, xml );
- }
-
- // Move matched elements from seed to results to keep them synchronized
- i = matcherOut.length;
- while ( i-- ) {
- if ( (elem = matcherOut[i]) &&
- (temp = postFinder ? indexOf( seed, elem ) : preMap[i]) > -1 ) {
-
- seed[temp] = !(results[temp] = elem);
- }
- }
- }
-
- // Add elements to results, through postFinder if defined
- } else {
- matcherOut = condense(
- matcherOut === results ?
- matcherOut.splice( preexisting, matcherOut.length ) :
- matcherOut
- );
- if ( postFinder ) {
- postFinder( null, results, matcherOut, xml );
- } else {
- push.apply( results, matcherOut );
- }
- }
- });
- }
-
- function matcherFromTokens( tokens ) {
- var checkContext, matcher, j,
- len = tokens.length,
- leadingRelative = Expr.relative[ tokens[0].type ],
- implicitRelative = leadingRelative || Expr.relative[" "],
- i = leadingRelative ? 1 : 0,
-
- // The foundational matcher ensures that elements are reachable from top-level context(s)
- matchContext = addCombinator( function( elem ) {
- return elem === checkContext;
- }, implicitRelative, true ),
- matchAnyContext = addCombinator( function( elem ) {
- return indexOf( checkContext, elem ) > -1;
- }, implicitRelative, true ),
- matchers = [ function( elem, context, xml ) {
- var ret = ( !leadingRelative && ( xml || context !== outermostContext ) ) || (
- (checkContext = context).nodeType ?
- matchContext( elem, context, xml ) :
- matchAnyContext( elem, context, xml ) );
- // Avoid hanging onto element (issue #299)
- checkContext = null;
- return ret;
- } ];
-
- for ( ; i < len; i++ ) {
- if ( (matcher = Expr.relative[ tokens[i].type ]) ) {
- matchers = [ addCombinator(elementMatcher( matchers ), matcher) ];
- } else {
- matcher = Expr.filter[ tokens[i].type ].apply( null, tokens[i].matches );
-
- // Return special upon seeing a positional matcher
- if ( matcher[ expando ] ) {
- // Find the next relative operator (if any) for proper handling
- j = ++i;
- for ( ; j < len; j++ ) {
- if ( Expr.relative[ tokens[j].type ] ) {
- break;
- }
- }
- return setMatcher(
- i > 1 && elementMatcher( matchers ),
- i > 1 && toSelector(
- // If the preceding token was a descendant combinator, insert an implicit any-element `*`
- tokens.slice( 0, i - 1 ).concat({ value: tokens[ i - 2 ].type === " " ? "*" : "" })
- ).replace( rtrim, "$1" ),
- matcher,
- i < j && matcherFromTokens( tokens.slice( i, j ) ),
- j < len && matcherFromTokens( (tokens = tokens.slice( j )) ),
- j < len && toSelector( tokens )
- );
- }
- matchers.push( matcher );
- }
- }
-
- return elementMatcher( matchers );
- }
-
- function matcherFromGroupMatchers( elementMatchers, setMatchers ) {
- var bySet = setMatchers.length > 0,
- byElement = elementMatchers.length > 0,
- superMatcher = function( seed, context, xml, results, outermost ) {
- var elem, j, matcher,
- matchedCount = 0,
- i = "0",
- unmatched = seed && [],
- setMatched = [],
- contextBackup = outermostContext,
- // We must always have either seed elements or outermost context
- elems = seed || byElement && Expr.find["TAG"]( "*", outermost ),
- // Use integer dirruns iff this is the outermost matcher
- dirrunsUnique = (dirruns += contextBackup == null ? 1 : Math.random() || 0.1),
- len = elems.length;
-
- if ( outermost ) {
- outermostContext = context === document || context || outermost;
- }
-
- // Add elements passing elementMatchers directly to results
- // Support: IE<9, Safari
- // Tolerate NodeList properties (IE: "length"; Safari: ) matching elements by id
- for ( ; i !== len && (elem = elems[i]) != null; i++ ) {
- if ( byElement && elem ) {
- j = 0;
- if ( !context && elem.ownerDocument !== document ) {
- setDocument( elem );
- xml = !documentIsHTML;
- }
- while ( (matcher = elementMatchers[j++]) ) {
- if ( matcher( elem, context || document, xml) ) {
- results.push( elem );
- break;
- }
- }
- if ( outermost ) {
- dirruns = dirrunsUnique;
- }
- }
-
- // Track unmatched elements for set filters
- if ( bySet ) {
- // They will have gone through all possible matchers
- if ( (elem = !matcher && elem) ) {
- matchedCount--;
- }
-
- // Lengthen the array for every element, matched or not
- if ( seed ) {
- unmatched.push( elem );
- }
- }
- }
-
- // `i` is now the count of elements visited above, and adding it to `matchedCount`
- // makes the latter nonnegative.
- matchedCount += i;
-
- // Apply set filters to unmatched elements
- // NOTE: This can be skipped if there are no unmatched elements (i.e., `matchedCount`
- // equals `i`), unless we didn't visit _any_ elements in the above loop because we have
- // no element matchers and no seed.
- // Incrementing an initially-string "0" `i` allows `i` to remain a string only in that
- // case, which will result in a "00" `matchedCount` that differs from `i` but is also
- // numerically zero.
- if ( bySet && i !== matchedCount ) {
- j = 0;
- while ( (matcher = setMatchers[j++]) ) {
- matcher( unmatched, setMatched, context, xml );
- }
-
- if ( seed ) {
- // Reintegrate element matches to eliminate the need for sorting
- if ( matchedCount > 0 ) {
- while ( i-- ) {
- if ( !(unmatched[i] || setMatched[i]) ) {
- setMatched[i] = pop.call( results );
- }
- }
- }
-
- // Discard index placeholder values to get only actual matches
- setMatched = condense( setMatched );
- }
-
- // Add matches to results
- push.apply( results, setMatched );
-
- // Seedless set matches succeeding multiple successful matchers stipulate sorting
- if ( outermost && !seed && setMatched.length > 0 &&
- ( matchedCount + setMatchers.length ) > 1 ) {
-
- Sizzle.uniqueSort( results );
- }
- }
-
- // Override manipulation of globals by nested matchers
- if ( outermost ) {
- dirruns = dirrunsUnique;
- outermostContext = contextBackup;
- }
-
- return unmatched;
- };
-
- return bySet ?
- markFunction( superMatcher ) :
- superMatcher;
- }
-
- compile = Sizzle.compile = function( selector, match /* Internal Use Only */ ) {
- var i,
- setMatchers = [],
- elementMatchers = [],
- cached = compilerCache[ selector + " " ];
-
- if ( !cached ) {
- // Generate a function of recursive functions that can be used to check each element
- if ( !match ) {
- match = tokenize( selector );
- }
- i = match.length;
- while ( i-- ) {
- cached = matcherFromTokens( match[i] );
- if ( cached[ expando ] ) {
- setMatchers.push( cached );
- } else {
- elementMatchers.push( cached );
- }
- }
-
- // Cache the compiled function
- cached = compilerCache( selector, matcherFromGroupMatchers( elementMatchers, setMatchers ) );
-
- // Save selector and tokenization
- cached.selector = selector;
- }
- return cached;
- };
-
- /**
- * A low-level selection function that works with Sizzle's compiled
- * selector functions
- * @param {String|Function} selector A selector or a pre-compiled
- * selector function built with Sizzle.compile
- * @param {Element} context
- * @param {Array} [results]
- * @param {Array} [seed] A set of elements to match against
- */
- select = Sizzle.select = function( selector, context, results, seed ) {
- var i, tokens, token, type, find,
- compiled = typeof selector === "function" && selector,
- match = !seed && tokenize( (selector = compiled.selector || selector) );
-
- results = results || [];
-
- // Try to minimize operations if there is only one selector in the list and no seed
- // (the latter of which guarantees us context)
- if ( match.length === 1 ) {
-
- // Reduce context if the leading compound selector is an ID
- tokens = match[0] = match[0].slice( 0 );
- if ( tokens.length > 2 && (token = tokens[0]).type === "ID" &&
- support.getById && context.nodeType === 9 && documentIsHTML &&
- Expr.relative[ tokens[1].type ] ) {
-
- context = ( Expr.find["ID"]( token.matches[0].replace(runescape, funescape), context ) || [] )[0];
- if ( !context ) {
- return results;
-
- // Precompiled matchers will still verify ancestry, so step up a level
- } else if ( compiled ) {
- context = context.parentNode;
- }
-
- selector = selector.slice( tokens.shift().value.length );
- }
-
- // Fetch a seed set for right-to-left matching
- i = matchExpr["needsContext"].test( selector ) ? 0 : tokens.length;
- while ( i-- ) {
- token = tokens[i];
-
- // Abort if we hit a combinator
- if ( Expr.relative[ (type = token.type) ] ) {
- break;
- }
- if ( (find = Expr.find[ type ]) ) {
- // Search, expanding context for leading sibling combinators
- if ( (seed = find(
- token.matches[0].replace( runescape, funescape ),
- rsibling.test( tokens[0].type ) && testContext( context.parentNode ) || context
- )) ) {
-
- // If seed is empty or no tokens remain, we can return early
- tokens.splice( i, 1 );
- selector = seed.length && toSelector( tokens );
- if ( !selector ) {
- push.apply( results, seed );
- return results;
- }
-
- break;
- }
- }
- }
- }
-
- // Compile and execute a filtering function if one is not provided
- // Provide `match` to avoid retokenization if we modified the selector above
- ( compiled || compile( selector, match ) )(
- seed,
- context,
- !documentIsHTML,
- results,
- !context || rsibling.test( selector ) && testContext( context.parentNode ) || context
- );
- return results;
- };
+function setFilters() {}
+setFilters.prototype = Expr.filters = Expr.pseudos;
+Expr.setFilters = new setFilters();
+
+tokenize = Sizzle.tokenize = function( selector, parseOnly ) {
+ var matched, match, tokens, type,
+ soFar, groups, preFilters,
+ cached = tokenCache[ selector + " " ];
+
+ if ( cached ) {
+ return parseOnly ? 0 : cached.slice( 0 );
+ }
+
+ soFar = selector;
+ groups = [];
+ preFilters = Expr.preFilter;
+
+ while ( soFar ) {
+
+ // Comma and first run
+ if ( !matched || ( match = rcomma.exec( soFar ) ) ) {
+ if ( match ) {
+
+ // Don't consume trailing commas as valid
+ soFar = soFar.slice( match[ 0 ].length ) || soFar;
+ }
+ groups.push( ( tokens = [] ) );
+ }
+
+ matched = false;
+
+ // Combinators
+ if ( ( match = rcombinators.exec( soFar ) ) ) {
+ matched = match.shift();
+ tokens.push( {
+ value: matched,
+
+ // Cast descendant combinators to space
+ type: match[ 0 ].replace( rtrim, " " )
+ } );
+ soFar = soFar.slice( matched.length );
+ }
+
+ // Filters
+ for ( type in Expr.filter ) {
+ if ( ( match = matchExpr[ type ].exec( soFar ) ) && ( !preFilters[ type ] ||
+ ( match = preFilters[ type ]( match ) ) ) ) {
+ matched = match.shift();
+ tokens.push( {
+ value: matched,
+ type: type,
+ matches: match
+ } );
+ soFar = soFar.slice( matched.length );
+ }
+ }
+
+ if ( !matched ) {
+ break;
+ }
+ }
+
+ // Return the length of the invalid excess
+ // if we're just parsing
+ // Otherwise, throw an error or return tokens
+ return parseOnly ?
+ soFar.length :
+ soFar ?
+ Sizzle.error( selector ) :
+
+ // Cache the tokens
+ tokenCache( selector, groups ).slice( 0 );
+};
+
+function toSelector( tokens ) {
+ var i = 0,
+ len = tokens.length,
+ selector = "";
+ for ( ; i < len; i++ ) {
+ selector += tokens[ i ].value;
+ }
+ return selector;
+}
+
+function addCombinator( matcher, combinator, base ) {
+ var dir = combinator.dir,
+ skip = combinator.next,
+ key = skip || dir,
+ checkNonElements = base && key === "parentNode",
+ doneName = done++;
+
+ return combinator.first ?
+
+ // Check against closest ancestor/preceding element
+ function( elem, context, xml ) {
+ while ( ( elem = elem[ dir ] ) ) {
+ if ( elem.nodeType === 1 || checkNonElements ) {
+ return matcher( elem, context, xml );
+ }
+ }
+ return false;
+ } :
+
+ // Check against all ancestor/preceding elements
+ function( elem, context, xml ) {
+ var oldCache, uniqueCache, outerCache,
+ newCache = [ dirruns, doneName ];
+
+ // We can't set arbitrary data on XML nodes, so they don't benefit from combinator caching
+ if ( xml ) {
+ while ( ( elem = elem[ dir ] ) ) {
+ if ( elem.nodeType === 1 || checkNonElements ) {
+ if ( matcher( elem, context, xml ) ) {
+ return true;
+ }
+ }
+ }
+ } else {
+ while ( ( elem = elem[ dir ] ) ) {
+ if ( elem.nodeType === 1 || checkNonElements ) {
+ outerCache = elem[ expando ] || ( elem[ expando ] = {} );
+
+ // Support: IE <9 only
+ // Defend against cloned attroperties (jQuery gh-1709)
+ uniqueCache = outerCache[ elem.uniqueID ] ||
+ ( outerCache[ elem.uniqueID ] = {} );
+
+ if ( skip && skip === elem.nodeName.toLowerCase() ) {
+ elem = elem[ dir ] || elem;
+ } else if ( ( oldCache = uniqueCache[ key ] ) &&
+ oldCache[ 0 ] === dirruns && oldCache[ 1 ] === doneName ) {
+
+ // Assign to newCache so results back-propagate to previous elements
+ return ( newCache[ 2 ] = oldCache[ 2 ] );
+ } else {
+
+ // Reuse newcache so results back-propagate to previous elements
+ uniqueCache[ key ] = newCache;
+
+ // A match means we're done; a fail means we have to keep checking
+ if ( ( newCache[ 2 ] = matcher( elem, context, xml ) ) ) {
+ return true;
+ }
+ }
+ }
+ }
+ }
+ return false;
+ };
+}
+
+function elementMatcher( matchers ) {
+ return matchers.length > 1 ?
+ function( elem, context, xml ) {
+ var i = matchers.length;
+ while ( i-- ) {
+ if ( !matchers[ i ]( elem, context, xml ) ) {
+ return false;
+ }
+ }
+ return true;
+ } :
+ matchers[ 0 ];
+}
+
+function multipleContexts( selector, contexts, results ) {
+ var i = 0,
+ len = contexts.length;
+ for ( ; i < len; i++ ) {
+ Sizzle( selector, contexts[ i ], results );
+ }
+ return results;
+}
+
+function condense( unmatched, map, filter, context, xml ) {
+ var elem,
+ newUnmatched = [],
+ i = 0,
+ len = unmatched.length,
+ mapped = map != null;
+
+ for ( ; i < len; i++ ) {
+ if ( ( elem = unmatched[ i ] ) ) {
+ if ( !filter || filter( elem, context, xml ) ) {
+ newUnmatched.push( elem );
+ if ( mapped ) {
+ map.push( i );
+ }
+ }
+ }
+ }
+
+ return newUnmatched;
+}
+
+function setMatcher( preFilter, selector, matcher, postFilter, postFinder, postSelector ) {
+ if ( postFilter && !postFilter[ expando ] ) {
+ postFilter = setMatcher( postFilter );
+ }
+ if ( postFinder && !postFinder[ expando ] ) {
+ postFinder = setMatcher( postFinder, postSelector );
+ }
+ return markFunction( function( seed, results, context, xml ) {
+ var temp, i, elem,
+ preMap = [],
+ postMap = [],
+ preexisting = results.length,
+
+ // Get initial elements from seed or context
+ elems = seed || multipleContexts(
+ selector || "*",
+ context.nodeType ? [ context ] : context,
+ []
+ ),
+
+ // Prefilter to get matcher input, preserving a map for seed-results synchronization
+ matcherIn = preFilter && ( seed || !selector ) ?
+ condense( elems, preMap, preFilter, context, xml ) :
+ elems,
+
+ matcherOut = matcher ?
+
+ // If we have a postFinder, or filtered seed, or non-seed postFilter or preexisting results,
+ postFinder || ( seed ? preFilter : preexisting || postFilter ) ?
+
+ // ...intermediate processing is necessary
+ [] :
+
+ // ...otherwise use results directly
+ results :
+ matcherIn;
+
+ // Find primary matches
+ if ( matcher ) {
+ matcher( matcherIn, matcherOut, context, xml );
+ }
+
+ // Apply postFilter
+ if ( postFilter ) {
+ temp = condense( matcherOut, postMap );
+ postFilter( temp, [], context, xml );
+
+ // Un-match failing elements by moving them back to matcherIn
+ i = temp.length;
+ while ( i-- ) {
+ if ( ( elem = temp[ i ] ) ) {
+ matcherOut[ postMap[ i ] ] = !( matcherIn[ postMap[ i ] ] = elem );
+ }
+ }
+ }
+
+ if ( seed ) {
+ if ( postFinder || preFilter ) {
+ if ( postFinder ) {
+
+ // Get the final matcherOut by condensing this intermediate into postFinder contexts
+ temp = [];
+ i = matcherOut.length;
+ while ( i-- ) {
+ if ( ( elem = matcherOut[ i ] ) ) {
+
+ // Restore matcherIn since elem is not yet a final match
+ temp.push( ( matcherIn[ i ] = elem ) );
+ }
+ }
+ postFinder( null, ( matcherOut = [] ), temp, xml );
+ }
+
+ // Move matched elements from seed to results to keep them synchronized
+ i = matcherOut.length;
+ while ( i-- ) {
+ if ( ( elem = matcherOut[ i ] ) &&
+ ( temp = postFinder ? indexOf( seed, elem ) : preMap[ i ] ) > -1 ) {
+
+ seed[ temp ] = !( results[ temp ] = elem );
+ }
+ }
+ }
+
+ // Add elements to results, through postFinder if defined
+ } else {
+ matcherOut = condense(
+ matcherOut === results ?
+ matcherOut.splice( preexisting, matcherOut.length ) :
+ matcherOut
+ );
+ if ( postFinder ) {
+ postFinder( null, results, matcherOut, xml );
+ } else {
+ push.apply( results, matcherOut );
+ }
+ }
+ } );
+}
+
+function matcherFromTokens( tokens ) {
+ var checkContext, matcher, j,
+ len = tokens.length,
+ leadingRelative = Expr.relative[ tokens[ 0 ].type ],
+ implicitRelative = leadingRelative || Expr.relative[ " " ],
+ i = leadingRelative ? 1 : 0,
+
+ // The foundational matcher ensures that elements are reachable from top-level context(s)
+ matchContext = addCombinator( function( elem ) {
+ return elem === checkContext;
+ }, implicitRelative, true ),
+ matchAnyContext = addCombinator( function( elem ) {
+ return indexOf( checkContext, elem ) > -1;
+ }, implicitRelative, true ),
+ matchers = [ function( elem, context, xml ) {
+ var ret = ( !leadingRelative && ( xml || context !== outermostContext ) ) || (
+ ( checkContext = context ).nodeType ?
+ matchContext( elem, context, xml ) :
+ matchAnyContext( elem, context, xml ) );
+
+ // Avoid hanging onto element (issue #299)
+ checkContext = null;
+ return ret;
+ } ];
+
+ for ( ; i < len; i++ ) {
+ if ( ( matcher = Expr.relative[ tokens[ i ].type ] ) ) {
+ matchers = [ addCombinator( elementMatcher( matchers ), matcher ) ];
+ } else {
+ matcher = Expr.filter[ tokens[ i ].type ].apply( null, tokens[ i ].matches );
+
+ // Return special upon seeing a positional matcher
+ if ( matcher[ expando ] ) {
+
+ // Find the next relative operator (if any) for proper handling
+ j = ++i;
+ for ( ; j < len; j++ ) {
+ if ( Expr.relative[ tokens[ j ].type ] ) {
+ break;
+ }
+ }
+ return setMatcher(
+ i > 1 && elementMatcher( matchers ),
+ i > 1 && toSelector(
+
+ // If the preceding token was a descendant combinator, insert an implicit any-element `*`
+ tokens
+ .slice( 0, i - 1 )
+ .concat( { value: tokens[ i - 2 ].type === " " ? "*" : "" } )
+ ).replace( rtrim, "$1" ),
+ matcher,
+ i < j && matcherFromTokens( tokens.slice( i, j ) ),
+ j < len && matcherFromTokens( ( tokens = tokens.slice( j ) ) ),
+ j < len && toSelector( tokens )
+ );
+ }
+ matchers.push( matcher );
+ }
+ }
+
+ return elementMatcher( matchers );
+}
+
+function matcherFromGroupMatchers( elementMatchers, setMatchers ) {
+ var bySet = setMatchers.length > 0,
+ byElement = elementMatchers.length > 0,
+ superMatcher = function( seed, context, xml, results, outermost ) {
+ var elem, j, matcher,
+ matchedCount = 0,
+ i = "0",
+ unmatched = seed && [],
+ setMatched = [],
+ contextBackup = outermostContext,
+
+ // We must always have either seed elements or outermost context
+ elems = seed || byElement && Expr.find[ "TAG" ]( "*", outermost ),
+
+ // Use integer dirruns iff this is the outermost matcher
+ dirrunsUnique = ( dirruns += contextBackup == null ? 1 : Math.random() || 0.1 ),
+ len = elems.length;
+
+ if ( outermost ) {
+
+ // Support: IE 11+, Edge 17 - 18+
+ // IE/Edge sometimes throw a "Permission denied" error when strict-comparing
+ // two documents; shallow comparisons work.
+ // eslint-disable-next-line eqeqeq
+ outermostContext = context == document || context || outermost;
+ }
+
+ // Add elements passing elementMatchers directly to results
+ // Support: IE<9, Safari
+ // Tolerate NodeList properties (IE: "length"; Safari: ) matching elements by id
+ for ( ; i !== len && ( elem = elems[ i ] ) != null; i++ ) {
+ if ( byElement && elem ) {
+ j = 0;
+
+ // Support: IE 11+, Edge 17 - 18+
+ // IE/Edge sometimes throw a "Permission denied" error when strict-comparing
+ // two documents; shallow comparisons work.
+ // eslint-disable-next-line eqeqeq
+ if ( !context && elem.ownerDocument != document ) {
+ setDocument( elem );
+ xml = !documentIsHTML;
+ }
+ while ( ( matcher = elementMatchers[ j++ ] ) ) {
+ if ( matcher( elem, context || document, xml ) ) {
+ results.push( elem );
+ break;
+ }
+ }
+ if ( outermost ) {
+ dirruns = dirrunsUnique;
+ }
+ }
+
+ // Track unmatched elements for set filters
+ if ( bySet ) {
+
+ // They will have gone through all possible matchers
+ if ( ( elem = !matcher && elem ) ) {
+ matchedCount--;
+ }
+
+ // Lengthen the array for every element, matched or not
+ if ( seed ) {
+ unmatched.push( elem );
+ }
+ }
+ }
+
+ // `i` is now the count of elements visited above, and adding it to `matchedCount`
+ // makes the latter nonnegative.
+ matchedCount += i;
+
+ // Apply set filters to unmatched elements
+ // NOTE: This can be skipped if there are no unmatched elements (i.e., `matchedCount`
+ // equals `i`), unless we didn't visit _any_ elements in the above loop because we have
+ // no element matchers and no seed.
+ // Incrementing an initially-string "0" `i` allows `i` to remain a string only in that
+ // case, which will result in a "00" `matchedCount` that differs from `i` but is also
+ // numerically zero.
+ if ( bySet && i !== matchedCount ) {
+ j = 0;
+ while ( ( matcher = setMatchers[ j++ ] ) ) {
+ matcher( unmatched, setMatched, context, xml );
+ }
+
+ if ( seed ) {
+
+ // Reintegrate element matches to eliminate the need for sorting
+ if ( matchedCount > 0 ) {
+ while ( i-- ) {
+ if ( !( unmatched[ i ] || setMatched[ i ] ) ) {
+ setMatched[ i ] = pop.call( results );
+ }
+ }
+ }
+
+ // Discard index placeholder values to get only actual matches
+ setMatched = condense( setMatched );
+ }
+
+ // Add matches to results
+ push.apply( results, setMatched );
+
+ // Seedless set matches succeeding multiple successful matchers stipulate sorting
+ if ( outermost && !seed && setMatched.length > 0 &&
+ ( matchedCount + setMatchers.length ) > 1 ) {
+
+ Sizzle.uniqueSort( results );
+ }
+ }
+
+ // Override manipulation of globals by nested matchers
+ if ( outermost ) {
+ dirruns = dirrunsUnique;
+ outermostContext = contextBackup;
+ }
+
+ return unmatched;
+ };
+
+ return bySet ?
+ markFunction( superMatcher ) :
+ superMatcher;
+}
+
+compile = Sizzle.compile = function( selector, match /* Internal Use Only */ ) {
+ var i,
+ setMatchers = [],
+ elementMatchers = [],
+ cached = compilerCache[ selector + " " ];
+
+ if ( !cached ) {
+
+ // Generate a function of recursive functions that can be used to check each element
+ if ( !match ) {
+ match = tokenize( selector );
+ }
+ i = match.length;
+ while ( i-- ) {
+ cached = matcherFromTokens( match[ i ] );
+ if ( cached[ expando ] ) {
+ setMatchers.push( cached );
+ } else {
+ elementMatchers.push( cached );
+ }
+ }
+
+ // Cache the compiled function
+ cached = compilerCache(
+ selector,
+ matcherFromGroupMatchers( elementMatchers, setMatchers )
+ );
+
+ // Save selector and tokenization
+ cached.selector = selector;
+ }
+ return cached;
+};
+
+/**
+ * A low-level selection function that works with Sizzle's compiled
+ * selector functions
+ * @param {String|Function} selector A selector or a pre-compiled
+ * selector function built with Sizzle.compile
+ * @param {Element} context
+ * @param {Array} [results]
+ * @param {Array} [seed] A set of elements to match against
+ */
+select = Sizzle.select = function( selector, context, results, seed ) {
+ var i, tokens, token, type, find,
+ compiled = typeof selector === "function" && selector,
+ match = !seed && tokenize( ( selector = compiled.selector || selector ) );
+
+ results = results || [];
+
+ // Try to minimize operations if there is only one selector in the list and no seed
+ // (the latter of which guarantees us context)
+ if ( match.length === 1 ) {
+
+ // Reduce context if the leading compound selector is an ID
+ tokens = match[ 0 ] = match[ 0 ].slice( 0 );
+ if ( tokens.length > 2 && ( token = tokens[ 0 ] ).type === "ID" &&
+ context.nodeType === 9 && documentIsHTML && Expr.relative[ tokens[ 1 ].type ] ) {
+
+ context = ( Expr.find[ "ID" ]( token.matches[ 0 ]
+ .replace( runescape, funescape ), context ) || [] )[ 0 ];
+ if ( !context ) {
+ return results;
+
+ // Precompiled matchers will still verify ancestry, so step up a level
+ } else if ( compiled ) {
+ context = context.parentNode;
+ }
+
+ selector = selector.slice( tokens.shift().value.length );
+ }
+
+ // Fetch a seed set for right-to-left matching
+ i = matchExpr[ "needsContext" ].test( selector ) ? 0 : tokens.length;
+ while ( i-- ) {
+ token = tokens[ i ];
+
+ // Abort if we hit a combinator
+ if ( Expr.relative[ ( type = token.type ) ] ) {
+ break;
+ }
+ if ( ( find = Expr.find[ type ] ) ) {
+
+ // Search, expanding context for leading sibling combinators
+ if ( ( seed = find(
+ token.matches[ 0 ].replace( runescape, funescape ),
+ rsibling.test( tokens[ 0 ].type ) && testContext( context.parentNode ) ||
+ context
+ ) ) ) {
+
+ // If seed is empty or no tokens remain, we can return early
+ tokens.splice( i, 1 );
+ selector = seed.length && toSelector( tokens );
+ if ( !selector ) {
+ push.apply( results, seed );
+ return results;
+ }
+
+ break;
+ }
+ }
+ }
+ }
+
+ // Compile and execute a filtering function if one is not provided
+ // Provide `match` to avoid retokenization if we modified the selector above
+ ( compiled || compile( selector, match ) )(
+ seed,
+ context,
+ !documentIsHTML,
+ results,
+ !context || rsibling.test( selector ) && testContext( context.parentNode ) || context
+ );
+ return results;
+};
// One-time assignments
// Sort stability
- support.sortStable = expando.split("").sort( sortOrder ).join("") === expando;
+support.sortStable = expando.split( "" ).sort( sortOrder ).join( "" ) === expando;
// Support: Chrome 14-35+
// Always assume duplicates if they aren't passed to the comparison function
- support.detectDuplicates = !!hasDuplicate;
+support.detectDuplicates = !!hasDuplicate;
// Initialize against the default document
- setDocument();
+setDocument();
// Support: Webkit<537.32 - Safari 6.0.3/Chrome 25 (fixed in Chrome 27)
// Detached nodes confoundingly follow *each other*
- support.sortDetached = assert(function( div1 ) {
- // Should return 1, but returns 4 (following)
- return div1.compareDocumentPosition( document.createElement("div") ) & 1;
- });
+support.sortDetached = assert( function( el ) {
+
+ // Should return 1, but returns 4 (following)
+ return el.compareDocumentPosition( document.createElement( "fieldset" ) ) & 1;
+} );
// Support: IE<8
// Prevent attribute/property "interpolation"
-// http://msdn.microsoft.com/en-us/library/ms536429%28VS.85%29.aspx
- if ( !assert(function( div ) {
- div.innerHTML = "";
- return div.firstChild.getAttribute("href") === "#" ;
- }) ) {
- addHandle( "type|href|height|width", function( elem, name, isXML ) {
- if ( !isXML ) {
- return elem.getAttribute( name, name.toLowerCase() === "type" ? 1 : 2 );
- }
- });
- }
+// https://msdn.microsoft.com/en-us/library/ms536429%28VS.85%29.aspx
+if ( !assert( function( el ) {
+ el.innerHTML = "";
+ return el.firstChild.getAttribute( "href" ) === "#";
+} ) ) {
+ addHandle( "type|href|height|width", function( elem, name, isXML ) {
+ if ( !isXML ) {
+ return elem.getAttribute( name, name.toLowerCase() === "type" ? 1 : 2 );
+ }
+ } );
+}
// Support: IE<9
// Use defaultValue in place of getAttribute("value")
- if ( !support.attributes || !assert(function( div ) {
- div.innerHTML = "";
- div.firstChild.setAttribute( "value", "" );
- return div.firstChild.getAttribute( "value" ) === "";
- }) ) {
- addHandle( "value", function( elem, name, isXML ) {
- if ( !isXML && elem.nodeName.toLowerCase() === "input" ) {
- return elem.defaultValue;
- }
- });
- }
+if ( !support.attributes || !assert( function( el ) {
+ el.innerHTML = "";
+ el.firstChild.setAttribute( "value", "" );
+ return el.firstChild.getAttribute( "value" ) === "";
+} ) ) {
+ addHandle( "value", function( elem, _name, isXML ) {
+ if ( !isXML && elem.nodeName.toLowerCase() === "input" ) {
+ return elem.defaultValue;
+ }
+ } );
+}
// Support: IE<9
// Use getAttributeNode to fetch booleans when getAttribute lies
- if ( !assert(function( div ) {
- return div.getAttribute("disabled") == null;
- }) ) {
- addHandle( booleans, function( elem, name, isXML ) {
- var val;
- if ( !isXML ) {
- return elem[ name ] === true ? name.toLowerCase() :
- (val = elem.getAttributeNode( name )) && val.specified ?
- val.value :
- null;
- }
- });
- }
+if ( !assert( function( el ) {
+ return el.getAttribute( "disabled" ) == null;
+} ) ) {
+ addHandle( booleans, function( elem, name, isXML ) {
+ var val;
+ if ( !isXML ) {
+ return elem[ name ] === true ? name.toLowerCase() :
+ ( val = elem.getAttributeNode( name ) ) && val.specified ?
+ val.value :
+ null;
+ }
+ } );
+}
// EXPOSE
- if ( typeof define === "function" && define.amd ) {
- define(function() { return Sizzle; });
+var _sizzle = window.Sizzle;
+
+Sizzle.noConflict = function() {
+ if ( window.Sizzle === Sizzle ) {
+ window.Sizzle = _sizzle;
+ }
+
+ return Sizzle;
+};
+
+if ( typeof define === "function" && define.amd ) {
+ define( function() {
+ return Sizzle;
+ } );
+
// Sizzle requires that there be a global window in Common-JS like environments
- } else if ( typeof module !== "undefined" && module.exports ) {
- module.exports = Sizzle;
- } else {
- window.Sizzle = Sizzle;
- }
+} else if ( typeof module !== "undefined" && module.exports ) {
+ module.exports = Sizzle;
+} else {
+ window.Sizzle = Sizzle;
+}
+
// EXPOSE
-})( window );
+} )( window );
\ No newline at end of file
diff --git a/web/src/main/scala/com/tribbloids/spookystuff/web/actions/Selector.scala b/web/src/main/scala/com/tribbloids/spookystuff/web/actions/Selector.scala
index f21192daa..6435932d9 100644
--- a/web/src/main/scala/com/tribbloids/spookystuff/web/actions/Selector.scala
+++ b/web/src/main/scala/com/tribbloids/spookystuff/web/actions/Selector.scala
@@ -1,8 +1,9 @@
package com.tribbloids.spookystuff.web.actions
-import com.tribbloids.spookystuff.selenium.BySizzleCssSelector
-import org.apache.spark.ml.dsl.utils.refl.ScalaUDT
+import com.tribbloids.spookystuff.selenium.BySizzleSelector
+import com.tribbloids.spookystuff.utils.IDMixin
import org.apache.spark.ml.dsl.utils.messaging.MessageRelay
+import org.apache.spark.ml.dsl.utils.refl.ScalaUDT
import org.apache.spark.sql.types.SQLUserDefinedType
import org.openqa.selenium.By
@@ -12,6 +13,8 @@ object Selector extends MessageRelay[Selector] {
final val SCHEMA = "By.sizzleCssSelector"
+ def bySizzle(v: String) = new BySizzleSelector(v)
+
final val factories: Seq[String => By] = {
Seq(
By.id(_),
@@ -22,7 +25,7 @@ object Selector extends MessageRelay[Selector] {
By.tagName(_),
By.xpath(_),
By.partialLinkText(_),
- v => new BySizzleCssSelector(v)
+ bySizzle(_)
)
}
final val factory_patterns = factories.map { fn =>
@@ -35,24 +38,28 @@ object Selector extends MessageRelay[Selector] {
for (tuple <- factory_patterns) {
val pattern = tuple._2
withPrefix match {
- case pattern(selector) =>
- return Selector(tuple._1(selector))
+ case pattern(str) =>
+ return Selector(tuple._1, str)
case _ =>
}
}
- Selector(new BySizzleCssSelector(v))
+ Selector(bySizzle, v)
}
override type M = String
- override def toMessage_>>(v: Selector) = v.toString
+ override def toMessage_>>(v: Selector): String = v.toString
}
class SelectorUDT extends ScalaUDT[Selector]
@SQLUserDefinedType(udt = classOf[SelectorUDT])
-case class Selector(by: By) {
+case class Selector(factory: String => By, pattern: String) extends IDMixin {
+
+ @transient lazy val by: By = factory(pattern)
override def toString: String = by.toString
+
+ override lazy val _id: By = by
}
diff --git a/web/src/main/scala/com/tribbloids/spookystuff/web/actions/WebInteraction.scala b/web/src/main/scala/com/tribbloids/spookystuff/web/actions/WebInteraction.scala
index 379ae6827..3ed292dbb 100644
--- a/web/src/main/scala/com/tribbloids/spookystuff/web/actions/WebInteraction.scala
+++ b/web/src/main/scala/com/tribbloids/spookystuff/web/actions/WebInteraction.scala
@@ -47,7 +47,6 @@ abstract class WebInteraction(
new org.openqa.selenium.interactions.Actions(session.driverOf(Web))
}
-
}
object WebInteraction {
diff --git a/web/src/main/scala/com/tribbloids/spookystuff/web/actions/WebTimed.scala b/web/src/main/scala/com/tribbloids/spookystuff/web/actions/WebTimed.scala
index 02950b1fa..d3fb86457 100644
--- a/web/src/main/scala/com/tribbloids/spookystuff/web/actions/WebTimed.scala
+++ b/web/src/main/scala/com/tribbloids/spookystuff/web/actions/WebTimed.scala
@@ -6,12 +6,17 @@ import com.tribbloids.spookystuff.web.conf.Web
import org.openqa.selenium.WebElement
import org.openqa.selenium.support.ui.{ExpectedConditions, WebDriverWait}
+import java.time.Duration
import java.util
+import scala.language.implicitConversions
+
trait WebTimed extends WebAction with Timed {
+ implicit def nanos2JDuration(v: Long): Duration = java.time.Duration.ofNanos(v)
+
def webDriverWait(session: Session): WebDriverWait =
- new WebDriverWait(session.driverOf(Web), this.timeout(session).max.toSeconds)
+ new WebDriverWait(session.driverOf(Web), this.timeout(session).max.toNanos)
def getClickableElement(selector: Selector, session: Session): WebElement = {
diff --git a/web/src/main/scala/com/tribbloids/spookystuff/web/conf/DesiredCapabilitiesView.scala b/web/src/main/scala/com/tribbloids/spookystuff/web/conf/DesiredCapabilitiesView.scala
new file mode 100644
index 000000000..19c59cb3c
--- /dev/null
+++ b/web/src/main/scala/com/tribbloids/spookystuff/web/conf/DesiredCapabilitiesView.scala
@@ -0,0 +1,106 @@
+package com.tribbloids.spookystuff.web.conf
+
+import com.tribbloids.spookystuff.SpookyContext
+import com.tribbloids.spookystuff.web.conf.WebDriverFactory.asSeleniumProxy
+import org.openqa.selenium.Platform
+import org.openqa.selenium.htmlunit.HtmlUnitDriver
+import org.openqa.selenium.logging.LoggingPreferences
+import org.openqa.selenium.phantomjs.PhantomJSDriverService
+import org.openqa.selenium.remote.CapabilityType.{ACCEPT_SSL_CERTS, PROXY, SUPPORTS_ALERTS, TAKES_SCREENSHOT}
+import org.openqa.selenium.remote.{CapabilityType, DesiredCapabilities}
+
+import scala.language.implicitConversions
+
+object DesiredCapabilitiesView {
+
+ object Defaults {
+ lazy val logPrefs: LoggingPreferences = {
+
+ val result = new LoggingPreferences()
+
+// result.enable(LogType.BROWSER, Level.SEVERE)
+ result
+ }
+
+ lazy val caps: DesiredCapabilities = {
+
+ val caps = new DesiredCapabilities()
+ caps.setPlatform(Platform.ANY)
+
+ caps.setJavascriptEnabled(true); //< not really needed: JS enabled by default
+
+ caps.setCapability(CapabilityType.LOGGING_PREFS, logPrefs)
+
+ // baseCaps.setCapability(CapabilityType.SUPPORTS_FINDING_BY_CSS, true)
+ // baseCaps.setCapability(CapabilityType.HAS_NATIVE_EVENTS, false)
+ caps.setCapability(TAKES_SCREENSHOT, true)
+ caps.setCapability(ACCEPT_SSL_CERTS, true)
+ caps.setCapability(SUPPORTS_ALERTS, true)
+ caps
+ }
+ }
+
+ lazy val default: DesiredCapabilitiesView = DesiredCapabilitiesView(Defaults.caps)
+
+ implicit def unbox(v: DesiredCapabilitiesView): DesiredCapabilities = v.self
+}
+
+case class DesiredCapabilitiesView(
+ self: DesiredCapabilities
+) {
+
+ def fork(): DesiredCapabilitiesView = this.copy(new DesiredCapabilities(self))
+
+ def configure(fn: DesiredCapabilities => Unit): this.type = {
+ fn(self)
+ this
+ }
+
+ case class Imported(
+ spooky: SpookyContext,
+ loadImage: Boolean = true
+ ) {
+
+ def base: DesiredCapabilitiesView = {
+ fork().configure { caps =>
+ val proxyOpt = Option(spooky.spookyConf.webProxy()).map { v =>
+ asSeleniumProxy(v)
+ }
+
+ proxyOpt.foreach { proxy =>
+ caps.setCapability(PROXY, proxy)
+ }
+
+ }
+ }
+
+ def htmlUnit: DesiredCapabilitiesView = {
+ base.configure { caps =>
+ caps.setBrowserName("htmlunit")
+
+ caps.setCapability(HtmlUnitDriver.DOWNLOAD_IMAGES_CAPABILITY, loadImage)
+ }
+ }
+
+ def phantomJS: DesiredCapabilitiesView = {
+ base.configure { caps =>
+ caps.setBrowserName("phantomjs")
+
+ caps.setCapability(PhantomJSDriverService.PHANTOMJS_PAGE_SETTINGS_PREFIX + "loadImages", loadImage)
+
+ caps.setCapability(
+ PhantomJSDriverService.PHANTOMJS_PAGE_SETTINGS_PREFIX + "resourceTimeout",
+ spooky.spookyConf.remoteResourceTimeout.max.toMillis
+ )
+
+ val headersOpt = Option(spooky.spookyConf.httpHeadersFactory).flatMap(v => Option(v.apply()))
+ headersOpt.foreach { headers =>
+ headers.foreach {
+ case (k, v) =>
+ caps.setCapability(PhantomJSDriverService.PHANTOMJS_PAGE_SETTINGS_PREFIX + k, v)
+ }
+ }
+ }
+ }
+ }
+}
diff --git a/web/src/main/scala/com/tribbloids/spookystuff/web/conf/WebDriverFactory.scala b/web/src/main/scala/com/tribbloids/spookystuff/web/conf/WebDriverFactory.scala
index 16dc75dff..703dc64cb 100644
--- a/web/src/main/scala/com/tribbloids/spookystuff/web/conf/WebDriverFactory.scala
+++ b/web/src/main/scala/com/tribbloids/spookystuff/web/conf/WebDriverFactory.scala
@@ -7,14 +7,12 @@ import com.tribbloids.spookystuff.dsl.BinaryDeployment
import com.tribbloids.spookystuff.session.{Session, WebProxySetting}
import com.tribbloids.spookystuff.utils.ConfUtils
import com.tribbloids.spookystuff.utils.io.LocalResolver
-import com.tribbloids.spookystuff.utils.lifespan.Lifespan
+import com.tribbloids.spookystuff.utils.lifespan.Cleanable.Lifespan
import com.tribbloids.spookystuff.web.session.CleanWebDriver
import org.apache.commons.io.FileUtils
+import org.openqa.selenium.Proxy
import org.openqa.selenium.htmlunit.HtmlUnitDriver
import org.openqa.selenium.phantomjs.{PhantomJSDriver, PhantomJSDriverService}
-import org.openqa.selenium.remote.CapabilityType._
-import org.openqa.selenium.remote.{BrowserType, CapabilityType, DesiredCapabilities}
-import org.openqa.selenium.{Capabilities, Platform, Proxy}
import java.io.File
import scala.util.Try
@@ -26,58 +24,33 @@ abstract class WebDriverFactory extends DriverFactory.Transient[CleanWebDriver]
override def factoryReset(driver: CleanWebDriver): Unit = {
driver.get("about:blank")
}
-
- def importHeaders(caps: DesiredCapabilities, spooky: SpookyContext): Unit = {
- val headersOpt = Option(spooky.spookyConf.httpHeadersFactory).flatMap(v => Option(v.apply()))
- headersOpt.foreach { headers =>
- headers.foreach {
- case (k, v) =>
- caps.setCapability(PhantomJSDriverService.PHANTOMJS_PAGE_SETTINGS_PREFIX + k, v)
- }
- }
- }
}
object WebDriverFactory {
import com.tribbloids.spookystuff.utils.CommonViews._
+ def asSeleniumProxy(s: WebProxySetting): Proxy = {
+ val seleniumProxy: Proxy = new Proxy
+ seleniumProxy.setProxyType(Proxy.ProxyType.MANUAL)
+ val proxyStr: String = s"${s.addr}:${s.port}"
+ seleniumProxy.setHttpProxy(proxyStr)
+ seleniumProxy.setSslProxy(proxyStr)
+ seleniumProxy.setSocksProxy(proxyStr)
+ seleniumProxy
+ }
+
case class HtmlUnit(
- browser: BrowserVersion = BrowserVersion.getDefault
+ browserV: BrowserVersion = BrowserVersion.getDefault
) extends WebDriverFactory {
- @transient lazy val baseCaps: DesiredCapabilities = new DesiredCapabilities(BrowserType.HTMLUNIT, "", Platform.ANY)
-
- def toSeleniumProxy(s: WebProxySetting): Proxy = {
- val seleniumProxy: Proxy = new Proxy
- seleniumProxy.setProxyType(Proxy.ProxyType.MANUAL)
- val proxyStr: String = s"${s.addr}:${s.port}"
- seleniumProxy.setHttpProxy(proxyStr)
- seleniumProxy.setSslProxy(proxyStr)
- seleniumProxy.setSocksProxy(proxyStr)
- seleniumProxy
- }
-
- def newCaps(capabilities: Capabilities, spooky: SpookyContext): DesiredCapabilities = {
- val newCaps = new DesiredCapabilities(baseCaps)
-
- importHeaders(newCaps, spooky)
-
- val proxy: WebProxySetting = spooky.spookyConf.webProxy()
-
- if (proxy != null) {
- newCaps.setCapability(PROXY, toSeleniumProxy(proxy))
- }
-
- newCaps.merge(capabilities)
- }
-
override def _createImpl(session: Session, lifespan: Lifespan): CleanWebDriver = {
- val cap = newCaps(null, session.spooky)
- val self = new HtmlUnitDriver(browser)
+ val caps = DesiredCapabilitiesView.default.Imported(session.spooky).htmlUnit
+
+ val self = new HtmlUnitDriver(browserV)
self.setJavascriptEnabled(true)
- self.setProxySettings(Proxy.extractFrom(cap))
+ self.setProxySettings(Proxy.extractFrom(caps))
val driver = new CleanWebDriver(self, lifespan)
driver
@@ -115,6 +88,17 @@ object WebDriverFactory {
val dstFile = new File(dst)
FileUtils.forceDelete(dstFile)
}
+
+ lazy val defaultBuilder: PhantomJSDriverService.Builder = {
+
+ new PhantomJSDriverService.Builder()
+ .withLogFile(new File("target/logs/phantomjsdriver.log"))
+// .withLogFile(new File("/dev/null"))
+ .usingCommandLineArguments(Array("--webdriver-loglevel=ERROR"))
+// .usingCommandLineArguments(Array.empty)
+// .usingGhostDriverCommandLineArguments(Array("service_log_path", "/tmp/ghostdriver.log"))
+ .usingGhostDriverCommandLineArguments(Array.empty)
+ }
}
case class PhantomJS(
@@ -129,50 +113,43 @@ object WebDriverFactory {
deployment.OnDriver(spooky.sparkContext).deployOnce
}
- @transient lazy val baseCaps: DesiredCapabilities = {
- val baseCaps = new DesiredCapabilities(BrowserType.PHANTOMJS, "", Platform.ANY)
-
- baseCaps.setJavascriptEnabled(true); //< not really needed: JS enabled by default
- baseCaps.setCapability(CapabilityType.SUPPORTS_FINDING_BY_CSS, true)
- // baseCaps.setCapability(CapabilityType.HAS_NATIVE_EVENTS, false)
- baseCaps.setCapability(TAKES_SCREENSHOT, true)
- baseCaps.setCapability(ACCEPT_SSL_CERTS, true)
- baseCaps.setCapability(SUPPORTS_ALERTS, true)
- baseCaps.setCapability(PhantomJSDriverService.PHANTOMJS_PAGE_SETTINGS_PREFIX + "loadImages", loadImages)
- baseCaps
- }
+ //called from executors
+ override def _createImpl(session: Session, lifespan: Lifespan): CleanWebDriver = {
- // baseCaps.setCapability(PhantomJSDriverService.PHANTOMJS_PAGE_SETTINGS_PREFIX+"resourceTimeout", Const.resourceTimeout*1000)
+ val deployment = deploy(session.spooky)
- def newCaps(spooky: SpookyContext, extra: Option[Capabilities] = None): DesiredCapabilities = {
- val newCaps = new DesiredCapabilities(baseCaps)
+ val caps = DesiredCapabilitiesView.default.Imported(session.spooky).phantomJS
+ caps.setCapability(PhantomJSDriverService.PHANTOMJS_EXECUTABLE_PATH_PROPERTY, deployment.verifiedLocalPath)
- val deployment = deploy(spooky)
+ lazy val service: PhantomJSDriverService = {
- val pathStr = deployment.verifiedLocalPath
+ val deployment = deploy(session.spooky)
+ val pathStr = deployment.verifiedLocalPath
- newCaps.setCapability(PhantomJSDriverService.PHANTOMJS_EXECUTABLE_PATH_PROPERTY, pathStr)
- newCaps.setCapability(
- PhantomJSDriverService.PHANTOMJS_PAGE_SETTINGS_PREFIX + "resourceTimeout",
- spooky.spookyConf.remoteResourceTimeout.max.toMillis
- )
- importHeaders(newCaps, spooky)
+ val proxyOpt = Option(session.spooky.spookyConf.webProxy()).map { v =>
+ asSeleniumProxy(v)
+ }
- val proxy = spooky.spookyConf.webProxy()
+ import scala.collection.JavaConverters._
- if (proxy != null)
- newCaps.setCapability(
- PhantomJSDriverService.PHANTOMJS_CLI_ARGS,
- Array("--proxy=" + proxy.addr + ":" + proxy.port, "--proxy-type=" + proxy.protocol)
- )
+ var builder = PhantomJS.defaultBuilder
+ .usingAnyFreePort()
+ .usingPhantomJSExecutable(new File(pathStr))
+ .withEnvironment(
+ Map(
+ "OPENSSL_CONF" -> "/dev/null" //https://github.com/bazelbuild/rules_closure/issues/351
+ ).asJava
+ )
- newCaps.merge(extra.orNull)
- }
+ proxyOpt.foreach { proxy =>
+ builder = builder.withProxy(proxy)
+ }
- //called from executors
- override def _createImpl(session: Session, lifespan: Lifespan): CleanWebDriver = {
- val caps = newCaps(session.spooky)
- val self = new PhantomJSDriver(caps)
+ builder.build
+ }
+
+// val self = new PhantomJSDriver(caps)
+ val self = new PhantomJSDriver(service, caps)
new CleanWebDriver(self, lifespan)
}
}
diff --git a/web/src/main/scala/com/tribbloids/spookystuff/web/session/CleanWebDriver.scala b/web/src/main/scala/com/tribbloids/spookystuff/web/session/CleanWebDriver.scala
index a0d312ec0..7d5638c6c 100644
--- a/web/src/main/scala/com/tribbloids/spookystuff/web/session/CleanWebDriver.scala
+++ b/web/src/main/scala/com/tribbloids/spookystuff/web/session/CleanWebDriver.scala
@@ -1,9 +1,12 @@
package com.tribbloids.spookystuff.web.session
import com.tribbloids.spookystuff.session.DriverLike
-import com.tribbloids.spookystuff.utils.lifespan.Lifespan
+import com.tribbloids.spookystuff.utils.lifespan.Cleanable.Lifespan
+import com.tribbloids.spookystuff.utils.{CommonConst, CommonUtils}
import org.openqa.selenium.{NoSuchSessionException, WebDriver}
+import org.slf4j.LoggerFactory
+import java.net.ConnectException
import scala.language.implicitConversions
object CleanWebDriver {
@@ -13,12 +16,31 @@ object CleanWebDriver {
class CleanWebDriver(
val self: WebDriver,
- override val _lifespan: Lifespan = Lifespan.TaskOrJVM()
+ override val _lifespan: Lifespan = Lifespan.TaskOrJVM().forShipping
) extends DriverLike {
override def cleanImpl(): Unit = {
- self.close()
- self.quit()
+ try {
+ CommonUtils.retry(CommonConst.driverClosingRetries) {
+ CommonUtils.withTimeout(CommonConst.driverClosingTimeout) {
+
+ self.close()
+ }
+ Thread.sleep(1000)
+
+ }
+ } catch {
+ case e: Throwable =>
+ LoggerFactory.getLogger(this.getClass).error("Failed to close ... will quit directly", e)
+ }
+
+ CommonUtils.retry(CommonConst.driverClosingRetries) {
+ CommonUtils.withTimeout(CommonConst.driverClosingTimeout) {
+
+ self.quit()
+ }
+ Thread.sleep(1000)
+ }
}
override def silentOnError(ee: Throwable): Boolean = {
diff --git a/web/src/test/scala/com/tribbloids/spookystuff/web/actions/AbstractTestTrace.scala b/web/src/test/scala/com/tribbloids/spookystuff/web/actions/AbstractTestTrace.scala
index 7b7cc454b..17f87f55e 100644
--- a/web/src/test/scala/com/tribbloids/spookystuff/web/actions/AbstractTestTrace.scala
+++ b/web/src/test/scala/com/tribbloids/spookystuff/web/actions/AbstractTestTrace.scala
@@ -4,7 +4,7 @@ import com.tribbloids.spookystuff.SpookyEnvFixture
import com.tribbloids.spookystuff.actions.{Delay, Loop, OAuthV2, TraceView, Wget}
import com.tribbloids.spookystuff.conf.DriverFactory
import com.tribbloids.spookystuff.doc.Doc
-import com.tribbloids.spookystuff.session.{AbstractSession, Session}
+import com.tribbloids.spookystuff.session.Session
import com.tribbloids.spookystuff.testutils.FunSpecx
import com.tribbloids.spookystuff.web.session.CleanWebDriver
@@ -97,7 +97,7 @@ abstract class AbstractTestTrace extends SpookyEnvFixture with FunSpecx {
+> TextInput("box", "something")
+> Snapshot()
+> WebDocIf(
- { (v: Doc, _: AbstractSession) =>
+ { (v: Doc, _: Session) =>
v.uri startsWith "http"
},
Click("o1")
diff --git a/web/src/test/scala/com/tribbloids/spookystuff/web/actions/TestTrace_PoolingHtmlUnit.scala b/web/src/test/scala/com/tribbloids/spookystuff/web/actions/TestTrace_HtmlUnit_TaskLocal.scala
similarity index 74%
rename from web/src/test/scala/com/tribbloids/spookystuff/web/actions/TestTrace_PoolingHtmlUnit.scala
rename to web/src/test/scala/com/tribbloids/spookystuff/web/actions/TestTrace_HtmlUnit_TaskLocal.scala
index c2d38fd33..33989cff1 100644
--- a/web/src/test/scala/com/tribbloids/spookystuff/web/actions/TestTrace_PoolingHtmlUnit.scala
+++ b/web/src/test/scala/com/tribbloids/spookystuff/web/actions/TestTrace_HtmlUnit_TaskLocal.scala
@@ -2,7 +2,7 @@ package com.tribbloids.spookystuff.web.actions
import com.tribbloids.spookystuff.web.conf.WebDriverFactory
-class TestTrace_PoolingHtmlUnit extends AbstractTestTrace {
+class TestTrace_HtmlUnit_TaskLocal extends AbstractTestTrace {
override lazy val driverFactory = WebDriverFactory.HtmlUnit().taskLocal
}
diff --git a/web/src/test/scala/com/tribbloids/spookystuff/web/actions/TestTrace_PoolingPhantomJS.scala b/web/src/test/scala/com/tribbloids/spookystuff/web/actions/TestTrace_PhantomJS_TaskLocal.scala
similarity index 74%
rename from web/src/test/scala/com/tribbloids/spookystuff/web/actions/TestTrace_PoolingPhantomJS.scala
rename to web/src/test/scala/com/tribbloids/spookystuff/web/actions/TestTrace_PhantomJS_TaskLocal.scala
index abd9a41b3..2f9de1d57 100644
--- a/web/src/test/scala/com/tribbloids/spookystuff/web/actions/TestTrace_PoolingPhantomJS.scala
+++ b/web/src/test/scala/com/tribbloids/spookystuff/web/actions/TestTrace_PhantomJS_TaskLocal.scala
@@ -2,7 +2,7 @@ package com.tribbloids.spookystuff.web.actions
import com.tribbloids.spookystuff.web.conf.WebDriverFactory
-class TestTrace_PoolingPhantomJS extends AbstractTestTrace {
+class TestTrace_PhantomJS_TaskLocal extends AbstractTestTrace {
override lazy val driverFactory = WebDriverFactory.PhantomJS().taskLocal
}
diff --git a/web/src/test/scala/com/tribbloids/spookystuff/web/conf/DriverFactorySpec.scala b/web/src/test/scala/com/tribbloids/spookystuff/web/conf/DriverFactorySpec.scala
index 1847506b3..793587511 100644
--- a/web/src/test/scala/com/tribbloids/spookystuff/web/conf/DriverFactorySpec.scala
+++ b/web/src/test/scala/com/tribbloids/spookystuff/web/conf/DriverFactorySpec.scala
@@ -1,7 +1,7 @@
package com.tribbloids.spookystuff.web.conf
import com.tribbloids.spookystuff.conf.DriverFactory.Transient
-import com.tribbloids.spookystuff.conf.{PluginSystem, Python, PythonDriverFactory}
+import com.tribbloids.spookystuff.conf.{DriverFactory, PluginSystem, Python, PythonDriverFactory}
import com.tribbloids.spookystuff.session.Session
import com.tribbloids.spookystuff.testutils.LocalPathDocsFixture
import com.tribbloids.spookystuff.web.actions.Visit
@@ -16,6 +16,8 @@ class DriverFactorySpec extends SpookyEnvFixture with LocalPathDocsFixture {
def driverFactory: Transient[Driver]
+ lazy val taskLocalDriverFactory: DriverFactory.TaskLocal[Driver] = driverFactory.taskLocal
+
it(s"$driverFactory can factoryReset") {
val session = new Session(spooky)
val driver = driverFactory.dispatch(session)
@@ -42,10 +44,16 @@ class DriverFactorySpec extends SpookyEnvFixture with LocalPathDocsFixture {
override lazy val driverFactory = WebDriverFactory.PhantomJS()
}
+ object HtmlUnitDriverCase extends BaseCase {
+ override val pluginSys: Web.type = Web
+ override lazy val driverFactory = WebDriverFactory.HtmlUnit()
+ }
+
PythonDriverCase
WebDriverCase
+ HtmlUnitDriverCase
- it("If the old driver is released, the second Pooling DriverFactory.get() should yield the same driver") {
+ it("If the old driver is released, the second taskLocal DriverFactory.get() should yield the same driver") {
val conf = Web.Conf(WebDriverFactory.PhantomJS().taskLocal)
val spooky = new SpookyContext(sql)