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)