You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by pw...@apache.org on 2015/05/01 10:37:43 UTC

[1/5] spark git commit: [SPARK-3468] [WEBUI] Timeline-View feature

Repository: spark
Updated Branches:
  refs/heads/master c24aeb6a3 -> 7fe0f3f2b


http://git-wip-us.apache.org/repos/asf/spark/blob/7fe0f3f2/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala
index c6dc336..cf3db0b 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala
@@ -47,6 +47,18 @@ class StageInfo(
     failureReason = Some(reason)
     completionTime = Some(System.currentTimeMillis)
   }
+
+  private[spark] def getStatusString: String = {
+    if (completionTime.isDefined) {
+      if (failureReason.isDefined) {
+        "failed"
+      } else {
+        "succeeded"
+      }
+    } else {
+      "running"
+    }
+  }
 }
 
 private[spark] object StageInfo {

http://git-wip-us.apache.org/repos/asf/spark/blob/7fe0f3f2/core/src/main/scala/org/apache/spark/ui/UIUtils.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala
index f078641..395af2e 100644
--- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala
+++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala
@@ -159,12 +159,17 @@ private[spark] object UIUtils extends Logging {
           type="text/css" />
     <link rel="stylesheet" href={prependBaseUri("/static/webui.css")}
           type="text/css" />
+    <link rel="stylesheet" href={prependBaseUri("/static/vis.min.css")}
+          typ="text/css" />
+    <link rel="stylesheet" href={prependBaseUri("/static/timeline-view.css")}></link>
     <script src={prependBaseUri("/static/sorttable.js")} ></script>
     <script src={prependBaseUri("/static/jquery-1.11.1.min.js")}></script>
+    <script src={prependBaseUri("/static/vis.min.js")}></script>
     <script src={prependBaseUri("/static/bootstrap-tooltip.js")}></script>
     <script src={prependBaseUri("/static/initialize-tooltips.js")}></script>
     <script src={prependBaseUri("/static/table.js")}></script>
     <script src={prependBaseUri("/static/additional-metrics.js")}></script>
+    <script src={prependBaseUri("/static/timeline-view.js")}></script>
   }
 
   /** Returns a spark page with correctly formatted headers */

http://git-wip-us.apache.org/repos/asf/spark/blob/7fe0f3f2/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala
index 69053fe..0a08b00 100644
--- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala
+++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala
@@ -24,6 +24,7 @@ import org.apache.spark.annotation.DeveloperApi
 import org.apache.spark.scheduler._
 import org.apache.spark.storage.{StorageStatus, StorageStatusListener}
 import org.apache.spark.ui.{SparkUI, SparkUITab}
+import org.apache.spark.ui.jobs.UIData.ExecutorUIData
 
 private[ui] class ExecutorsTab(parent: SparkUI) extends SparkUITab(parent, "executors") {
   val listener = parent.executorsListener
@@ -54,12 +55,22 @@ class ExecutorsListener(storageStatusListener: StorageStatusListener) extends Sp
   val executorToShuffleRead = HashMap[String, Long]()
   val executorToShuffleWrite = HashMap[String, Long]()
   val executorToLogUrls = HashMap[String, Map[String, String]]()
+  val executorIdToData = HashMap[String, ExecutorUIData]()
 
   def storageStatusList: Seq[StorageStatus] = storageStatusListener.storageStatusList
 
   override def onExecutorAdded(executorAdded: SparkListenerExecutorAdded): Unit = synchronized {
     val eid = executorAdded.executorId
     executorToLogUrls(eid) = executorAdded.executorInfo.logUrlMap
+    executorIdToData(eid) = ExecutorUIData(executorAdded.time)
+  }
+
+  override def onExecutorRemoved(
+      executorRemoved: SparkListenerExecutorRemoved): Unit = synchronized {
+    val eid = executorRemoved.executorId
+    val uiData = executorIdToData(eid)
+    uiData.finishTime = Some(executorRemoved.time)
+    uiData.finishReason = Some(executorRemoved.reason)
   }
 
   override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = synchronized {

http://git-wip-us.apache.org/repos/asf/spark/blob/7fe0f3f2/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala
index bd923d7..a7ea12b 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala
@@ -17,17 +17,183 @@
 
 package org.apache.spark.ui.jobs
 
-import scala.xml.{Node, NodeSeq}
+import scala.collection.mutable.{HashMap, ListBuffer}
+import scala.xml.{Node, NodeSeq, Unparsed}
 
+import java.util.Date
 import javax.servlet.http.HttpServletRequest
 
-import org.apache.spark.ui.{WebUIPage, UIUtils}
-import org.apache.spark.ui.jobs.UIData.JobUIData
+import org.apache.spark.ui.{UIUtils, WebUIPage}
+import org.apache.spark.ui.jobs.UIData.{ExecutorUIData, JobUIData}
+import org.apache.spark.JobExecutionStatus
 
 /** Page showing list of all ongoing and recently finished jobs */
 private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") {
-  private val startTime: Option[Long] = parent.sc.map(_.startTime)
-  private val listener = parent.listener
+  private val JOBS_LEGEND =
+    <div class="legend-area"><svg width="150px" height="85px">
+      <rect class="succeeded-job-legend"
+        x="5px" y="5px" width="20px" height="15px" rx="2px" ry="2px"></rect>
+      <text x="35px" y="17px">Succeeded</text>
+      <rect class="failed-job-legend"
+        x="5px" y="30px" width="20px" height="15px" rx="2px" ry="2px"></rect>
+      <text x="35px" y="42px">Failed</text>
+      <rect class="running-job-legend"
+        x="5px" y="55px" width="20px" height="15px" rx="2px" ry="2px"></rect>
+      <text x="35px" y="67px">Running</text>
+    </svg></div>.toString.filter(_ != '\n')
+
+  private val EXECUTORS_LEGEND =
+    <div class="legend-area"><svg width="150px" height="55px">
+      <rect class="executor-added-legend"
+        x="5px" y="5px" width="20px" height="15px" rx="2px" ry="2px"></rect>
+      <text x="35px" y="17px">Added</text>
+      <rect class="executor-removed-legend"
+        x="5px" y="30px" width="20px" height="15px" rx="2px" ry="2px"></rect>
+      <text x="35px" y="42px">Removed</text>
+    </svg></div>.toString.filter(_ != '\n')
+
+  private def getLastStageNameAndDescription(job: JobUIData): (String, String) = {
+    val lastStageInfo = Option(job.stageIds)
+      .filter(_.nonEmpty)
+      .flatMap { ids => parent.jobProgresslistener.stageIdToInfo.get(ids.max)}
+    val lastStageData = lastStageInfo.flatMap { s =>
+      parent.jobProgresslistener.stageIdToData.get((s.stageId, s.attemptId))
+    }
+    val name = lastStageInfo.map(_.name).getOrElse("(Unknown Stage Name)")
+    val description = lastStageData.flatMap(_.description).getOrElse("")
+    (name, description)
+  }
+
+  private def makeJobEvent(jobUIDatas: Seq[JobUIData]): Seq[String] = {
+    jobUIDatas.filter { jobUIData =>
+      jobUIData.status != JobExecutionStatus.UNKNOWN && jobUIData.submissionTime.isDefined
+    }.map { jobUIData =>
+      val jobId = jobUIData.jobId
+      val status = jobUIData.status
+      val (jobName, jobDescription) = getLastStageNameAndDescription(jobUIData)
+      val displayJobDescription = if (jobDescription.isEmpty) jobName else jobDescription
+      val submissionTime = jobUIData.submissionTime.get
+      val completionTimeOpt = jobUIData.completionTime
+      val completionTime = completionTimeOpt.getOrElse(System.currentTimeMillis())
+      val classNameByStatus = status match {
+        case JobExecutionStatus.SUCCEEDED => "succeeded"
+        case JobExecutionStatus.FAILED => "failed"
+        case JobExecutionStatus.RUNNING => "running"
+      }
+
+      val jobEventJsonAsStr =
+        s"""
+           |{
+           |  'className': 'job application-timeline-object ${classNameByStatus}',
+           |  'group': 'jobs',
+           |  'start': new Date(${submissionTime}),
+           |  'end': new Date(${completionTime}),
+           |  'content': '<div class="application-timeline-content"' +
+           |     'data-html="true" data-placement="top" data-toggle="tooltip"' +
+           |     'data-title="${displayJobDescription} (Job ${jobId})<br>Status: ${status}<br>' +
+           |     'Submission Time: ${UIUtils.formatDate(new Date(submissionTime))}' +
+           |     '${
+                     if (status != JobExecutionStatus.RUNNING) {
+                       s"""<br>Completion Time: ${UIUtils.formatDate(new Date(completionTime))}"""
+                     } else {
+                       ""
+                     }
+                  }">' +
+           |    '${displayJobDescription} (Job ${jobId})</div>'
+           |}
+         """.stripMargin
+      jobEventJsonAsStr
+    }
+  }
+
+  private def makeExecutorEvent(executorUIDatas: HashMap[String, ExecutorUIData]): Seq[String] = {
+    val events = ListBuffer[String]()
+    executorUIDatas.foreach {
+      case (executorId, event) =>
+        val addedEvent =
+          s"""
+             |{
+             |  'className': 'executor added',
+             |  'group': 'executors',
+             |  'start': new Date(${event.startTime}),
+             |  'content': '<div class="executor-event-content"' +
+             |    'data-toggle="tooltip" data-placement="bottom"' +
+             |    'data-title="Executor ${executorId}<br>' +
+             |    'Added at ${UIUtils.formatDate(new Date(event.startTime))}"' +
+             |    'data-html="true">Executor ${executorId} added</div>'
+             |}
+           """.stripMargin
+        events += addedEvent
+
+        if (event.finishTime.isDefined) {
+          val removedEvent =
+            s"""
+               |{
+               |  'className': 'executor removed',
+               |  'group': 'executors',
+               |  'start': new Date(${event.finishTime.get}),
+               |  'content': '<div class="executor-event-content"' +
+               |    'data-toggle="tooltip" data-placement="bottom"' +
+               |    'data-title="Executor ${executorId}<br>' +
+               |    'Removed at ${UIUtils.formatDate(new Date(event.finishTime.get))}' +
+               |    '${
+                        if (event.finishReason.isDefined) {
+                          s"""<br>Reason: ${event.finishReason.get}"""
+                        } else {
+                          ""
+                        }
+                     }"' +
+               |    'data-html="true">Executor ${executorId} removed</div>'
+               |}
+             """.stripMargin
+          events += removedEvent
+        }
+    }
+    events.toSeq
+  }
+
+  private def makeTimeline(
+      jobs: Seq[JobUIData],
+      executors: HashMap[String, ExecutorUIData],
+      startTime: Long): Seq[Node] = {
+
+    val jobEventJsonAsStrSeq = makeJobEvent(jobs)
+    val executorEventJsonAsStrSeq = makeExecutorEvent(executors)
+
+    val groupJsonArrayAsStr =
+      s"""
+          |[
+          |  {
+          |    'id': 'executors',
+          |    'content': '<div>Executors</div>${EXECUTORS_LEGEND}',
+          |  },
+          |  {
+          |    'id': 'jobs',
+          |    'content': '<div>Jobs</div>${JOBS_LEGEND}',
+          |  }
+          |]
+        """.stripMargin
+
+    val eventArrayAsStr =
+      (jobEventJsonAsStrSeq ++ executorEventJsonAsStrSeq).mkString("[", ",", "]")
+
+    <span class="expand-application-timeline">
+      <span class="expand-application-timeline-arrow arrow-closed"></span>
+      <strong>Event Timeline</strong>
+    </span> ++
+    <div id="application-timeline" class="collapsed">
+      <div class="control-panel">
+        <div id="application-timeline-zoom-lock">
+          <input type="checkbox" checked="checked"></input>
+          <span>Zoom Lock</span>
+        </div>
+      </div>
+    </div> ++
+    <script type="text/javascript">
+      {Unparsed(s"drawApplicationTimeline(${groupJsonArrayAsStr}," +
+      s"${eventArrayAsStr}, ${startTime});")}
+    </script>
+  }
 
   private def jobsTable(jobs: Seq[JobUIData]): Seq[Node] = {
     val someJobHasJobGroup = jobs.exists(_.jobGroup.isDefined)
@@ -42,15 +208,7 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") {
     }
 
     def makeRow(job: JobUIData): Seq[Node] = {
-      val lastStageInfo = Option(job.stageIds)
-        .filter(_.nonEmpty)
-        .flatMap { ids => listener.stageIdToInfo.get(ids.max) }
-      val lastStageData = lastStageInfo.flatMap { s =>
-        listener.stageIdToData.get((s.stageId, s.attemptId))
-      }
-
-      val lastStageName = lastStageInfo.map(_.name).getOrElse("(Unknown Stage Name)")
-      val lastStageDescription = lastStageData.flatMap(_.description).getOrElse("")
+      val (lastStageName, lastStageDescription) = getLastStageNameAndDescription(job)
       val duration: Option[Long] = {
         job.submissionTime.map { start =>
           val end = job.completionTime.getOrElse(System.currentTimeMillis())
@@ -61,7 +219,7 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") {
       val formattedSubmissionTime = job.submissionTime.map(UIUtils.formatDate).getOrElse("Unknown")
       val detailUrl =
         "%s/jobs/job?id=%s".format(UIUtils.prependBaseUri(parent.basePath), job.jobId)
-      <tr>
+      <tr id={"job-" + job.jobId}>
         <td sorttable_customkey={job.jobId.toString}>
           {job.jobId} {job.jobGroup.map(id => s"($id)").getOrElse("")}
         </td>
@@ -95,11 +253,12 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") {
   }
 
   def render(request: HttpServletRequest): Seq[Node] = {
+    val listener = parent.jobProgresslistener
     listener.synchronized {
+      val startTime = listener.startTime
       val activeJobs = listener.activeJobs.values.toSeq
       val completedJobs = listener.completedJobs.reverse.toSeq
       val failedJobs = listener.failedJobs.reverse.toSeq
-      val now = System.currentTimeMillis
 
       val activeJobsTable =
         jobsTable(activeJobs.sortBy(_.submissionTime.getOrElse(-1L)).reverse)
@@ -115,11 +274,11 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") {
       val summary: NodeSeq =
         <div>
           <ul class="unstyled">
-            {if (startTime.isDefined) {
+            {if (parent.sc.isDefined) {
               // Total duration is not meaningful unless the UI is live
               <li>
                 <strong>Total Duration: </strong>
-                {UIUtils.formatDuration(now - startTime.get)}
+                {UIUtils.formatDuration(System.currentTimeMillis() - startTime)}
               </li>
             }}
             <li>
@@ -154,6 +313,10 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") {
         </div>
 
       var content = summary
+      val executorListener = parent.executorListener
+      content ++= makeTimeline(activeJobs ++ completedJobs ++ failedJobs,
+          executorListener.executorIdToData, startTime)
+
       if (shouldShowActiveJobs) {
         content ++= <h4 id="active">Active Jobs ({activeJobs.size})</h4> ++
           activeJobsTable
@@ -166,6 +329,7 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") {
         content ++= <h4 id ="failed">Failed Jobs ({failedJobs.size})</h4> ++
           failedJobsTable
       }
+
       val helpText = """A job is triggered by an action, like "count()" or "saveAsTextFile()".""" +
         " Click on a job's title to see information about the stages of tasks associated with" +
         " the job."

http://git-wip-us.apache.org/repos/asf/spark/blob/7fe0f3f2/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala
index 7541d3e..dd968e1 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala
@@ -17,20 +17,167 @@
 
 package org.apache.spark.ui.jobs
 
-import scala.collection.mutable
-import scala.xml.{NodeSeq, Node}
+import java.util.Date
+
+import scala.collection.mutable.{Buffer, HashMap, ListBuffer}
+import scala.xml.{NodeSeq, Node, Unparsed}
 
 import javax.servlet.http.HttpServletRequest
 
 import org.apache.spark.JobExecutionStatus
 import org.apache.spark.scheduler.StageInfo
 import org.apache.spark.ui.{UIUtils, WebUIPage}
+import org.apache.spark.ui.jobs.UIData.ExecutorUIData
 
 /** Page showing statistics and stage list for a given job */
 private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") {
-  private val listener = parent.listener
+  private val STAGES_LEGEND =
+    <div class="legend-area"><svg width="150px" height="85px">
+      <rect class="completed-stage-legend"
+        x="5px" y="5px" width="20px" height="15px" rx="2px" ry="2px"></rect>
+      <text x="35px" y="17px">Completed</text>
+      <rect class="failed-stage-legend"
+        x="5px" y="30px" width="20px" height="15px" rx="2px" ry="2px"></rect>
+      <text x="35px" y="42px">Failed</text>
+      <rect class="active-stage-legend"
+        x="5px" y="55px" width="20px" height="15px" rx="2px" ry="2px"></rect>
+      <text x="35px" y="67px">Active</text>
+    </svg></div>.toString.filter(_ != '\n')
+
+  private val EXECUTORS_LEGEND =
+    <div class="legend-area"><svg width="150px" height="55px">
+      <rect class="executor-added-legend"
+        x="5px" y="5px" width="20px" height="15px" rx="2px" ry="2px"></rect>
+      <text x="35px" y="17px">Added</text>
+      <rect class="executor-removed-legend"
+        x="5px" y="30px" width="20px" height="15px" rx="2px" ry="2px"></rect>
+      <text x="35px" y="42px">Removed</text>
+    </svg></div>.toString.filter(_ != '\n')
+
+  private def makeStageEvent(stageInfos: Seq[StageInfo]): Seq[String] = {
+    stageInfos.map { stage =>
+      val stageId = stage.stageId
+      val attemptId = stage.attemptId
+      val name = stage.name
+      val status = stage.getStatusString
+      val submissionTime = stage.submissionTime.get
+      val completionTime = stage.completionTime.getOrElse(System.currentTimeMillis())
+
+      s"""
+         |{
+         |  'className': 'stage job-timeline-object ${status}',
+         |  'group': 'stages',
+         |  'start': new Date(${submissionTime}),
+         |  'end': new Date(${completionTime}),
+         |  'content': '<div class="job-timeline-content" data-toggle="tooltip"' +
+         |   'data-placement="top" data-html="true"' +
+         |   'data-title="${name} (Stage ${stageId}.${attemptId})<br>' +
+         |   'Status: ${status.toUpperCase}<br>' +
+         |   'Submission Time: ${UIUtils.formatDate(new Date(submissionTime))}' +
+         |   '${
+                 if (status != "running") {
+                   s"""<br>Completion Time: ${UIUtils.formatDate(new Date(completionTime))}"""
+                 } else {
+                   ""
+                 }
+              }">' +
+         |    '${name} (Stage ${stageId}.${attemptId})</div>',
+         |}
+       """.stripMargin
+    }
+  }
+
+  def makeExecutorEvent(executorUIDatas: HashMap[String, ExecutorUIData]): Seq[String] = {
+    val events = ListBuffer[String]()
+    executorUIDatas.foreach {
+      case (executorId, event) =>
+        val addedEvent =
+          s"""
+             |{
+             |  'className': 'executor added',
+             |  'group': 'executors',
+             |  'start': new Date(${event.startTime}),
+             |  'content': '<div class="executor-event-content"' +
+             |    'data-toggle="tooltip" data-placement="bottom"' +
+             |    'data-title="Executor ${executorId}<br>' +
+             |    'Added at ${UIUtils.formatDate(new Date(event.startTime))}"' +
+             |    'data-html="true">Executor ${executorId} added</div>'
+             |}
+           """.stripMargin
+        events += addedEvent
+
+        if (event.finishTime.isDefined) {
+          val removedEvent =
+            s"""
+               |{
+               |  'className': 'executor removed',
+               |  'group': 'executors',
+               |  'start': new Date(${event.finishTime.get}),
+               |  'content': '<div class="executor-event-content"' +
+               |    'data-toggle="tooltip" data-placement="bottom"' +
+               |    'data-title="Executor ${executorId}<br>' +
+               |    'Removed at ${UIUtils.formatDate(new Date(event.finishTime.get))}' +
+               |    '${
+                        if (event.finishReason.isDefined) {
+                          s"""<br>Reason: ${event.finishReason.get}"""
+                        } else {
+                          ""
+                        }
+                     }"' +
+               |    'data-html="true">Executor ${executorId} removed</div>'
+               |}
+             """.stripMargin
+            events += removedEvent
+        }
+    }
+    events.toSeq
+  }
+
+  private def  makeTimeline(
+      stages: Seq[StageInfo],
+      executors: HashMap[String, ExecutorUIData],
+      appStartTime: Long): Seq[Node] = {
+
+    val stageEventJsonAsStrSeq = makeStageEvent(stages)
+    val executorsJsonAsStrSeq = makeExecutorEvent(executors)
+
+    val groupJsonArrayAsStr =
+      s"""
+          |[
+          |  {
+          |    'id': 'executors',
+          |    'content': '<div>Executors</div>${EXECUTORS_LEGEND}',
+          |  },
+          |  {
+          |    'id': 'stages',
+          |    'content': '<div>Stages</div>${STAGES_LEGEND}',
+          |  }
+          |]
+        """.stripMargin
+
+    val eventArrayAsStr =
+      (stageEventJsonAsStrSeq ++ executorsJsonAsStrSeq).mkString("[", ",", "]")
+
+    <span class="expand-job-timeline">
+      <span class="expand-job-timeline-arrow arrow-closed"></span>
+      <strong>Event Timeline</strong>
+    </span> ++
+    <div id="job-timeline" class="collapsed">
+      <div class="control-panel">
+        <div id="job-timeline-zoom-lock">
+          <input type="checkbox" checked="checked"></input>
+          <span>Zoom Lock</span>
+        </div>
+      </div>
+    </div> ++
+    <script type="text/javascript">
+      {Unparsed(s"drawJobTimeline(${groupJsonArrayAsStr}, ${eventArrayAsStr}, ${appStartTime});")}
+    </script>
+  }
 
   def render(request: HttpServletRequest): Seq[Node] = {
+    val listener = parent.jobProgresslistener
+
     listener.synchronized {
       val parameterId = request.getParameter("id")
       require(parameterId != null && parameterId.nonEmpty, "Missing id parameter")
@@ -54,11 +201,11 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") {
           new StageInfo(stageId, 0, "Unknown", 0, Seq.empty, "Unknown"))
       }
 
-      val activeStages = mutable.Buffer[StageInfo]()
-      val completedStages = mutable.Buffer[StageInfo]()
+      val activeStages = Buffer[StageInfo]()
+      val completedStages = Buffer[StageInfo]()
       // If the job is completed, then any pending stages are displayed as "skipped":
-      val pendingOrSkippedStages = mutable.Buffer[StageInfo]()
-      val failedStages = mutable.Buffer[StageInfo]()
+      val pendingOrSkippedStages = Buffer[StageInfo]()
+      val failedStages = Buffer[StageInfo]()
       for (stage <- stages) {
         if (stage.submissionTime.isEmpty) {
           pendingOrSkippedStages += stage
@@ -75,18 +222,18 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") {
 
       val activeStagesTable =
         new StageTableBase(activeStages.sortBy(_.submissionTime).reverse,
-          parent.basePath, parent.listener, isFairScheduler = parent.isFairScheduler,
+          parent.basePath, parent.jobProgresslistener, isFairScheduler = parent.isFairScheduler,
           killEnabled = parent.killEnabled)
       val pendingOrSkippedStagesTable =
         new StageTableBase(pendingOrSkippedStages.sortBy(_.stageId).reverse,
-          parent.basePath, parent.listener, isFairScheduler = parent.isFairScheduler,
+          parent.basePath, parent.jobProgresslistener, isFairScheduler = parent.isFairScheduler,
           killEnabled = false)
       val completedStagesTable =
         new StageTableBase(completedStages.sortBy(_.submissionTime).reverse, parent.basePath,
-          parent.listener, isFairScheduler = parent.isFairScheduler, killEnabled = false)
+          parent.jobProgresslistener, isFairScheduler = parent.isFairScheduler, killEnabled = false)
       val failedStagesTable =
         new FailedStageTable(failedStages.sortBy(_.submissionTime).reverse, parent.basePath,
-          parent.listener, isFairScheduler = parent.isFairScheduler)
+          parent.jobProgresslistener, isFairScheduler = parent.isFairScheduler)
 
       val shouldShowActiveStages = activeStages.nonEmpty
       val shouldShowPendingStages = !isComplete && pendingOrSkippedStages.nonEmpty
@@ -154,6 +301,11 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") {
         </div>
 
       var content = summary
+      val appStartTime = listener.startTime
+      val executorListener = parent.executorListener
+      content ++= makeTimeline(activeStages ++ completedStages ++ failedStages,
+          executorListener.executorIdToData, appStartTime)
+
       if (shouldShowActiveStages) {
         content ++= <h4 id="active">Active Stages ({activeStages.size})</h4> ++
           activeStagesTable.toNodeSeq

http://git-wip-us.apache.org/repos/asf/spark/blob/7fe0f3f2/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala
index 6255968..d6d716d 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala
@@ -50,6 +50,9 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging {
   type PoolName = String
   type ExecutorId = String
 
+  // Applicatin:
+  @volatile var startTime = -1L
+
   // Jobs:
   val activeJobs = new HashMap[JobId, JobUIData]
   val completedJobs = ListBuffer[JobUIData]()
@@ -75,6 +78,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging {
 
   // Misc:
   val executorIdToBlockManagerId = HashMap[ExecutorId, BlockManagerId]()
+
   def blockManagerIds: Seq[BlockManagerId] = executorIdToBlockManagerId.values.toSeq
 
   var schedulingMode: Option[SchedulingMode] = None
@@ -516,6 +520,9 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging {
     }
   }
 
+  override def onApplicationStart(appStarted: SparkListenerApplicationStart) {
+    startTime = appStarted.time
+  }
 }
 
 private object JobProgressListener {

http://git-wip-us.apache.org/repos/asf/spark/blob/7fe0f3f2/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala
index 7ffcf29..342787f 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala
@@ -24,8 +24,10 @@ import org.apache.spark.ui.{SparkUI, SparkUITab}
 private[ui] class JobsTab(parent: SparkUI) extends SparkUITab(parent, "jobs") {
   val sc = parent.sc
   val killEnabled = parent.killEnabled
-  def isFairScheduler: Boolean = listener.schedulingMode.exists(_ == SchedulingMode.FAIR)
-  val listener = parent.jobProgressListener
+  def isFairScheduler: Boolean =
+    jobProgresslistener.schedulingMode.exists(_ == SchedulingMode.FAIR)
+  val jobProgresslistener = parent.jobProgressListener
+  val executorListener = parent.executorsListener
 
   attachPage(new AllJobsPage(this))
   attachPage(new JobPage(this))

http://git-wip-us.apache.org/repos/asf/spark/blob/7fe0f3f2/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala
index cb72890..6d8c7e1 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala
@@ -174,7 +174,8 @@ private[ui] class StageTableBase(
   }
 
   /** Render an HTML row that represents a stage */
-  private def renderStageRow(s: StageInfo): Seq[Node] = <tr>{stageRow(s)}</tr>
+  private def renderStageRow(s: StageInfo): Seq[Node] =
+    <tr id={"stage-" + s.stageId + "-" + s.attemptId}>{stageRow(s)}</tr>
 }
 
 private[ui] class FailedStageTable(

http://git-wip-us.apache.org/repos/asf/spark/blob/7fe0f3f2/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala
index 935c8a4..3d96113 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala
@@ -108,4 +108,9 @@ private[spark] object UIData {
       var taskInfo: TaskInfo,
       var taskMetrics: Option[TaskMetrics] = None,
       var errorMessage: Option[String] = None)
+
+  case class ExecutorUIData(
+      val startTime: Long,
+      var finishTime: Option[Long] = None,
+      var finishReason: Option[String] = None)
 }


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


[5/5] spark git commit: [SPARK-3468] [WEBUI] Timeline-View feature

Posted by pw...@apache.org.
[SPARK-3468] [WEBUI] Timeline-View feature

I sometimes trouble-shoot and analyse the cause of long time spending job.

At the time, I find the stages which spends long time or fails, then I find the tasks which spends long time or fails, next I analyse the proportion of each phase in a task.

Another case, I find executors which spends long time for running a task and analyse the details of a task.

In such situation, I think it's helpful to visualize timeline  for each application, job, task and the details of proportion of activity for each task.

I added 3 timeline features to existing Web UI.

[Application Timeline View]
This view shows following things.

* When each executor was added/removed and the reason why it's removed.
* When each job  was started/finished.
* Status of each job.

![screenshot from 2015-04-01 16 49 25](https://cloud.githubusercontent.com/assets/4736016/6936886/e35fd582-d891-11e4-980d-8de13f50e442.png)

[Stage Timeline View]
Similar to Application Timeline View, this view shows following things.

* When each executor was added/removed and the reason why it's removed.
* When each job was started/finished.
* Status of each stage.

![screenshot from 2015-04-01 16 50 59](https://cloud.githubusercontent.com/assets/4736016/6936900/0dca6526-d892-11e4-84a8-efd9037af444.png)

[Task Assignment Timeline View]
This view shows following things.

* When each task started/finished
* How long each task spent and the proportion.
* Status of each task.
* Where each task ran on.

![screenshot from 2015-04-01 16 51 54](https://cloud.githubusercontent.com/assets/4736016/6936910/20fd5acc-d892-11e4-9018-80e463881fc2.png)

All the view above is zoomable by mouse wheel action and scrollable by drag action.

Author: Kousuke Saruta <sa...@oss.nttdata.co.jp>

Closes #2342 from sarutak/timeline-viewer-feature and squashes the following commits:

11fe67d [Kousuke Saruta] Fixed conflict
79ac03d [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into timeline-viewer-feature
a91abd3 [Kousuke Saruta] Merge branch 'master' of https://github.com/apache/spark into timeline-viewer-feature
ef34a5b [Kousuke Saruta] Implement tooltip using bootstrap
b09d0c5 [Kousuke Saruta] Move `stroke` and `fill` attribute of rect elements to css
d3c63c8 [Kousuke Saruta] Fixed a little bit bugs
a36291b [Kousuke Saruta] Merge branch 'master' of https://github.com/apache/spark into timeline-viewer-feature
28714b6 [Kousuke Saruta] Fixed highlight issue
0dc4278 [Kousuke Saruta] Addressed most of Patrics's feedbacks
8110acf [Kousuke Saruta] Added scroll limit to Job timeline
974a64a [Kousuke Saruta] Removed unused function
ee7a7f0 [Kousuke Saruta] Refactored
6a91872 [Kousuke Saruta] Temporary commit
6693f34 [Kousuke Saruta] Added link to job/stage box in the timeline in order to move to corresponding row when we click
8f88222 [Kousuke Saruta] Added job/stage description
aeed4b1 [Kousuke Saruta] Removed stage timeline
fc1696c [Kousuke Saruta] Merge branch 'timeline-viewer-feature' of github.com:sarutak/spark into timeline-viewer-feature
999ccd4 [Kousuke Saruta] Improved scalability
0fc6a31 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into timeline-viewer-feature
19815ae [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into timeline-viewer-feature
68b7540 [Kousuke Saruta] Merge branch 'timeline-viewer-feature' of github.com:sarutak/spark into timeline-viewer-feature
52b5f0b [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into timeline-viewer-feature
dec85db [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into timeline-viewer-feature
fcdab7d [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into timeline-viewer-feature
dab7cc1 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into timeline-viewer-feature
09cce97 [Kousuke Saruta] Cleanuped
16f82cf [Kousuke Saruta] Cleanuped
9fb522e [Kousuke Saruta] Cleanuped
d05f2c2 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into timeline-viewer-feature
e85e9aa [Kousuke Saruta] Cleanup: Added TimelineViewUtils.scala
a76e569 [Kousuke Saruta] Removed unused setting in timeline-view.css
5ce1b21 [Kousuke Saruta] Added vis.min.js, vis.min.css and vis.map to .rat-exclude
082f709 [Kousuke Saruta] Added Timeline-View feature for Applications, Jobs and Stages


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/7fe0f3f2
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/7fe0f3f2
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/7fe0f3f2

Branch: refs/heads/master
Commit: 7fe0f3f2b46c61a5cc4af9166781624409fda8a4
Parents: c24aeb6
Author: Kousuke Saruta <sa...@oss.nttdata.co.jp>
Authored: Fri May 1 01:39:56 2015 -0700
Committer: Patrick Wendell <pa...@databricks.com>
Committed: Fri May 1 01:39:56 2015 -0700

----------------------------------------------------------------------
 .rat-excludes                                   |   3 +
 .../apache/spark/ui/static/timeline-view.css    | 182 +++++++++++++++++
 .../org/apache/spark/ui/static/timeline-view.js | 168 ++++++++++++++++
 .../org/apache/spark/ui/static/vis.map          |   1 +
 .../org/apache/spark/ui/static/vis.min.css      |   1 +
 .../org/apache/spark/ui/static/vis.min.js       |  39 ++++
 .../org/apache/spark/scheduler/StageInfo.scala  |  12 ++
 .../scala/org/apache/spark/ui/UIUtils.scala     |   5 +
 .../org/apache/spark/ui/exec/ExecutorsTab.scala |  11 +
 .../org/apache/spark/ui/jobs/AllJobsPage.scala  | 200 +++++++++++++++++--
 .../org/apache/spark/ui/jobs/JobPage.scala      | 174 +++++++++++++++-
 .../spark/ui/jobs/JobProgressListener.scala     |   7 +
 .../org/apache/spark/ui/jobs/JobsTab.scala      |   6 +-
 .../org/apache/spark/ui/jobs/StageTable.scala   |   3 +-
 .../scala/org/apache/spark/ui/jobs/UIData.scala |   5 +
 15 files changed, 785 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/7fe0f3f2/.rat-excludes
----------------------------------------------------------------------
diff --git a/.rat-excludes b/.rat-excludes
index 8aca5a7..4468da1 100644
--- a/.rat-excludes
+++ b/.rat-excludes
@@ -30,6 +30,9 @@ log4j-defaults.properties
 bootstrap-tooltip.js
 jquery-1.11.1.min.js
 sorttable.js
+vis.min.js
+vis.min.css
+vis.map
 .*avsc
 .*txt
 .*json

http://git-wip-us.apache.org/repos/asf/spark/blob/7fe0f3f2/core/src/main/resources/org/apache/spark/ui/static/timeline-view.css
----------------------------------------------------------------------
diff --git a/core/src/main/resources/org/apache/spark/ui/static/timeline-view.css b/core/src/main/resources/org/apache/spark/ui/static/timeline-view.css
new file mode 100644
index 0000000..35ef14e
--- /dev/null
+++ b/core/src/main/resources/org/apache/spark/ui/static/timeline-view.css
@@ -0,0 +1,182 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+div#application-timeline, div#job-timeline {
+  margin-bottom: 30px;
+}
+
+#application-timeline div.legend-area {
+  margin-top: 5px;
+}
+
+.vis.timeline div.content {
+  width: 100%;
+}
+
+.vis.timeline .item.stage {
+  cursor: pointer;
+}
+
+.vis.timeline .item.stage.succeeded {
+  background-color: #D5DDF6;
+}
+
+.vis.timeline .item.stage.succeeded.selected {
+   background-color: #D5DDF6;
+   border-color: #97B0F8;
+   z-index: auto;
+}
+
+.legend-area rect.completed-stage-legend {
+  fill: #D5DDF6;
+  stroke: #97B0F8;
+}
+
+.vis.timeline .item.stage.failed {
+   background-color: #FF5475;
+}
+
+.vis.timeline .item.stage.failed.selected {
+   background-color: #FF5475;
+   border-color: #97B0F8;
+   z-index: auto;
+}
+
+.legend-area rect.failed-stage-legend {
+  fill: #FF5475;
+  stroke: #97B0F8;
+}
+
+.vis.timeline .item.stage.running {
+   background-color: #FDFFCA;
+}
+
+.vis.timeline .item.stage.running.selected {
+   background-color: #FDFFCA;
+   border-color: #97B0F8;
+   z-index: auto;
+}
+
+.legend-area rect.active-stage-legend {
+  fill: #FDFFCA;
+  stroke: #97B0F8;
+}
+
+.vis.timeline .item.job {
+  cursor: pointer;
+}
+
+.vis.timeline .item.job.succeeded {
+  background-color: #D5DDF6;
+}
+
+.vis.timeline .item.job.succeeded.selected {
+   background-color: #D5DDF6;
+   border-color: #97B0F8;
+   z-index: auto;
+}
+
+.legend-area rect.succeeded-job-legend {
+  fill: #D5DDF6;
+  stroke: #97B0F8;
+}
+
+.vis.timeline .item.job.failed {
+   background-color: #FF5475;
+}
+
+.vis.timeline .item.job.failed.selected {
+   background-color: #FF5475;
+   border-color: #97B0F8;
+   z-index: auto;
+}
+
+.legend-area rect.failed-job-legend {
+  fill: #FF5475;
+  stroke: #97B0F8;
+}
+
+.vis.timeline .item.job.running {
+   background-color: #FDFFCA;
+}
+
+.vis.timeline .item.job.running.selected {
+   background-color: #FDFFCA;
+   border-color: #97B0F8;
+   z-index: auto;
+}
+
+.legend-area rect.running-job-legend {
+  fill: #FDFFCA;
+  stroke: #97B0F8;
+}
+
+.vis.timeline .item.executor.added {
+  background-color: #D5DDF6;
+}
+
+.legend-area rect.executor-added-legend {
+  fill: #D5DDF6;
+  stroke: #97B0F8;
+}
+
+.vis.timeline .item.executor.removed {
+  background-color: #EBCA59;
+}
+
+.legend-area rect.executor-removed-legend {
+  fill: #EBCA59;
+  stroke: #97B0F8;
+}
+
+.vis.timeline .item.executor.selected {
+  border-color: #FFC200;
+  background-color: #FFF785;
+  z-index: 2;
+}
+
+tr.corresponding-item-hover>td, tr.corresponding-item-hover>th {
+  background-color: #FFE1FA !important;
+}
+
+#application-timeline.collapsed {
+  display: none;
+}
+
+#job-timeline.collapsed {
+  display: none;
+}
+
+.control-panel {
+  margin-bottom: 5px;
+}
+
+span.expand-application-timeline, span.expand-job-timeline {
+  cursor: pointer;
+}
+
+.control-panel input+span {
+  cursor: pointer;
+}
+
+.vis.timeline .item.range .content {
+  position: unset;
+}
+
+.vis.timeline .item .tooltip-inner {
+  max-width: unset !important;
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/7fe0f3f2/core/src/main/resources/org/apache/spark/ui/static/timeline-view.js
----------------------------------------------------------------------
diff --git a/core/src/main/resources/org/apache/spark/ui/static/timeline-view.js b/core/src/main/resources/org/apache/spark/ui/static/timeline-view.js
new file mode 100644
index 0000000..e4a891d
--- /dev/null
+++ b/core/src/main/resources/org/apache/spark/ui/static/timeline-view.js
@@ -0,0 +1,168 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+function drawApplicationTimeline(groupArray, eventObjArray, startTime) {
+  var groups = new vis.DataSet(groupArray);
+  var items = new vis.DataSet(eventObjArray);
+  var container = $("#application-timeline")[0];
+  var options = {
+    groupOrder: function(a, b) {
+      return a.value - b.value
+    },
+    editable: false,
+    showCurrentTime: false,
+    min: startTime,
+    zoomable: false
+  };
+
+  var applicationTimeline = new vis.Timeline(container);
+  applicationTimeline.setOptions(options);
+  applicationTimeline.setGroups(groups);
+  applicationTimeline.setItems(items);
+
+  setupZoomable("#application-timeline-zoom-lock", applicationTimeline);
+  setupExecutorEventAction();
+
+  function setupJobEventAction() {
+    $(".item.range.job.application-timeline-object").each(function() {
+      var getJobId = function(baseElem) {
+        var jobIdText = $($(baseElem).find(".application-timeline-content")[0]).text();
+        var jobId = jobIdText.match("\\(Job (\\d+)\\)")[1];
+       return jobId;
+      };
+
+      $(this).click(function() {
+        window.location.href = "job/?id=" + getJobId(this);
+      });
+
+      $(this).hover(
+        function() {
+          $("#job-" + getJobId(this)).addClass("corresponding-item-hover");
+          $($(this).find("div.application-timeline-content")[0]).tooltip("show");
+        },
+        function() {
+          $("#job-" + getJobId(this)).removeClass("corresponding-item-hover");
+          $($(this).find("div.application-timeline-content")[0]).tooltip("hide");
+        }
+      );
+    });
+  }
+
+  setupJobEventAction();
+
+  $("span.expand-application-timeline").click(function() {
+    $("#application-timeline").toggleClass('collapsed');
+
+    // Switch the class of the arrow from open to closed.
+    $(this).find('.expand-application-timeline-arrow').toggleClass('arrow-open');
+    $(this).find('.expand-application-timeline-arrow').toggleClass('arrow-closed');
+  });
+}
+
+function drawJobTimeline(groupArray, eventObjArray, startTime) {
+  var groups = new vis.DataSet(groupArray);
+  var items = new vis.DataSet(eventObjArray);
+  var container = $('#job-timeline')[0];
+  var options = {
+    groupOrder: function(a, b) {
+      return a.value - b.value;
+    },
+    editable: false,
+    showCurrentTime: false,
+    min: startTime,
+    zoomable: false,
+  };
+
+  var jobTimeline = new vis.Timeline(container);
+  jobTimeline.setOptions(options);
+  jobTimeline.setGroups(groups);
+  jobTimeline.setItems(items);
+
+  setupZoomable("#job-timeline-zoom-lock", jobTimeline);
+  setupExecutorEventAction();
+
+  function setupStageEventAction() {
+    $(".item.range.stage.job-timeline-object").each(function() {
+      var getStageIdAndAttempt = function(baseElem) {
+        var stageIdText = $($(baseElem).find(".job-timeline-content")[0]).text();
+        var stageIdAndAttempt = stageIdText.match("\\(Stage (\\d+\\.\\d+)\\)")[1].split(".");
+        return stageIdAndAttempt;
+      };
+
+      $(this).click(function() {
+        var idAndAttempt = getStageIdAndAttempt(this);
+        var id = idAndAttempt[0];
+        var attempt = idAndAttempt[1];
+        window.location.href = "../../stages/stage/?id=" + id + "&attempt=" + attempt;
+      });
+
+      $(this).hover(
+        function() {
+          var idAndAttempt = getStageIdAndAttempt(this);
+          var id = idAndAttempt[0];
+          var attempt = idAndAttempt[1];
+          $("#stage-" + id + "-" + attempt).addClass("corresponding-item-hover");
+          $($(this).find("div.job-timeline-content")[0]).tooltip("show");
+        },
+        function() {
+          var idAndAttempt = getStageIdAndAttempt(this);
+          var id = idAndAttempt[0];
+          var attempt = idAndAttempt[1];
+          $("#stage-" + id + "-" + attempt).removeClass("corresponding-item-hover");
+          $($(this).find("div.job-timeline-content")[0]).tooltip("hide");
+        }
+      );
+    });
+  }
+
+  setupStageEventAction();
+
+  $("span.expand-job-timeline").click(function() {
+    $("#job-timeline").toggleClass('collapsed');
+
+    // Switch the class of the arrow from open to closed.
+    $(this).find('.expand-job-timeline-arrow').toggleClass('arrow-open');
+    $(this).find('.expand-job-timeline-arrow').toggleClass('arrow-closed');
+  });
+}
+
+function setupExecutorEventAction() {
+  $(".item.box.executor").each(function () {
+    $(this).hover(
+      function() {
+        $($(this).find(".executor-event-content")[0]).tooltip("show");
+      },
+      function() {
+        $($(this).find(".executor-event-content")[0]).tooltip("hide");
+      }
+    );
+  });
+}
+
+function setupZoomable(id, timeline) {
+  $(id + '>input[type="checkbox"]').click(function() {
+    if (this.checked) {
+      timeline.setOptions({zoomable: false});
+    } else {
+      timeline.setOptions({zoomable: true});
+    }
+  });
+
+  $(id + ">span").click(function() {
+    $(this).parent().find('input:checkbox').trigger('click');
+  });
+}


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


[3/5] spark git commit: [SPARK-3468] [WEBUI] Timeline-View feature

Posted by pw...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/7fe0f3f2/core/src/main/resources/org/apache/spark/ui/static/vis.min.css
----------------------------------------------------------------------
diff --git a/core/src/main/resources/org/apache/spark/ui/static/vis.min.css b/core/src/main/resources/org/apache/spark/ui/static/vis.min.css
new file mode 100644
index 0000000..a390c40
--- /dev/null
+++ b/core/src/main/resources/org/apache/spark/ui/static/vis.min.css
@@ -0,0 +1 @@
+.vis .overlay{position:absolute;top:0;left:0;width:100%;height:100%;z-index:10}.vis-active{box-shadow:0 0 10px #86d5f8}.vis [class*=span]{min-height:0;width:auto}.vis.timeline.root{position:relative;border:1px solid #bfbfbf;overflow:hidden;padding:0;margin:0;box-sizing:border-box}.vis.timeline .vispanel{position:absolute;padding:0;margin:0;box-sizing:border-box}.vis.timeline .vispanel.bottom,.vis.timeline .vispanel.center,.vis.timeline .vispanel.left,.vis.timeline .vispanel.right,.vis.timeline .vispanel.top{border:1px #bfbfbf}.vis.timeline .vispanel.center,.vis.timeline .vispanel.left,.vis.timeline .vispanel.right{border-top-style:solid;border-bottom-style:solid;overflow:hidden}.vis.timeline .vispanel.bottom,.vis.timeline .vispanel.center,.vis.timeline .vispanel.top{border-left-style:solid;border-right-style:solid}.vis.timeline .background{overflow:hidden}.vis.timeline .vispanel>.content{position:relative}.vis.timeline .vispanel .shadow{position:absolute;width:100%;height:1px;box-sh
 adow:0 0 10px rgba(0,0,0,.8)}.vis.timeline .vispanel .shadow.top{top:-1px;left:0}.vis.timeline .vispanel .shadow.bottom{bottom:-1px;left:0}.vis.timeline .labelset{position:relative;overflow:hidden;box-sizing:border-box}.vis.timeline .labelset .vlabel{position:relative;left:0;top:0;width:100%;color:#4d4d4d;box-sizing:border-box;border-bottom:1px solid #bfbfbf}.vis.timeline .labelset .vlabel:last-child{border-bottom:none}.vis.timeline .labelset .vlabel .inner{display:inline-block;padding:5px}.vis.timeline .labelset .vlabel .inner.hidden{padding:0}.vis.timeline .itemset{position:relative;padding:0;margin:0;box-sizing:border-box}.vis.timeline .itemset .background,.vis.timeline .itemset .foreground{position:absolute;width:100%;height:100%;overflow:visible}.vis.timeline .axis{position:absolute;width:100%;height:0;left:0;z-index:1}.vis.timeline .foreground .group{position:relative;box-sizing:border-box;border-bottom:1px solid #bfbfbf}.vis.timeline .foreground .group:last-child{border-botto
 m:none}.vis.timeline .item{position:absolute;color:#1A1A1A;border-color:#97B0F8;border-width:1px;background-color:#D5DDF6;display:inline-block;padding:5px}.vis.timeline .item.selected{border-color:#FFC200;background-color:#FFF785;z-index:2}.vis.timeline .editable .item.selected{cursor:move}.vis.timeline .item.point.selected{background-color:#FFF785}.vis.timeline .item.box{text-align:center;border-style:solid;border-radius:2px}.vis.timeline .item.point{background:0 0}.vis.timeline .item.dot{position:absolute;padding:0;border-width:4px;border-style:solid;border-radius:4px}.vis.timeline .item.range{border-style:solid;border-radius:2px;box-sizing:border-box}.vis.timeline .item.background{overflow:hidden;border:none;background-color:rgba(213,221,246,.4);box-sizing:border-box;padding:0;margin:0}.vis.timeline .item.range .content{position:relative;display:inline-block;max-width:100%;overflow:hidden}.vis.timeline .item.background .content{position:absolute;display:inline-block;overflow:hidd
 en;max-width:100%;margin:5px}.vis.timeline .item.line{padding:0;position:absolute;width:0;border-left-width:1px;border-left-style:solid}.vis.timeline .item .content{white-space:nowrap;overflow:hidden}.vis.timeline .item .delete{background:url(img/timeline/delete.png) top center no-repeat;position:absolute;width:24px;height:24px;top:0;right:-24px;cursor:pointer}.vis.timeline .item.range .drag-left{position:absolute;width:24px;height:100%;top:0;left:-4px;cursor:w-resize}.vis.timeline .item.range .drag-right{position:absolute;width:24px;height:100%;top:0;right:-4px;cursor:e-resize}.vis.timeline .timeaxis{position:relative;overflow:hidden}.vis.timeline .timeaxis.foreground{top:0;left:0;width:100%}.vis.timeline .timeaxis.background{position:absolute;top:0;left:0;width:100%;height:100%}.vis.timeline .timeaxis .text{position:absolute;color:#4d4d4d;padding:3px;white-space:nowrap}.vis.timeline .timeaxis .text.measure{position:absolute;padding-left:0;padding-right:0;margin-left:0;margin-right
 :0;visibility:hidden}.vis.timeline .timeaxis .grid.vertical{position:absolute;border-left:1px solid}.vis.timeline .timeaxis .grid.minor{border-color:#e5e5e5}.vis.timeline .timeaxis .grid.major{border-color:#bfbfbf}.vis.timeline .currenttime{background-color:#FF7F6E;width:2px;z-index:1}.vis.timeline .customtime{background-color:#6E94FF;width:2px;cursor:move;z-index:1}.vis.timeline .vispanel.background.horizontal .grid.horizontal{position:absolute;width:100%;height:0;border-bottom:1px solid}.vis.timeline .vispanel.background.horizontal .grid.minor{border-color:#e5e5e5}.vis.timeline .vispanel.background.horizontal .grid.major{border-color:#bfbfbf}.vis.timeline .dataaxis .yAxis.major{width:100%;position:absolute;color:#4d4d4d;white-space:nowrap}.vis.timeline .dataaxis .yAxis.major.measure{padding:0;margin:0;border:0;visibility:hidden;width:auto}.vis.timeline .dataaxis .yAxis.minor{position:absolute;width:100%;color:#bebebe;white-space:nowrap}.vis.timeline .dataaxis .yAxis.minor.measure{
 padding:0;margin:0;border:0;visibility:hidden;width:auto}.vis.timeline .dataaxis .yAxis.title{position:absolute;color:#4d4d4d;white-space:nowrap;bottom:20px;text-align:center}.vis.timeline .dataaxis .yAxis.title.measure{padding:0;margin:0;visibility:hidden;width:auto}.vis.timeline .dataaxis .yAxis.title.left{bottom:0;-webkit-transform-origin:left top;-moz-transform-origin:left top;-ms-transform-origin:left top;-o-transform-origin:left top;transform-origin:left bottom;-webkit-transform:rotate(-90deg);-moz-transform:rotate(-90deg);-ms-transform:rotate(-90deg);-o-transform:rotate(-90deg);transform:rotate(-90deg)}.vis.timeline .dataaxis .yAxis.title.right{bottom:0;-webkit-transform-origin:right bottom;-moz-transform-origin:right bottom;-ms-transform-origin:right bottom;-o-transform-origin:right bottom;transform-origin:right bottom;-webkit-transform:rotate(90deg);-moz-transform:rotate(90deg);-ms-transform:rotate(90deg);-o-transform:rotate(90deg);transform:rotate(90deg)}.vis.timeline .leg
 end{background-color:rgba(247,252,255,.65);padding:5px;border-color:#b3b3b3;border-style:solid;border-width:1px;box-shadow:2px 2px 10px rgba(154,154,154,.55)}.vis.timeline .legendText{white-space:nowrap;display:inline-block}.vis.timeline .graphGroup0{fill:#4f81bd;fill-opacity:0;stroke-width:2px;stroke:#4f81bd}.vis.timeline .graphGroup1{fill:#f79646;fill-opacity:0;stroke-width:2px;stroke:#f79646}.vis.timeline .graphGroup2{fill:#8c51cf;fill-opacity:0;stroke-width:2px;stroke:#8c51cf}.vis.timeline .graphGroup3{fill:#75c841;fill-opacity:0;stroke-width:2px;stroke:#75c841}.vis.timeline .graphGroup4{fill:#ff0100;fill-opacity:0;stroke-width:2px;stroke:#ff0100}.vis.timeline .graphGroup5{fill:#37d8e6;fill-opacity:0;stroke-width:2px;stroke:#37d8e6}.vis.timeline .graphGroup6{fill:#042662;fill-opacity:0;stroke-width:2px;stroke:#042662}.vis.timeline .graphGroup7{fill:#00ff26;fill-opacity:0;stroke-width:2px;stroke:#00ff26}.vis.timeline .graphGroup8{fill:#f0f;fill-opacity:0;stroke-width:2px;stroke:#
 f0f}.vis.timeline .graphGroup9{fill:#8f3938;fill-opacity:0;stroke-width:2px;stroke:#8f3938}.vis.timeline .fill{fill-opacity:.1;stroke:none}.vis.timeline .bar{fill-opacity:.5;stroke-width:1px}.vis.timeline .point{stroke-width:2px;fill-opacity:1}.vis.timeline .legendBackground{stroke-width:1px;fill-opacity:.9;fill:#fff;stroke:#c2c2c2}.vis.timeline .outline{stroke-width:1px;fill-opacity:1;fill:#fff;stroke:#e5e5e5}.vis.timeline .iconFill{fill-opacity:.3;stroke:none}div.network-manipulationDiv{border-width:0;border-bottom:1px;border-style:solid;border-color:#d6d9d8;background:#fff;background:-moz-linear-gradient(top,#fff 0,#fcfcfc 48%,#fafafa 50%,#fcfcfc 100%);background:-webkit-gradient(linear,left top,left bottom,color-stop(0,#fff),color-stop(48%,#fcfcfc),color-stop(50%,#fafafa),color-stop(100%,#fcfcfc));background:-webkit-linear-gradient(top,#fff 0,#fcfcfc 48%,#fafafa 50%,#fcfcfc 100%);background:-o-linear-gradient(top,#fff 0,#fcfcfc 48%,#fafafa 50%,#fcfcfc 100%);background:-ms-linear
 -gradient(top,#fff 0,#fcfcfc 48%,#fafafa 50%,#fcfcfc 100%);background:linear-gradient(to bottom,#fff 0,#fcfcfc 48%,#fafafa 50%,#fcfcfc 100%);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffffff', endColorstr='#fcfcfc', GradientType=0);position:absolute;left:0;top:0;width:100%;height:30px}div.network-manipulation-editMode{position:absolute;left:0;top:0;height:30px;margin-top:20px}div.network-manipulation-closeDiv{position:absolute;right:0;top:0;width:30px;height:30px;background-position:20px 3px;background-repeat:no-repeat;background-image:url(img/network/cross.png);cursor:pointer;-webkit-touch-callout:none;-webkit-user-select:none;-khtml-user-select:none;-moz-user-select:none;-ms-user-select:none;user-select:none}div.network-manipulation-closeDiv:hover{opacity:.6}span.network-manipulationUI{font-family:verdana;font-size:12px;-moz-border-radius:15px;border-radius:15px;display:inline-block;background-position:0 0;background-repeat:no-repeat;height:24px;margin:-14px
  0 0 10px;vertical-align:middle;cursor:pointer;padding:0 8px;-webkit-touch-callout:none;-webkit-user-select:none;-khtml-user-select:none;-moz-user-select:none;-ms-user-select:none;user-select:none}span.network-manipulationUI:hover{box-shadow:1px 1px 8px rgba(0,0,0,.2)}span.network-manipulationUI:active{box-shadow:1px 1px 8px rgba(0,0,0,.5)}span.network-manipulationUI.back{background-image:url(img/network/backIcon.png)}span.network-manipulationUI.none:hover{box-shadow:1px 1px 8px transparent;cursor:default}span.network-manipulationUI.none:active{box-shadow:1px 1px 8px transparent}span.network-manipulationUI.none{padding:0}span.network-manipulationUI.notification{margin:2px;font-weight:700}span.network-manipulationUI.add{background-image:url(img/network/addNodeIcon.png)}span.network-manipulationUI.edit{background-image:url(img/network/editIcon.png)}span.network-manipulationUI.edit.editmode{background-color:#fcfcfc;border-style:solid;border-width:1px;border-color:#ccc}span.network-mani
 pulationUI.connect{background-image:url(img/network/connectIcon.png)}span.network-manipulationUI.delete{background-image:url(img/network/deleteIcon.png)}span.network-manipulationLabel{margin:0 0 0 23px;line-height:25px}div.network-seperatorLine{display:inline-block;width:1px;height:20px;background-color:#bdbdbd;margin:5px 7px 0 15px}div.network-navigation_wrapper{position:absolute;left:0;top:0;width:100%;height:100%}div.network-navigation{width:34px;height:34px;-moz-border-radius:17px;border-radius:17px;position:absolute;display:inline-block;background-position:2px 2px;background-repeat:no-repeat;cursor:pointer;-webkit-touch-callout:none;-webkit-user-select:none;-khtml-user-select:none;-moz-user-select:none;-ms-user-select:none;user-select:none}div.network-navigation:hover{box-shadow:0 0 3px 3px rgba(56,207,21,.3)}div.network-navigation:active{box-shadow:0 0 1px 3px rgba(56,207,21,.95)}div.network-navigation.up{background-image:url(img/network/upArrow.png);bottom:50px;left:55px}div.
 network-navigation.down{background-image:url(img/network/downArrow.png);bottom:10px;left:55px}div.network-navigation.left{background-image:url(img/network/leftArrow.png);bottom:10px;left:15px}div.network-navigation.right{background-image:url(img/network/rightArrow.png);bottom:10px;left:95px}div.network-navigation.zoomIn{background-image:url(img/network/plus.png);bottom:10px;right:15px}div.network-navigation.zoomOut{background-image:url(img/network/minus.png);bottom:10px;right:55px}div.network-navigation.zoomExtends{background-image:url(img/network/zoomExtends.png);bottom:50px;right:15px}
\ No newline at end of file


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


[2/5] spark git commit: [SPARK-3468] [WEBUI] Timeline-View feature

Posted by pw...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/7fe0f3f2/core/src/main/resources/org/apache/spark/ui/static/vis.min.js
----------------------------------------------------------------------
diff --git a/core/src/main/resources/org/apache/spark/ui/static/vis.min.js b/core/src/main/resources/org/apache/spark/ui/static/vis.min.js
new file mode 100644
index 0000000..4af2c81
--- /dev/null
+++ b/core/src/main/resources/org/apache/spark/ui/static/vis.min.js
@@ -0,0 +1,39 @@
+/**
+ * vis.js
+ * https://github.com/almende/vis
+ *
+ * A dynamic, browser-based visualization library.
+ *
+ * @version 3.9.0
+ * @date    2015-01-16
+ *
+ * @license
+ * Copyright (C) 2011-2014 Almende B.V, http://almende.com
+ *
+ * Vis.js is dual licensed under both
+ *
+ * * The Apache 2.0 License
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * and
+ *
+ * * The MIT License
+ *   http://opensource.org/licenses/MIT
+ *
+ * Vis.js may be distributed under either license.
+ */
+"use strict";!function(t,e){"object"==typeof exports&&"object"==typeof module?module.exports=e():"function"==typeof define&&define.amd?define(e):"object"==typeof exports?exports.vis=e():t.vis=e()}(this,function(){return function(t){function e(s){if(i[s])return i[s].exports;var o=i[s]={exports:{},id:s,loaded:!1};return t[s].call(o.exports,o,o.exports,e),o.loaded=!0,o.exports}var i={};return e.m=t,e.c=i,e.p="",e(0)}([function(t,e,i){e.util=i(1),e.DOMutil=i(2),e.DataSet=i(3),e.DataView=i(4),e.Queue=i(5),e.Graph3d=i(6),e.graph3d={Camera:i(7),Filter:i(8),Point2d:i(9),Point3d:i(10),Slider:i(11),StepNumber:i(12)},e.Timeline=i(13),e.Graph2d=i(14),e.timeline={DateUtil:i(15),DataStep:i(16),Range:i(17),stack:i(18),TimeStep:i(19),components:{items:{Item:i(31),BackgroundItem:i(32),BoxItem:i(33),PointItem:i(34),RangeItem:i(35)},Component:i(20),CurrentTime:i(21),CustomTime:i(22),DataAxis:i(23),GraphGroup:i(24),Group:i(25),BackgroundGroup:i(26),ItemSet:i(27),Legend:i(28),LineGraph:i(29),TimeAxis:i(
 30)}},e.Network=i(36),e.network={Edge:i(37),Groups:i(38),Images:i(39),Node:i(40),Popup:i(41),dotparser:i(42),gephiParser:i(43)},e.Graph=function(){throw new Error("Graph is renamed to Network. Please create a graph as new vis.Network(...)")},e.moment=i(44),e.hammer=i(45),e.Hammer=i(45)},function(t,e,i){var s=i(44);e.isNumber=function(t){return t instanceof Number||"number"==typeof t},e.isString=function(t){return t instanceof String||"string"==typeof t},e.isDate=function(t){if(t instanceof Date)return!0;if(e.isString(t)){var i=o.exec(t);if(i)return!0;if(!isNaN(Date.parse(t)))return!0}return!1},e.isDataTable=function(t){return"undefined"!=typeof google&&google.visualization&&google.visualization.DataTable&&t instanceof google.visualization.DataTable},e.randomUUID=function(){var t=function(){return Math.floor(65536*Math.random()).toString(16)};return t()+t()+"-"+t()+"-"+t()+"-"+t()+"-"+t()+t()+t()},e.extend=function(t){for(var e=1,i=arguments.length;i>e;e++){var s=arguments[e];for(var
  o in s)s.hasOwnProperty(o)&&(t[o]=s[o])}return t},e.selectiveExtend=function(t,e){if(!Array.isArray(t))throw new Error("Array with property names expected as first argument");for(var i=2;i<arguments.length;i++)for(var s=arguments[i],o=0;o<t.length;o++){var n=t[o];s.hasOwnProperty(n)&&(e[n]=s[n])}return e},e.selectiveDeepExtend=function(t,i,s){if(Array.isArray(s))throw new TypeError("Arrays are not supported by deepExtend");for(var o=2;o<arguments.length;o++)for(var n=arguments[o],r=0;r<t.length;r++){var a=t[r];if(n.hasOwnProperty(a))if(s[a]&&s[a].constructor===Object)void 0===i[a]&&(i[a]={}),i[a].constructor===Object?e.deepExtend(i[a],s[a]):i[a]=s[a];else{if(Array.isArray(s[a]))throw new TypeError("Arrays are not supported by deepExtend");i[a]=s[a]}}return i},e.selectiveNotDeepExtend=function(t,i,s){if(Array.isArray(s))throw new TypeError("Arrays are not supported by deepExtend");for(var o in s)if(s.hasOwnProperty(o)&&-1==t.indexOf(o))if(s[o]&&s[o].constructor===Object)void 0===i[o
 ]&&(i[o]={}),i[o].constructor===Object?e.deepExtend(i[o],s[o]):i[o]=s[o];else{if(Array.isArray(s[o]))throw new TypeError("Arrays are not supported by deepExtend");i[o]=s[o]}return i},e.deepExtend=function(t,i){if(Array.isArray(i))throw new TypeError("Arrays are not supported by deepExtend");for(var s in i)if(i.hasOwnProperty(s))if(i[s]&&i[s].constructor===Object)void 0===t[s]&&(t[s]={}),t[s].constructor===Object?e.deepExtend(t[s],i[s]):t[s]=i[s];else{if(Array.isArray(i[s]))throw new TypeError("Arrays are not supported by deepExtend");t[s]=i[s]}return t},e.equalArray=function(t,e){if(t.length!=e.length)return!1;for(var i=0,s=t.length;s>i;i++)if(t[i]!=e[i])return!1;return!0},e.convert=function(t,i){var n;if(void 0===t)return void 0;if(null===t)return null;if(!i)return t;if("string"!=typeof i&&!(i instanceof String))throw new Error("Type must be a string");switch(i){case"boolean":case"Boolean":return Boolean(t);case"number":case"Number":return Number(t.valueOf());case"string":case"Stri
 ng":return String(t);case"Date":if(e.isNumber(t))return new Date(t);if(t instanceof Date)return new Date(t.valueOf());if(s.isMoment(t))return new Date(t.valueOf());if(e.isString(t))return n=o.exec(t),n?new Date(Number(n[1])):s(t).toDate();throw new Error("Cannot convert object of type "+e.getType(t)+" to type Date");case"Moment":if(e.isNumber(t))return s(t);if(t instanceof Date)return s(t.valueOf());if(s.isMoment(t))return s(t);if(e.isString(t))return n=o.exec(t),s(n?Number(n[1]):t);throw new Error("Cannot convert object of type "+e.getType(t)+" to type Date");case"ISODate":if(e.isNumber(t))return new Date(t);if(t instanceof Date)return t.toISOString();if(s.isMoment(t))return t.toDate().toISOString();if(e.isString(t))return n=o.exec(t),n?new Date(Number(n[1])).toISOString():new Date(t).toISOString();throw new Error("Cannot convert object of type "+e.getType(t)+" to type ISODate");case"ASPDate":if(e.isNumber(t))return"/Date("+t+")/";if(t instanceof Date)return"/Date("+t.valueOf()+")/
 ";if(e.isString(t)){n=o.exec(t);var r;return r=n?new Date(Number(n[1])).valueOf():new Date(t).valueOf(),"/Date("+r+")/"}throw new Error("Cannot convert object of type "+e.getType(t)+" to type ASPDate");default:throw new Error('Unknown type "'+i+'"')}};var o=/^\/?Date\((\-?\d+)/i;e.getType=function(t){var e=typeof t;return"object"==e?null==t?"null":t instanceof Boolean?"Boolean":t instanceof Number?"Number":t instanceof String?"String":Array.isArray(t)?"Array":t instanceof Date?"Date":"Object":"number"==e?"Number":"boolean"==e?"Boolean":"string"==e?"String":e},e.getAbsoluteLeft=function(t){return t.getBoundingClientRect().left},e.getAbsoluteTop=function(t){return t.getBoundingClientRect().top},e.addClassName=function(t,e){var i=t.className.split(" ");-1==i.indexOf(e)&&(i.push(e),t.className=i.join(" "))},e.removeClassName=function(t,e){var i=t.className.split(" "),s=i.indexOf(e);-1!=s&&(i.splice(s,1),t.className=i.join(" "))},e.forEach=function(t,e){var i,s;if(Array.isArray(t))for(i=
 0,s=t.length;s>i;i++)e(t[i],i,t);else for(i in t)t.hasOwnProperty(i)&&e(t[i],i,t)},e.toArray=function(t){var e=[];for(var i in t)t.hasOwnProperty(i)&&e.push(t[i]);return e},e.updateProperty=function(t,e,i){return t[e]!==i?(t[e]=i,!0):!1},e.addEventListener=function(t,e,i,s){t.addEventListener?(void 0===s&&(s=!1),"mousewheel"===e&&navigator.userAgent.indexOf("Firefox")>=0&&(e="DOMMouseScroll"),t.addEventListener(e,i,s)):t.attachEvent("on"+e,i)},e.removeEventListener=function(t,e,i,s){t.removeEventListener?(void 0===s&&(s=!1),"mousewheel"===e&&navigator.userAgent.indexOf("Firefox")>=0&&(e="DOMMouseScroll"),t.removeEventListener(e,i,s)):t.detachEvent("on"+e,i)},e.preventDefault=function(t){t||(t=window.event),t.preventDefault?t.preventDefault():t.returnValue=!1},e.getTarget=function(t){t||(t=window.event);var e;return t.target?e=t.target:t.srcElement&&(e=t.srcElement),void 0!=e.nodeType&&3==e.nodeType&&(e=e.parentNode),e},e.option={},e.option.asBoolean=function(t,e){return"function"==t
 ypeof t&&(t=t()),null!=t?0!=t:e||null},e.option.asNumber=function(t,e){return"function"==typeof t&&(t=t()),null!=t?Number(t)||e||null:e||null},e.option.asString=function(t,e){return"function"==typeof t&&(t=t()),null!=t?String(t):e||null},e.option.asSize=function(t,i){return"function"==typeof t&&(t=t()),e.isString(t)?t:e.isNumber(t)?t+"px":i||null},e.option.asElement=function(t,e){return"function"==typeof t&&(t=t()),t||e||null},e.hexToRGB=function(t){var e=/^#?([a-f\d])([a-f\d])([a-f\d])$/i;t=t.replace(e,function(t,e,i,s){return e+e+i+i+s+s});var i=/^#?([a-f\d]{2})([a-f\d]{2})([a-f\d]{2})$/i.exec(t);return i?{r:parseInt(i[1],16),g:parseInt(i[2],16),b:parseInt(i[3],16)}:null},e.RGBToHex=function(t,e,i){return"#"+((1<<24)+(t<<16)+(e<<8)+i).toString(16).slice(1)},e.parseColor=function(t){var i;if(e.isString(t)){if(e.isValidRGB(t)){var s=t.substr(4).substr(0,t.length-5).split(",");t=e.RGBToHex(s[0],s[1],s[2])}if(e.isValidHex(t)){var o=e.hexToHSV(t),n={h:o.h,s:.45*o.s,v:Math.min(1,1.05*o.
 v)},r={h:o.h,s:Math.min(1,1.25*o.v),v:.6*o.v},a=e.HSVToHex(r.h,r.h,r.v),h=e.HSVToHex(n.h,n.s,n.v);i={background:t,border:a,highlight:{background:h,border:a},hover:{background:h,border:a}}}else i={background:t,border:t,highlight:{background:t,border:t},hover:{background:t,border:t}}}else i={},i.background=t.background||"white",i.border=t.border||i.background,e.isString(t.highlight)?i.highlight={border:t.highlight,background:t.highlight}:(i.highlight={},i.highlight.background=t.highlight&&t.highlight.background||i.background,i.highlight.border=t.highlight&&t.highlight.border||i.border),e.isString(t.hover)?i.hover={border:t.hover,background:t.hover}:(i.hover={},i.hover.background=t.hover&&t.hover.background||i.background,i.hover.border=t.hover&&t.hover.border||i.border);return i},e.RGBToHSV=function(t,e,i){t/=255,e/=255,i/=255;var s=Math.min(t,Math.min(e,i)),o=Math.max(t,Math.max(e,i));if(s==o)return{h:0,s:0,v:s};var n=t==s?e-i:i==s?t-e:i-t,r=t==s?3:i==s?1:5,a=60*(r-n/(o-s))/360,h=(o-s
 )/o,d=o;return{h:a,s:h,v:d}};var n={split:function(t){var e={};return t.split(";").forEach(function(t){if(""!=t.trim()){var i=t.split(":"),s=i[0].trim(),o=i[1].trim();e[s]=o}}),e},join:function(t){return Object.keys(t).map(function(e){return e+": "+t[e]}).join("; ")}};e.addCssText=function(t,i){var s=n.split(t.style.cssText),o=n.split(i),r=e.extend(s,o);t.style.cssText=n.join(r)},e.removeCssText=function(t,e){var i=n.split(t.style.cssText),s=n.split(e);for(var o in s)s.hasOwnProperty(o)&&delete i[o];t.style.cssText=n.join(i)},e.HSVToRGB=function(t,e,i){var s,o,n,r=Math.floor(6*t),a=6*t-r,h=i*(1-e),d=i*(1-a*e),l=i*(1-(1-a)*e);switch(r%6){case 0:s=i,o=l,n=h;break;case 1:s=d,o=i,n=h;break;case 2:s=h,o=i,n=l;break;case 3:s=h,o=d,n=i;break;case 4:s=l,o=h,n=i;break;case 5:s=i,o=h,n=d}return{r:Math.floor(255*s),g:Math.floor(255*o),b:Math.floor(255*n)}},e.HSVToHex=function(t,i,s){var o=e.HSVToRGB(t,i,s);return e.RGBToHex(o.r,o.g,o.b)},e.hexToHSV=function(t){var i=e.hexToRGB(t);return e.RGBT
 oHSV(i.r,i.g,i.b)},e.isValidHex=function(t){var e=/(^#[0-9A-F]{6}$)|(^#[0-9A-F]{3}$)/i.test(t);return e},e.isValidRGB=function(t){t=t.replace(" ","");var e=/rgb\((\d{1,3}),(\d{1,3}),(\d{1,3})\)/i.test(t);return e},e.selectiveBridgeObject=function(t,i){if("object"==typeof i){for(var s=Object.create(i),o=0;o<t.length;o++)i.hasOwnProperty(t[o])&&"object"==typeof i[t[o]]&&(s[t[o]]=e.bridgeObject(i[t[o]]));return s}return null},e.bridgeObject=function(t){if("object"==typeof t){var i=Object.create(t);for(var s in t)t.hasOwnProperty(s)&&"object"==typeof t[s]&&(i[s]=e.bridgeObject(t[s]));return i}return null},e.mergeOptions=function(t,e,i){if(void 0!==e[i])if("boolean"==typeof e[i])t[i].enabled=e[i];else{t[i].enabled=!0;for(var s in e[i])e[i].hasOwnProperty(s)&&(t[i][s]=e[i][s])}},e.binarySearchCustom=function(t,e,i,s){for(var o=1e4,n=0,r=0,a=t.length-1;a>=r&&o>n;){var h=Math.floor((r+a)/2),d=t[h],l=void 0===s?d[i]:d[i][s],c=e(l);if(0==c)return h;-1==c?r=h+1:a=h-1,n++}return-1},e.binarySear
 chValue=function(t,e,i,s){for(var o,n,r,a,h=1e4,d=0,l=0,c=t.length-1;c>=l&&h>d;){if(a=Math.floor(.5*(c+l)),o=t[Math.max(0,a-1)][i],n=t[a][i],r=t[Math.min(t.length-1,a+1)][i],n==e)return a;if(e>o&&n>e)return"before"==s?Math.max(0,a-1):a;if(e>n&&r>e)return"before"==s?a:Math.min(t.length-1,a+1);e>n?l=a+1:c=a-1,d++}return-1},e.easeInOutQuad=function(t,e,i,s){var o=i-e;return t/=s/2,1>t?o/2*t*t+e:(t--,-o/2*(t*(t-2)-1)+e)},e.easingFunctions={linear:function(t){return t},easeInQuad:function(t){return t*t},easeOutQuad:function(t){return t*(2-t)},easeInOutQuad:function(t){return.5>t?2*t*t:-1+(4-2*t)*t},easeInCubic:function(t){return t*t*t},easeOutCubic:function(t){return--t*t*t+1},easeInOutCubic:function(t){return.5>t?4*t*t*t:(t-1)*(2*t-2)*(2*t-2)+1},easeInQuart:function(t){return t*t*t*t},easeOutQuart:function(t){return 1- --t*t*t*t},easeInOutQuart:function(t){return.5>t?8*t*t*t*t:1-8*--t*t*t*t},easeInQuint:function(t){return t*t*t*t*t},easeOutQuint:function(t){return 1+--t*t*t*t*t},easeInO
 utQuint:function(t){return.5>t?16*t*t*t*t*t:1+16*--t*t*t*t*t}}},function(t,e){e.prepareElements=function(t){for(var e in t)t.hasOwnProperty(e)&&(t[e].redundant=t[e].used,t[e].used=[])},e.cleanupElements=function(t){for(var e in t)if(t.hasOwnProperty(e)&&t[e].redundant){for(var i=0;i<t[e].redundant.length;i++)t[e].redundant[i].parentNode.removeChild(t[e].redundant[i]);t[e].redundant=[]}},e.getSVGElement=function(t,e,i){var s;return e.hasOwnProperty(t)?e[t].redundant.length>0?(s=e[t].redundant[0],e[t].redundant.shift()):(s=document.createElementNS("http://www.w3.org/2000/svg",t),i.appendChild(s)):(s=document.createElementNS("http://www.w3.org/2000/svg",t),e[t]={used:[],redundant:[]},i.appendChild(s)),e[t].used.push(s),s},e.getDOMElement=function(t,e,i,s){var o;return e.hasOwnProperty(t)?e[t].redundant.length>0?(o=e[t].redundant[0],e[t].redundant.shift()):(o=document.createElement(t),void 0!==s?i.insertBefore(o,s):i.appendChild(o)):(o=document.createElement(t),e[t]={used:[],redundant:[
 ]},void 0!==s?i.insertBefore(o,s):i.appendChild(o)),e[t].used.push(o),o},e.drawPoint=function(t,i,s,o,n){var r;return"circle"==s.options.drawPoints.style?(r=e.getSVGElement("circle",o,n),r.setAttributeNS(null,"cx",t),r.setAttributeNS(null,"cy",i),r.setAttributeNS(null,"r",.5*s.options.drawPoints.size)):(r=e.getSVGElement("rect",o,n),r.setAttributeNS(null,"x",t-.5*s.options.drawPoints.size),r.setAttributeNS(null,"y",i-.5*s.options.drawPoints.size),r.setAttributeNS(null,"width",s.options.drawPoints.size),r.setAttributeNS(null,"height",s.options.drawPoints.size)),void 0!==s.options.drawPoints.styles&&r.setAttributeNS(null,"style",s.group.options.drawPoints.styles),r.setAttributeNS(null,"class",s.className+" point"),r},e.drawBar=function(t,i,s,o,n,r,a){if(0!=o){0>o&&(o*=-1,i-=o);var h=e.getSVGElement("rect",r,a);h.setAttributeNS(null,"x",t-.5*s),h.setAttributeNS(null,"y",i),h.setAttributeNS(null,"width",s),h.setAttributeNS(null,"height",o),h.setAttributeNS(null,"class",n)}}},function(t,
 e,i){function s(t,e){if(!t||Array.isArray(t)||o.isDataTable(t)||(e=t,t=null),this._options=e||{},this._data={},this._fieldId=this._options.fieldId||"id",this._type={},this._options.type)for(var i in this._options.type)if(this._options.type.hasOwnProperty(i)){var s=this._options.type[i];this._type[i]="Date"==s||"ISODate"==s||"ASPDate"==s?"Date":s}if(this._options.convert)throw new Error('Option "convert" is deprecated. Use "type" instead.');this._subscribers={},t&&this.add(t),this.setOptions(e)}var o=i(1),n=i(5);s.prototype.setOptions=function(t){t&&void 0!==t.queue&&(t.queue===!1?this._queue&&(this._queue.destroy(),delete this._queue):(this._queue||(this._queue=n.extend(this,{replace:["add","update","remove"]})),"object"==typeof t.queue&&this._queue.setOptions(t.queue)))},s.prototype.on=function(t,e){var i=this._subscribers[t];i||(i=[],this._subscribers[t]=i),i.push({callback:e})},s.prototype.subscribe=s.prototype.on,s.prototype.off=function(t,e){var i=this._subscribers[t];i&&(this.
 _subscribers[t]=i.filter(function(t){return t.callback!=e}))},s.prototype.unsubscribe=s.prototype.off,s.prototype._trigger=function(t,e,i){if("*"==t)throw new Error("Cannot trigger event *");var s=[];t in this._subscribers&&(s=s.concat(this._subscribers[t])),"*"in this._subscribers&&(s=s.concat(this._subscribers["*"]));for(var o=0;o<s.length;o++){var n=s[o];n.callback&&n.callback(t,e,i||null)}},s.prototype.add=function(t,e){var i,s=[],n=this;if(Array.isArray(t))for(var r=0,a=t.length;a>r;r++)i=n._addItem(t[r]),s.push(i);else if(o.isDataTable(t))for(var h=this._getColumnNames(t),d=0,l=t.getNumberOfRows();l>d;d++){for(var c={},p=0,u=h.length;u>p;p++){var m=h[p];c[m]=t.getValue(d,p)}i=n._addItem(c),s.push(i)}else{if(!(t instanceof Object))throw new Error("Unknown dataType");i=n._addItem(t),s.push(i)}return s.length&&this._trigger("add",{items:s},e),s},s.prototype.update=function(t,e){var i=[],s=[],n=[],r=this,a=r._fieldId,h=function(t){var e=t[a];r._data[e]?(e=r._updateItem(t),s.push(e
 ),n.push(t)):(e=r._addItem(t),i.push(e))};if(Array.isArray(t))for(var d=0,l=t.length;l>d;d++)h(t[d]);else if(o.isDataTable(t))for(var c=this._getColumnNames(t),p=0,u=t.getNumberOfRows();u>p;p++){for(var m={},f=0,g=c.length;g>f;f++){var v=c[f];m[v]=t.getValue(p,f)}h(m)}else{if(!(t instanceof Object))throw new Error("Unknown dataType");h(t)}return i.length&&this._trigger("add",{items:i},e),s.length&&this._trigger("update",{items:s,data:n},e),i.concat(s)},s.prototype.get=function(){var t,e,i,s,n=this,r=o.getType(arguments[0]);"String"==r||"Number"==r?(t=arguments[0],i=arguments[1],s=arguments[2]):"Array"==r?(e=arguments[0],i=arguments[1],s=arguments[2]):(i=arguments[0],s=arguments[1]);var a;if(i&&i.returnType){var h=["DataTable","Array","Object"];if(a=-1==h.indexOf(i.returnType)?"Array":i.returnType,s&&a!=o.getType(s))throw new Error('Type of parameter "data" ('+o.getType(s)+") does not correspond with specified options.type ("+i.type+")");if("DataTable"==a&&!o.isDataTable(s))throw new
  Error('Parameter "data" must be a DataTable when options.type is "DataTable"')}else a=s&&"DataTable"==o.getType(s)?"DataTable":"Array";var d,l,c,p,u=i&&i.type||this._options.type,m=i&&i.filter,f=[];if(void 0!=t)d=n._getItem(t,u),m&&!m(d)&&(d=null);else if(void 0!=e)for(c=0,p=e.length;p>c;c++)d=n._getItem(e[c],u),(!m||m(d))&&f.push(d);else for(l in this._data)this._data.hasOwnProperty(l)&&(d=n._getItem(l,u),(!m||m(d))&&f.push(d));if(i&&i.order&&void 0==t&&this._sort(f,i.order),i&&i.fields){var g=i.fields;if(void 0!=t)d=this._filterFields(d,g);else for(c=0,p=f.length;p>c;c++)f[c]=this._filterFields(f[c],g)}if("DataTable"==a){var v=this._getColumnNames(s);if(void 0!=t)n._appendRow(s,v,d);else for(c=0;c<f.length;c++)n._appendRow(s,v,f[c]);return s}if("Object"==a){var y={};for(c=0;c<f.length;c++)y[f[c].id]=f[c];return y}if(void 0!=t)return d;if(s){for(c=0,p=f.length;p>c;c++)s.push(f[c]);return s}return f},s.prototype.getIds=function(t){var e,i,s,o,n,r=this._data,a=t&&t.filter,h=t&&t.ord
 er,d=t&&t.type||this._options.type,l=[];if(a)if(h){n=[];for(s in r)r.hasOwnProperty(s)&&(o=this._getItem(s,d),a(o)&&n.push(o));for(this._sort(n,h),e=0,i=n.length;i>e;e++)l[e]=n[e][this._fieldId]}else for(s in r)r.hasOwnProperty(s)&&(o=this._getItem(s,d),a(o)&&l.push(o[this._fieldId]));else if(h){n=[];for(s in r)r.hasOwnProperty(s)&&n.push(r[s]);for(this._sort(n,h),e=0,i=n.length;i>e;e++)l[e]=n[e][this._fieldId]}else for(s in r)r.hasOwnProperty(s)&&(o=r[s],l.push(o[this._fieldId]));return l},s.prototype.getDataSet=function(){return this},s.prototype.forEach=function(t,e){var i,s,o=e&&e.filter,n=e&&e.type||this._options.type,r=this._data;if(e&&e.order)for(var a=this.get(e),h=0,d=a.length;d>h;h++)i=a[h],s=i[this._fieldId],t(i,s);else for(s in r)r.hasOwnProperty(s)&&(i=this._getItem(s,n),(!o||o(i))&&t(i,s))},s.prototype.map=function(t,e){var i,s=e&&e.filter,o=e&&e.type||this._options.type,n=[],r=this._data;for(var a in r)r.hasOwnProperty(a)&&(i=this._getItem(a,o),(!s||s(i))&&n.push(t(i,
 a)));return e&&e.order&&this._sort(n,e.order),n},s.prototype._filterFields=function(t,e){var i={};for(var s in t)t.hasOwnProperty(s)&&-1!=e.indexOf(s)&&(i[s]=t[s]);return i},s.prototype._sort=function(t,e){if(o.isString(e)){var i=e;t.sort(function(t,e){var s=t[i],o=e[i];return s>o?1:o>s?-1:0})}else{if("function"!=typeof e)throw new TypeError("Order must be a function or a string");t.sort(e)}},s.prototype.remove=function(t,e){var i,s,o,n=[];if(Array.isArray(t))for(i=0,s=t.length;s>i;i++)o=this._remove(t[i]),null!=o&&n.push(o);else o=this._remove(t),null!=o&&n.push(o);return n.length&&this._trigger("remove",{items:n},e),n},s.prototype._remove=function(t){if(o.isNumber(t)||o.isString(t)){if(this._data[t])return delete this._data[t],t}else if(t instanceof Object){var e=t[this._fieldId];if(e&&this._data[e])return delete this._data[e],e}return null},s.prototype.clear=function(t){var e=Object.keys(this._data);return this._data={},this._trigger("remove",{items:e},t),e},s.prototype.max=funct
 ion(t){var e=this._data,i=null,s=null;for(var o in e)if(e.hasOwnProperty(o)){var n=e[o],r=n[t];null!=r&&(!i||r>s)&&(i=n,s=r)}return i},s.prototype.min=function(t){var e=this._data,i=null,s=null;for(var o in e)if(e.hasOwnProperty(o)){var n=e[o],r=n[t];null!=r&&(!i||s>r)&&(i=n,s=r)}return i},s.prototype.distinct=function(t){var e,i=this._data,s=[],n=this._options.type&&this._options.type[t]||null,r=0;for(var a in i)if(i.hasOwnProperty(a)){var h=i[a],d=h[t],l=!1;for(e=0;r>e;e++)if(s[e]==d){l=!0;break}l||void 0===d||(s[r]=d,r++)}if(n)for(e=0;e<s.length;e++)s[e]=o.convert(s[e],n);return s},s.prototype._addItem=function(t){var e=t[this._fieldId];if(void 0!=e){if(this._data[e])throw new Error("Cannot add item: item with id "+e+" already exists")}else e=o.randomUUID(),t[this._fieldId]=e;var i={};for(var s in t)if(t.hasOwnProperty(s)){var n=this._type[s];i[s]=o.convert(t[s],n)}return this._data[e]=i,e},s.prototype._getItem=function(t,e){var i,s,n=this._data[t];if(!n)return null;var r={};if(e
 )for(i in n)n.hasOwnProperty(i)&&(s=n[i],r[i]=o.convert(s,e[i]));else for(i in n)n.hasOwnProperty(i)&&(s=n[i],r[i]=s);return r},s.prototype._updateItem=function(t){var e=t[this._fieldId];if(void 0==e)throw new Error("Cannot update item: item has no id (item: "+JSON.stringify(t)+")");var i=this._data[e];if(!i)throw new Error("Cannot update item: no item with id "+e+" found");for(var s in t)if(t.hasOwnProperty(s)){var n=this._type[s];i[s]=o.convert(t[s],n)}return e},s.prototype._getColumnNames=function(t){for(var e=[],i=0,s=t.getNumberOfColumns();s>i;i++)e[i]=t.getColumnId(i)||t.getColumnLabel(i);return e},s.prototype._appendRow=function(t,e,i){for(var s=t.addRow(),o=0,n=e.length;n>o;o++){var r=e[o];t.setValue(s,o,i[r])}},t.exports=s},function(t,e,i){function s(t,e){this._data=null,this._ids={},this._options=e||{},this._fieldId="id",this._subscribers={};var i=this;this.listener=function(){i._onEvent.apply(i,arguments)},this.setData(t)}var o=i(1),n=i(3);s.prototype.setData=function(t){
 var e,i,s;if(this._data){this._data.unsubscribe&&this._data.unsubscribe("*",this.listener),e=[];for(var o in this._ids)this._ids.hasOwnProperty(o)&&e.push(o);this._ids={},this._trigger("remove",{items:e})}if(this._data=t,this._data){for(this._fieldId=this._options.fieldId||this._data&&this._data.options&&this._data.options.fieldId||"id",e=this._data.getIds({filter:this._options&&this._options.filter}),i=0,s=e.length;s>i;i++)o=e[i],this._ids[o]=!0;this._trigger("add",{items:e}),this._data.on&&this._data.on("*",this.listener)}},s.prototype.get=function(){var t,e,i,s=this,n=o.getType(arguments[0]);"String"==n||"Number"==n||"Array"==n?(t=arguments[0],e=arguments[1],i=arguments[2]):(e=arguments[0],i=arguments[1]);var r=o.extend({},this._options,e);this._options.filter&&e&&e.filter&&(r.filter=function(t){return s._options.filter(t)&&e.filter(t)});var a=[];return void 0!=t&&a.push(t),a.push(r),a.push(i),this._data&&this._data.get.apply(this._data,a)},s.prototype.getIds=function(t){var e;if
 (this._data){var i,s=this._options.filter;i=t&&t.filter?s?function(e){return s(e)&&t.filter(e)}:t.filter:s,e=this._data.getIds({filter:i,order:t&&t.order})}else e=[];return e},s.prototype.getDataSet=function(){for(var t=this;t instanceof s;)t=t._data;return t||null},s.prototype._onEvent=function(t,e,i){var s,o,n,r,a=e&&e.items,h=this._data,d=[],l=[],c=[];if(a&&h){switch(t){case"add":for(s=0,o=a.length;o>s;s++)n=a[s],r=this.get(n),r&&(this._ids[n]=!0,d.push(n));break;case"update":for(s=0,o=a.length;o>s;s++)n=a[s],r=this.get(n),r?this._ids[n]?l.push(n):(this._ids[n]=!0,d.push(n)):this._ids[n]&&(delete this._ids[n],c.push(n));break;case"remove":for(s=0,o=a.length;o>s;s++)n=a[s],this._ids[n]&&(delete this._ids[n],c.push(n))}d.length&&this._trigger("add",{items:d},i),l.length&&this._trigger("update",{items:l},i),c.length&&this._trigger("remove",{items:c},i)}},s.prototype.on=n.prototype.on,s.prototype.off=n.prototype.off,s.prototype._trigger=n.prototype._trigger,s.prototype.subscribe=s.pr
 ototype.on,s.prototype.unsubscribe=s.prototype.off,t.exports=s},function(t){function e(t){this.delay=null,this.max=1/0,this._queue=[],this._timeout=null,this._extended=null,this.setOptions(t)}e.prototype.setOptions=function(t){t&&"undefined"!=typeof t.delay&&(this.delay=t.delay),t&&"undefined"!=typeof t.max&&(this.max=t.max),this._flushIfNeeded()},e.extend=function(t,i){var s=new e(i);if(void 0!==t.flush)throw new Error("Target object already has a property flush");t.flush=function(){s.flush()};var o=[{name:"flush",original:void 0}];if(i&&i.replace)for(var n=0;n<i.replace.length;n++){var r=i.replace[n];o.push({name:r,original:t[r]}),s.replace(t,r)}return s._extended={object:t,methods:o},s},e.prototype.destroy=function(){if(this.flush(),this._extended){for(var t=this._extended.object,e=this._extended.methods,i=0;i<e.length;i++){var s=e[i];s.original?t[s.name]=s.original:delete t[s.name]}this._extended=null}},e.prototype.replace=function(t,e){var i=this,s=t[e];if(!s)throw new Error("M
 ethod "+e+" undefined");t[e]=function(){for(var t=[],e=0;e<arguments.length;e++)t[e]=arguments[e];i.queue({args:t,fn:s,context:this})}},e.prototype.queue=function(t){this._queue.push("function"==typeof t?{fn:t}:t),this._flushIfNeeded()},e.prototype._flushIfNeeded=function(){if(this._queue.length>this.max&&this.flush(),clearTimeout(this._timeout),this.queue.length>0&&"number"==typeof this.delay){var t=this;this._timeout=setTimeout(function(){t.flush()},this.delay)}},e.prototype.flush=function(){for(;this._queue.length>0;){var t=this._queue.shift();t.fn.apply(t.context||t.fn,t.args||[])}},t.exports=e},function(t,e,i){function s(t,e,i){if(!(this instanceof s))throw new SyntaxError("Constructor must be called with the new operator");this.containerElement=t,this.width="400px",this.height="400px",this.margin=10,this.defaultXCenter="55%",this.defaultYCenter="50%",this.xLabel="x",this.yLabel="y",this.zLabel="z";var o=function(t){return t};this.xValueLabel=o,this.yValueLabel=o,this.zValueLab
 el=o,this.filterLabel="time",this.legendLabel="value",this.style=s.STYLE.DOT,this.showPerspective=!0,this.showGrid=!0,this.keepAspectRatio=!0,this.showShadow=!1,this.showGrayBottom=!1,this.showTooltip=!1,this.verticalRatio=.5,this.animationInterval=1e3,this.animationPreload=!1,this.camera=new p,this.eye=new l(0,0,-1),this.dataTable=null,this.dataPoints=null,this.colX=void 0,this.colY=void 0,this.colZ=void 0,this.colValue=void 0,this.colFilter=void 0,this.xMin=0,this.xStep=void 0,this.xMax=1,this.yMin=0,this.yStep=void 0,this.yMax=1,this.zMin=0,this.zStep=void 0,this.zMax=1,this.valueMin=0,this.valueMax=1,this.xBarWidth=1,this.yBarWidth=1,this.colorAxis="#4D4D4D",this.colorGrid="#D3D3D3",this.colorDot="#7DC1FF",this.colorDotBorder="#3267D2",this.create(),this.setOptions(i),e&&this.setData(e)}function o(t){return"clientX"in t?t.clientX:t.targetTouches[0]&&t.targetTouches[0].clientX||0}function n(t){return"clientY"in t?t.clientY:t.targetTouches[0]&&t.targetTouches[0].clientY||0}var r=i
 (56),a=i(3),h=i(4),d=i(1),l=i(10),c=i(9),p=i(7),u=i(8),m=i(11),f=i(12);r(s.prototype),s.prototype._setScale=function(){this.scale=new l(1/(this.xMax-this.xMin),1/(this.yMax-this.yMin),1/(this.zMax-this.zMin)),this.keepAspectRatio&&(this.scale.x<this.scale.y?this.scale.y=this.scale.x:this.scale.x=this.scale.y),this.scale.z*=this.verticalRatio,this.scale.value=1/(this.valueMax-this.valueMin);var t=(this.xMax+this.xMin)/2*this.scale.x,e=(this.yMax+this.yMin)/2*this.scale.y,i=(this.zMax+this.zMin)/2*this.scale.z;this.camera.setArmLocation(t,e,i)},s.prototype._convert3Dto2D=function(t){var e=this._convertPointToTranslation(t);return this._convertTranslationToScreen(e)},s.prototype._convertPointToTranslation=function(t){var e=t.x*this.scale.x,i=t.y*this.scale.y,s=t.z*this.scale.z,o=this.camera.getCameraLocation().x,n=this.camera.getCameraLocation().y,r=this.camera.getCameraLocation().z,a=Math.sin(this.camera.getCameraRotation().x),h=Math.cos(this.camera.getCameraRotation().x),d=Math.sin(t
 his.camera.getCameraRotation().y),c=Math.cos(this.camera.getCameraRotation().y),p=Math.sin(this.camera.getCameraRotation().z),u=Math.cos(this.camera.getCameraRotation().z),m=c*(p*(i-n)+u*(e-o))-d*(s-r),f=a*(c*(s-r)+d*(p*(i-n)+u*(e-o)))+h*(u*(i-n)-p*(e-o)),g=h*(c*(s-r)+d*(p*(i-n)+u*(e-o)))-a*(u*(i-n)-p*(e-o));return new l(m,f,g)},s.prototype._convertTranslationToScreen=function(t){var e,i,s=this.eye.x,o=this.eye.y,n=this.eye.z,r=t.x,a=t.y,h=t.z;return this.showPerspective?(e=(r-s)*(n/h),i=(a-o)*(n/h)):(e=r*-(n/this.camera.getArmLength()),i=a*-(n/this.camera.getArmLength())),new c(this.xcenter+e*this.frame.canvas.clientWidth,this.ycenter-i*this.frame.canvas.clientWidth)},s.prototype._setBackgroundColor=function(t){var e="white",i="gray",s=1;if("string"==typeof t)e=t,i="none",s=0;else if("object"==typeof t)void 0!==t.fill&&(e=t.fill),void 0!==t.stroke&&(i=t.stroke),void 0!==t.strokeWidth&&(s=t.strokeWidth);else if(void 0!==t)throw"Unsupported type of backgroundColor";this.frame.style.b
 ackgroundColor=e,this.frame.style.borderColor=i,this.frame.style.borderWidth=s+"px",this.frame.style.borderStyle="solid"},s.STYLE={BAR:0,BARCOLOR:1,BARSIZE:2,DOT:3,DOTLINE:4,DOTCOLOR:5,DOTSIZE:6,GRID:7,LINE:8,SURFACE:9},s.prototype._getStyleNumber=function(t){switch(t){case"dot":return s.STYLE.DOT;case"dot-line":return s.STYLE.DOTLINE;case"dot-color":return s.STYLE.DOTCOLOR;case"dot-size":return s.STYLE.DOTSIZE;case"line":return s.STYLE.LINE;case"grid":return s.STYLE.GRID;case"surface":return s.STYLE.SURFACE;case"bar":return s.STYLE.BAR;case"bar-color":return s.STYLE.BARCOLOR;case"bar-size":return s.STYLE.BARSIZE}return-1},s.prototype._determineColumnIndexes=function(t){if(this.style===s.STYLE.DOT||this.style===s.STYLE.DOTLINE||this.style===s.STYLE.LINE||this.style===s.STYLE.GRID||this.style===s.STYLE.SURFACE||this.style===s.STYLE.BAR)this.colX=0,this.colY=1,this.colZ=2,this.colValue=void 0,t.getNumberOfColumns()>3&&(this.colFilter=3);else{if(this.style!==s.STYLE.DOTCOLOR&&this.styl
 e!==s.STYLE.DOTSIZE&&this.style!==s.STYLE.BARCOLOR&&this.style!==s.STYLE.BARSIZE)throw'Unknown style "'+this.style+'"';this.colX=0,this.colY=1,this.colZ=2,this.colValue=3,t.getNumberOfColumns()>4&&(this.colFilter=4)}},s.prototype.getNumberOfRows=function(t){return t.length},s.prototype.getNumberOfColumns=function(t){var e=0;for(var i in t[0])t[0].hasOwnProperty(i)&&e++;return e},s.prototype.getDistinctValues=function(t,e){for(var i=[],s=0;s<t.length;s++)-1==i.indexOf(t[s][e])&&i.push(t[s][e]);return i},s.prototype.getColumnRange=function(t,e){for(var i={min:t[0][e],max:t[0][e]},s=0;s<t.length;s++)i.min>t[s][e]&&(i.min=t[s][e]),i.max<t[s][e]&&(i.max=t[s][e]);return i},s.prototype._dataInitialize=function(t){var e=this;if(this.dataSet&&this.dataSet.off("*",this._onChange),void 0!==t){Array.isArray(t)&&(t=new a(t));var i;if(!(t instanceof a||t instanceof h))throw new Error("Array, DataSet, or DataView expected");if(i=t.get(),0!=i.length){this.dataSet=t,this.dataTable=i,this._onChange=f
 unction(){e.setData(e.dataSet)},this.dataSet.on("*",this._onChange),this.colX="x",this.colY="y",this.colZ="z",this.colValue="style",this.colFilter="filter",i[0].hasOwnProperty("filter")&&void 0===this.dataFilter&&(this.dataFilter=new u(t,this.colFilter,this),this.dataFilter.setOnLoadCallback(function(){e.redraw()}));var o=this.style==s.STYLE.BAR||this.style==s.STYLE.BARCOLOR||this.style==s.STYLE.BARSIZE;if(o){if(void 0!==this.defaultXBarWidth)this.xBarWidth=this.defaultXBarWidth;else{var n=this.getDistinctValues(i,this.colX);this.xBarWidth=n[1]-n[0]||1}if(void 0!==this.defaultYBarWidth)this.yBarWidth=this.defaultYBarWidth;else{var r=this.getDistinctValues(i,this.colY);this.yBarWidth=r[1]-r[0]||1}}var d=this.getColumnRange(i,this.colX);o&&(d.min-=this.xBarWidth/2,d.max+=this.xBarWidth/2),this.xMin=void 0!==this.defaultXMin?this.defaultXMin:d.min,this.xMax=void 0!==this.defaultXMax?this.defaultXMax:d.max,this.xMax<=this.xMin&&(this.xMax=this.xMin+1),this.xStep=void 0!==this.defaultXSt
 ep?this.defaultXStep:(this.xMax-this.xMin)/5;var l=this.getColumnRange(i,this.colY);o&&(l.min-=this.yBarWidth/2,l.max+=this.yBarWidth/2),this.yMin=void 0!==this.defaultYMin?this.defaultYMin:l.min,this.yMax=void 0!==this.defaultYMax?this.defaultYMax:l.max,this.yMax<=this.yMin&&(this.yMax=this.yMin+1),this.yStep=void 0!==this.defaultYStep?this.defaultYStep:(this.yMax-this.yMin)/5;
+var c=this.getColumnRange(i,this.colZ);if(this.zMin=void 0!==this.defaultZMin?this.defaultZMin:c.min,this.zMax=void 0!==this.defaultZMax?this.defaultZMax:c.max,this.zMax<=this.zMin&&(this.zMax=this.zMin+1),this.zStep=void 0!==this.defaultZStep?this.defaultZStep:(this.zMax-this.zMin)/5,void 0!==this.colValue){var p=this.getColumnRange(i,this.colValue);this.valueMin=void 0!==this.defaultValueMin?this.defaultValueMin:p.min,this.valueMax=void 0!==this.defaultValueMax?this.defaultValueMax:p.max,this.valueMax<=this.valueMin&&(this.valueMax=this.valueMin+1)}this._setScale()}}},s.prototype._getDataPoints=function(t){var e,i,o,n,r,a,h=[];if(this.style===s.STYLE.GRID||this.style===s.STYLE.SURFACE){var d=[],c=[];for(o=0;o<this.getNumberOfRows(t);o++)e=t[o][this.colX]||0,i=t[o][this.colY]||0,-1===d.indexOf(e)&&d.push(e),-1===c.indexOf(i)&&c.push(i);var p=function(t,e){return t-e};d.sort(p),c.sort(p);var u=[];for(o=0;o<t.length;o++){e=t[o][this.colX]||0,i=t[o][this.colY]||0,n=t[o][this.colZ]||0;
 var m=d.indexOf(e),f=c.indexOf(i);void 0===u[m]&&(u[m]=[]);var g=new l;g.x=e,g.y=i,g.z=n,r={},r.point=g,r.trans=void 0,r.screen=void 0,r.bottom=new l(e,i,this.zMin),u[m][f]=r,h.push(r)}for(e=0;e<u.length;e++)for(i=0;i<u[e].length;i++)u[e][i]&&(u[e][i].pointRight=e<u.length-1?u[e+1][i]:void 0,u[e][i].pointTop=i<u[e].length-1?u[e][i+1]:void 0,u[e][i].pointCross=e<u.length-1&&i<u[e].length-1?u[e+1][i+1]:void 0)}else for(o=0;o<t.length;o++)a=new l,a.x=t[o][this.colX]||0,a.y=t[o][this.colY]||0,a.z=t[o][this.colZ]||0,void 0!==this.colValue&&(a.value=t[o][this.colValue]||0),r={},r.point=a,r.bottom=new l(a.x,a.y,this.zMin),r.trans=void 0,r.screen=void 0,h.push(r);return h},s.prototype.create=function(){for(;this.containerElement.hasChildNodes();)this.containerElement.removeChild(this.containerElement.firstChild);this.frame=document.createElement("div"),this.frame.style.position="relative",this.frame.style.overflow="hidden",this.frame.canvas=document.createElement("canvas"),this.frame.canvas
 .style.position="relative",this.frame.appendChild(this.frame.canvas);var t=document.createElement("DIV");t.style.color="red",t.style.fontWeight="bold",t.style.padding="10px",t.innerHTML="Error: your browser does not support HTML canvas",this.frame.canvas.appendChild(t),this.frame.filter=document.createElement("div"),this.frame.filter.style.position="absolute",this.frame.filter.style.bottom="0px",this.frame.filter.style.left="0px",this.frame.filter.style.width="100%",this.frame.appendChild(this.frame.filter);var e=this,i=function(t){e._onMouseDown(t)},s=function(t){e._onTouchStart(t)},o=function(t){e._onWheel(t)},n=function(t){e._onTooltip(t)};d.addEventListener(this.frame.canvas,"keydown",onkeydown),d.addEventListener(this.frame.canvas,"mousedown",i),d.addEventListener(this.frame.canvas,"touchstart",s),d.addEventListener(this.frame.canvas,"mousewheel",o),d.addEventListener(this.frame.canvas,"mousemove",n),this.containerElement.appendChild(this.frame)},s.prototype.setSize=function(t,
 e){this.frame.style.width=t,this.frame.style.height=e,this._resizeCanvas()},s.prototype._resizeCanvas=function(){this.frame.canvas.style.width="100%",this.frame.canvas.style.height="100%",this.frame.canvas.width=this.frame.canvas.clientWidth,this.frame.canvas.height=this.frame.canvas.clientHeight,this.frame.filter.style.width=this.frame.canvas.clientWidth-20+"px"},s.prototype.animationStart=function(){if(!this.frame.filter||!this.frame.filter.slider)throw"No animation available";this.frame.filter.slider.play()},s.prototype.animationStop=function(){this.frame.filter&&this.frame.filter.slider&&this.frame.filter.slider.stop()},s.prototype._resizeCenter=function(){this.xcenter="%"===this.defaultXCenter.charAt(this.defaultXCenter.length-1)?parseFloat(this.defaultXCenter)/100*this.frame.canvas.clientWidth:parseFloat(this.defaultXCenter),this.ycenter="%"===this.defaultYCenter.charAt(this.defaultYCenter.length-1)?parseFloat(this.defaultYCenter)/100*(this.frame.canvas.clientHeight-this.frame
 .filter.clientHeight):parseFloat(this.defaultYCenter)},s.prototype.setCameraPosition=function(t){void 0!==t&&(void 0!==t.horizontal&&void 0!==t.vertical&&this.camera.setArmRotation(t.horizontal,t.vertical),void 0!==t.distance&&this.camera.setArmLength(t.distance),this.redraw())},s.prototype.getCameraPosition=function(){var t=this.camera.getArmRotation();return t.distance=this.camera.getArmLength(),t},s.prototype._readData=function(t){this._dataInitialize(t,this.style),this.dataPoints=this.dataFilter?this.dataFilter._getDataPoints():this._getDataPoints(this.dataTable),this._redrawFilter()},s.prototype.setData=function(t){this._readData(t),this.redraw(),this.animationAutoStart&&this.dataFilter&&this.animationStart()},s.prototype.setOptions=function(t){var e=void 0;if(this.animationStop(),void 0!==t){if(void 0!==t.width&&(this.width=t.width),void 0!==t.height&&(this.height=t.height),void 0!==t.xCenter&&(this.defaultXCenter=t.xCenter),void 0!==t.yCenter&&(this.defaultYCenter=t.yCenter),
 void 0!==t.filterLabel&&(this.filterLabel=t.filterLabel),void 0!==t.legendLabel&&(this.legendLabel=t.legendLabel),void 0!==t.xLabel&&(this.xLabel=t.xLabel),void 0!==t.yLabel&&(this.yLabel=t.yLabel),void 0!==t.zLabel&&(this.zLabel=t.zLabel),void 0!==t.xValueLabel&&(this.xValueLabel=t.xValueLabel),void 0!==t.yValueLabel&&(this.yValueLabel=t.yValueLabel),void 0!==t.zValueLabel&&(this.zValueLabel=t.zValueLabel),void 0!==t.style){var i=this._getStyleNumber(t.style);-1!==i&&(this.style=i)}void 0!==t.showGrid&&(this.showGrid=t.showGrid),void 0!==t.showPerspective&&(this.showPerspective=t.showPerspective),void 0!==t.showShadow&&(this.showShadow=t.showShadow),void 0!==t.tooltip&&(this.showTooltip=t.tooltip),void 0!==t.showAnimationControls&&(this.showAnimationControls=t.showAnimationControls),void 0!==t.keepAspectRatio&&(this.keepAspectRatio=t.keepAspectRatio),void 0!==t.verticalRatio&&(this.verticalRatio=t.verticalRatio),void 0!==t.animationInterval&&(this.animationInterval=t.animationInter
 val),void 0!==t.animationPreload&&(this.animationPreload=t.animationPreload),void 0!==t.animationAutoStart&&(this.animationAutoStart=t.animationAutoStart),void 0!==t.xBarWidth&&(this.defaultXBarWidth=t.xBarWidth),void 0!==t.yBarWidth&&(this.defaultYBarWidth=t.yBarWidth),void 0!==t.xMin&&(this.defaultXMin=t.xMin),void 0!==t.xStep&&(this.defaultXStep=t.xStep),void 0!==t.xMax&&(this.defaultXMax=t.xMax),void 0!==t.yMin&&(this.defaultYMin=t.yMin),void 0!==t.yStep&&(this.defaultYStep=t.yStep),void 0!==t.yMax&&(this.defaultYMax=t.yMax),void 0!==t.zMin&&(this.defaultZMin=t.zMin),void 0!==t.zStep&&(this.defaultZStep=t.zStep),void 0!==t.zMax&&(this.defaultZMax=t.zMax),void 0!==t.valueMin&&(this.defaultValueMin=t.valueMin),void 0!==t.valueMax&&(this.defaultValueMax=t.valueMax),void 0!==t.cameraPosition&&(e=t.cameraPosition),void 0!==e?(this.camera.setArmRotation(e.horizontal,e.vertical),this.camera.setArmLength(e.distance)):(this.camera.setArmRotation(1,.5),this.camera.setArmLength(1.7))}this.
 _setBackgroundColor(t&&t.backgroundColor),this.setSize(this.width,this.height),this.dataTable&&this.setData(this.dataTable),this.animationAutoStart&&this.dataFilter&&this.animationStart()},s.prototype.redraw=function(){if(void 0===this.dataPoints)throw"Error: graph data not initialized";this._resizeCanvas(),this._resizeCenter(),this._redrawSlider(),this._redrawClear(),this._redrawAxis(),this.style===s.STYLE.GRID||this.style===s.STYLE.SURFACE?this._redrawDataGrid():this.style===s.STYLE.LINE?this._redrawDataLine():this.style===s.STYLE.BAR||this.style===s.STYLE.BARCOLOR||this.style===s.STYLE.BARSIZE?this._redrawDataBar():this._redrawDataDot(),this._redrawInfo(),this._redrawLegend()},s.prototype._redrawClear=function(){var t=this.frame.canvas,e=t.getContext("2d");e.clearRect(0,0,t.width,t.height)},s.prototype._redrawLegend=function(){var t;if(this.style===s.STYLE.DOTCOLOR||this.style===s.STYLE.DOTSIZE){var e,i,o=.02*this.frame.clientWidth;this.style===s.STYLE.DOTSIZE?(e=o/2,i=o/2+2*o):(
 e=20,i=20);var n=Math.max(.25*this.frame.clientHeight,100),r=this.margin,a=this.frame.clientWidth-this.margin,h=a-i,d=r+n}var l=this.frame.canvas,c=l.getContext("2d");if(c.lineWidth=1,c.font="14px arial",this.style===s.STYLE.DOTCOLOR){var p=0,u=n;for(t=p;u>t;t++){var m=(t-p)/(u-p),g=240*m,v=this._hsv2rgb(g,1,1);c.strokeStyle=v,c.beginPath(),c.moveTo(h,r+t),c.lineTo(a,r+t),c.stroke()}c.strokeStyle=this.colorAxis,c.strokeRect(h,r,i,n)}if(this.style===s.STYLE.DOTSIZE&&(c.strokeStyle=this.colorAxis,c.fillStyle=this.colorDot,c.beginPath(),c.moveTo(h,r),c.lineTo(a,r),c.lineTo(a-i+e,d),c.lineTo(h,d),c.closePath(),c.fill(),c.stroke()),this.style===s.STYLE.DOTCOLOR||this.style===s.STYLE.DOTSIZE){var y=5,b=new f(this.valueMin,this.valueMax,(this.valueMax-this.valueMin)/5,!0);for(b.start(),b.getCurrent()<this.valueMin&&b.next();!b.end();)t=d-(b.getCurrent()-this.valueMin)/(this.valueMax-this.valueMin)*n,c.beginPath(),c.moveTo(h-y,t),c.lineTo(h,t),c.stroke(),c.textAlign="right",c.textBaseline="
 middle",c.fillStyle=this.colorAxis,c.fillText(b.getCurrent(),h-2*y,t),b.next();c.textAlign="right",c.textBaseline="top";var _=this.legendLabel;c.fillText(_,a,d+this.margin)}},s.prototype._redrawFilter=function(){if(this.frame.filter.innerHTML="",this.dataFilter){var t={visible:this.showAnimationControls},e=new m(this.frame.filter,t);this.frame.filter.slider=e,this.frame.filter.style.padding="10px",e.setValues(this.dataFilter.values),e.setPlayInterval(this.animationInterval);var i=this,s=function(){var t=e.getIndex();i.dataFilter.selectValue(t),i.dataPoints=i.dataFilter._getDataPoints(),i.redraw()};e.setOnChangeCallback(s)}else this.frame.filter.slider=void 0},s.prototype._redrawSlider=function(){void 0!==this.frame.filter.slider&&this.frame.filter.slider.redraw()},s.prototype._redrawInfo=function(){if(this.dataFilter){var t=this.frame.canvas,e=t.getContext("2d");e.font="14px arial",e.lineStyle="gray",e.fillStyle="gray",e.textAlign="left",e.textBaseline="top";var i=this.margin,s=this
 .margin;e.fillText(this.dataFilter.getLabel()+": "+this.dataFilter.getSelectedValue(),i,s)}},s.prototype._redrawAxis=function(){var t,e,i,s,o,n,r,a,h,d,c,p,u,m=this.frame.canvas,g=m.getContext("2d");g.font=24/this.camera.getArmLength()+"px arial";var v=.025/this.scale.x,y=.025/this.scale.y,b=5/this.camera.getArmLength(),_=this.camera.getArmRotation().horizontal;for(g.lineWidth=1,s=void 0===this.defaultXStep,i=new f(this.xMin,this.xMax,this.xStep,s),i.start(),i.getCurrent()<this.xMin&&i.next();!i.end();){var x=i.getCurrent();this.showGrid?(t=this._convert3Dto2D(new l(x,this.yMin,this.zMin)),e=this._convert3Dto2D(new l(x,this.yMax,this.zMin)),g.strokeStyle=this.colorGrid,g.beginPath(),g.moveTo(t.x,t.y),g.lineTo(e.x,e.y),g.stroke()):(t=this._convert3Dto2D(new l(x,this.yMin,this.zMin)),e=this._convert3Dto2D(new l(x,this.yMin+v,this.zMin)),g.strokeStyle=this.colorAxis,g.beginPath(),g.moveTo(t.x,t.y),g.lineTo(e.x,e.y),g.stroke(),t=this._convert3Dto2D(new l(x,this.yMax,this.zMin)),e=this._
 convert3Dto2D(new l(x,this.yMax-v,this.zMin)),g.strokeStyle=this.colorAxis,g.beginPath(),g.moveTo(t.x,t.y),g.lineTo(e.x,e.y),g.stroke()),r=Math.cos(_)>0?this.yMin:this.yMax,o=this._convert3Dto2D(new l(x,r,this.zMin)),Math.cos(2*_)>0?(g.textAlign="center",g.textBaseline="top",o.y+=b):Math.sin(2*_)<0?(g.textAlign="right",g.textBaseline="middle"):(g.textAlign="left",g.textBaseline="middle"),g.fillStyle=this.colorAxis,g.fillText("  "+this.xValueLabel(i.getCurrent())+"  ",o.x,o.y),i.next()}for(g.lineWidth=1,s=void 0===this.defaultYStep,i=new f(this.yMin,this.yMax,this.yStep,s),i.start(),i.getCurrent()<this.yMin&&i.next();!i.end();)this.showGrid?(t=this._convert3Dto2D(new l(this.xMin,i.getCurrent(),this.zMin)),e=this._convert3Dto2D(new l(this.xMax,i.getCurrent(),this.zMin)),g.strokeStyle=this.colorGrid,g.beginPath(),g.moveTo(t.x,t.y),g.lineTo(e.x,e.y),g.stroke()):(t=this._convert3Dto2D(new l(this.xMin,i.getCurrent(),this.zMin)),e=this._convert3Dto2D(new l(this.xMin+y,i.getCurrent(),this.z
 Min)),g.strokeStyle=this.colorAxis,g.beginPath(),g.moveTo(t.x,t.y),g.lineTo(e.x,e.y),g.stroke(),t=this._convert3Dto2D(new l(this.xMax,i.getCurrent(),this.zMin)),e=this._convert3Dto2D(new l(this.xMax-y,i.getCurrent(),this.zMin)),g.strokeStyle=this.colorAxis,g.beginPath(),g.moveTo(t.x,t.y),g.lineTo(e.x,e.y),g.stroke()),n=Math.sin(_)>0?this.xMin:this.xMax,o=this._convert3Dto2D(new l(n,i.getCurrent(),this.zMin)),Math.cos(2*_)<0?(g.textAlign="center",g.textBaseline="top",o.y+=b):Math.sin(2*_)>0?(g.textAlign="right",g.textBaseline="middle"):(g.textAlign="left",g.textBaseline="middle"),g.fillStyle=this.colorAxis,g.fillText("  "+this.yValueLabel(i.getCurrent())+"  ",o.x,o.y),i.next();for(g.lineWidth=1,s=void 0===this.defaultZStep,i=new f(this.zMin,this.zMax,this.zStep,s),i.start(),i.getCurrent()<this.zMin&&i.next(),n=Math.cos(_)>0?this.xMin:this.xMax,r=Math.sin(_)<0?this.yMin:this.yMax;!i.end();)t=this._convert3Dto2D(new l(n,r,i.getCurrent())),g.strokeStyle=this.colorAxis,g.beginPath(),g.mo
 veTo(t.x,t.y),g.lineTo(t.x-b,t.y),g.stroke(),g.textAlign="right",g.textBaseline="middle",g.fillStyle=this.colorAxis,g.fillText(this.zValueLabel(i.getCurrent())+" ",t.x-5,t.y),i.next();g.lineWidth=1,t=this._convert3Dto2D(new l(n,r,this.zMin)),e=this._convert3Dto2D(new l(n,r,this.zMax)),g.strokeStyle=this.colorAxis,g.beginPath(),g.moveTo(t.x,t.y),g.lineTo(e.x,e.y),g.stroke(),g.lineWidth=1,p=this._convert3Dto2D(new l(this.xMin,this.yMin,this.zMin)),u=this._convert3Dto2D(new l(this.xMax,this.yMin,this.zMin)),g.strokeStyle=this.colorAxis,g.beginPath(),g.moveTo(p.x,p.y),g.lineTo(u.x,u.y),g.stroke(),p=this._convert3Dto2D(new l(this.xMin,this.yMax,this.zMin)),u=this._convert3Dto2D(new l(this.xMax,this.yMax,this.zMin)),g.strokeStyle=this.colorAxis,g.beginPath(),g.moveTo(p.x,p.y),g.lineTo(u.x,u.y),g.stroke(),g.lineWidth=1,t=this._convert3Dto2D(new l(this.xMin,this.yMin,this.zMin)),e=this._convert3Dto2D(new l(this.xMin,this.yMax,this.zMin)),g.strokeStyle=this.colorAxis,g.beginPath(),g.moveTo(t
 .x,t.y),g.lineTo(e.x,e.y),g.stroke(),t=this._convert3Dto2D(new l(this.xMax,this.yMin,this.zMin)),e=this._convert3Dto2D(new l(this.xMax,this.yMax,this.zMin)),g.strokeStyle=this.colorAxis,g.beginPath(),g.moveTo(t.x,t.y),g.lineTo(e.x,e.y),g.stroke();var w=this.xLabel;w.length>0&&(c=.1/this.scale.y,n=(this.xMin+this.xMax)/2,r=Math.cos(_)>0?this.yMin-c:this.yMax+c,o=this._convert3Dto2D(new l(n,r,this.zMin)),Math.cos(2*_)>0?(g.textAlign="center",g.textBaseline="top"):Math.sin(2*_)<0?(g.textAlign="right",g.textBaseline="middle"):(g.textAlign="left",g.textBaseline="middle"),g.fillStyle=this.colorAxis,g.fillText(w,o.x,o.y));var S=this.yLabel;S.length>0&&(d=.1/this.scale.x,n=Math.sin(_)>0?this.xMin-d:this.xMax+d,r=(this.yMin+this.yMax)/2,o=this._convert3Dto2D(new l(n,r,this.zMin)),Math.cos(2*_)<0?(g.textAlign="center",g.textBaseline="top"):Math.sin(2*_)>0?(g.textAlign="right",g.textBaseline="middle"):(g.textAlign="left",g.textBaseline="middle"),g.fillStyle=this.colorAxis,g.fillText(S,o.x,o.y)
 );var M=this.zLabel;M.length>0&&(h=30,n=Math.cos(_)>0?this.xMin:this.xMax,r=Math.sin(_)<0?this.yMin:this.yMax,a=(this.zMin+this.zMax)/2,o=this._convert3Dto2D(new l(n,r,a)),g.textAlign="right",g.textBaseline="middle",g.fillStyle=this.colorAxis,g.fillText(M,o.x-h,o.y))},s.prototype._hsv2rgb=function(t,e,i){var s,o,n,r,a,h;switch(r=i*e,a=Math.floor(t/60),h=r*(1-Math.abs(t/60%2-1)),a){case 0:s=r,o=h,n=0;break;case 1:s=h,o=r,n=0;break;case 2:s=0,o=r,n=h;break;case 3:s=0,o=h,n=r;break;case 4:s=h,o=0,n=r;break;case 5:s=r,o=0,n=h;break;default:s=0,o=0,n=0}return"RGB("+parseInt(255*s)+","+parseInt(255*o)+","+parseInt(255*n)+")"},s.prototype._redrawDataGrid=function(){var t,e,i,o,n,r,a,h,d,c,p,u,m,f=this.frame.canvas,g=f.getContext("2d");if(!(void 0===this.dataPoints||this.dataPoints.length<=0)){for(n=0;n<this.dataPoints.length;n++){var v=this._convertPointToTranslation(this.dataPoints[n].point),y=this._convertTranslationToScreen(v);this.dataPoints[n].trans=v,this.dataPoints[n].screen=y;var b
 =this._convertPointToTranslation(this.dataPoints[n].bottom);this.dataPoints[n].dist=this.showPerspective?b.length():-b.z}var _=function(t,e){return e.dist-t.dist};if(this.dataPoints.sort(_),this.style===s.STYLE.SURFACE){for(n=0;n<this.dataPoints.length;n++)if(t=this.dataPoints[n],e=this.dataPoints[n].pointRight,i=this.dataPoints[n].pointTop,o=this.dataPoints[n].pointCross,void 0!==t&&void 0!==e&&void 0!==i&&void 0!==o){if(this.showGrayBottom||this.showShadow){var x=l.subtract(o.trans,t.trans),w=l.subtract(i.trans,e.trans),S=l.crossProduct(x,w),M=S.length();r=S.z>0}else r=!0;r?(m=(t.point.z+e.point.z+i.point.z+o.point.z)/4,c=240*(1-(m-this.zMin)*this.scale.z/this.verticalRatio),p=1,this.showShadow?(u=Math.min(1+S.x/M/2,1),a=this._hsv2rgb(c,p,u),h=a):(u=1,a=this._hsv2rgb(c,p,u),h=this.colorAxis)):(a="gray",h=this.colorAxis),d=.5,g.lineWidth=d,g.fillStyle=a,g.strokeStyle=h,g.beginPath(),g.moveTo(t.screen.x,t.screen.y),g.lineTo(e.screen.x,e.screen.y),g.lineTo(o.screen.x,o.screen.y),g.li
 neTo(i.screen.x,i.screen.y),g.closePath(),g.fill(),g.stroke()}}else for(n=0;n<this.dataPoints.length;n++)t=this.dataPoints[n],e=this.dataPoints[n].pointRight,i=this.dataPoints[n].pointTop,void 0!==t&&(d=this.showPerspective?2/-t.trans.z:2*-(this.eye.z/this.camera.getArmLength())),void 0!==t&&void 0!==e&&(m=(t.point.z+e.point.z)/2,c=240*(1-(m-this.zMin)*this.scale.z/this.verticalRatio),g.lineWidth=d,g.strokeStyle=this._hsv2rgb(c,1,1),g.beginPath(),g.moveTo(t.screen.x,t.screen.y),g.lineTo(e.screen.x,e.screen.y),g.stroke()),void 0!==t&&void 0!==i&&(m=(t.point.z+i.point.z)/2,c=240*(1-(m-this.zMin)*this.scale.z/this.verticalRatio),g.lineWidth=d,g.strokeStyle=this._hsv2rgb(c,1,1),g.beginPath(),g.moveTo(t.screen.x,t.screen.y),g.lineTo(i.screen.x,i.screen.y),g.stroke())}},s.prototype._redrawDataDot=function(){var t,e=this.frame.canvas,i=e.getContext("2d");if(!(void 0===this.dataPoints||this.dataPoints.length<=0)){for(t=0;t<this.dataPoints.length;t++){var o=this._convertPointToTranslation(th
 is.dataPoints[t].point),n=this._convertTranslationToScreen(o);this.dataPoints[t].trans=o,this.dataPoints[t].screen=n;var r=this._convertPointToTranslation(this.dataPoints[t].bottom);this.dataPoints[t].dist=this.showPerspective?r.length():-r.z}var a=function(t,e){return e.dist-t.dist};this.dataPoints.sort(a);var h=.02*this.frame.clientWidth;for(t=0;t<this.dataPoints.length;t++){var d=this.dataPoints[t];if(this.style===s.STYLE.DOTLINE){var l=this._convert3Dto2D(d.bottom);i.lineWidth=1,i.strokeStyle=this.colorGrid,i.beginPath(),i.moveTo(l.x,l.y),i.lineTo(d.screen.x,d.screen.y),i.stroke()}var c;c=this.style===s.STYLE.DOTSIZE?h/2+2*h*(d.point.value-this.valueMin)/(this.valueMax-this.valueMin):h;var p;p=this.showPerspective?c/-d.trans.z:c*-(this.eye.z/this.camera.getArmLength()),0>p&&(p=0);var u,m,f;this.style===s.STYLE.DOTCOLOR?(u=240*(1-(d.point.value-this.valueMin)*this.scale.value),m=this._hsv2rgb(u,1,1),f=this._hsv2rgb(u,1,.8)):this.style===s.STYLE.DOTSIZE?(m=this.colorDot,f=this.col
 orDotBorder):(u=240*(1-(d.point.z-this.zMin)*this.scale.z/this.verticalRatio),m=this._hsv2rgb(u,1,1),f=this._hsv2rgb(u,1,.8)),i.lineWidth=1,i.strokeStyle=f,i.fillStyle=m,i.beginPath(),i.arc(d.screen.x,d.screen.y,p,0,2*Math.PI,!0),i.fill(),i.stroke()}}},s.prototype._redrawDataBar=function(){var t,e,i,o,n=this.frame.canvas,r=n.getContext("2d");if(!(void 0===this.dataPoints||this.dataPoints.length<=0)){for(t=0;t<this.dataPoints.length;t++){var a=this._convertPointToTranslation(this.dataPoints[t].point),h=this._convertTranslationToScreen(a);this.dataPoints[t].trans=a,this.dataPoints[t].screen=h;var d=this._convertPointToTranslation(this.dataPoints[t].bottom);this.dataPoints[t].dist=this.showPerspective?d.length():-d.z}var c=function(t,e){return e.dist-t.dist};this.dataPoints.sort(c);var p=this.xBarWidth/2,u=this.yBarWidth/2;for(t=0;t<this.dataPoints.length;t++){var m,f,g,v=this.dataPoints[t];this.style===s.STYLE.BARCOLOR?(m=240*(1-(v.point.value-this.valueMin)*this.scale.value),f=this._
 hsv2rgb(m,1,1),g=this._hsv2rgb(m,1,.8)):this.style===s.STYLE.BARSIZE?(f=this.colorDot,g=this.colorDotBorder):(m=240*(1-(v.point.z-this.zMin)*this.scale.z/this.verticalRatio),f=this._hsv2rgb(m,1,1),g=this._hsv2rgb(m,1,.8)),this.style===s.STYLE.BARSIZE&&(p=this.xBarWidth/2*((v.point.value-this.valueMin)/(this.valueMax-this.valueMin)*.8+.2),u=this.yBarWidth/2*((v.point.value-this.valueMin)/(this.valueMax-this.valueMin)*.8+.2));var y=this,b=v.point,_=[{point:new l(b.x-p,b.y-u,b.z)},{point:new l(b.x+p,b.y-u,b.z)},{point:new l(b.x+p,b.y+u,b.z)},{point:new l(b.x-p,b.y+u,b.z)}],x=[{point:new l(b.x-p,b.y-u,this.zMin)},{point:new l(b.x+p,b.y-u,this.zMin)},{point:new l(b.x+p,b.y+u,this.zMin)},{point:new l(b.x-p,b.y+u,this.zMin)}];_.forEach(function(t){t.screen=y._convert3Dto2D(t.point)}),x.forEach(function(t){t.screen=y._convert3Dto2D(t.point)});var w=[{corners:_,center:l.avg(x[0].point,x[2].point)},{corners:[_[0],_[1],x[1],x[0]],center:l.avg(x[1].point,x[0].point)},{corners:[_[1],_[2],x[2],x[
 1]],center:l.avg(x[2].point,x[1].point)},{corners:[_[2],_[3],x[3],x[2]],center:l.avg(x[3].point,x[2].point)},{corners:[_[3],_[0],x[0],x[3]],center:l.avg(x[0].point,x[3].point)}];for(v.surfaces=w,e=0;e<w.length;e++){i=w[e];var S=this._convertPointToTranslation(i.center);i.dist=this.showPerspective?S.length():-S.z}for(w.sort(function(t,e){var i=e.dist-t.dist;return i?i:t.corners===_?1:e.corners===_?-1:0}),r.lineWidth=1,r.strokeStyle=g,r.fillStyle=f,e=2;e<w.length;e++)i=w[e],o=i.corners,r.beginPath(),r.moveTo(o[3].screen.x,o[3].screen.y),r.lineTo(o[0].screen.x,o[0].screen.y),r.lineTo(o[1].screen.x,o[1].screen.y),r.lineTo(o[2].screen.x,o[2].screen.y),r.lineTo(o[3].screen.x,o[3].screen.y),r.fill(),r.stroke()}}},s.prototype._redrawDataLine=function(){var t,e,i=this.frame.canvas,s=i.getContext("2d");if(!(void 0===this.dataPoints||this.dataPoints.length<=0)){for(e=0;e<this.dataPoints.length;e++){var o=this._convertPointToTranslation(this.dataPoints[e].point),n=this._convertTranslationToScre
 en(o);this.dataPoints[e].trans=o,this.dataPoints[e].screen=n}for(this.dataPoints.length>0&&(t=this.dataPoints[0],s.lineWidth=1,s.strokeStyle="blue",s.beginPath(),s.moveTo(t.screen.x,t.screen.y)),e=1;e<this.dataPoints.length;e++)t=this.dataPoints[e],s.lineTo(t.screen.x,t.screen.y);this.dataPoints.length>0&&s.stroke()}},s.prototype._onMouseDown=function(t){if(t=t||window.event,this.leftButtonDown&&this._onMouseUp(t),this.leftButtonDown=t.which?1===t.which:1===t.button,this.leftButtonDown||this.touchDown){this.startMouseX=o(t),this.startMouseY=n(t),this.startStart=new Date(this.start),this.startEnd=new Date(this.end),this.startArmRotation=this.camera.getArmRotation(),this.frame.style.cursor="move";var e=this;this.onmousemove=function(t){e._onMouseMove(t)},this.onmouseup=function(t){e._onMouseUp(t)},d.addEventListener(document,"mousemove",e.onmousemove),d.addEventListener(document,"mouseup",e.onmouseup),d.preventDefault(t)}},s.prototype._onMouseMove=function(t){t=t||window.event;var e=p
 arseFloat(o(t))-this.startMouseX,i=parseFloat(n(t))-this.startMouseY,s=this.startArmRotation.horizontal+e/200,r=this.startArmRotation.vertical+i/200,a=4,h=Math.sin(a/360*2*Math.PI);Math.abs(Math.sin(s))<h&&(s=Math.round(s/Math.PI)*Math.PI-.001),Math.abs(Math.cos(s))<h&&(s=(Math.round(s/Math.PI-.5)+.5)*Math.PI-.001),Math.abs(Math.sin(r))<h&&(r=Math.round(r/Math.PI)*Math.PI),Math.abs(Math.cos(r))<h&&(r=(Math.round(r/Math.PI-.5)+.5)*Math.PI),this.camera.setArmRotation(s,r),this.redraw();var l=this.getCameraPosition();this.emit("cameraPositionChange",l),d.preventDefault(t)},s.prototype._onMouseUp=function(t){this.frame.style.cursor="auto",this.leftButtonDown=!1,d.removeEventListener(document,"mousemove",this.onmousemove),d.removeEventListener(document,"mouseup",this.onmouseup),d.preventDefault(t)},s.prototype._onTooltip=function(t){var e=300,i=this.frame.getBoundingClientRect(),s=o(t)-i.left,r=n(t)-i.top;if(this.showTooltip){if(this.tooltipTimeout&&clearTimeout(this.tooltipTimeout),this
 .leftButtonDown)return void this._hideTooltip();if(this.tooltip&&this.tooltip.dataPoint){var a=this._dataPointFromXY(s,r);a!==this.tooltip.dataPoint&&(a?this._showTooltip(a):this._hideTooltip())}else{var h=this;this.tooltipTimeout=setTimeout(function(){h.tooltipTimeout=null;var t=h._dataPointFromXY(s,r);t&&h._showTooltip(t)},e)}}},s.prototype._onTouchStart=function(t){this.touchDown=!0;var e=this;this.ontouchmove=function(t){e._onTouchMove(t)},this.ontouchend=function(t){e._onTouchEnd(t)},d.addEventListener(document,"touchmove",e.ontouchmove),d.addEventListener(document,"touchend",e.ontouchend),this._onMouseDown(t)},s.prototype._onTouchMove=function(t){this._onMouseMove(t)},s.prototype._onTouchEnd=function(t){this.touchDown=!1,d.removeEventListener(document,"touchmove",this.ontouchmove),d.removeEventListener(document,"touchend",this.ontouchend),this._onMouseUp(t)},s.prototype._onWheel=function(t){t||(t=window.event);var e=0;if(t.wheelDelta?e=t.wheelDelta/120:t.detail&&(e=-t.detail/3
 ),e){var i=this.camera.getArmLength(),s=i*(1-e/10);this.camera.setArmLength(s),this.redraw(),this._hideTooltip()}var o=this.getCameraPosition();this.emit("cameraPositionChange",o),d.preventDefault(t)},s.prototype._insideTriangle=function(t,e){function i(t){return t>0?1:0>t?-1:0}var s=e[0],o=e[1],n=e[2],r=i((o.x-s.x)*(t.y-s.y)-(o.y-s.y)*(t.x-s.x)),a=i((n.x-o.x)*(t.y-o.y)-(n.y-o.y)*(t.x-o.x)),h=i((s.x-n.x)*(t.y-n.y)-(s.y-n.y)*(t.x-n.x));return!(0!=r&&0!=a&&r!=a||0!=a&&0!=h&&a!=h||0!=r&&0!=h&&r!=h)},s.prototype._dataPointFromXY=function(t,e){var i,o=100,n=null,r=null,a=null,h=new c(t,e);if(this.style===s.STYLE.BAR||this.style===s.STYLE.BARCOLOR||this.style===s.STYLE.BARSIZE)for(i=this.dataPoints.length-1;i>=0;i--){n=this.dataPoints[i];var d=n.surfaces;if(d)for(var l=d.length-1;l>=0;l--){var p=d[l],u=p.corners,m=[u[0].screen,u[1].screen,u[2].screen],f=[u[2].screen,u[3].screen,u[0].screen];if(this._insideTriangle(h,m)||this._insideTriangle(h,f))return n}}else for(i=0;i<this.dataPoints.le
 ngth;i++){n=this.dataPoints[i];var g=n.screen;if(g){var v=Math.abs(t-g.x),y=Math.abs(e-g.y),b=Math.sqrt(v*v+y*y);(null===a||a>b)&&o>b&&(a=b,r=n)}}return r},s.prototype._showTooltip=function(t){var e,i,s;this.tooltip?(e=this.tooltip.dom.content,i=this.tooltip.dom.line,s=this.tooltip.dom.dot):(e=document.createElement("div"),e.style.position="absolute",e.style.padding="10px",e.style.border="1px solid #4d4d4d",e.style.color="#1a1a1a",e.style.background="rgba(255,255,255,0.7)",e.style.borderRadius="2px",e.style.boxShadow="5px 5px 10px rgba(128,128,128,0.5)",i=document.createElement("div"),i.style.position="absolute",i.style.height="40px",i.style.width="0",i.style.borderLeft="1px solid #4d4d4d",s=document.createElement("div"),s.style.position="absolute",s.style.height="0",s.style.width="0",s.style.border="5px solid #4d4d4d",s.style.borderRadius="5px",this.tooltip={dataPoint:null,dom:{content:e,line:i,dot:s}}),this._hideTooltip(),this.tooltip.dataPoint=t,e.innerHTML="function"==typeof thi
 s.showTooltip?this.showTooltip(t.point):"<table><tr><td>x:</td><td>"+t.point.x+"</td></tr><tr><td>y:</td><td>"+t.point.y+"</td></tr><tr><td>z:</td><td>"+t.point.z+"</td></tr></table>",e.style.left="0",e.style.top="0",this.frame.appendChild(e),this.frame.appendChild(i),this.frame.appendChild(s);var o=e.offsetWidth,n=e.offsetHeight,r=i.offsetHeight,a=s.offsetWidth,h=s.offsetHeight,d=t.screen.x-o/2;d=Math.min(Math.max(d,10),this.frame.clientWidth-10-o),i.style.left=t.screen.x+"px",i.style.top=t.screen.y-r+"px",e.style.left=d+"px",e.style.top=t.screen.y-r-n+"px",s.style.left=t.screen.x-a/2+"px",s.style.top=t.screen.y-h/2+"px"},s.prototype._hideTooltip=function(){if(this.tooltip){this.tooltip.dataPoint=null;for(var t in this.tooltip.dom)if(this.tooltip.dom.hasOwnProperty(t)){var e=this.tooltip.dom[t];e&&e.parentNode&&e.parentNode.removeChild(e)}}},t.exports=s},function(t,e,i){function s(){this.armLocation=new o,this.armRotation={},this.armRotation.horizontal=0,this.armRotation.vertical=0
 ,this.armLength=1.7,this.cameraLocation=new o,this.cameraRotation=new o(.5*Math.PI,0,0),this.calculateCameraOrientation()}var o=i(10);s.prototype.setArmLocation=function(t,e,i){this.armLocation.x=t,this.armLocation.y=e,this.armLocation.z=i,this.calculateCameraOrientation()},s.prototype.setArmRotation=function(t,e){void 0!==t&&(this.armRotation.horizontal=t),void 0!==e&&(this.armRotation.vertical=e,this.armRotation.vertical<0&&(this.armRotation.vertical=0),this.armRotation.vertical>.5*Math.PI&&(this.armRotation.vertical=.5*Math.PI)),(void 0!==t||void 0!==e)&&this.calculateCameraOrientation()},s.prototype.getArmRotation=function(){var t={};return t.horizontal=this.armRotation.horizontal,t.vertical=this.armRotation.vertical,t},s.prototype.setArmLength=function(t){void 0!==t&&(this.armLength=t,this.armLength<.71&&(this.armLength=.71),this.armLength>5&&(this.armLength=5),this.calculateCameraOrientation())},s.prototype.getArmLength=function(){return this.armLength},s.prototype.getCameraLo
 cation=function(){return this.cameraLocation},s.prototype.getCameraRotation=function(){return this.cameraRotation},s.prototype.calculateCameraOrientation=function(){this.cameraLocation.x=this.armLocation.x-this.armLength*Math.sin(this.armRotation.horizontal)*Math.cos(this.armRotation.vertical),this.cameraLocation.y=this.armLocation.y-this.armLength*Math.cos(this.armRotation.horizontal)*Math.cos(this.armRotation.vertical),this.cameraLocation.z=this.armLocation.z+this.armLength*Math.sin(this.armRotation.vertical),this.cameraRotation.x=Math.PI/2-this.armRotation.vertical,this.cameraRotation.y=0,this.cameraRotation.z=-this.armRotation.horizontal},t.exports=s},function(t,e,i){function s(t,e,i){this.data=t,this.column=e,this.graph=i,this.index=void 0,this.value=void 0,this.values=i.getDistinctValues(t.get(),this.column),this.values.sort(function(t,e){return t>e?1:e>t?-1:0}),this.values.length>0&&this.selectValue(0),this.dataPoints=[],this.loaded=!1,this.onLoadCallback=void 0,i.animationPr
 eload?(this.loaded=!1,this.loadInBackground()):this.loaded=!0}var o=i(4);s.prototype.isLoaded=function(){return this.loaded},s.prototype.getLoadedProgress=function(){for(var t=this.values.length,e=0;this.dataPoints[e];)e++;return Math.round(e/t*100)},s.prototype.getLabel=function(){return this.graph.filterLabel},s.prototype.getColumn=function(){return this.column},s.prototype.getSelectedValue=function(){return void 0===this.index?void 0:this.values[this.index]},s.prototype.getValues=function(){return this.values},s.prototype.getValue=function(t){if(t>=this.values.length)throw"Error: index out of range";return this.values[t]},s.prototype._getDataPoints=function(t){if(void 0===t&&(t=this.index),void 0===t)return[];var e;if(this.dataPoints[t])e=this.dataPoints[t];else{var i={};i.column=this.column,i.value=this.values[t];var s=new o(this.data,{filter:function(t){return t[i.column]==i.value}}).get();e=this.graph._getDataPoints(s),this.dataPoints[t]=e}return e},s.prototype.setOnLoadCallba
 ck=function(t){this.onLoadCallback=t},s.prototype.selectValue=function(t){if(t>=this.values.length)throw"Error: index out of range";this.index=t,this.value=this.values[t]},s.prototype.loadInBackground=function(t){void 0===t&&(t=0);var e=this.graph.frame;if(t<this.values.length){{this._getDataPoints(t)}void 0===e.progress&&(e.progress=document.createElement("DIV"),e.progress.style.position="absolute",e.progress.style.color="gray",e.appendChild(e.progress));var i=this.getLoadedProgress();e.progress.innerHTML="Loading animation... "+i+"%",e.progress.style.bottom="60px",e.progress.style.left="10px";var s=this;setTimeout(function(){s.loadInBackground(t+1)},10),this.loaded=!1}else this.loaded=!0,void 0!==e.progress&&(e.removeChild(e.progress),e.progress=void 0),this.onLoadCallback&&this.onLoadCallback()},t.exports=s},function(t){function e(t,e){this.x=void 0!==t?t:0,this.y=void 0!==e?e:0}t.exports=e},function(t){function e(t,e,i){this.x=void 0!==t?t:0,this.y=void 0!==e?e:0,this.z=void 0!=
 =i?i:0}e.subtract=function(t,i){var s=new e;return s.x=t.x-i.x,s.y=t.y-i.y,s.z=t.z-i.z,s},e.add=function(t,i){var s=new e;
+return s.x=t.x+i.x,s.y=t.y+i.y,s.z=t.z+i.z,s},e.avg=function(t,i){return new e((t.x+i.x)/2,(t.y+i.y)/2,(t.z+i.z)/2)},e.crossProduct=function(t,i){var s=new e;return s.x=t.y*i.z-t.z*i.y,s.y=t.z*i.x-t.x*i.z,s.z=t.x*i.y-t.y*i.x,s},e.prototype.length=function(){return Math.sqrt(this.x*this.x+this.y*this.y+this.z*this.z)},t.exports=e},function(t,e,i){function s(t,e){if(void 0===t)throw"Error: No container element defined";if(this.container=t,this.visible=e&&void 0!=e.visible?e.visible:!0,this.visible){this.frame=document.createElement("DIV"),this.frame.style.width="100%",this.frame.style.position="relative",this.container.appendChild(this.frame),this.frame.prev=document.createElement("INPUT"),this.frame.prev.type="BUTTON",this.frame.prev.value="Prev",this.frame.appendChild(this.frame.prev),this.frame.play=document.createElement("INPUT"),this.frame.play.type="BUTTON",this.frame.play.value="Play",this.frame.appendChild(this.frame.play),this.frame.next=document.createElement("INPUT"),this.f
 rame.next.type="BUTTON",this.frame.next.value="Next",this.frame.appendChild(this.frame.next),this.frame.bar=document.createElement("INPUT"),this.frame.bar.type="BUTTON",this.frame.bar.style.position="absolute",this.frame.bar.style.border="1px solid red",this.frame.bar.style.width="100px",this.frame.bar.style.height="6px",this.frame.bar.style.borderRadius="2px",this.frame.bar.style.MozBorderRadius="2px",this.frame.bar.style.border="1px solid #7F7F7F",this.frame.bar.style.backgroundColor="#E5E5E5",this.frame.appendChild(this.frame.bar),this.frame.slide=document.createElement("INPUT"),this.frame.slide.type="BUTTON",this.frame.slide.style.margin="0px",this.frame.slide.value=" ",this.frame.slide.style.position="relative",this.frame.slide.style.left="-100px",this.frame.appendChild(this.frame.slide);var i=this;this.frame.slide.onmousedown=function(t){i._onMouseDown(t)},this.frame.prev.onclick=function(t){i.prev(t)},this.frame.play.onclick=function(t){i.togglePlay(t)},this.frame.next.onclic
 k=function(t){i.next(t)}}this.onChangeCallback=void 0,this.values=[],this.index=void 0,this.playTimeout=void 0,this.playInterval=1e3,this.playLoop=!0}var o=i(1);s.prototype.prev=function(){var t=this.getIndex();t>0&&(t--,this.setIndex(t))},s.prototype.next=function(){var t=this.getIndex();t<this.values.length-1&&(t++,this.setIndex(t))},s.prototype.playNext=function(){var t=new Date,e=this.getIndex();e<this.values.length-1?(e++,this.setIndex(e)):this.playLoop&&(e=0,this.setIndex(e));var i=new Date,s=i-t,o=Math.max(this.playInterval-s,0),n=this;this.playTimeout=setTimeout(function(){n.playNext()},o)},s.prototype.togglePlay=function(){void 0===this.playTimeout?this.play():this.stop()},s.prototype.play=function(){this.playTimeout||(this.playNext(),this.frame&&(this.frame.play.value="Stop"))},s.prototype.stop=function(){clearInterval(this.playTimeout),this.playTimeout=void 0,this.frame&&(this.frame.play.value="Play")},s.prototype.setOnChangeCallback=function(t){this.onChangeCallback=t},s
 .prototype.setPlayInterval=function(t){this.playInterval=t},s.prototype.getPlayInterval=function(){return this.playInterval},s.prototype.setPlayLoop=function(t){this.playLoop=t},s.prototype.onChange=function(){void 0!==this.onChangeCallback&&this.onChangeCallback()},s.prototype.redraw=function(){if(this.frame){this.frame.bar.style.top=this.frame.clientHeight/2-this.frame.bar.offsetHeight/2+"px",this.frame.bar.style.width=this.frame.clientWidth-this.frame.prev.clientWidth-this.frame.play.clientWidth-this.frame.next.clientWidth-30+"px";var t=this.indexToLeft(this.index);this.frame.slide.style.left=t+"px"}},s.prototype.setValues=function(t){this.values=t,this.values.length>0?this.setIndex(0):this.index=void 0},s.prototype.setIndex=function(t){if(!(t<this.values.length))throw"Error: index out of range";this.index=t,this.redraw(),this.onChange()},s.prototype.getIndex=function(){return this.index},s.prototype.get=function(){return this.values[this.index]},s.prototype._onMouseDown=function
 (t){var e=t.which?1===t.which:1===t.button;if(e){this.startClientX=t.clientX,this.startSlideX=parseFloat(this.frame.slide.style.left),this.frame.style.cursor="move";var i=this;this.onmousemove=function(t){i._onMouseMove(t)},this.onmouseup=function(t){i._onMouseUp(t)},o.addEventListener(document,"mousemove",this.onmousemove),o.addEventListener(document,"mouseup",this.onmouseup),o.preventDefault(t)}},s.prototype.leftToIndex=function(t){var e=parseFloat(this.frame.bar.style.width)-this.frame.slide.clientWidth-10,i=t-3,s=Math.round(i/e*(this.values.length-1));return 0>s&&(s=0),s>this.values.length-1&&(s=this.values.length-1),s},s.prototype.indexToLeft=function(t){var e=parseFloat(this.frame.bar.style.width)-this.frame.slide.clientWidth-10,i=t/(this.values.length-1)*e,s=i+3;return s},s.prototype._onMouseMove=function(t){var e=t.clientX-this.startClientX,i=this.startSlideX+e,s=this.leftToIndex(i);this.setIndex(s),o.preventDefault()},s.prototype._onMouseUp=function(){this.frame.style.curso
 r="auto",o.removeEventListener(document,"mousemove",this.onmousemove),o.removeEventListener(document,"mouseup",this.onmouseup),o.preventDefault()},t.exports=s},function(t){function e(t,e,i,s){this._start=0,this._end=0,this._step=1,this.prettyStep=!0,this.precision=5,this._current=0,this.setRange(t,e,i,s)}e.prototype.setRange=function(t,e,i,s){this._start=t?t:0,this._end=e?e:0,this.setStep(i,s)},e.prototype.setStep=function(t,i){void 0===t||0>=t||(void 0!==i&&(this.prettyStep=i),this._step=this.prettyStep===!0?e.calculatePrettyStep(t):t)},e.calculatePrettyStep=function(t){var e=function(t){return Math.log(t)/Math.LN10},i=Math.pow(10,Math.round(e(t))),s=2*Math.pow(10,Math.round(e(t/2))),o=5*Math.pow(10,Math.round(e(t/5))),n=i;return Math.abs(s-t)<=Math.abs(n-t)&&(n=s),Math.abs(o-t)<=Math.abs(n-t)&&(n=o),0>=n&&(n=1),n},e.prototype.getCurrent=function(){return parseFloat(this._current.toPrecision(this.precision))},e.prototype.getStep=function(){return this._step},e.prototype.start=funct
 ion(){this._current=this._start-this._start%this._step},e.prototype.next=function(){this._current+=this._step},e.prototype.end=function(){return this._current>this._end},t.exports=e},function(t,e,i){function s(t,e,i,r){if(!(this instanceof s))throw new SyntaxError("Constructor must be called with the new operator");if(!(Array.isArray(i)||i instanceof n)&&i instanceof Object){var h=r;r=i,i=h}var u=this;this.defaultOptions={start:null,end:null,autoResize:!0,orientation:"bottom",width:null,height:null,maxHeight:null,minHeight:null},this.options=o.deepExtend({},this.defaultOptions),this._create(t),this.components=[],this.body={dom:this.dom,domProps:this.props,emitter:{on:this.on.bind(this),off:this.off.bind(this),emit:this.emit.bind(this)},hiddenDates:[],util:{snap:null,toScreen:u._toScreen.bind(u),toGlobalScreen:u._toGlobalScreen.bind(u),toTime:u._toTime.bind(u),toGlobalTime:u._toGlobalTime.bind(u)}},this.range=new a(this.body),this.components.push(this.range),this.body.range=this.rang
 e,this.timeAxis=new d(this.body),this.components.push(this.timeAxis),this.body.util.snap=this.timeAxis.snap.bind(this.timeAxis),this.currentTime=new l(this.body),this.components.push(this.currentTime),this.customTime=new c(this.body),this.components.push(this.customTime),this.itemSet=new p(this.body),this.components.push(this.itemSet),this.itemsData=null,this.groupsData=null,r&&this.setOptions(r),i&&this.setGroups(i),e?this.setItems(e):this.redraw()}var o=(i(56),i(45),i(1)),n=i(3),r=i(4),a=i(17),h=i(46),d=i(30),l=i(21),c=i(22),p=i(27);s.prototype=new h,s.prototype.setItems=function(t){var e,i=null==this.itemsData;if(e=t?t instanceof n||t instanceof r?t:new n(t,{type:{start:"Date",end:"Date"}}):null,this.itemsData=e,this.itemSet&&this.itemSet.setItems(e),i)if(void 0!=this.options.start||void 0!=this.options.end){if(void 0==this.options.start||void 0==this.options.end)var s=this._getDataRange();var o=void 0!=this.options.start?this.options.start:s.start,a=void 0!=this.options.end?this
 .options.end:s.end;this.setWindow(o,a,{animate:!1})}else this.fit({animate:!1})},s.prototype.setGroups=function(t){var e;e=t?t instanceof n||t instanceof r?t:new n(t):null,this.groupsData=e,this.itemSet.setGroups(e)},s.prototype.setSelection=function(t,e){this.itemSet&&this.itemSet.setSelection(t),e&&e.focus&&this.focus(t,e)},s.prototype.getSelection=function(){return this.itemSet&&this.itemSet.getSelection()||[]},s.prototype.focus=function(t,e){if(this.itemsData&&void 0!=t){var i=Array.isArray(t)?t:[t],s=this.itemsData.getDataSet().get(i,{type:{start:"Date",end:"Date"}}),o=null,n=null;if(s.forEach(function(t){var e=t.start.valueOf(),i="end"in t?t.end.valueOf():t.start.valueOf();(null===o||o>e)&&(o=e),(null===n||i>n)&&(n=i)}),null!==o&&null!==n){var r=(o+n)/2,a=Math.max(this.range.end-this.range.start,1.1*(n-o)),h=e&&void 0!==e.animate?e.animate:!0;this.range.setRange(r-a/2,r+a/2,h)}}},s.prototype.getItemRange=function(){var t=this.itemsData.getDataSet(),e=null,i=null;if(t){var s=t.
 min("start");e=s?o.convert(s.start,"Date").valueOf():null;var n=t.max("start");n&&(i=o.convert(n.start,"Date").valueOf());var r=t.max("end");r&&(i=null==i?o.convert(r.end,"Date").valueOf():Math.max(i,o.convert(r.end,"Date").valueOf()))}return{min:null!=e?new Date(e):null,max:null!=i?new Date(i):null}},t.exports=s},function(t,e,i){function s(t,e,i,s){if(!(Array.isArray(i)||i instanceof n)&&i instanceof Object){var r=s;s=i,i=r}var h=this;this.defaultOptions={start:null,end:null,autoResize:!0,orientation:"bottom",width:null,height:null,maxHeight:null,minHeight:null},this.options=o.deepExtend({},this.defaultOptions),this._create(t),this.components=[],this.body={dom:this.dom,domProps:this.props,emitter:{on:this.on.bind(this),off:this.off.bind(this),emit:this.emit.bind(this)},hiddenDates:[],util:{snap:null,toScreen:h._toScreen.bind(h),toGlobalScreen:h._toGlobalScreen.bind(h),toTime:h._toTime.bind(h),toGlobalTime:h._toGlobalTime.bind(h)}},this.range=new a(this.body),this.components.push(th
 is.range),this.body.range=this.range,this.timeAxis=new d(this.body),this.components.push(this.timeAxis),this.body.util.snap=this.timeAxis.snap.bind(this.timeAxis),this.currentTime=new l(this.body),this.components.push(this.currentTime),this.customTime=new c(this.body),this.components.push(this.customTime),this.linegraph=new p(this.body),this.components.push(this.linegraph),this.itemsData=null,this.groupsData=null,s&&this.setOptions(s),i&&this.setGroups(i),e?this.setItems(e):this.redraw()}var o=(i(56),i(45),i(1)),n=i(3),r=i(4),a=i(17),h=i(46),d=i(30),l=i(21),c=i(22),p=i(29);s.prototype=new h,s.prototype.setItems=function(t){var e,i=null==this.itemsData;if(e=t?t instanceof n||t instanceof r?t:new n(t,{type:{start:"Date",end:"Date"}}):null,this.itemsData=e,this.linegraph&&this.linegraph.setItems(e),i)if(void 0!=this.options.start||void 0!=this.options.end){var s=void 0!=this.options.start?this.options.start:null,o=void 0!=this.options.end?this.options.end:null;this.setWindow(s,o,{anima
 te:!1})}else this.fit({animate:!1})},s.prototype.setGroups=function(t){var e;e=t?t instanceof n||t instanceof r?t:new n(t):null,this.groupsData=e,this.linegraph.setGroups(e)},s.prototype.getLegend=function(t,e,i){return void 0===e&&(e=15),void 0===i&&(i=15),void 0!==this.linegraph.groups[t]?this.linegraph.groups[t].getLegend(e,i):"cannot find group:"+t},s.prototype.isGroupVisible=function(t){return void 0!==this.linegraph.groups[t]?this.linegraph.groups[t].visible&&(void 0===this.linegraph.options.groups.visibility[t]||1==this.linegraph.options.groups.visibility[t]):!1},s.prototype.getItemRange=function(){var t=null,e=null;for(var i in this.linegraph.groups)if(this.linegraph.groups.hasOwnProperty(i)&&1==this.linegraph.groups[i].visible)for(var s=0;s<this.linegraph.groups[i].itemsData.length;s++){var n=this.linegraph.groups[i].itemsData[s],r=o.convert(n.x,"Date").valueOf();t=null==t?r:t>r?r:t,e=null==e?r:r>e?r:e}return{min:null!=t?new Date(t):null,max:null!=e?new Date(e):null}},t.exp
 orts=s},function(t,e,i){var s=i(44);e.convertHiddenOptions=function(t,e){if(t.hiddenDates=[],e&&1==Array.isArray(e)){for(var i=0;i<e.length;i++)if(void 0===e[i].repeat){var o={};o.start=s(e[i].start).toDate().valueOf(),o.end=s(e[i].end).toDate().valueOf(),t.hiddenDates.push(o)}t.hiddenDates.sort(function(t,e){return t.start-e.start})}},e.updateHiddenDates=function(t,i){if(i&&void 0!==t.domProps.centerContainer.width){e.convertHiddenOptions(t,i);for(var o=s(t.range.start),n=s(t.range.end),r=t.range.end-t.range.start,a=r/t.domProps.centerContainer.width,h=0;h<i.length;h++)if(void 0!==i[h].repeat){var d=s(i[h].start),l=s(i[h].end);if("Invalid Date"==d._d)throw new Error("Supplied start date is not valid: "+i[h].start);if("Invalid Date"==l._d)throw new Error("Supplied end date is not valid: "+i[h].end);var c=l-d;if(c>=4*a){var p=0,u=n.clone();switch(i[h].repeat){case"daily":d.day()!=l.day()&&(p=1),d.dayOfYear(o.dayOfYear()),d.year(o.year()),d.subtract(7,"days"),l.dayOfYear(o.dayOfYear()
 ),l.year(o.year()),l.subtract(7-p,"days"),u.add(1,"weeks");break;case"weekly":var m=l.diff(d,"days"),f=d.day();d.date(o.date()),d.month(o.month()),d.year(o.year()),l=d.clone(),d.day(f),l.day(f),l.add(m,"days"),d.subtract(1,"weeks"),l.subtract(1,"weeks"),u.add(1,"weeks");break;case"monthly":d.month()!=l.month()&&(p=1),d.month(o.month()),d.year(o.year()),d.subtract(1,"months"),l.month(o.month()),l.year(o.year()),l.subtract(1,"months"),l.add(p,"months"),u.add(1,"months");break;case"yearly":d.year()!=l.year()&&(p=1),d.year(o.year()),d.subtract(1,"years"),l.year(o.year()),l.subtract(1,"years"),l.add(p,"years"),u.add(1,"years");break;default:return void console.log("Wrong repeat format, allowed are: daily, weekly, monthly, yearly. Given:",i[h].repeat)}for(;u>d;)switch(t.hiddenDates.push({start:d.valueOf(),end:l.valueOf()}),i[h].repeat){case"daily":d.add(1,"days"),l.add(1,"days");break;case"weekly":d.add(1,"weeks"),l.add(1,"weeks");break;case"monthly":d.add(1,"months"),l.add(1,"months");br
 eak;case"yearly":d.add(1,"y"),l.add(1,"y");break;default:return void console.log("Wrong repeat format, allowed are: daily, weekly, monthly, yearly. Given:",i[h].repeat)}t.hiddenDates.push({start:d.valueOf(),end:l.valueOf()})}}e.removeDuplicates(t);var g=e.isHidden(t.range.start,t.hiddenDates),v=e.isHidden(t.range.end,t.hiddenDates),y=t.range.start,b=t.range.end;1==g.hidden&&(y=1==t.range.startToFront?g.startDate-1:g.endDate+1),1==v.hidden&&(b=1==t.range.endToFront?v.startDate-1:v.endDate+1),(1==g.hidden||1==v.hidden)&&t.range._applyRange(y,b)}},e.removeDuplicates=function(t){for(var e=t.hiddenDates,i=[],s=0;s<e.length;s++)for(var o=0;o<e.length;o++)s!=o&&1!=e[o].remove&&1!=e[s].remove&&(e[o].start>=e[s].start&&e[o].end<=e[s].end?e[o].remove=!0:e[o].start>=e[s].start&&e[o].start<=e[s].end?(e[s].end=e[o].end,e[o].remove=!0):e[o].end>=e[s].start&&e[o].end<=e[s].end&&(e[s].start=e[o].start,e[o].remove=!0));for(var s=0;s<e.length;s++)e[s].remove!==!0&&i.push(e[s]);t.hiddenDates=i,t.hidde
 nDates.sort(function(t,e){return t.start-e.start})},e.printDates=function(t){for(var e=0;e<t.length;e++)console.log(e,new Date(t[e].start),new Date(t[e].end),t[e].start,t[e].end,t[e].remove)},e.stepOverHiddenDates=function(t,e){for(var i=!1,o=t.current.valueOf(),n=0;n<t.hiddenDates.length;n++){var r=t.hiddenDates[n].start,a=t.hiddenDates[n].end;if(o>=r&&a>o){i=!0;break}}if(1==i&&o<t._end.valueOf()&&o!=e){var h=s(e),d=s(a);h.year()!=d.year()?t.switchedYear=!0:h.month()!=d.month()?t.switchedMonth=!0:h.dayOfYear()!=d.dayOfYear()&&(t.switchedDay=!0),t.current=d.toDate()}},e.toScreen=function(t,i,s){if(0==t.body.hiddenDates.length){var o=t.range.conversion(s);return(i.valueOf()-o.offset)*o.scale}var n=e.isHidden(i,t.body.hiddenDates);1==n.hidden&&(i=n.startDate);var r=e.getHiddenDurationBetween(t.body.hiddenDates,t.range.start,t.range.end);i=e.correctTimeForHidden(t.body.hiddenDates,t.range,i);var o=t.range.conversion(s,r);return(i.valueOf()-o.offset)*o.scale},e.toTime=function(t,i,s){if
 (0==t.body.hiddenDates.length){var o=t.range.conversion(s);return new Date(i/o.scale+o.offset)}var n=e.getHiddenDurationBetween(t.body.hiddenDates,t.range.start,t.range.end),r=t.range.end-t.range.start-n,a=r*i/s,h=e.getAccumulatedHiddenDuration(t.body.hiddenDates,t.range,a),d=new Date(h+a+t.range.start);return d},e.getHiddenDurationBetween=function(t,e,i){for(var s=0,o=0;o<t.length;o++){var n=t[o].start,r=t[o].end;n>=e&&i>r&&(s+=r-n)}return s},e.correctTimeForHidden=function(t,i,o){return o=s(o).toDate().valueOf(),o-=e.getHiddenDurationBefore(t,i,o)},e.getHiddenDurationBefore=function(t,e,i){var o=0;i=s(i).toDate().valueOf();for(var n=0;n<t.length;n++){var r=t[n].start,a=t[n].end;r>=e.start&&a<e.end&&i>=a&&(o+=a-r)}return o},e.getAccumulatedHiddenDuration=function(t,e,i){for(var s=0,o=0,n=e.start,r=0;r<t.length;r++){var a=t[r].start,h=t[r].end;if(a>=e.start&&h<e.end){if(o+=a-n,n=h,o>=i)break;s+=h-a}}return s},e.snapAwayFromHidden=function(t,i,s,o){var n=e.isHidden(i,t);return 1==n.h
 idden?0>s?1==o?n.startDate-(n.endDate-i)-1:n.startDate-1:1==o?n.endDate+(i-n.startDate)+1:n.endDate+1:i},e.isHidden=function(t,e){for(var i=0;i<e.length;i++){var s=e[i].start,o=e[i].end;if(t>=s&&o>t)return{hidden:!0,startDate:s,endDate:o}}return{hidden:!1,startDate:s,endDate:o}}},function(t){function e(t,e,i,s,o,n){this.current=0,this.autoScale=!0,this.stepIndex=0,this.step=1,this.scale=1,this.marginStart,this.marginEnd,this.deadSpace=0,this.majorSteps=[1,2,5,10],this.minorSteps=[.25,.5,1,2],this.alignZeros=n,this.setRange(t,e,i,s,o)}e.prototype.setRange=function(t,e,i,s,o){this._start=void 0===o.min?t:o.min,this._end=void 0===o.max?e:o.max,this._start==this._end&&(this._start-=.75,this._end+=1),1==this.autoScale&&this.setMinimumStep(i,s),this.setFirst(o)},e.prototype.setMinimumStep=function(t,e){var i=this._end-this._start,s=1.2*i,o=t*(s/e),n=Math.round(Math.log(s)/Math.LN10),r=-1,a=Math.pow(10,n),h=0;0>n&&(h=n);for(var d=!1,l=h;Math.abs(l)<=Math.abs(n);l++){a=Math.pow(10,l);for(va
 r c=0;c<this.minorSteps.length;c++){var p=a*this.minorSteps[c];if(p>=o){d=!0,r=c;break}}if(1==d)break}this.stepIndex=r,this.scale=a,this.step=a*this.minorSteps[r]},e.prototype.setFirst=function(t){void 0===t&&(t={});var e=void 0===t.min?this._start-2*this.scale*this.minorSteps[this.stepIndex]:t.min,i=void 0===t.max?this._end+this.scale*this.minorSteps[this.stepIndex]:t.max;this.marginEnd=void 0===t.max?this.roundToMinor(i):t.max,this.marginStart=void 0===t.min?this.roundToMinor(e):t.min,1==this.alignZeros&&(this.marginEnd-this.marginStart)%this.step!=0&&(this.marginEnd+=this.marginEnd%this.step),this.deadSpace=this.roundToMinor(i)-i+this.roundToMinor(e)-e,this.marginRange=this.marginEnd-this.marginStart,this.current=this.marginEnd},e.prototype.roundToMinor=function(t){var e=t-t%(this.scale*this.minorSteps[this.stepIndex]);return t%(this.scale*this.minorSteps[this.stepIndex])>.5*this.scale*this.minorSteps[this.stepIndex]?e+this.scale*this.minorSteps[this.stepIndex]:e},e.prototype.has
 Next=function(){return this.current>=this.marginStart},e.prototype.next=function(){var t=this.current;this.current-=this.step,this.current==t&&(this.current=this._end)},e.prototype.previous=function(){this.current+=this.step,this.marginEnd+=this.step,this.marginRange=this.marginEnd-this.marginStart},e.prototype.getCurrent=function(t){var e=Math.abs(this.current)<this.step/2?0:this.current,i=""+Number(e).toPrecision(5);if(void 0===t||isNaN(Number(t))){if(-1!=i.indexOf(",")||-1!=i.indexOf("."))for(var s=i.length-1;s>0;s--){if("0"!=i[s]){if("."==i[s]||","==i[s]){i=i.slice(0,s);break}break}i=i.slice(0,s)}}else{var o="",n=i.indexOf("e");if(-1!=n&&(o=i.slice(n),i=i.slice(0,n)),n=Math.max(i.indexOf(","),i.indexOf(".")),-1===n?(0!==t&&(i+="."),n=i.length+t):0!==t&&(n+=t+1),n>i.length)for(var r=n-i.length;r>0;r--)i+="0";else i=i.slice(0,n);i+=o}return i},e.prototype.snap=function(){},e.prototype.isMajor=function(){return this.current%(this.scale*this.majorSteps[this.stepIndex])==0},t.exports
 =e},function(t,e,i){function s(t,e){var i=a().hours(0).minutes(0).seconds(0).milliseconds(0);this.start=i.clone().add(-3,"days").valueOf(),this.end=i.clone().add(4,"days").valueOf(),this.body=t,this.deltaDifference=0,this.scaleOffset=0,this.startToFront=!1,this.endToFront=!0,this.defaultOptions={start:null,end:null,direction:"horizontal",moveable:!0,zoomable:!0,min:null,max:null,zoomMin:10,zoomMax:31536e10},this.options=r.extend({},this.defaultOptions),this.props={touch:{}},this.animateTimer=null,this.body.emitter.on("panstart",this._onDragStart.bind(this)),this.body.emitter.on("panmove",this._onDrag.bind(this)),this.body.emitter.on("panend",this._onDragEnd.bind(this)),this.body.emitter.on("press",this._onHold.bind(this)),this.body.emitter.on("mousewheel",this._onMouseWheel.bind(this)),this.body.emitter.on("touch",this._onTouch.bind(this)),this.body.emitter.on("pinch",this._onPinch.bind(this)),this.setOptions(e)}function o(t){if("horizontal"!=t&&"vertical"!=t)throw new TypeError('Un
 known direction "'+t+'". Choose "horizontal" or "vertical".')}function n(t,e){return{x:t.x-r.getAbsoluteLeft(e),y:t.y-r.getAbsoluteTop(e)}}var r=i(1),a=(i(47),i(44)),h=i(20),d=i(15);s.prototype=new h,s.prototype.setOptions=function(t){if(t){var e=["direction","min","max","zoomMin","zoomMax","moveable","zoomable","activate","hiddenDates"];r.selectiveExtend(e,this.options,t),("start"in t||"end"in t)&&this.setRange(t.start,t.end)}},s.prototype.setRange=function(t,e,i,s){s!==!0&&(s=!1);var o=void 0!=t?r.convert(t,"Date").valueOf():null,n=void 0!=e?r.convert(e,"Date").valueOf():null;if(this._cancelAnimation(),i){var a=this,h=this.start,l=this.end,c="number"==typeof i?i:500,p=(new Date).valueOf(),u=!1,m=function(){if(!a.props.touch.dragging){var t=(new Date).valueOf(),e=t-p,i=e>c,g=i||null===o?o:r.easeInOutQuad(e,h,o,c),v=i||null===n?n:r.easeInOutQuad(e,l,n,c);f=a._applyRange(g,v),d.updateHiddenDates(a.body,a.options.hiddenDates),u=u||f,f&&a.body.emitter.emit("rangechange",{start:new Date
 (a.start),end:new Date(a.end),byUser:s}),i?u&&a.body.emitter.emit("rangechanged",{start:new Date(a.start),end:new Date(a.end),byUser:s}):a.animateTimer=setTimeout(m,20)}};return m()}var f=this._applyRange(o,n);if(d.updateHiddenDates(this.body,this.options.hiddenDates),f){var g={start:new Date(this.start),end:new Date(this.end),byUser:s};this.body.emitter.emit("rangechange",g),this.body.emitter.emit("rangechanged",g)}},s.prototype._cancelAnimation=function(){this.animateTimer&&(clearTimeout(this.animateTimer),this.animateTimer=null)},s.prototype._applyRange=function(t,e){var i,s=null!=t?r.convert(t,"Date").valueOf():this.start,o=null!=e?r.convert(e,"Date").valueOf():this.end,n=null!=this.options.max?r.convert(this.options.max,"Date").valueOf():null,a=null!=this.options.min?r.convert(this.options.min,"Date").valueOf():null;if(isNaN(s)||null===s)throw new Error('Invalid start "'+t+'"');if(isNaN(o)||null===o)throw new Error('Invalid end "'+e+'"');if(s>o&&(o=s),null!==a&&a>s&&(i=a-s,s+=i
 ,o+=i,null!=n&&o>n&&(o=n)),null!==n&&o>n&&(i=o-n,s-=i,o-=i,null!=a&&a>s&&(s=a)),null!==this.options.zoomMin){var h=parseFloat(this.options.zoomMin);0>h&&(h=0),h>o-s&&(this.end-this.start===h?(s=this.start,o=this.end):(i=h-(o-s),s-=i/2,o+=i/2))}if(null!==this.options.zoomMax){var d=parseFloat(this.options.zoomMax);0>d&&(d=0),o-s>d&&(this.end-this.start===d?(s=this.start,o=this.end):(i=o-s-d,s+=i/2,o-=i/2))}var l=this.start!=s||this.end!=o;return s>=this.start&&s<=this.end||o>=this.start&&o<=this.end||this.start>=s&&this.start<=o||this.end>=s&&this.end<=o||this.body.emitter.emit("checkRangedItems"),this.start=s,this.end=o,l},s.prototype.getRange=function(){return{start:this.start,end:this.end}},s.prototype.conversion=function(t,e){return s.conversion(this.start,this.end,t,e)},s.conversion=function(t,e,i,s){return void 0===s&&(s=0),0!=i&&e-t!=0?{offset:t,scale:i/(e-t-s)}:{offset:0,scale:1}},s.prototype._onDragStart=function(t){this.deltaDifference=0,this.previousDelta=0,this.options.mo
 veable&&this.props.touch.allowDragging&&(this.props.touch.start=this.start,this.props.touch.end=this.end,this.props.touch.dragging=!0,this.body.dom.root&&(this.body.dom.root.style.cursor="move"),t.preventDefault())},s.prototype._onDrag=function(t){if(this.options.moveable&&

<TRUNCATED>

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


[4/5] spark git commit: [SPARK-3468] [WEBUI] Timeline-View feature

Posted by pw...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/7fe0f3f2/core/src/main/resources/org/apache/spark/ui/static/vis.map
----------------------------------------------------------------------
diff --git a/core/src/main/resources/org/apache/spark/ui/static/vis.map b/core/src/main/resources/org/apache/spark/ui/static/vis.map
new file mode 100644
index 0000000..9be8b61
--- /dev/null
+++ b/core/src/main/resources/org/apache/spark/ui/static/vis.map
@@ -0,0 +1 @@
+{"version":3,"file":"vis.map","sources":["./dist/vis.js"],"names":["root","factory","exports","module","define","amd","this","modules","__webpack_require__","moduleId","installedModules","id","loaded","call","m","c","p","util","DOMutil","DataSet","DataView","Queue","Graph3d","graph3d","Camera","Filter","Point2d","Point3d","Slider","StepNumber","Timeline","Graph2d","timeline","DateUtil","DataStep","Range","stack","TimeStep","components","items","Item","BackgroundItem","BoxItem","PointItem","RangeItem","Component","CurrentTime","CustomTime","DataAxis","GraphGroup","Group","BackgroundGroup","ItemSet","Legend","LineGraph","TimeAxis","Network","network","Edge","Groups","Images","Node","Popup","dotparser","gephiParser","Graph","Error","moment","hammer","Hammer","isNumber","object","Number","isString","String","isDate","Date","match","ASPDateRegex","exec","isNaN","parse","isDataTable","google","visualization","DataTable","randomUUID","S4","Math","floor","random","toString","extend","a","i"
 ,"len","arguments","length","other","prop","hasOwnProperty","selectiveExtend","props","Array","isArray","selectiveDeepExtend","b","TypeError","constructor","Object","undefined","deepExtend","selectiveNotDeepExtend","indexOf","equalArray","convert","type","Boolean","valueOf","isMoment","toDate","getType","toISOString","value","getAbsoluteLeft","elem","getBoundingClientRect","left","getAbsoluteTop","top","addClassName","className","classes","split","push","join","removeClassName","index","splice","forEach","callback","toArray","array","updateProperty","key","addEventListener","element","action","listener","useCapture","navigator","userAgent","attachEvent","removeEventListener","detachEvent","preventDefault","event","window","returnValue","getTarget","target","srcElement","nodeType","parentNode","option","asBoolean","defaultValue","asNumber","asString","asSize","asElement","hexToRGB","hex","shorthandRegex","replace","r","g","result","parseInt","RGBToHex","red","green","blue","slice","p
 arseColor","color","isValidRGB","rgb","substr","isValidHex","hsv","hexToHSV","lighterColorHSV","h","s","v","min","darkerColorHSV","darkerColorHex","HSVToHex","lighterColorHex","background","border","highlight","hover","RGBToHSV","minRGB","maxRGB","max","d","hue","saturation","cssUtil","cssText","styles","style","trim","parts","keys","map","addCssText","currentStyles","newStyles","removeCssText","removeStyles","HSVToRGB","f","q","t","isOk","test","selectiveBridgeObject","fields","referenceObject","objectTo","create","bridgeObject","mergeOptions","mergeTarget","options","enabled","binarySearchCustom","orderedItems","searchFunction","field","field2","maxIterations","iteration","low","high","middle","item","searchResult","binarySearchValue","sidePreference","prevValue","nextValue","easeInOutQuad","start","end","duration","change","easingFunctions","linear","easeInQuad","easeOutQuad","easeInCubic","easeOutCubic","easeInOutCubic","easeInQuart","easeOutQuart","easeInOutQuart","easeInQuint"
 ,"easeOutQuint","easeInOutQuint","prepareElements","JSONcontainer","elementType","redundant","used","cleanupElements","removeChild","getSVGElement","svgContainer","shift","document","createElementNS","appendChild","getDOMElement","DOMContainer","insertBefore","createElement","drawPoint","x","y","group","point","drawPoints","setAttributeNS","size","drawBar","width","height","rect","data","_options","_data","_fieldId","fieldId","_type","_subscribers","add","setOptions","prototype","queue","_queue","destroy","on","subscribers","subscribe","off","filter","unsubscribe","_trigger","params","senderId","concat","subscriber","addedIds","me","_addItem","columns","_getColumnNames","row","rows","getNumberOfRows","col","cols","getValue","update","updatedIds","updatedData","addOrUpdate","_updateItem","get","ids","firstType","returnType","allowedValues","itemId","_getItem","order","_sort","_filterFields","_appendRow","getIds","getDataSet","mappedItems","filteredItem","name","sort","av","bv","remov
 e","removedId","removedIds","_remove","clear","maxField","itemField","minField","distinct","values","fieldType","count","exists","types","raw","converted","JSON","stringify","dataTable","getNumberOfColumns","getColumnId","getColumnLabel","addRow","setValue","_ids","_onEvent","apply","setData","viewOptions","getArguments","defaultFilter","dataSet","added","updated","removed","delay","Infinity","_timeout","_extended","_flushIfNeeded","flush","methods","original","method","args","fn","context","entry","clearTimeout","setTimeout","container","SyntaxError","containerElement","margin","defaultXCenter","defaultYCenter","xLabel","yLabel","zLabel","passValueFn","xValueLabel","yValueLabel","zValueLabel","filterLabel","legendLabel","STYLE","DOT","showPerspective","showGrid","keepAspectRatio","showShadow","showGrayBottom","showTooltip","verticalRatio","animationInterval","animationPreload","camera","eye","dataPoints","colX","colY","colZ","colValue","colFilter","xMin","xStep","xMax","yMin","ySte
 p","yMax","zMin","zStep","zMax","valueMin","valueMax","xBarWidth","yBarWidth","colorAxis","colorGrid","colorDot","colorDotBorder","getMouseX","clientX","targetTouches","getMouseY","clientY","Emitter","_setScale","scale","z","xCenter","yCenter","zCenter","setArmLocation","_convert3Dto2D","point3d","translation","_convertPointToTranslation","_convertTranslationToScreen","ax","ay","az","cx","getCameraLocation","cy","cz","sinTx","sin","getCameraRotation","cosTx","cos","sinTy","cosTy","sinTz","cosTz","dx","dy","dz","bx","by","ex","ey","ez","getArmLength","xcenter","frame","canvas","clientWidth","ycenter","_setBackgroundColor","backgroundColor","fill","stroke","strokeWidth","borderColor","borderWidth","borderStyle","BAR","BARCOLOR","BARSIZE","DOTLINE","DOTCOLOR","DOTSIZE","GRID","LINE","SURFACE","_getStyleNumber","styleName","_determineColumnIndexes","counter","column","getDistinctValues","distinctValues","getColumnRange","minMax","_dataInitialize","rawData","_onChange","dataFilter","setO
 nLoadCallback","redraw","withBars","defaultXBarWidth","dataX","defaultYBarWidth","dataY","xRange","defaultXMin","defaultXMax","defaultXStep","yRange","defaultYMin","defaultYMax","defaultYStep","zRange","defaultZMin","defaultZMax","defaultZStep","valueRange","defaultValueMin","defaultValueMax","_getDataPoints","obj","sortNumber","dataMatrix","xIndex","yIndex","trans","screen","bottom","pointRight","pointTop","pointCross","hasChildNodes","firstChild","position","overflow","noCanvas","fontWeight","padding","innerHTML","onmousedown","_onMouseDown","ontouchstart","_onTouchStart","onmousewheel","_onWheel","ontooltip","_onTooltip","onkeydown","setSize","_resizeCanvas","clientHeight","animationStart","slider","play","animationStop","stop","_resizeCenter","charAt","parseFloat","setCameraPosition","pos","horizontal","vertical","setArmRotation","distance","setArmLength","getCameraPosition","getArmRotation","_readData","_redrawFilter","animationAutoStart","cameraPosition","styleNumber","tooltip
 ","showAnimationControls","_redrawSlider","_redrawClear","_redrawAxis","_redrawDataGrid","_redrawDataLine","_redrawDataBar","_redrawDataDot","_redrawInfo","_redrawLegend","ctx","getContext","clearRect","widthMin","widthMax","dotSize","right","lineWidth","font","ymin","ymax","_hsv2rgb","strokeStyle","beginPath","moveTo","lineTo","strokeRect","fillStyle","closePath","gridLineLen","step","getCurrent","next","textAlign","textBaseline","fillText","label","visible","setValues","setPlayInterval","onchange","getIndex","selectValue","setOnChangeCallback","lineStyle","getLabel","getSelectedValue","from","to","prettyStep","text","xText","yText","zText","offset","xOffset","yOffset","xMin2d","xMax2d","gridLenX","gridLenY","textMargin","armAngle","H","S","V","R","G","B","C","Hi","X","abs","cross","topSideVisible","zAvg","transBottom","dist","sortDepth","aDiff","subtract","bDiff","crossproduct","crossProduct","radius","arc","PI","j","surface","corners","xWidth","yWidth","surfaces","center","avg","
 transCenter","diff","leftButtonDown","_onMouseUp","which","button","touchDown","startMouseX","startMouseY","startStart","startEnd","startArmRotation","cursor","onmousemove","_onMouseMove","onmouseup","diffX","diffY","horizontalNew","verticalNew","snapAngle","snapValue","round","parameters","emit","boundingRect","mouseX","mouseY","tooltipTimeout","_hideTooltip","dataPoint","_dataPointFromXY","_showTooltip","ontouchmove","_onTouchMove","ontouchend","_onTouchEnd","delta","wheelDelta","detail","oldLength","newLength","_insideTriangle","triangle","sign","as","bs","cs","distMax","closestDataPoint","closestDist","triangle1","triangle2","distX","distY","sqrt","content","line","dot","dom","borderRadius","boxShadow","borderLeft","contentWidth","offsetWidth","contentHeight","offsetHeight","lineHeight","dotWidth","dotHeight","armLocation","armRotation","armLength","cameraLocation","cameraRotation","calculateCameraOrientation","rot","graph","onLoadCallback","loadInBackground","isLoaded","getLoad
 edProgress","getColumn","getValues","dataView","progress","sub","sum","prev","bar","MozBorderRadius","slide","onclick","togglePlay","onChangeCallback","playTimeout","playInterval","playLoop","setIndex","playNext","interval","clearInterval","getPlayInterval","setPlayLoop","doLoop","onChange","indexToLeft","startClientX","startSlideX","leftToIndex","_start","_end","_step","precision","_current","setRange","setStep","calculatePrettyStep","log10","log","LN10","step1","pow","step2","step5","toPrecision","getStep","groups","forthArgument","defaultOptions","autoResize","orientation","maxHeight","minHeight","_create","body","domProps","emitter","bind","hiddenDates","snap","toScreen","_toScreen","toGlobalScreen","_toGlobalScreen","toTime","_toTime","toGlobalTime","_toGlobalTime","range","timeAxis","currentTime","customTime","itemSet","itemsData","groupsData","setGroups","setItems","Core","newDataSet","initialLoad","dataRange","_getDataRange","setWindow","animate","fit","setSelection","focus"
 ,"getSelection","itemData","e","getItemRange","dataset","minItem","maxStartItem","maxEndItem","linegraph","getLegend","groupId","isGroupVisible","visibility","convertHiddenOptions","repeat","dateItem","updateHiddenDates","centerContainer","totalRange","pixelTime","startDate","endDate","_d","runUntil","clone","day","dayOfYear","year","dayOffset","date","month","console","removeDuplicates","startHidden","isHidden","endHidden","rangeStart","rangeEnd","hidden","startToFront","endToFront","_applyRange","safeDates","printDates","dates","stepOverHiddenDates","timeStep","previousTime","stepInHidden","currentValue","current","newValue","switchedYear","switchedMonth","switchedDay","time","conversion","getHiddenDurationBetween","correctTimeForHidden","hiddenDuration","totalDuration","partialDuration","accumulatedHiddenDuration","getAccumulatedHiddenDuration","newTime","getHiddenDurationBefore","timeOffset","requiredDuration","previousPoint","snapAwayFromHidden","direction","correctionEnabled",
 "minimumStep","containerHeight","customRange","alignZeros","autoScale","stepIndex","marginStart","marginEnd","deadSpace","majorSteps","minorSteps","setMinimumStep","setFirst","safeSize","minimumStepValue","orderOfMagnitude","minorStepIdx","magnitudefactor","solutionFound","stepSize","niceStart","niceEnd","roundToMinor","marginRange","rounded","hasNext","previous","decimals","exp","cnt","isMajor","now","hours","minutes","seconds","milliseconds","deltaDifference","scaleOffset","moveable","zoomable","zoomMin","zoomMax","touch","animateTimer","_onDragStart","_onDrag","_onDragEnd","_onHold","_onMouseWheel","_onTouch","_onPinch","validateDirection","getPointer","byUser","_cancelAnimation","initStart","initEnd","initTime","anyChanged","dragging","done","changed","newStart","newEnd","getRange","totalHidden","previousDelta","allowDragging","deltaX","deltaY","diffRange","safeStart","safeEnd","pointer","pageX","pageY","pointerDate","_pointerToDate","zoom","centerDate","hiddenDurationBefore","h
 iddenDurationAfter","move","EPSILON","orderByStart","orderByEnd","aTime","bTime","force","iMax","axis","collidingItem","jj","collision","nostack","subgroups","newTop","subgroup","format","FORMAT","minorLabels","millisecond","second","minute","hour","weekday","majorLabels","setFormat","defaultFormat","first","setFullYear","getFullYear","setMonth","setDate","setHours","setMinutes","setSeconds","setMilliseconds","getMilliseconds","getSeconds","getMinutes","getHours","getDate","getMonth","setScale","newScale","newStep","setAutoScale","enable","stepYear","stepMonth","stepDay","stepHour","stepMinute","stepSecond","stepMillisecond","getLabelMinor","getLabelMajor","getClassName","even","today","isSame","currentWeek","currentMonth","currentYear","locale","lang","toLowerCase","_isResized","resized","_previousWidth","_previousHeight","showCurrentTime","locales","parent","backgroundVertical","title","toUpperCase","substring","currentTimeTimer","setCurrentTime","getCurrentTime","showCustomTime",
 "eventParams","drag","setCustomTime","getCustomTime","stopPropagation","svg","linegraphOptions","showMinorLabels","showMajorLabels","icons","majorLinesOffset","minorLinesOffset","labelOffsetX","labelOffsetY","iconWidth","linegraphSVG","DOMelements","lines","labels","conversionFactor","minWidth","stepPixels","stepPixelsForced","zeroCrossing","lineOffset","master","svgElements","iconsRemoved","amountOfGroups","lineContainer","scrollTop","addGroup","graphOptions","updateGroup","removeGroup","hide","show","display","_redrawGroupIcons","iconHeight","iconOffset","drawIcon","_cleanupIcons","backgroundHorizontal","activeGroups","_calculateCharSize","minorLabelHeight","minorCharHeight","majorLabelHeight","majorCharHeight","minorLineWidth","minorLineHeight","majorLineWidth","majorLineHeight","_redrawLabels","_redrawTitle","amountOfSteps","stepDifference","zeroStepDifference","valueAtZero","marginStartPos","maxLabelSize","_redrawLabel","_redrawLine","titleWidth","titleCharHeight","convertValue
 ","invertedValue","convertedValue","characterHeight","largestWidth","majorCharWidth","minorCharWidth","textMinor","createTextNode","measureCharMinor","textMajor","measureCharMajor","textTitle","measureCharTitle","titleCharWidth","groupsUsingDefaultStyles","usingDefaultStyle","zeroPosition","Line","Bar","Points","setZeroPosition","catmullRom","parametrization","alpha","SVGcontainer","path","fillPath","fillHeight","outline","shaded","barWidth","bar1Height","bar2Height","icon","yAxisOrientation","getYRange","groupData","draw","framework","subgroupIndex","subgroupOrderer","subgroupOrder","visibleItems","byStart","byEnd","checkRangedItems","inner","foreground","marker","Element","getLabelWidth","restack","_updateVisibleItems","markerHeight","lastMarkerHeight","dirty","displayed","_calculateHeight","offsetTop","offsetLeft","ii","repositionY","resetSubgroups","labelSet","setParent","orderSubgroups","_checkIfVisible","sortArray","sortField","removeFromDataSet","removeItem","startArray","end
 Array","oldVisibleItems","visibleItemsLookup","lowerBound","upperBound","_checkIfVisibleWithReference","initialPosByStart","_traceVisible","initialPosByEnd","repositionX","initialPos","breakCondition","isVisible","align","groupOrder","selectable","editable","updateTime","onAdd","onUpdate","onMove","onRemove","onMoving","itemOptions","itemListeners","_onAdd","_onUpdate","_onRemove","groupListeners","_onAddGroups","_onUpdateGroups","_onRemoveGroups","groupIds","selection","stackDirty","touchParams","UNGROUPED","BACKGROUND","box","_updateUngrouped","backgroundGroup","isFirst","_onSelectItem","_onMultiSelectItem","_onAddItem","addCallback","Function","markDirty","unselect","select","getVisibleItems","rawVisibleItems","_deselect","_orderGroups","visibleInterval","zoomed","lastVisibleInterval","lastWidth","firstGroup","_firstGroup","firstMargin","nonFirstMargin","groupMargin","groupResized","firstGroupIndex","firstGroupId","ungrouped","_getGroupId","getLabelSet","oldItemsData","getItems",
 "_order","getGroups","_getType","_removeItem","groupOptions","oldGroupId","oldGroup","_constructByEndArray","itemFromTarget","dragLeftItem","dragRightItem","itemProps","selected","initialX","newProps","initial","groupFromTarget","_updateItemProps","_moveToGroup","changes","ctrlKey","srcEvent","shiftKey","oldSelection","newSelection","xAbs","newItem","_getItemRange","_item","itemSetFromTarget","side","iconSize","iconSpacing","textArea","scrollableHeight","drawLegendIcons","getComputedStyle","paddingTop","defaultGroup","sampling","graphHeight","barChart","handleOverlap","dataAxis","legend","abortedGraphUpdate","updateSVGheight","updateSVGheightOnResize","lastStart","COUNTER","BarGraphFunctions","yAxisLeft","yAxisRight","legendLeft","legendRight","_updateAllGroupData","_updateGroup","groupsContent","ungroupedCounter","forceGraphUpdate","_updateGraph","rangePerPixelInv","preprocessedGroupData","processedGroupData","groupRanges","changeCalled","minDate","maxDate","_getRelevantData","_app
 lySampling","_convertXcoordinates","_getYRanges","_updateYAxis","MAX_CYCLES","_convertYcoordinates","dataContainer","guess","increment","amountOfPoints","xDistance","pointsPerPixel","ceil","sampledData","barCombinedDataLeft","barCombinedDataRight","getStackedBarYRange","minVal","maxVal","yAxisLeftUsed","yAxisRightUsed","minLeft","minRight","maxLeft","maxRight","ignore","_toggleAxisVisiblity","drawIcons","axisUsed","datapoints","xValue","yValue","extractedData","svgHeight","majorTexts","minorTexts","lineTop","parentChanged","foregroundNextSibling","nextSibling","backgroundNextSibling","_repaintLabels","timeLabelsize","cur","prevLine","xPrev","xFirstMajorLabel","_repaintMinorText","_repaintMajorText","_repaintMajorLine","_repaintMinorLine","leftTime","leftText","widthText","arr","pop","childNodes","nodeValue","_repaintDeleteButton","anchor","deleteButton","_updateContents","template","_updateTitle","removeAttribute","_updateDataAttributes","dataAttributes","attributes","setAttribute",
 "_updateStyle","emptyContent","baseClassName","onTop","itemSubgroup","itemSetHeight","marginLeft","maxWidth","_repaintDragLeft","_repaintDragRight","contentLeft","parentWidth","boxWidth","dragLeft","dragRight","_determineBrowserMethod","_initializeMixinLoaders","renderRefreshRate","renderTimestep","renderTime","physicsTime","runDoubleSpeed","physicsDiscreteStepsize","initializing","triggerFunctions","edit","editEdge","connect","del","nodes","mass","radiusMin","radiusMax","shape","image","fontColor","fontSize","fontFace","fontFill","fontStrokeWidth","fontStrokeColor","level","borderWidthSelected","edges","widthSelectionMultiplier","hoverWidth","labelAlignment","arrowScaleFactor","dash","gap","altLength","inheritColor","configurePhysics","physics","barnesHut","thetaInverted","gravitationalConstant","centralGravity","springLength","springConstant","damping","repulsion","nodeDistance","hierarchicalRepulsion","clustering","initialMaxNodes","clusterThreshold","reduceToNodes","chainThresho
 ld","clusterEdgeThreshold","sectorThreshold","screenSizeThreshold","fontSizeMultiplier","maxFontSize","forceAmplification","distanceAmplification","edgeGrowth","nodeScaling","maxNodeSizeIncrements","activeAreaBoxSize","clusterLevelDifference","navigation","keyboard","speed","dataManipulation","initiallyVisible","hierarchicalLayout","levelSeparation","nodeSpacing","layout","freezeForStabilization","smoothCurves","dynamic","roundness","maxVelocity","minVelocity","stabilize","stabilizationIterations","zoomExtentOnStabilize","dragNetwork","dragNodes","hideEdgesOnDrag","hideNodesOnDrag","constants","pixelRatio","hoverObj","controlNodesActive","navigationHammers","existing","_new","animationSpeed","animationEasingFunction","easingTime","sourceScale","targetScale","sourceTranslation","targetTranslation","lockedOnNodeId","lockedOnNodeOffset","touchTime","images","setOnloadCallback","_redraw","xIncrement","yIncrement","zoomIncrement","_loadPhysicsSystem","_loadSectorSystem","_loadClusterSyst
 em","_loadSelectionSystem","_loadHierarchySystem","_setTranslation","freezeSimulation","cachedFunctions","startedStabilization","stabilized","draggingNodes","calculationNodes","calculationNodeIndices","nodeIndices","canvasTopLeft","canvasBottomRight","pointerPosition","areaCenter","previousScale","nodesData","edgesData","nodesListeners","_addNodes","_updateNodes","_removeNodes","edgesListeners","_addEdges","_updateEdges","_removeEdges","moving","timer","_setupHierarchicalLayout","zoomExtent","startWithClustering","keycharm","hammerUtil","MixinLoader","Activator","browserType","requiresTimeout","_getScriptPath","scripts","getElementsByTagName","src","_getRange","node","minY","maxY","minX","maxX","nodeId","boundingBox","_findCenter","animationOptions","initialZoom","disableStart","zoomLevel","numberOfNodes","factor","yDistance","xZoomLevel","yZoomLevel","animation","_updateNodeIndexList","_clearNodeIndexList","idx","_createManipulatorBar","dotData","DOTToGraph","gephi","gephiData","pa
 rseGephi","_setNodes","_setEdges","_putDataInSector","_resetLevels","_stabilize","onEdit","onEditEdge","onConnect","onDelete","editMode","newColorObj","groupname","clickToUse","activator","_createKeyBinds","_loadNavigationControls","_loadManipulationSystem","_configureSmoothCurves","devicePixelRatio","webkitBackingStorePixelRatio","mozBackingStorePixelRatio","msBackingStorePixelRatio","oBackingStorePixelRatio","backingStorePixelRatio","setTransform","pinch","set","_onTap","_onDoubleTap","onTouch","_onMouseMoveTitle","reset","isActive","_moveUp","_yStopMoving","_moveDown","_moveLeft","_xStopMoving","_moveRight","_zoomIn","_stopZoom","_zoomOut","_deleteSelected","_cleanupPhysicsConfiguration","_recursiveDOMDelete","DOMobject","_getPointer","pinched","_getScale","_handleTouch","_handleDragStart","_getNodeAt","_getTranslation","isSelected","_selectObject","nodeIds","objectId","selectionObj","xFixed","yFixed","_handleOnDrag","releaseNode","_XconvertDOMtoCanvas","_XconvertCanvasToDOM","_Y
 convertDOMtoCanvas","_YconvertCanvasToDOM","_handleDragEnd","_handleTap","_handleDoubleTap","_handleOnHold","_onRelease","_handleOnRelease","_zoom","scaleOld","preScaleDragPointer","DOMtoCanvas","scaleFrac","tx","ty","updateClustersDefault","postScaleDragPointer","canvasToDOM","popupObj","_checkHidePopup","checkShow","_checkShowPopup","popupTimer","edgeId","_getEdgeAt","_hoverObject","_blurObject","lastPopupNode","nodeUnderCursor","overlappingNodes","isOverlappingWith","getTitle","overlappingEdges","edge","connected","popup","setPosition","setText","emitEvent","oldWidth","oldHeight","oldNodesData","_updateSelection","angle","_updateCalculationNodes","_reconnectEdges","_updateValueRange","updateLabels","changedData","setProperties","properties","oldEdgesData","oldEdge","disconnect","showInternalIds","_createBezierNodes","via","sectors","dynamicEdges","setValueRange","w","save","translate","_doInAllSectors","restore","offsetX","offsetY","_drawNodes","alwaysShow","setScaleAndPos","inAr
 ea","sMax","_drawEdges","_drawControlNodes","_freezeDefinedNodes","_physicsTick","_restoreFrozenNodes","fixedData","_isMoving","vmin","isMoving","_discreteStepNodes","nodesPresent","discreteStepLimited","discreteStep","vminCorrected","_revertPhysicsState","revertPosition","_revertPhysicsTick","_doInAllActiveSectors","_doInSupportSector","mainMovingStatus","supportMovingStatus","mainMoving","_animationStep","_handleNavigation","startTime","renderStartTime","requestAnimationFrame","mozRequestAnimationFrame","webkitRequestAnimationFrame","msRequestAnimationFrame","iterations","toggleFreeze","parentEdgeId","internalMultiplier","positionBezierNode","mixin","storePosition","storePositions","dataArray","allowedToMoveX","allowedToMoveY","getPositions","focusOnNode","nodePosition","lockedOnNode","easingFunction","animateView","locked","_transitionRedraw","viewCenter","distanceFromCenter","_classicRedraw","_lockedRedraw","active","getScale","getCenterCoordinates","getBoundingBox","networkCons
 tants","fromId","toId","widthSelected","labelDimensions","yLine","dirtyLabel","fromBackup","toBackup","originalFromId","originalToId","widthFixed","lengthFixed","controlNodesEnabled","controlNodes","positions","connectedNode","_drawLine","_drawArrow","_drawArrowCenter","_drawDashLine","attachEdge","detachEdge","xFrom","yFrom","xTo","yTo","xObj","yObj","_getDistanceToEdge","_getColor","colorObj","_getLineWidth","_line","midpointX","midpointY","_pointOnLine","_label","resize","_circle","_pointOnCircle","networkScaleInv","_getViaCoordinates","xVia","yVia","quadraticCurveTo","lineCount","measureText","_rotateForLabelAlignment","_drawLabelRect","_drawLabelText","angleInDegrees","atan2","rotate","lineMargin","fillRect","lineJoin","strokeText","setLineDash","pattern","lineDashOffset","lineCap","dashedLine","percentage","arrow","_pointOnBezier","_findBorderPosition","distanceToBorder","distanceToNodes","difference","threshold","arrowPos","guidePos","edgeSegmentLength","toBorderDist","toBord
 erPoint","x1","y1","x2","y2","x3","y3","lastX","lastY","minDistance","_getDistanceToLine","px","py","something","u","nodeIdFrom","nodeIdTo","getControlNodeFromPosition","getControlNodeToPosition","_enableControlNodes","_disableControlNodes","_getSelectedControlNode","fromDistance","toDistance","_restoreControlNodes","controlnodeFromPos","fromBorderDist","fromBorderPoint","controlnodeToPos","defaultIndex","DEFAULT","imageBroken","load","url","brokenUrl","img","Image","onload","onerror","error","imagelist","grouplist","reroutedEdges","fontDrawThreshold","horizontalAlignLeft","verticalAlignTop","baseRadiusValue","radiusFixed","preassignedLevel","hierarchyEnumerated","fx","fy","vx","vy","previousState","resetCluster","dynamicEdgesLength","clusterSession","clusterSizeWidthFactor","clusterSizeHeightFactor","clusterSizeRadiusFactor","growthIndicator","networkScale","formationScale","clusterSize","containedNodes","containedEdges","clusterSessions","originalLabel","triggerFunction","groupObj
 ","imageObj","brokenImage","_drawDatabase","_resizeDatabase","_drawBox","_resizeBox","_drawCircle","_resizeCircle","_drawEllipse","_resizeEllipse","_drawImage","_resizeImage","_drawCircularImage","_resizeCircularImage","_drawText","_resizeText","_drawDot","_resizeShape","_drawSquare","_drawTriangle","_drawTriangleDown","_drawStar","_reset","clearSizeCache","_setForce","_addForce","storeState","isFixed","velocity","getDistance","_drawImageAtPosition","globalAlpha","drawImage","_drawImageLabel","getTextSize","_swapToImageResizeWhenImageLoaded","diameter","centerX","centerY","_drawRawCircle","circle","clip","textSize","clusterLineWidth","selectionLineWidth","roundRect","database","defaultSize","ellipse","_drawShape","radiusMultiplier","baseline","labelUnderNode","inView","clearVelocity","updateVelocity","massBeforeClustering","energyBefore","styleAttr","fontFamily","WebkitBorderRadius","whiteSpace","parseDOT","parseGraph","nextPreview","isAlphaNumeric","regexAlphaNumeric","merge","o","
 addNode","graphs","attr","addEdge","createEdge","getToken","tokenType","TOKENTYPE","NULL","token","isComment","DELIMITER","c2","DELIMITERS","IDENTIFIER","newSyntaxError","UNKNOWN","chop","strict","parseStatements","parseStatement","subgraph","parseSubgraph","parseEdge","parseAttributeStatement","parseNodeStatement","subgraphs","parseAttributeList","message","maxLength","forEach2","array1","array2","elem1","elem2","graphData","dotNode","graphNode","convertEdge","dotEdge","graphEdge","subEdge","{","}","[","]",";","=",",","->","--","gephiJSON","allowedToMove","gEdges","gNodes","gEdge","source","gNode","propagating","onMouseWheel","leftContainer","rightContainer","shadowTop","shadowBottom","shadowTopLeft","shadowBottomLeft","shadowTopRight","shadowBottomRight","_redrawTimer","touchAction","listeners","events","scrollTopMin","redrawCount","_initAutoResize","component","_stopAutoResize","what","getWindow","borderRootHeight","borderRootWidth","autoHeight","centerWidth","_updateScrollTop","
 visibilityTop","visibilityBottom","MAX_REDRAWS","repaint","_startAutoResize","_onResize","lastHeight","watchTimer","setInterval","_setScrollTop","_getScrollTop","inputHandler","onRelease","isFinal","offTouch","offRelease","custom","back","editNode","addDescription","edgeDescription","editEdgeDescription","createEdgeError","deleteClusterError","CanvasRenderingContext2D","square","s2","ir","triangleDown","star","n","r2d","kappa","ox","oy","xe","ye","xm","ym","bezierCurveTo","wEllipse","hEllipse","ymb","yeb","xt","yt","xi","yi","xl","yl","xr","yr","dashArray","dashLength","dashCount","slope","distRemaining","dashIndex","_catmullRom","_linear","dFill","_catmullRomUniform","p0","p1","p2","p3","bp1","bp2","normalization","d1","d2","d3","A","N","M","d3powA","d2powA","d3pow2A","d2pow2A","d1pow2A","d1powA","Bargraph","barCombinedData","coreDistance","drawData","combinedData","intersections","barPoints","_getDataIntersections","heightOffset","_getSafeDrawData","nextKey","amount","resolved","p
 revKey","accumulated","groupLabel","_getStackedBarYRange","xpos","PhysicsMixin","ClusterMixin","SectorsMixin","SelectionMixin","ManipulationMixin","NavigationMixin","HierarchicalLayoutMixin","_loadMixin","sourceVariable","mixinFunction","_clearMixin","_loadSelectedForceSolver","_loadPhysicsConfiguration","hubThreshold","activeSector","drawingNode","blockConnectingEdgeSelection","forceAppendSelection","manipulationDiv","editModeDiv","closeDiv","_cleanNavigation","_loadNavigationElements","overlay","prevent_default","_onTapOverlay","windowHammer","_hasParent","deactivate","escListener","activate","unbind","_callbacks","once","self","removeListener","removeAllListeners","callbacks","cb","hasListeners","__WEBPACK_AMD_DEFINE_RESULT__","global","dfl","hasOwnProp","defaultParsingFlags","empty","unusedTokens","unusedInput","charsLeftOver","nullInput","invalidMonth","invalidFormat","userInvalidated","iso","printMsg","msg","suppressDeprecationWarnings","warn","deprecate","firstTime","deprecat
 eSimple","deprecations","padToken","func","leftZeroFill","ordinalizeToken","period","localeData","ordinal","monthDiff","anchor2","adjust","wholeMonthDiff","meridiemFixWrap","meridiem","isPm","meridiemHour","isPM","Locale","Moment","config","skipOverflow","checkOverflow","copyConfig","updateInProgress","updateOffset","Duration","normalizedInput","normalizeObjectUnits","years","quarters","quarter","months","weeks","week","days","_milliseconds","_days","_months","_locale","_bubble","val","_isAMomentObject","_i","_f","_l","_strict","_tzm","_isUTC","_offset","_pf","momentProperties","absRound","number","targetLength","forceSign","output","positiveMomentsDifference","base","res","isAfter","momentsDifference","makeAs","isBefore","createAdder","dur","tmp","addOrSubtractDurationFromMoment","mom","isAdding","setTime","rawSetter","rawGetter","rawMonthSetter","input","compareArrays","dontConvert","lengthDiff","diffs","toInt","normalizeUnits","units","lowered","unitAliases","camelFunctions","inp
 utObject","normalizedProp","makeList","setter","getter","results","utc","argumentForCoercion","coercedNumber","isFinite","daysInMonth","UTC","getUTCDate","weeksInYear","dow","doy","weekOfYear","daysInYear","isLeapYear","_a","MONTH","DATE","YEAR","HOUR","MINUTE","SECOND","MILLISECOND","_overflowDayOfYear","isValid","_isValid","getTime","bigHour","normalizeLocale","chooseLocale","names","loadLocale","oldLocale","hasModule","code","model","local","removeFormattingTokens","makeFormatFunction","formattingTokens","formatTokenFunctions","formatMoment","expandFormat","formatFunctions","invalidDate","replaceLongDateFormatTokens","longDateFormat","localFormattingTokens","lastIndex","getParseRegexForToken","parseTokenOneDigit","parseTokenThreeDigits","parseTokenFourDigits","parseTokenOneToFourDigits","parseTokenSignedNumber","parseTokenSixDigits","parseTokenOneToSixDigits","parseTokenTwoDigits","parseTokenOneToThreeDigits","parseTokenWord","_meridiemParse","parseTokenOffsetMs","parseTokenTimes
 tampMs","parseTokenTimezone","parseTokenT","parseTokenDigits","parseTokenOneOrTwoDigits","_ordinalParse","_ordinalParseLenient","RegExp","regexpEscape","unescapeFormat","utcOffsetFromString","string","possibleTzMatches","tzChunk","parseTimezoneChunker","addTimeToArrayFromToken","datePartArray","monthsParse","_dayOfYear","parseTwoDigitYear","_meridiem","_useUTC","weekdaysParse","_w","invalidWeekday","dayOfYearFromWeekInfo","weekYear","temp","GG","W","E","_week","gg","dayOfYearFromWeeks","dateFromConfig","currentDate","yearToUse","currentDateArray","makeUTCDate","getUTCMonth","_nextDay","makeDate","setUTCMinutes","getUTCMinutes","dateFromObject","getUTCFullYear","makeDateFromStringAndFormat","ISO_8601","parseISO","parsedInput","tokens","skipped","stringLength","totalParsedInputLength","matched","p4","makeDateFromStringAndArray","tempConfig","bestMoment","scoreToBeat","currentScore","NaN","score","l","isoRegex","isoDates","isoTimes","makeDateFromString","createFromInputFallback","makeD
 ateFromInput","aspNetJsonRegex","ms","setUTCFullYear","parseWeekday","substituteTimeAgo","withoutSuffix","isFuture","relativeTime","posNegDuration","relativeTimeThresholds","firstDayOfWeek","firstDayOfWeekOfYear","adjustedMoment","daysToDayOfWeek","daysToAdd","getUTCDay","makeMoment","invalid","preparse","pickBy","moments","dayOfMonth","unit","makeAccessor","keepTime","daysToYears","yearsToDays","makeDurationGetter","makeGlobal","shouldDeprecate","ender","oldGlobalMoment","globalScope","VERSION","aspNetTimeSpanJsonRegex","isoDurationRegex","isoFormat","unitMillisecondFactors","Milliseconds","Seconds","Minutes","Hours","Days","Months","Years","D","Q","DDD","dayofyear","isoweekday","isoweek","weekyear","isoweekyear","ordinalizeTokens","paddedTokens","MMM","monthsShort","MMMM","dd","weekdaysMin","ddd","weekdaysShort","dddd","weekdays","isoWeek","YY","YYYY","YYYYY","YYYYYY","gggg","ggggg","isoWeekYear","GGGG","GGGGG","isoWeekday","SS","SSS","SSSS","Z","utcOffset","ZZ","zoneAbbr","zz","z
 oneName","unix","lists","DDDD","_monthsShort","monthName","regex","_monthsParse","_longMonthsParse","_shortMonthsParse","_weekdays","_weekdaysShort","_weekdaysMin","weekdayName","_weekdaysParse","_longDateFormat","LTS","LT","L","LL","LLL","LLLL","isLower","_calendar","sameDay","nextDay","nextWeek","lastDay","lastWeek","sameElse","calendar","_relativeTime","future","past","mm","hh","MM","yy","pastFuture","_ordinal","postformat","firstDayOfYear","_invalidDate","ret","parseIso","diffRes","isDuration","inp","version","relativeTimeThreshold","limit","defineLocale","_abbr","abbr","langData","flags","parseZone","isDSTShifted","parsingFlags","invalidAt","keepLocalTime","_dateUtcOffset","inputString","asFloat","that","zoneDiff","humanize","fromNow","sod","startOf","isDST","getDay","endOf","inputMs","isBetween","zone","localAdjust","_changeInProgress","isLocal","isUtcOffset","isUtc","hasAlignedHourOffset","isoWeeksInYear","weekInfo","newLocaleData","getTimezoneOffset","isoWeeks","toJSON","isU
 TC","withSuffix","toIsoString","asSeconds","asMilliseconds","asMinutes","asHours","asDays","asWeeks","asMonths","asYears","ordinalParse","require","noGlobal","__WEBPACK_AMD_DEFINE_FACTORY__","__WEBPACK_AMD_DEFINE_ARRAY__","_exportFunctions","_bound","keydown","keyup","_keys","fromCharCode","down","handleEvent","up","keyCode","bound","bindAll","getKey","newBindings","_firstTarget","propagatedHandler","_handled","stopped","firstTarget","_handlers","Manager","PropagatingHammer","_on","_off","_destroy","handler","exportName","setTimeoutContext","timeout","bindFn","invokeArrayArg","arg","each","iterator","dest","inherit","child","childP","baseP","_super","boolOrFn","TYPE_FUNCTION","ifUndefined","val1","val2","addEventListeners","splitStr","removeEventListeners","hasParent","inStr","str","find","inArray","findByKey","uniqueArray","prefixed","property","prefix","camelProp","VENDOR_PREFIXES","uniqueId","_uniqueId","getWindowForElement","doc","ownerDocument","defaultView","parentWindow","Inp
 ut","manager","inputTarget","domHandler","ev","init","createInputInstance","Type","inputClass","SUPPORT_POINTER_EVENTS","PointerEventInput","SUPPORT_ONLY_TOUCH","TouchInput","SUPPORT_TOUCH","TouchMouseInput","MouseInput","eventType","pointersLen","pointers","changedPointersLen","changedPointers","INPUT_START","INPUT_END","INPUT_CANCEL","session","computeInputData","recognize","prevInput","pointersLength","firstInput","simpleCloneInputData","firstMultiple","offsetCenter","getCenter","timeStamp","deltaTime","getAngle","computeDeltaXY","offsetDirection","getDirection","rotation","getRotation","computeIntervalInputData","offsetDelta","prevDelta","velocityX","velocityY","last","lastInterval","COMPUTE_INTERVAL","getVelocity","DIRECTION_NONE","DIRECTION_LEFT","DIRECTION_RIGHT","DIRECTION_UP","DIRECTION_DOWN","PROPS_XY","PROPS_CLIENT_XY","evEl","MOUSE_ELEMENT_EVENTS","evWin","MOUSE_WINDOW_EVENTS","allow","pressed","POINTER_ELEMENT_EVENTS","POINTER_WINDOW_EVENTS","store","pointerEvents","Sin
 gleTouchInput","evTarget","SINGLE_TOUCH_TARGET_EVENTS","SINGLE_TOUCH_WINDOW_EVENTS","started","normalizeSingleTouches","all","touches","changedTouches","TOUCH_TARGET_EVENTS","targetIds","getTouches","allTouches","INPUT_MOVE","identifier","changedTargetTouches","mouse","TouchAction","cleanTouchActions","actions","TOUCH_ACTION_NONE","hasPanX","TOUCH_ACTION_PAN_X","hasPanY","TOUCH_ACTION_PAN_Y","TOUCH_ACTION_MANIPULATION","TOUCH_ACTION_AUTO","Recognizer","defaults","state","STATE_POSSIBLE","simultaneous","requireFail","stateStr","STATE_CANCELLED","STATE_ENDED","STATE_CHANGED","STATE_BEGAN","directionStr","getRecognizerByNameIfManager","otherRecognizer","recognizer","AttrRecognizer","PanRecognizer","pX","pY","PinchRecognizer","PressRecognizer","_timer","_input","RotateRecognizer","SwipeRecognizer","TapRecognizer","pTime","pCenter","recognizers","preset","handlers","toggleCssProps","recognizeWith","requireFailure","cssProps","triggerDomEvent","gestureEvent","createEvent","initEvent","ges
 ture","dispatchEvent","TEST_ELEMENT","MOBILE_REGEX","INPUT_TYPE_TOUCH","INPUT_TYPE_PEN","INPUT_TYPE_MOUSE","INPUT_TYPE_KINECT","DIRECTION_HORIZONTAL","DIRECTION_VERTICAL","DIRECTION_ALL","MOUSE_INPUT_MAP","mousedown","mousemove","mouseup","pointerType","POINTER_INPUT_MAP","pointerdown","pointermove","pointerup","pointercancel","pointerout","IE10_POINTER_TYPE_ENUM",2,3,4,5,"MSPointerEvent","removePointer","eventTypeNormalized","isTouch","storeIndex","pointerId","SINGLE_TOUCH_INPUT_MAP","touchstart","touchmove","touchend","touchcancel","TOUCH_INPUT_MAP","inputEvent","inputData","isMouse","PREFIXED_TOUCH_ACTION","NATIVE_TOUCH_ACTION","TOUCH_ACTION_COMPUTE","compute","getTouchAction","preventDefaults","prevented","hasNone","preventSrc","STATE_RECOGNIZED","STATE_FAILED","dropRecognizeWith","dropRequireFailure","hasRequireFailures","canRecognizeWith","withState","tryEmit","canEmit","inputDataClone","process","attrTest","optionPointers","isRecognized","directionTest","hasMoved","inOut","va
 lidPointers","validMovement","validTime","taps","posThreshold","validTouchTime","failTimeout","validInterval","validMultiTap","tapCount","domEvents","userSelect","touchSelect","touchCallout","contentZooming","userDrag","tapHighlightColor","STOP","FORCED_STOP","curRecognizer","Tap","Pan","Swipe","Pinch","Rotate","Press","clusterToFit","maxNumberOfNodes","reposition","maxLevels","forceAggregateHubs","normalizeClusterLevels","increaseClusterLevel","repositionNodes","openCluster","isMovingBeforeClustering","_nodeInActiveArea","_sector","_addSector","decreaseClusterLevel","_expandClusterNode","_updateDynamicEdges","updateClusters","zoomDirection","recursive","doNotStart","amountOfNodes","_collapseSector","_formClusters","_openClusters","_openClustersBySize","_aggregateHubs","handleChains","chainPercentage","_getChainFraction","_reduceAmountOfChains","_getHubSize","_formClustersByHub","openAll","containedNodeId","childNode","_expelChildFromParent","_unselectAll","_releaseContainedEdges","
 _connectEdgeBackToChild","_validateEdges","othersPresent","childNodeId","_repositionBezierNodes","_formClustersByZoom","_forceClustersByZoom","minLength","_addToCluster","_clusterToSmallestNeighbour","smallestNeighbour","smallestNeighbourNode","neighbour","onlyEqual","_formClusterFromHub","hubNode","absorptionSizeOffset","allowCluster","edgesIdarray","amountOfInitialEdges","_addToContainedEdges","_connectEdgeToCluster","_containCircularEdgesFromNode","massBefore","correction","edgeToId","edgeFromId","k","_addToReroutedEdges","maxLevel","minLevel","clusterLevel","targetLevel","average","averageSquared","hubCounter","largestHub","variance","standardDeviation","fraction","reduceAmount","chains","total","_switchToSector","sectorId","sectorType","_switchToActiveSector","_switchToFrozenSector","_switchToSupportSector","_loadLatestSector","_previousSector","_setActiveSector","newId","_forgetLastSector","_createNewSector","_deleteActiveSector","_deleteFrozenSector","_freezeSector","_activat
 eSector","_mergeThisWithFrozen","_collapseThisToSingleCluster","sector","unqiueIdentifier","previousSector","runFunction","argument","returnValues","_doInAllFrozenSectors","_drawSectorNodes","_drawAllSectorNodes","_getNodesOverlappingWith","_getAllNodesOverlappingWith","_pointerToPositionObject","positionObject","_getEdgesOverlappingWith","_getAllEdgesOverlappingWith","_addToSelection","_addToHover","_removeFromSelection","doNotTrigger","_unselectClusters","_getSelectedNodeCount","_getSelectedNode","_getSelectedEdge","_getSelectedEdgeCount","_getSelectedObjectCount","_selectionIsEmpty","_clusterInSelection","_selectConnectedEdges","_hoverConnectedEdges","_unselectConnectedEdges","append","highlightEdges","overrideSelectable","DOM","_manipulationReleaseOverload","_navigationReleaseOverload","getSelectedNodes","edgeIds","getSelectedEdges","idArray","selectNodes","RangeError","selectEdges","_clearManipulatorBar","manipulationDOM","_restoreOverloadedFunctions","functionName","_toggleEdi
 tMode","toolbar","boundFunction","edgeBeingEdited","selectedControlNode","_createAddNodeToolbar","_createAddEdgeToolbar","_editNode","_createEditEdgeToolbar","_addNode","_handleConnect","_finishConnect","_selectControlNode","_controlNodeDrag","_releaseControlNode","newNode","_editEdge","alert","supportNodes","targetNode","connectionEdge","connectFromId","_createEdge","defaultData","finalizedData","sourceNodeId","targetNodeId","selectedNodes","selectedEdges","navigationDivs","navigationDivActions","_stopMovement","_zoomExtent","hubsize","definedLevel","undefinedLevel","_changeConstants","_determineLevels","_determineLevelsDirected","distribution","_getDistribution","_placeNodesByHierarchy","minPos","_placeBranchNodes","maxCount","_setLevel","firstNode","_setLevelDirected","parentId","parentLevel","nodeMoved","_restoreNodes","graphToggleSmoothCurves","graph_toggleSmooth","getElementById","graphRepositionNodes","showValueOfRange","graphGenerateOptions","optionsSpecific","radioButton1",
 "radioButton2","checked","backupConstants","optionsDiv","switchConfigurations","radioButton","querySelector","tableId","table","constantsVariableName","valueId","rangeValue","_overWriteGraphConstants","RepulsionMixin","HierarchialRepulsionMixin","BarnesHutMixin","_toggleBarnesHut","barnesHutTree","_initializeForceCalculation","_calculateForces","_calculateGravitationalForces","_calculateNodeForces","_calculateSpringForcesWithSupport","_calculateHierarchicalSpringForces","_calculateSpringForces","supportNodeId","gravity","gravityForce","edgeLength","springForce","combinedClusterSize","node1","node2","node3","_calculateSpringForce","physicsConfiguration","hierarchicalLayoutDirections","parentElement","rangeElement","radioButton3","graph_repositionNodes","graph_generateOptions","dynamicSmoothCurves","nameArray","webpackContext","req","resolve","repulsingForce","a_base","minimumDistance","steepness","springFx","springFy","totalFx","totalFy","correctionFx","correctionFy","nodeCount","_fo
 rmBarnesHutTree","_getForceContribution","children","NW","NE","SW","SE","parentBranch","childrenCount","centerOfMass","calcSize","MAX_VALUE","sizeDiff","minimumTreeSize","rootSize","halfRootSize","_splitBranch","_placeInTree","_updateBranchMass","totalMass","totalMassInv","biggestSize","skipMassUpdate","_placeInRegion","region","containedNode","_insertRegion","childSize","_drawTree","_drawBranch","branch","webpackPolyfill","paths","__webpack_amd_options__"],"mappings":";;;;;;;;;;;;;;;;;;;;;;;;AAyBA,cAEA,SAA2CA,EAAMC,GAC1B,gBAAZC,UAA0C,gBAAXC,QACxCA,OAAOD,QAAUD,IACQ,kBAAXG,SAAyBA,OAAOC,IAC9CD,OAAOH,GACmB,gBAAZC,SACdA,QAAa,IAAID,IAEjBD,EAAU,IAAIC,KACbK,KAAM,WACT,MAAgB,UAAUC,GAKhB,QAASC,GAAoBC,GAG5B,GAAGC,EAAiBD,GACnB,MAAOC,GAAiBD,GAAUP,OAGnC,IAAIC,GAASO,EAAiBD,IAC7BP,WACAS,GAAIF,EACJG,QAAQ,EAUT,OANAL,GAAQE,GAAUI,KAAKV,EAAOD,QAASC,EAAQA,EAAOD,QAASM,GAG/DL,EAAOS,QAAS,EAGTT,EAAOD,QAvBf,GAAIQ,KAqCJ,OATAF,GAAoBM,EAAIP,EAGxBC,EAAoBO,EAAIL,EAGxBF,EAAoBQ,EAAI,GAGjBR,EAAoB,KAK/B,SAASL,EAAQD,EA
 ASM,GAG9BN,EAAQe,KAAOT,EAAoB,GACnCN,EAAQgB,QAAUV,EAAoB,GAGtCN,EAAQiB,QAAUX,EAAoB,GACtCN,EAAQkB,SAAWZ,EAAoB,GACvCN,EAAQmB,MAAQb,EAAoB,GAGpCN,EAAQoB,QAAUd,EAAoB,GACtCN,EAAQqB,SACNC,OAAQhB,EAAoB,GAC5BiB,OAAQjB,EAAoB,GAC5BkB,QAASlB,EAAoB,GAC7BmB,QAASnB,EAAoB,IAC7BoB,OAAQpB,EAAoB,IAC5BqB,WAAYrB,EAAoB,KAIlCN,EAAQ4B,SAAWtB,EAAoB,IACvCN,EAAQ6B,QAAUvB,EAAoB,IACtCN,EAAQ8B,UACNC,SAAUzB,EAAoB,IAC9B0B,SAAU1B,EAAoB,IAC9B2B,MAAO3B,EAAoB,IAC3B4B,MAAO5B,EAAoB,IAC3B6B,SAAU7B,EAAoB,IAE9B8B,YACEC,OACEC,KAAMhC,EAAoB,IAC1BiC,eAAgBjC,EAAoB,IACpCkC,QAASlC,EAAoB,IAC7BmC,UAAWnC,EAAoB,IAC/BoC,UAAWpC,EAAoB,KAGjCqC,UAAWrC,EAAoB,IAC/BsC,YAAatC,EAAoB,IACjCuC,WAAYvC,EAAoB,IAChCwC,SAAUxC,EAAoB,IAC9ByC,WAAYzC,EAAoB,IAChC0C,MAAO1C,EAAoB,IAC3B2C,gBAAiB3C,EAAoB,IACrC4C,QAAS5C,EAAoB,IAC7B6C,OAAQ7C,EAAoB,IAC5B8C,UAAW9C,EAAoB,IAC/B+C,SAAU/C,EAAoB,MAKlCN,EAAQsD,QAAUhD,EAAoB,IACtCN,EAAQuD,SACNC,KAAMlD,EAAoB,IAC1BmD,OAAQnD,EAAoB,IAC5BoD,OAAQpD,EAAoB,IAC5BqD,KAAMrD,EAAoB,IAC1BsD,MAAOtD,EAAoB,IAC3BuD,UAAWvD,EAAoB,IAC/BwD,YAAax
 D,EAAoB,KAInCN,EAAQ+D,MAAQ,WACd,KAAM,IAAIC,OAAM,+EAIlBhE,EAAQiE,OAAS3D,EAAoB,IACrCN,EAAQkE,OAAS5D,EAAoB,IACrCN,EAAQmE,OAAS7D,EAAoB,KAKjC,SAASL,EAAQD,EAASM,GAM9B,GAAI2D,GAAS3D,EAAoB,GAOjCN,GAAQoE,SAAW,SAASC,GAC1B,MAAQA,aAAkBC,SAA2B,gBAAVD,IAQ7CrE,EAAQuE,SAAW,SAASF,GAC1B,MAAQA,aAAkBG,SAA2B,gBAAVH,IAQ7CrE,EAAQyE,OAAS,SAASJ,GACxB,GAAIA,YAAkBK,MACpB,OAAO,CAEJ,IAAI1E,EAAQuE,SAASF,GAAS,CAEjC,GAAIM,GAAQC,EAAaC,KAAKR,EAC9B,IAAIM,EACF,OAAO,CAEJ,KAAKG,MAAMJ,KAAKK,MAAMV,IACzB,OAAO,EAIX,OAAO,GAQTrE,EAAQgF,YAAc,SAASX,GAC7B,MAA4B,mBAAb,SACVY,OAAoB,eACpBA,OAAOC,cAAuB,WAC9Bb,YAAkBY,QAAOC,cAAcC,WAQ9CnF,EAAQoF,WAAa,WACnB,GAAIC,GAAK,WACP,MAAOC,MAAKC,MACQ,MAAhBD,KAAKE,UACPC,SAAS,IAGb,OACIJ,KAAOA,IAAO,IACVA,IAAO,IACPA,IAAO,IACPA,IAAO,IACPA,IAAOA,IAAOA,KAWxBrF,EAAQ0F,OAAS,SAAUC,GACzB,IAAK,GAAIC,GAAI,EAAGC,EAAMC,UAAUC,OAAYF,EAAJD,EAASA,IAAK,CACpD,GAAII,GAAQF,UAAUF,EACtB,KAAK,GAAIK,KAAQD,GACXA,EAAME,eAAeD,KACvBN,EAAEM,GAAQD,EAAMC,IAKtB,MAAON,IAWT3F,EAAQmG,gBAAkB,SAAUC,EAAOT,GACzC,IAAKU,MAAMC,QAAQF,GACjB,KAA
 M,IAAIpC,OAAM,uDAGlB,KAAK,GAAI4B,GAAI,EAAGA,EAAIE,UAAUC,OAAQH,IAGpC,IAAK,GAFDI,GAAQF,UAAUF,GAEb9E,EAAI,EAAGA,EAAIsF,EAAML,OAAQjF,IAAK,CACrC,GAAImF,GAAOG,EAAMtF,EACbkF,GAAME,eAAeD,KACvBN,EAAEM,GAAQD,EAAMC,IAItB,MAAON,IAWT3F,EAAQuG,oBAAsB,SAAUH,EAAOT,EAAGa,GAEhD,GAAIH,MAAMC,QAAQE,GAChB,KAAM,IAAIC,WAAU,yCAEtB,KAAK,GAAIb,GAAI,EAAGA,EAAIE,UAAUC,OAAQH,IAEpC,IAAK,GADDI,GAAQF,UAAUF,GACb9E,EAAI,EAAGA,EAAIsF,EAAML,OAAQjF,IAAK,CACrC,GAAImF,GAAOG,EAAMtF,EACjB,IAAIkF,EAAME,eAAeD,GACvB,GAAIO,EAAEP,IAASO,EAAEP,GAAMS,cAAgBC,OACrBC,SAAZjB,EAAEM,KACJN,EAAEM,OAEAN,EAAEM,GAAMS,cAAgBC,OAC1B3G,EAAQ6G,WAAWlB,EAAEM,GAAOO,EAAEP,IAG9BN,EAAEM,GAAQO,EAAEP,OAET,CAAA,GAAII,MAAMC,QAAQE,EAAEP,IACzB,KAAM,IAAIQ,WAAU,yCAEpBd,GAAEM,GAAQO,EAAEP,IAMpB,MAAON,IAWT3F,EAAQ8G,uBAAyB,SAAUV,EAAOT,EAAGa,GAEnD,GAAIH,MAAMC,QAAQE,GAChB,KAAM,IAAIC,WAAU,yCAEtB,KAAK,GAAIR,KAAQO,GACf,GAAIA,EAAEN,eAAeD,IACQ,IAAvBG,EAAMW,QAAQd,GAChB,GAAIO,EAAEP,IAASO,EAAEP,GAAMS,cAAgBC,OACrBC,SAAZjB,EAAEM,KACJN,EAAEM,OAEAN,EAAEM,GAAMS,cAAgBC,OAC1B3G,EAA
 Q6G,WAAWlB,EAAEM,GAAOO,EAAEP,IAG9BN,EAAEM,GAAQO,EAAEP,OAET,CAAA,GAAII,MAAMC,QAAQE,EAAEP,IACzB,KAAM,IAAIQ,WAAU,yCAEpBd,GAAEM,GAAQO,EAAEP,GAKpB,MAAON,IAST3F,EAAQ6G,WAAa,SAASlB,EAAGa,GAE/B,GAAIH,MAAMC,QAAQE,GAChB,KAAM,IAAIC,WAAU,yCAGtB,KAAK,GAAIR,KAAQO,GACf,GAAIA,EAAEN,eAAeD,GACnB,GAAIO,EAAEP,IAASO,EAAEP,GAAMS,cAAgBC,OACrBC,SAAZjB,EAAEM,KACJN,EAAEM,OAEAN,EAAEM,GAAMS,cAAgBC,OAC1B3G,EAAQ6G,WAAWlB,EAAEM,GAAOO,EAAEP,IAG9BN,EAAEM,GAAQO,EAAEP,OAET,CAAA,GAAII,MAAMC,QAAQE,EAAEP,IACzB,KAAM,IAAIQ,WAAU,yCAEpBd,GAAEM,GAAQO,EAAEP,GAIlB,MAAON,IAUT3F,EAAQgH,WAAa,SAAUrB,EAAGa,GAChC,GAAIb,EAAEI,QAAUS,EAAET,OAAQ,OAAO,CAEjC,KAAK,GAAIH,GAAI,EAAGC,EAAMF,EAAEI,OAAYF,EAAJD,EAASA,IACvC,GAAID,EAAEC,IAAMY,EAAEZ,GAAI,OAAO,CAG3B,QAAO,GAYT5F,EAAQiH,QAAU,SAAS5C,EAAQ6C,GACjC,GAAIvC,EAEJ,IAAeiC,SAAXvC,EACF,MAAOuC,OAET,IAAe,OAAXvC,EACF,MAAO,KAGT,KAAK6C,EACH,MAAO7C,EAET,IAAsB,gBAAT6C,MAAwBA,YAAgB1C,SACnD,KAAM,IAAIR,OAAM,wBAIlB,QAAQkD,GACN,IAAK,UACL,IAAK,UACH,MAAOC,SAAQ9C,EAEjB,KAAK,SACL,IAAK,SACH,MAAOC,QAAOD,EAAO+C,UAE
 vB,KAAK,SACL,IAAK,SACH,MAAO5C,QAAOH,EAEhB,KAAK,OACH,GAAIrE,EAAQoE,SAASC,GACnB,MAAO,IAAIK,MAAKL,EAElB,IAAIA,YAAkBK,MACpB,MAAO,IAAIA,MAAKL,EAAO+C,UAEpB,IAAInD,EAAOoD,SAAShD,GACvB,MAAO,IAAIK,MAAKL,EAAO+C,UAEzB,IAAIpH,EAAQuE,SAASF,GAEnB,MADAM,GAAQC,EAAaC,KAAKR,GACtBM,EAEK,GAAID,MAAKJ,OAAOK,EAAM,KAGtBV,EAAOI,GAAQiD,QAIxB,MAAM,IAAItD,OACN,iCAAmChE,EAAQuH,QAAQlD,GAC/C,gBAGZ,KAAK,SACH,GAAIrE,EAAQoE,SAASC,GACnB,MAAOJ,GAAOI,EAEhB,IAAIA,YAAkBK,MACpB,MAAOT,GAAOI,EAAO+C,UAElB,IAAInD,EAAOoD,SAAShD,GACvB,MAAOJ,GAAOI,EAEhB,IAAIrE,EAAQuE,SAASF,GAEnB,MADAM,GAAQC,EAAaC,KAAKR,GAGjBJ,EAFLU,EAEYL,OAAOK,EAAM,IAGbN,EAIhB,MAAM,IAAIL,OACN,iCAAmChE,EAAQuH,QAAQlD,GAC/C,gBAGZ,KAAK,UACH,GAAIrE,EAAQoE,SAASC,GACnB,MAAO,IAAIK,MAAKL,EAEb,IAAIA,YAAkBK,MACzB,MAAOL,GAAOmD,aAEX,IAAIvD,EAAOoD,SAAShD,GACvB,MAAOA,GAAOiD,SAASE,aAEpB,IAAIxH,EAAQuE,SAASF,GAExB,MADAM,GAAQC,EAAaC,KAAKR,GACtBM,EAEK,GAAID,MAAKJ,OAAOK,EAAM,KAAK6C,cAG3B,GAAI9C,MAAKL,GAAQmD,aAI1B,MAAM,IAAIxD,OACN,iCAAmChE,EAAQuH,QAAQlD,GAC/C,mBAGZ,KAAK,UACH,GAAIrE,E
 AAQoE,SAASC,GACnB,MAAO,SAAWA,EAAS,IAExB,IAAIA,YAAkBK,MACzB,MAAO,SAAWL,EAAO+C,UAAY,IAElC,IAAIpH,EAAQuE,SAASF,GAAS,CACjCM,EAAQC,EAAaC,KAAKR,EAC1B,IAAIoD,EAQJ,OALEA,GAFE9C,EAEM,GAAID,MAAKJ,OAAOK,EAAM,KAAKyC,UAG3B,GAAI1C,MAAKL,GAAQ+C,UAEpB,SAAWK,EAAQ,KAG1B,KAAM,IAAIzD,OACN,iCAAmChE,EAAQuH,QAAQlD,GAC/C,mBAGZ,SACE,KAAM,IAAIL,OAAM,iBAAmBkD,EAAO,MAOhD,IAAItC,GAAe,qBAOnB5E,GAAQuH,QAAU,SAASlD,GACzB,GAAI6C,SAAc7C,EAElB,OAAY,UAAR6C,EACY,MAAV7C,EACK,OAELA,YAAkB8C,SACb,UAEL9C,YAAkBC,QACb,SAELD,YAAkBG,QACb,SAEL6B,MAAMC,QAAQjC,GACT,QAELA,YAAkBK,MACb,OAEF,SAEQ,UAARwC,EACA,SAEQ,WAARA,EACA,UAEQ,UAARA,EACA,SAGFA,GASTlH,EAAQ0H,gBAAkB,SAASC,GACjC,MAAOA,GAAKC,wBAAwBC,MAStC7H,EAAQ8H,eAAiB,SAASH,GAChC,MAAOA,GAAKC,wBAAwBG,KAQtC/H,EAAQgI,aAAe,SAASL,EAAMM,GACpC,GAAIC,GAAUP,EAAKM,UAAUE,MAAM,IACD,KAA9BD,EAAQnB,QAAQkB,KAClBC,EAAQE,KAAKH,GACbN,EAAKM,UAAYC,EAAQG,KAAK,OASlCrI,EAAQsI,gBAAkB,SAASX,EAAMM,GACvC,GAAIC,GAAUP,EAAKM,UAAUE,MAAM,KAC/BI,EAAQL,EAAQnB,QAAQkB,EACf,KAATM,IACFL,EAAQM,OAAOD,EAAO,GACtBZ,EAAKM,UAAYC,E
 AAQG,KAAK,OAalCrI,EAAQyI,QAAU,SAASpE,EAAQqE,GACjC,GAAI9C,GACAC,CACJ,IAAIQ,MAAMC,QAAQjC,GAEhB,IAAKuB,EAAI,EAAGC,EAAMxB,EAAO0B,OAAYF,EAAJD,EAASA,IACxC8C,EAASrE,EAAOuB,GAAIA,EAAGvB,OAKzB,KAAKuB,IAAKvB,GACJA,EAAO6B,eAAeN,IACxB8C,EAASrE,EAAOuB,GAAIA,EAAGvB,IAY/BrE,EAAQ2I,QAAU,SAAStE,GACzB,GAAIuE,KAEJ,KAAK,GAAI3C,KAAQ5B,GACXA,EAAO6B,eAAeD,IAAO2C,EAAMR,KAAK/D,EAAO4B,GAGrD,OAAO2C,IAUT5I,EAAQ6I,eAAiB,SAASxE,EAAQyE,EAAKrB,GAC7C,MAAIpD,GAAOyE,KAASrB,GAClBpD,EAAOyE,GAAOrB,GACP,IAGA,GAYXzH,EAAQ+I,iBAAmB,SAASC,EAASC,EAAQC,EAAUC,GACzDH,EAAQD,kBACSnC,SAAfuC,IACFA,GAAa,GAEA,eAAXF,GAA2BG,UAAUC,UAAUtC,QAAQ,YAAc,IACvEkC,EAAS,kBAGXD,EAAQD,iBAAiBE,EAAQC,EAAUC,IAE3CH,EAAQM,YAAY,KAAOL,EAAQC,IAWvClJ,EAAQuJ,oBAAsB,SAASP,EAASC,EAAQC,EAAUC,GAC5DH,EAAQO,qBAES3C,SAAfuC,IACFA,GAAa,GAEA,eAAXF,GAA2BG,UAAUC,UAAUtC,QAAQ,YAAc,IACvEkC,EAAS,kBAGXD,EAAQO,oBAAoBN,EAAQC,EAAUC,IAG9CH,EAAQQ,YAAY,KAAOP,EAAQC,IAOvClJ,EAAQyJ,eAAiB,SAAUC,GAC5BA,IACHA,EAAQC,OAAOD,OAEbA,EAAMD,eACRC,EAAMD,iBAGNC,EAAME,aAAc,GASxB5J,EAAQ6J,UAAY,SAASH
 ,GAEtBA,IACHA,EAAQC,OAAOD,MAGjB,IAAII,EAcJ,OAZIJ,GAAMI,OACRA,EAASJ,EAAMI,OAERJ,EAAMK,aACbD,EAASJ,EAAMK,YAGMnD,QAAnBkD,EAAOE,UAA4C,GAAnBF,EAAOE,WAEzCF,EAASA,EAAOG,YAGXH,GAGT9J,EAAQkK,UAQRlK,EAAQkK,OAAOC,UAAY,SAAU1C,EAAO2C,GAK1C,MAJoB,kBAAT3C,KACTA,EAAQA,KAGG,MAATA,EACe,GAATA,EAGH2C,GAAgB,MASzBpK,EAAQkK,OAAOG,SAAW,SAAU5C,EAAO2C,GAKzC,MAJoB,kBAAT3C,KACTA,EAAQA,KAGG,MAATA,EACKnD,OAAOmD,IAAU2C,GAAgB,KAGnCA,GAAgB,MASzBpK,EAAQkK,OAAOI,SAAW,SAAU7C,EAAO2C,GAKzC,MAJoB,kBAAT3C,KACTA,EAAQA,KAGG,MAATA,EACKjD,OAAOiD,GAGT2C,GAAgB,MASzBpK,EAAQkK,OAAOK,OAAS,SAAU9C,EAAO2C,GAKvC,MAJoB,kBAAT3C,KACTA,EAAQA,KAGNzH,EAAQuE,SAASkD,GACZA,EAEAzH,EAAQoE,SAASqD,GACjBA,EAAQ,KAGR2C,GAAgB,MAU3BpK,EAAQkK,OAAOM,UAAY,SAAU/C,EAAO2C,GAK1C,MAJoB,kBAAT3C,KACTA,EAAQA,KAGHA,GAAS2C,GAAgB,MASlCpK,EAAQyK,SAAW,SAASC,GAE1B,GAAIC,GAAiB,kCACrBD,GAAMA,EAAIE,QAAQD,EAAgB,SAAS/J,EAAGiK,EAAGC,EAAGtE,GAChD,MAAOqE,GAAIA,EAAIC,EAAIA,EAAItE,EAAIA,GAE/B,IAAIuE,GAAS,4CAA4ClG,KAAK6F,EAC9D,OAAOK,IACHF,EAAGG,SAASD,EAAO,GAAI,IACvBD,EAAGE,SAASD,
 EAAO,GAAI,IACvBvE,EAAGwE,SAASD,EAAO,GAAI,KACvB,MAWN/K,EAAQiL,SAAW,SAASC,EAAIC,EAAMC,GACpC,MAAO,MAAQ,GAAK,KAAOF,GAAO,KAAOC,GAAS,GAAKC,GAAM3F,SAAS,IAAI4F,MAAM,IASlFrL,EAAQsL,WAAa,SAASC,GAC5B,GAAI1K,EACJ,IAAIb,EAAQuE,SAASgH,GAAQ,CAC3B,GAAIvL,EAAQwL,WAAWD,GAAQ,CAC7B,GAAIE,GAAMF,EAAMG,OAAO,GAAGA,OAAO,EAAEH,EAAMxF,OAAO,GAAGoC,MAAM,IACzDoD,GAAQvL,EAAQiL,SAASQ,EAAI,GAAGA,EAAI,GAAGA,EAAI,IAE7C,GAAIzL,EAAQ2L,WAAWJ,GAAQ,CAC7B,GAAIK,GAAM5L,EAAQ6L,SAASN,GACvBO,GAAmBC,EAAEH,EAAIG,EAAEC,EAAU,IAARJ,EAAII,EAASC,EAAE3G,KAAK4G,IAAI,EAAU,KAARN,EAAIK,IAC3DE,GAAmBJ,EAAEH,EAAIG,EAAEC,EAAE1G,KAAK4G,IAAI,EAAU,KAARN,EAAIK,GAAUA,EAAQ,GAANL,EAAIK,GAC5DG,EAAkBpM,EAAQqM,SAASF,EAAeJ,EAAGI,EAAeJ,EAAGI,EAAeF,GACtFK,EAAkBtM,EAAQqM,SAASP,EAAgBC,EAAED,EAAgBE,EAAEF,EAAgBG,EAE3FpL,IACE0L,WAAYhB,EACZiB,OAAOJ,EACPK,WACEF,WAAWD,EACXE,OAAOJ,GAETM,OACEH,WAAWD,EACXE,OAAOJ,QAKXvL,IACE0L,WAAWhB,EACXiB,OAAOjB,EACPkB,WACEF,WAAWhB,EACXiB,OAAOjB,GAETmB,OACEH,WAAWhB,EACXiB,OAAOjB,QAMb1K,MACAA,EAAE0L,WAAahB,EAAMgB,YAAc,QACnC1L,EAAE2L
 ,OAASjB,EAAMiB,QAAU3L,EAAE0L,WAEzBvM,EAAQuE,SAASgH,EAAMkB,WACzB5L,EAAE4L,WACAD,OAAQjB,EAAMkB,UACdF,WAAYhB,EAAMkB,YAIpB5L,EAAE4L,aACF5L,EAAE4L,UAAUF,WAAahB,EAAMkB,WAAalB,EAAMkB,UAAUF,YAAc1L,EAAE0L,WAC5E1L,EAAE4L,UAAUD,OAASjB,EAAMkB,WAAalB,EAAMkB,UAAUD,QAAU3L,EAAE2L,QAGlExM,EAAQuE,SAASgH,EAAMmB,OACzB7L,EAAE6L,OACAF,OAAQjB,EAAMmB,MACdH,WAAYhB,EAAMmB,QAIpB7L,EAAE6L,SACF7L,EAAE6L,MAAMH,WAAahB,EAAMmB,OAASnB,EAAMmB,MAAMH,YAAc1L,EAAE0L,WAChE1L,EAAE6L,MAAMF,OAASjB,EAAMmB,OAASnB,EAAMmB,MAAMF,QAAU3L,EAAE2L,OAI5D,OAAO3L,IAYTb,EAAQ2M,SAAW,SAASzB,EAAIC,EAAMC,GACpCF,GAAQ,IAAKC,GAAY,IAAKC,GAAU,GACxC,IAAIwB,GAAStH,KAAK4G,IAAIhB,EAAI5F,KAAK4G,IAAIf,EAAMC,IACrCyB,EAASvH,KAAKwH,IAAI5B,EAAI5F,KAAKwH,IAAI3B,EAAMC,GAGzC,IAAIwB,GAAUC,EACZ,OAAQd,EAAE,EAAEC,EAAE,EAAEC,EAAEW,EAIpB,IAAIG,GAAK7B,GAAK0B,EAAUzB,EAAMC,EAASA,GAAMwB,EAAU1B,EAAIC,EAAQC,EAAKF,EACpEa,EAAKb,GAAK0B,EAAU,EAAMxB,GAAMwB,EAAU,EAAI,EAC9CI,EAAM,IAAIjB,EAAIgB,GAAGF,EAASD,IAAS,IACnCK,GAAcJ,EAASD,GAAQC,EAC/BpF,EAAQoF,CACZ,QAAQd,EAAEiB,EAAIhB,EAAE
 iB,EAAWhB,EAAExE,GAG/B,IAAIyF,IAEF/E,MAAO,SAAUgF,GACf,GAAIC,KAWJ,OATAD,GAAQhF,MAAM,KAAKM,QAAQ,SAAU4E,GACnC,GAAoB,IAAhBA,EAAMC,OAAc,CACtB,GAAIC,GAAQF,EAAMlF,MAAM,KACpBW,EAAMyE,EAAM,GAAGD,OACf7F,EAAQ8F,EAAM,GAAGD,MACrBF,GAAOtE,GAAOrB,KAIX2F,GAIT/E,KAAM,SAAU+E,GACd,MAAOzG,QAAO6G,KAAKJ,GACdK,IAAI,SAAU3E,GACb,MAAOA,GAAM,KAAOsE,EAAOtE,KAE5BT,KAAK,OASdrI,GAAQ0N,WAAa,SAAU1E,EAASmE,GACtC,GAAIQ,GAAgBT,EAAQ/E,MAAMa,EAAQqE,MAAMF,SAC5CS,EAAYV,EAAQ/E,MAAMgF,GAC1BC,EAASpN,EAAQ0F,OAAOiI,EAAeC,EAE3C5E,GAAQqE,MAAMF,QAAUD,EAAQ7E,KAAK+E,IAQvCpN,EAAQ6N,cAAgB,SAAU7E,EAASmE,GACzC,GAAIC,GAASF,EAAQ/E,MAAMa,EAAQqE,MAAMF,SACrCW,EAAeZ,EAAQ/E,MAAMgF,EAEjC,KAAK,GAAIrE,KAAOgF,GACVA,EAAa5H,eAAe4C,UACvBsE,GAAOtE,EAIlBE,GAAQqE,MAAMF,QAAUD,EAAQ7E,KAAK+E,IAWvCpN,EAAQ+N,SAAW,SAAShC,EAAGC,EAAGC,GAChC,GAAIpB,GAAGC,EAAGtE,EAENZ,EAAIN,KAAKC,MAAU,EAAJwG,GACfiC,EAAQ,EAAJjC,EAAQnG,EACZ9E,EAAImL,GAAK,EAAID,GACbiC,EAAIhC,GAAK,EAAI+B,EAAIhC,GACjBkC,EAAIjC,GAAK,GAAK,EAAI+B,GAAKhC,EAE3B,QAAQpG,EAAI,GACV,IAAK,GAAGiF,EAAIoB,EAAGnB,E
 AAIoD,EAAG1H,EAAI1F,CAAG,MAC7B,KAAK,GAAG+J,EAAIoD,EAAGnD,EAAImB,EAAGzF,EAAI1F,CAAG,MAC7B,KAAK,GAAG+J,EAAI/J,EAAGgK,EAAImB,EAAGzF,EAAI0H,CAAG,MAC7B,KAAK,GAAGrD,EAAI/J,EAAGgK,EAAImD,EAAGzH,EAAIyF,CAAG,MAC7B,KAAK,GAAGpB,EAAIqD,EAAGpD,EAAIhK,EAAG0F,EAAIyF,CAAG,MAC7B,KAAK,GAAGpB,EAAIoB,EAAGnB,EAAIhK,EAAG0F,EAAIyH,EAG5B,OAAQpD,EAAEvF,KAAKC,MAAU,IAAJsF,GAAUC,EAAExF,KAAKC,MAAU,IAAJuF,GAAUtE,EAAElB,KAAKC,MAAU,IAAJiB,KAGrExG,EAAQqM,SAAW,SAASN,EAAGC,EAAGC,GAChC,GAAIR,GAAMzL,EAAQ+N,SAAShC,EAAGC,EAAGC,EACjC,OAAOjM,GAAQiL,SAASQ,EAAIZ,EAAGY,EAAIX,EAAGW,EAAIjF,IAG5CxG,EAAQ6L,SAAW,SAASnB,GAC1B,GAAIe,GAAMzL,EAAQyK,SAASC,EAC3B,OAAO1K,GAAQ2M,SAASlB,EAAIZ,EAAGY,EAAIX,EAAGW,EAAIjF,IAG5CxG,EAAQ2L,WAAa,SAASjB,GAC5B,GAAIyD,GAAO,qCAAqCC,KAAK1D,EACrD,OAAOyD,IAGTnO,EAAQwL,WAAa,SAASC,GAC5BA,EAAMA,EAAIb,QAAQ,IAAI,GACtB,IAAIuD,GAAO,wCAAwCC,KAAK3C,EACxD,OAAO0C,IAUTnO,EAAQqO,sBAAwB,SAASC,EAAQC,GAC/C,GAA8B,gBAAnBA,GAA6B,CAEtC,IAAK,GADDC,GAAW7H,OAAO8H,OAAOF,GACpB3I,EAAI,EAAGA,EAAI0I,EAAOvI,OAAQH,IAC7B2I,EAAgBrI,eAAeo
 I,EAAO1I,KACC,gBAA9B2I,GAAgBD,EAAO1I,MAChC4I,EAASF,EAAO1I,IAAM5F,EAAQ0O,aAAaH,EAAgBD,EAAO1I,KAIxE,OAAO4I,GAGP,MAAO,OAWXxO,EAAQ0O,aAAe,SAASH,GAC9B,GAA8B,gBAAnBA,GAA6B,CACtC,GAAIC,GAAW7H,OAAO8H,OAAOF,EAC7B,KAAK,GAAI3I,KAAK2I,GACRA,EAAgBrI,eAAeN,IACA,gBAAtB2I,GAAgB3I,KACzB4I,EAAS5I,GAAK5F,EAAQ0O,aAAaH,EAAgB3I,IAIzD,OAAO4I,GAGP,MAAO,OAcXxO,EAAQ2O,aAAe,SAAUC,EAAaC,EAAS3E,GACrD,GAAwBtD,SAApBiI,EAAQ3E,GACV,GAA8B,iBAAnB2E,GAAQ3E,GACjB0E,EAAY1E,GAAQ4E,QAAUD,EAAQ3E,OAEnC,CACH0E,EAAY1E,GAAQ4E,SAAU,CAC9B,KAAK,GAAI7I,KAAQ4I,GAAQ3E,GACnB2E,EAAQ3E,GAAQhE,eAAeD,KACjC2I,EAAY1E,GAAQjE,GAAQ4I,EAAQ3E,GAAQjE,MAmBtDjG,EAAQ+O,mBAAqB,SAASC,EAAcC,EAAgBC,EAAOC,GAMzE,IALA,GAAIC,GAAgB,IAChBC,EAAY,EACZC,EAAM,EACNC,EAAOP,EAAajJ,OAAS,EAEnBwJ,GAAPD,GAA2BF,EAAZC,GAA2B,CAC/C,GAAIG,GAASlK,KAAKC,OAAO+J,EAAMC,GAAQ,GAEnCE,EAAOT,EAAaQ,GACpB/H,EAAoBb,SAAXuI,EAAwBM,EAAKP,GAASO,EAAKP,GAAOC,GAE3DO,EAAeT,EAAexH,EAClC,IAAoB,GAAhBiI,EACF,MAAOF,EAEgB,KAAhBE,EACPJ,EAAME,EAAS,EAGfD,EAAOC,EAAS,EAGlBH,IAGF,MAAO,IAeTrP,EAAQ2P,kBAAoB,
 SAASX,EAAclF,EAAQoF,EAAOU,GAOhE,IANA,GAIIC,GAAWpI,EAAOqI,EAAWN,EAJ7BJ,EAAgB,IAChBC,EAAY,EACZC,EAAM,EACNC,EAAOP,EAAajJ,OAAS,EAGnBwJ,GAAPD,GAA2BF,EAAZC,GAA2B,CAO/C,GALAG,EAASlK,KAAKC,MAAM,IAAKgK,EAAKD,IAC9BO,EAAYb,EAAa1J,KAAKwH,IAAI,EAAE0C,EAAS,IAAIN,GACjDzH,EAAYuH,EAAaQ,GAAQN,GACjCY,EAAYd,EAAa1J,KAAK4G,IAAI8C,EAAajJ,OAAO,EAAEyJ,EAAS,IAAIN,GAEjEzH,GAASqC,EACX,MAAO0F,EAEJ,IAAgB1F,EAAZ+F,GAAsBpI,EAAQqC,EACrC,MAAyB,UAAlB8F,EAA6BtK,KAAKwH,IAAI,EAAE0C,EAAS,GAAKA,CAE1D,IAAY1F,EAARrC,GAAkBqI,EAAYhG,EACrC,MAAyB,UAAlB8F,EAA6BJ,EAASlK,KAAK4G,IAAI8C,EAAajJ,OAAO,EAAEyJ,EAAS,EAGzE1F,GAARrC,EACF6H,EAAME,EAAS,EAGfD,EAAOC,EAAS,EAGpBH,IAIF,MAAO,IAYTrP,EAAQ+P,cAAgB,SAAU7B,EAAG8B,EAAOC,EAAKC,GAC/C,GAAIC,GAASF,EAAMD,CAEnB,OADA9B,IAAKgC,EAAS,EACN,EAAJhC,EAAciC,EAAO,EAAEjC,EAAEA,EAAI8B,GACjC9B,KACQiC,EAAO,GAAKjC,GAAGA,EAAE,GAAK,GAAK8B,IAUrChQ,EAAQoQ,iBAENC,OAAQ,SAAUnC,GAChB,MAAOA,IAGToC,WAAY,SAAUpC,GACpB,MAAOA,GAAIA,GAGbqC,YAAa,SAAUrC,GACrB,MAAOA,IAAK,EAAIA,IAGlB6B,cAAe,SAAU7B,GACvB,MAAW,GAAJA,EAAS,EAAIA,E
 AAIA,EAAI,IAAM,EAAI,EAAIA,GAAKA,GAGjDsC,YAAa,SAAUtC,GACrB,MAAOA,GAAIA,EAAIA,GAGjBuC,aAAc,SAAUvC,GACtB,QAAUA,EAAKA,EAAIA,EAAI,GAGzBwC,eAAgB,SAAUxC,GACxB,MAAW,GAAJA,EAAS,EAAIA,EAAIA,EAAIA,GAAKA,EAAI,IAAM,EAAIA,EAAI,IAAM,EAAIA,EAAI,GAAK,GAGxEyC,YAAa,SAAUzC,GACrB,MAAOA,GAAIA,EAAIA,EAAIA,GAGrB0C,aAAc,SAAU1C,GACtB,MAAO,MAAOA,EAAKA,EAAIA,EAAIA,GAG7B2C,eAAgB,SAAU3C,GACxB,MAAW,GAAJA,EAAS,EAAIA,EAAIA,EAAIA,EAAIA,EAAI,EAAI,IAAOA,EAAKA,EAAIA,EAAIA,GAG9D4C,YAAa,SAAU5C,GACrB,MAAOA,GAAIA,EAAIA,EAAIA,EAAIA,GAGzB6C,aAAc,SAAU7C,GACtB,MAAO,KAAOA,EAAKA,EAAIA,EAAIA,EAAIA,GAGjC8C,eAAgB,SAAU9C,GACxB,MAAW,GAAJA,EAAS,GAAKA,EAAIA,EAAIA,EAAIA,EAAIA,EAAI,EAAI,KAAQA,EAAKA,EAAIA,EAAIA,EAAIA,KAMtE,SAASjO,EAAQD,GASrBA,EAAQiR,gBAAkB,SAASC,GAEjC,IAAK,GAAIC,KAAeD,GAClBA,EAAchL,eAAeiL,KAC/BD,EAAcC,GAAaC,UAAYF,EAAcC,GAAaE,KAClEH,EAAcC,GAAaE,UAYjCrR,EAAQsR,gBAAkB,SAASJ,GAEjC,IAAK,GAAIC,KAAeD,GACtB,GAAIA,EAAchL,eAAeiL,IAC3BD,EAAcC,GAAaC,UAAW,CACxC,IAAK,GAAIxL,GAAI,EAAGA,EAAIsL,EAAcC,GAAaC,UAAUrL,OAAQH,IAC/DsL,EAAcC,GAAaC
 ,UAAUxL,GAAGqE,WAAWsH,YAAYL,EAAcC,GAAaC,UAAUxL,GAEtGsL,GAAcC,GAAaC,eAgBnCpR,EAAQwR,cAAgB,SAAUL,EAAaD,EAAeO,GAC5D,GAAIzI,EAqBJ,OAnBIkI,GAAchL,eAAeiL,GAE3BD,EAAcC,GAAaC,UAAUrL,OAAS,GAChDiD,EAAUkI,EAAcC,GAAaC,UAAU,GAC/CF,EAAcC,GAAaC,UAAUM,UAIrC1I,EAAU2I,SAASC,gBAAgB,6BAA8BT,GACjEM,EAAaI,YAAY7I,KAK3BA,EAAU2I,SAASC,gBAAgB,6BAA8BT,GACjED,EAAcC,IAAgBE,QAAUD,cACxCK,EAAaI,YAAY7I,IAE3BkI,EAAcC,GAAaE,KAAKjJ,KAAKY,GAC9BA,GAcThJ,EAAQ8R,cAAgB,SAAUX,EAAaD,EAAea,EAAcC,GAC1E,GAAIhJ,EA+BJ,OA7BIkI,GAAchL,eAAeiL,GAE3BD,EAAcC,GAAaC,UAAUrL,OAAS,GAChDiD,EAAUkI,EAAcC,GAAaC,UAAU,GAC/CF,EAAcC,GAAaC,UAAUM,UAIrC1I,EAAU2I,SAASM,cAAcd,GACZvK,SAAjBoL,EACFD,EAAaC,aAAahJ,EAASgJ,GAGnCD,EAAaF,YAAY7I,KAM7BA,EAAU2I,SAASM,cAAcd,GACjCD,EAAcC,IAAgBE,QAAUD,cACnBxK,SAAjBoL,EACFD,EAAaC,aAAahJ,EAASgJ,GAGnCD,EAAaF,YAAY7I,IAG7BkI,EAAcC,GAAaE,KAAKjJ,KAAKY,GAC9BA,GAkBThJ,EAAQkS,UAAY,SAASC,EAAGC,EAAGC,EAAOnB,EAAeO,GACvD,GAAIa,EAmBJ,OAlBsC,UAAlCD,EAAMxD,QAAQ0D,WAAWlF,OAC3BiF,EAAQtS,EAAQwR,cAAc,SAASN,EAAcO,GACrDa,EAAME,eAAe,KAAM,KAA
 ML,GACjCG,EAAME,eAAe,KAAM,KAAMJ,GACjCE,EAAME,eAAe,KAAM,IAAK,GAAMH,EAAMxD,QAAQ0D,WAAWE,QAG/DH,EAAQtS,EAAQwR,cAAc,OAAON,EAAcO,GACnDa,EAAME,eAAe,KAAM,IAAKL,EAAI,GAAIE,EAAMxD,QAAQ0D,WAAWE,MACjEH,EAAME,eAAe,KAAM,IAAKJ,EAAI,GAAIC,EAAMxD,QAAQ0D,WAAWE,MACjEH,EAAME,eAAe,KAAM,QAASH,EAAMxD,QAAQ0D,WAAWE,MAC7DH,EAAME,eAAe,KAAM,SAAUH,EAAMxD,QAAQ0D,WAAWE,OAGzB7L,SAApCyL,EAAMxD,QAAQ0D,WAAWnF,QAC1BkF,EAAME,eAAe,KAAM,QAASH,EAAMA,MAAMxD,QAAQ0D,WAAWnF,QAErEkF,EAAME,eAAe,KAAM,QAASH,EAAMpK,UAAY,UAC/CqK,GAUTtS,EAAQ0S,QAAU,SAAUP,EAAGC,EAAGO,EAAOC,EAAQ3K,EAAWiJ,EAAeO,GACzE,GAAc,GAAVmB,EAAa,CACF,EAATA,IACFA,GAAU,GACVR,GAAKQ,EAEP,IAAIC,GAAO7S,EAAQwR,cAAc,OAAON,EAAeO,EACvDoB,GAAKL,eAAe,KAAM,IAAKL,EAAI,GAAMQ,GACzCE,EAAKL,eAAe,KAAM,IAAKJ,GAC/BS,EAAKL,eAAe,KAAM,QAASG,GACnCE,EAAKL,eAAe,KAAM,SAAUI,GACpCC,EAAKL,eAAe,KAAM,QAASvK,MAMnC,SAAShI,EAAQD,EAASM,GAgD9B,QAASW,GAAS6R,EAAMjE,GActB,IAZIiE,GAASzM,MAAMC,QAAQwM,IAAU/R,EAAKiE,YAAY8N,KACpDjE,EAAUiE,EACVA,EAAO,MAGT1S,KAAK2S,SAAWlE,MAChBzO,KAAK4S,SACL5S,KAAK6S,SAAW7S,KA
 AK2S,SAASG,SAAW,KACzC9S,KAAK+S,SAID/S,KAAK2S,SAAS7L,KAChB,IAAK,GAAIgI,KAAS9O,MAAK2S,SAAS7L,KAC9B,GAAI9G,KAAK2S,SAAS7L,KAAKhB,eAAegJ,GAAQ,CAC5C,GAAIzH,GAAQrH,KAAK2S,SAAS7L,KAAKgI,EAE7B9O,MAAK+S,MAAMjE,GADA,QAATzH,GAA4B,WAATA,GAA+B,WAATA,EACvB,OAGAA,EAO5B,GAAIrH,KAAK2S,SAAS9L,QAChB,KAAM,IAAIjD,OAAM,sDAGlB5D,MAAKgT,gBAGDN,GACF1S,KAAKiT,IAAIP,GAGX1S,KAAKkT,WAAWzE,GAtFlB,GAAI9N,GAAOT,EAAoB,GAC3Ba,EAAQb,EAAoB,EAiGhCW,GAAQsS,UAAUD,WAAa,SAASzE,GAClCA,GAA6BjI,SAAlBiI,EAAQ2E,QACjB3E,EAAQ2E,SAAU,EAEhBpT,KAAKqT,SACPrT,KAAKqT,OAAOC,gBACLtT,MAAKqT,SAKTrT,KAAKqT,SACRrT,KAAKqT,OAAStS,EAAMuE,OAAOtF,MACzBwK,SAAU,MAAO,SAAU,aAIF,gBAAlBiE,GAAQ2E,OACjBpT,KAAKqT,OAAOH,WAAWzE,EAAQ2E,UAevCvS,EAAQsS,UAAUI,GAAK,SAASjK,EAAOhB,GACrC,GAAIkL,GAAcxT,KAAKgT,aAAa1J,EAC/BkK,KACHA,KACAxT,KAAKgT,aAAa1J,GAASkK,GAG7BA,EAAYxL,MACVM,SAAUA,KAKdzH,EAAQsS,UAAUM,UAAY5S,EAAQsS,UAAUI,GAOhD1S,EAAQsS,UAAUO,IAAM,SAASpK,EAAOhB,GACtC,GAAIkL,GAAcxT,KAAKgT,aAAa1J,EAChCkK,KACFxT,KAAKgT,aAAa1J,GAASkK,EAAYG,OAAO,SAAU7K,GACtD,MAAQA,GAASR,U
 AAYA,MAMnCzH,EAAQsS,UAAUS,YAAc/S,EAAQsS,UAAUO,IASlD7S,EAAQsS,UAAUU,SAAW,SAAUvK,EAAOwK,EAAQC,GACpD,GAAa,KAATzK,EACF,KAAM,IAAI1F,OAAM,yBAGlB,IAAI4P,KACAlK,KAAStJ,MAAKgT,eAChBQ,EAAcA,EAAYQ,OAAOhU,KAAKgT,aAAa1J,KAEjD,KAAOtJ,MAAKgT,eACdQ,EAAcA,EAAYQ,OAAOhU,KAAKgT,aAAa,MAGrD,KAAK,GAAIxN,GAAI,EAAGA,EAAIgO,EAAY7N,OAAQH,IAAK,CAC3C,GAAIyO,GAAaT,EAAYhO,EACzByO,GAAW3L,UACb2L,EAAW3L,SAASgB,EAAOwK,EAAQC,GAAY,QAYrDlT,EAAQsS,UAAUF,IAAM,SAAUP,EAAMqB,GACtC,GACI1T,GADA6T,KAEAC,EAAKnU,IAET,IAAIiG,MAAMC,QAAQwM,GAEhB,IAAK,GAAIlN,GAAI,EAAGC,EAAMiN,EAAK/M,OAAYF,EAAJD,EAASA,IAC1CnF,EAAK8T,EAAGC,SAAS1B,EAAKlN,IACtB0O,EAASlM,KAAK3H,OAGb,IAAIM,EAAKiE,YAAY8N,GAGxB,IAAK,GADD2B,GAAUrU,KAAKsU,gBAAgB5B,GAC1B6B,EAAM,EAAGC,EAAO9B,EAAK+B,kBAAyBD,EAAND,EAAYA,IAAO,CAElE,IAAK,GADDlF,MACKqF,EAAM,EAAGC,EAAON,EAAQ1O,OAAcgP,EAAND,EAAYA,IAAO,CAC1D,GAAI5F,GAAQuF,EAAQK,EACpBrF,GAAKP,GAAS4D,EAAKkC,SAASL,EAAKG,GAGnCrU,EAAK8T,EAAGC,SAAS/E,GACjB6E,EAASlM,KAAK3H,OAGb,CAAA,KAAIqS,YAAgBnM,SAMvB,KAAM,IAAI3C,OAAM,mBAJhBvD,GAAK8T,EAAGC,S
 AAS1B,GACjBwB,EAASlM,KAAK3H,GAUhB,MAJI6T,GAASvO,QACX3F,KAAK6T,SAAS,OAAQ5R,MAAOiS,GAAWH,GAGnCG,GASTrT,EAAQsS,UAAU0B,OAAS,SAAUnC,EAAMqB,GACzC,GAAIG,MACAY,KACAC,KACAZ,EAAKnU,KACL8S,EAAUqB,EAAGtB,SAEbmC,EAAc,SAAU3F,GAC1B,GAAIhP,GAAKgP,EAAKyD,EACVqB,GAAGvB,MAAMvS,IAEXA,EAAK8T,EAAGc,YAAY5F,GACpByF,EAAW9M,KAAK3H,GAChB0U,EAAY/M,KAAKqH,KAIjBhP,EAAK8T,EAAGC,SAAS/E,GACjB6E,EAASlM,KAAK3H,IAIlB,IAAI4F,MAAMC,QAAQwM,GAEhB,IAAK,GAAIlN,GAAI,EAAGC,EAAMiN,EAAK/M,OAAYF,EAAJD,EAASA,IAC1CwP,EAAYtC,EAAKlN,QAGhB,IAAI7E,EAAKiE,YAAY8N,GAGxB,IAAK,GADD2B,GAAUrU,KAAKsU,gBAAgB5B,GAC1B6B,EAAM,EAAGC,EAAO9B,EAAK+B,kBAAyBD,EAAND,EAAYA,IAAO,CAElE,IAAK,GADDlF,MACKqF,EAAM,EAAGC,EAAON,EAAQ1O,OAAcgP,EAAND,EAAYA,IAAO,CAC1D,GAAI5F,GAAQuF,EAAQK,EACpBrF,GAAKP,GAAS4D,EAAKkC,SAASL,EAAKG,GAGnCM,EAAY3F,OAGX,CAAA,KAAIqD,YAAgBnM,SAKvB,KAAM,IAAI3C,OAAM,mBAHhBoR,GAAYtC,GAad,MAPIwB,GAASvO,QACX3F,KAAK6T,SAAS,OAAQ5R,MAAOiS,GAAWH,GAEtCe,EAAWnP,QACb3F,KAAK6T,SAAS,UAAW5R,MAAO6S,EAAYpC,KAAMqC,GAAchB,GAG3DG,EAASF,OAAOc,IAsCzBjU,EAAQsS,UAAU+
 B,IAAM,WACtB,GAGI7U,GAAI8U,EAAK1G,EAASiE,EAHlByB,EAAKnU,KAILoV,EAAYzU,EAAKwG,QAAQzB,UAAU,GACtB,WAAb0P,GAAsC,UAAbA,GAE3B/U,EAAKqF,UAAU,GACf+I,EAAU/I,UAAU,GACpBgN,EAAOhN,UAAU,IAEG,SAAb0P,GAEPD,EAAMzP,UAAU,GAChB+I,EAAU/I,UAAU,GACpBgN,EAAOhN,UAAU,KAIjB+I,EAAU/I,UAAU,GACpBgN,EAAOhN,UAAU,GAInB,IAAI2P,EACJ,IAAI5G,GAAWA,EAAQ4G,WAAY,CACjC,GAAIC,IAAiB,YAAa,QAAS,SAG3C,IAFAD,EAA0D,IAA7CC,EAAc3O,QAAQ8H,EAAQ4G,YAAoB,QAAU5G,EAAQ4G,WAE7E3C,GAAS2C,GAAc1U,EAAKwG,QAAQuL,GACtC,KAAM,IAAI9O,OAAM,6BAA+BjD,EAAKwG,QAAQuL,GAAQ,sDACVjE,EAAQ3H,KAAO,IAE3E,IAAkB,aAAduO,IAA8B1U,EAAKiE,YAAY8N,GACjD,KAAM,IAAI9O,OAAM,6EAKlByR,GADO3C,GAC6B,aAAtB/R,EAAKwG,QAAQuL,GAAwB,YAGtC,OAIf,IAEgBrD,GAAMkG,EAAQ/P,EAAGC,EAF7BqB,EAAO2H,GAAWA,EAAQ3H,MAAQ9G,KAAK2S,SAAS7L,KAChD6M,EAASlF,GAAWA,EAAQkF,OAC5B1R,IAGJ,IAAUuE,QAANnG,EAEFgP,EAAO8E,EAAGqB,SAASnV,EAAIyG,GACnB6M,IAAWA,EAAOtE,KACpBA,EAAO,UAGN,IAAW7I,QAAP2O,EAEP,IAAK3P,EAAI,EAAGC,EAAM0P,EAAIxP,OAAYF,EAAJD,EAASA,IACrC6J,EAAO8E,EAAGqB,SAASL,EAAI3P,GAAIsB,KACtB6M,GAAUA,EAAOtE,KACpBpN,
 EAAM+F,KAAKqH,OAMf,KAAKkG,IAAUvV,MAAK4S,MACd5S,KAAK4S,MAAM9M,eAAeyP,KAC5BlG,EAAO8E,EAAGqB,SAASD,EAAQzO,KACtB6M,GAAUA,EAAOtE,KACpBpN,EAAM+F,KAAKqH,GAYnB,IALIZ,GAAWA,EAAQgH,OAAejP,QAANnG,GAC9BL,KAAK0V,MAAMzT,EAAOwM,EAAQgH,OAIxBhH,GAAWA,EAAQP,OAAQ,CAC7B,GAAIA,GAASO,EAAQP,MACrB,IAAU1H,QAANnG,EACFgP,EAAOrP,KAAK2V,cAActG,EAAMnB,OAGhC,KAAK1I,EAAI,EAAGC,EAAMxD,EAAM0D,OAAYF,EAAJD,EAASA,IACvCvD,EAAMuD,GAAKxF,KAAK2V,cAAc1T,EAAMuD,GAAI0I,GAM9C,GAAkB,aAAdmH,EAA2B,CAC7B,GAAIhB,GAAUrU,KAAKsU,gBAAgB5B,EACnC,IAAUlM,QAANnG,EAEF8T,EAAGyB,WAAWlD,EAAM2B,EAAShF,OAI7B,KAAK7J,EAAI,EAAGA,EAAIvD,EAAM0D,OAAQH,IAC5B2O,EAAGyB,WAAWlD,EAAM2B,EAASpS,EAAMuD,GAGvC,OAAOkN,GAEJ,GAAkB,UAAd2C,EAAwB,CAC/B,GAAI1K,KACJ,KAAKnF,EAAI,EAAGA,EAAIvD,EAAM0D,OAAQH,IAC5BmF,EAAO1I,EAAMuD,GAAGnF,IAAM4B,EAAMuD,EAE9B,OAAOmF,GAIP,GAAUnE,QAANnG,EAEF,MAAOgP,EAIP,IAAIqD,EAAM,CAER,IAAKlN,EAAI,EAAGC,EAAMxD,EAAM0D,OAAYF,EAAJD,EAASA,IACvCkN,EAAK1K,KAAK/F,EAAMuD,GAElB,OAAOkN,GAIP,MAAOzQ,IAcfpB,EAAQsS,UAAU0C,OAAS,SAAUpH,GACnC,GAIIjJ,GACAC,EACApF
 ,EACAgP,EACApN,EARAyQ,EAAO1S,KAAK4S,MACZe,EAASlF,GAAWA,EAAQkF,OAC5B8B,EAAQhH,GAAWA,EAAQgH,MAC3B3O,EAAO2H,GAAWA,EAAQ3H,MAAQ9G,KAAK2S,SAAS7L,KAMhDqO,IAEJ,IAAIxB,EAEF,GAAI8B,EAAO,CAETxT,IACA,KAAK5B,IAAMqS,GACLA,EAAK5M,eAAezF,KACtBgP,EAAOrP,KAAKwV,SAASnV,EAAIyG,GACrB6M,EAAOtE,IACTpN,EAAM+F,KAAKqH,GAOjB,KAFArP,KAAK0V,MAAMzT,EAAOwT,GAEbjQ,EAAI,EAAGC,EAAMxD,EAAM0D,OAAYF,EAAJD,EAASA,IACvC2P,EAAI3P,GAAKvD,EAAMuD,GAAGxF,KAAK6S,cAKzB,KAAKxS,IAAMqS,GACLA,EAAK5M,eAAezF,KACtBgP,EAAOrP,KAAKwV,SAASnV,EAAIyG,GACrB6M,EAAOtE,IACT8F,EAAInN,KAAKqH,EAAKrP,KAAK6S,gBAQ3B,IAAI4C,EAAO,CAETxT,IACA,KAAK5B,IAAMqS,GACLA,EAAK5M,eAAezF,IACtB4B,EAAM+F,KAAK0K,EAAKrS,GAMpB,KAFAL,KAAK0V,MAAMzT,EAAOwT,GAEbjQ,EAAI,EAAGC,EAAMxD,EAAM0D,OAAYF,EAAJD,EAASA,IACvC2P,EAAI3P,GAAKvD,EAAMuD,GAAGxF,KAAK6S,cAKzB,KAAKxS,IAAMqS,GACLA,EAAK5M,eAAezF,KACtBgP,EAAOqD,EAAKrS,GACZ8U,EAAInN,KAAKqH,EAAKrP,KAAK6S,WAM3B,OAAOsC,IAOTtU,EAAQsS,UAAU2C,WAAa,WAC7B,MAAO9V,OAaTa,EAAQsS,UAAU9K,QAAU,SAAUC,EAAUmG,GAC9C,GAGIY,GACAhP,EAJAsT,EAASlF,GAAWA,EAAQ
 kF,OAC5B7M,EAAO2H,GAAWA,EAAQ3H,MAAQ9G,KAAK2S,SAAS7L,KAChD4L,EAAO1S,KAAK4S,KAIhB,IAAInE,GAAWA,EAAQgH,MAIrB,IAAK,GAFDxT,GAAQjC,KAAKkV,IAAIzG,GAEZjJ,EAAI,EAAGC,EAAMxD,EAAM0D,OAAYF,EAAJD,EAASA,IAC3C6J,EAAOpN,EAAMuD,GACbnF,EAAKgP,EAAKrP,KAAK6S,UACfvK,EAAS+G,EAAMhP,OAKjB,KAAKA,IAAMqS,GACLA,EAAK5M,eAAezF,KACtBgP,EAAOrP,KAAKwV,SAASnV,EAAIyG,KACpB6M,GAAUA,EAAOtE,KACpB/G,EAAS+G,EAAMhP,KAkBzBQ,EAAQsS,UAAU9F,IAAM,SAAU/E,EAAUmG,GAC1C,GAIIY,GAJAsE,EAASlF,GAAWA,EAAQkF,OAC5B7M,EAAO2H,GAAWA,EAAQ3H,MAAQ9G,KAAK2S,SAAS7L,KAChDiP,KACArD,EAAO1S,KAAK4S,KAIhB,KAAK,GAAIvS,KAAMqS,GACTA,EAAK5M,eAAezF,KACtBgP,EAAOrP,KAAKwV,SAASnV,EAAIyG,KACpB6M,GAAUA,EAAOtE,KACpB0G,EAAY/N,KAAKM,EAAS+G,EAAMhP,IAUtC,OAJIoO,IAAWA,EAAQgH,OACrBzV,KAAK0V,MAAMK,EAAatH,EAAQgH,OAG3BM,GAUTlV,EAAQsS,UAAUwC,cAAgB,SAAUtG,EAAMnB,GAChD,GAAI8H,KAEJ,KAAK,GAAIlH,KAASO,GACZA,EAAKvJ,eAAegJ,IAAoC,IAAzBZ,EAAOvH,QAAQmI,KAChDkH,EAAalH,GAASO,EAAKP,GAI/B,OAAOkH,IASTnV,EAAQsS,UAAUuC,MAAQ,SAAUzT,EAAOwT,GACzC,GAAI9U,EAAKwD,SAASsR,GAAQ,CAExB,GAAIQ,GAAOR,CA
 CXxT,GAAMiU,KAAK,SAAU3Q,EAAGa,GACtB,GAAI+P,GAAK5Q,EAAE0Q,GACPG,EAAKhQ,EAAE6P,EACX,OAAQE,GAAKC,EAAM,EAAWA,EAALD,EAAW,GAAK,QAGxC,CAAA,GAAqB,kBAAVV,GAOd,KAAM,IAAIpP,WAAU,uCALpBpE,GAAMiU,KAAKT,KAgBf5U,EAAQsS,UAAUkD,OAAS,SAAUhW,EAAI0T,GACvC,GACIvO,GAAGC,EAAK6Q,EADRC,IAGJ,IAAItQ,MAAMC,QAAQ7F,GAChB,IAAKmF,EAAI,EAAGC,EAAMpF,EAAGsF,OAAYF,EAAJD,EAASA,IACpC8Q,EAAYtW,KAAKwW,QAAQnW,EAAGmF,IACX,MAAb8Q,GACFC,EAAWvO,KAAKsO,OAKpBA,GAAYtW,KAAKwW,QAAQnW,GACR,MAAbiW,GACFC,EAAWvO,KAAKsO,EAQpB,OAJIC,GAAW5Q,QACb3F,KAAK6T,SAAS,UAAW5R,MAAOsU,GAAaxC,GAGxCwC,GAST1V,EAAQsS,UAAUqD,QAAU,SAAUnW,GACpC,GAAIM,EAAKqD,SAAS3D,IAAOM,EAAKwD,SAAS9D,IACrC,GAAIL,KAAK4S,MAAMvS,GAEb,aADOL,MAAK4S,MAAMvS,GACXA,MAGN,IAAIA,YAAckG,QAAQ,CAC7B,GAAIgP,GAASlV,EAAGL,KAAK6S,SACrB,IAAI0C,GAAUvV,KAAK4S,MAAM2C,GAEvB,aADOvV,MAAK4S,MAAM2C,GACXA,EAGX,MAAO,OAQT1U,EAAQsS,UAAUsD,MAAQ,SAAU1C,GAClC,GAAIoB,GAAM5O,OAAO6G,KAAKpN,KAAK4S,MAM3B,OAJA5S,MAAK4S,SAEL5S,KAAK6T,SAAS,UAAW5R,MAAOkT,GAAMpB,GAE/BoB,GAQTtU,EAAQsS,UAAUzG,IAAM,SAAUoC,GAChC,GAAI4D,GA
 AO1S,KAAK4S,MACZlG,EAAM,KACNgK,EAAW,IAEf,KAAK,GAAIrW,KAAMqS,GACb,GAAIA,EAAK5M,eAAezF,GAAK,CAC3B,GAAIgP,GAAOqD,EAAKrS,GACZsW,EAAYtH,EAAKP,EACJ,OAAb6H,KAAuBjK,GAAOiK,EAAYD,KAC5ChK,EAAM2C,EACNqH,EAAWC,GAKjB,MAAOjK,IAQT7L,EAAQsS,UAAUrH,IAAM,SAAUgD,GAChC,GAAI4D,GAAO1S,KAAK4S,MACZ9G,EAAM,KACN8K,EAAW,IAEf,KAAK,GAAIvW,KAAMqS,GACb,GAAIA,EAAK5M,eAAezF,GAAK,CAC3B,GAAIgP,GAAOqD,EAAKrS,GACZsW,EAAYtH,EAAKP,EACJ,OAAb6H,KAAuB7K,GAAmB8K,EAAZD,KAChC7K,EAAMuD,EACNuH,EAAWD,GAKjB,MAAO7K,IAUTjL,EAAQsS,UAAU0D,SAAW,SAAU/H,GACrC,GAIItJ,GAJAkN,EAAO1S,KAAK4S,MACZkE,KACAC,EAAY/W,KAAK2S,SAAS7L,MAAQ9G,KAAK2S,SAAS7L,KAAKgI,IAAU,KAC/DkI,EAAQ,CAGZ,KAAK,GAAInR,KAAQ6M,GACf,GAAIA,EAAK5M,eAAeD,GAAO,CAC7B,GAAIwJ,GAAOqD,EAAK7M,GACZwB,EAAQgI,EAAKP,GACbmI,GAAS,CACb,KAAKzR,EAAI,EAAOwR,EAAJxR,EAAWA,IACrB,GAAIsR,EAAOtR,IAAM6B,EAAO,CACtB4P,GAAS,CACT,OAGCA,GAAqBzQ,SAAVa,IACdyP,EAAOE,GAAS3P,EAChB2P,KAKN,GAAID,EACF,IAAKvR,EAAI,EAAGA,EAAIsR,EAAOnR,OAAQH,IAC7BsR,EAAOtR,GAAK7E,EAAKkG,QAAQiQ,EAAOtR,GAAIuR,EAIxC,OAAOD,IASTjW,EAAQsS,UA
 AUiB,SAAW,SAAU/E,GACrC,GAAIhP,GAAKgP,EAAKrP,KAAK6S,SAEnB,IAAUrM,QAANnG,GAEF,GAAIL,KAAK4S,MAAMvS,GAEb,KAAM,IAAIuD,OAAM,iCAAmCvD,EAAK,uBAK1DA,GAAKM,EAAKqE,aACVqK,EAAKrP,KAAK6S,UAAYxS,CAGxB,IAAIsM,KACJ,KAAK,GAAImC,KAASO,GAChB,GAAIA,EAAKvJ,eAAegJ,GAAQ,CAC9B,GAAIiI,GAAY/W,KAAK+S,MAAMjE,EAC3BnC,GAAEmC,GAASnO,EAAKkG,QAAQwI,EAAKP,GAAQiI,GAKzC,MAFA/W,MAAK4S,MAAMvS,GAAMsM,EAEVtM,GAUTQ,EAAQsS,UAAUqC,SAAW,SAAUnV,EAAI6W,GACzC,GAAIpI,GAAOzH,EAGP8P,EAAMnX,KAAK4S,MAAMvS,EACrB,KAAK8W,EACH,MAAO,KAIT,IAAIC,KACJ,IAAIF,EACF,IAAKpI,IAASqI,GACRA,EAAIrR,eAAegJ,KACrBzH,EAAQ8P,EAAIrI,GACZsI,EAAUtI,GAASnO,EAAKkG,QAAQQ,EAAO6P,EAAMpI,SAMjD,KAAKA,IAASqI,GACRA,EAAIrR,eAAegJ,KACrBzH,EAAQ8P,EAAIrI,GACZsI,EAAUtI,GAASzH,EAIzB,OAAO+P,IAWTvW,EAAQsS,UAAU8B,YAAc,SAAU5F,GACxC,GAAIhP,GAAKgP,EAAKrP,KAAK6S,SACnB,IAAUrM,QAANnG,EACF,KAAM,IAAIuD,OAAM,6CAA+CyT,KAAKC,UAAUjI,GAAQ,IAExF,IAAI1C,GAAI3M,KAAK4S,MAAMvS,EACnB,KAAKsM,EAEH,KAAM,IAAI/I,OAAM,uCAAyCvD,EAAK,SAIhE,KAAK,GAAIyO,KAASO,GAChB,GAAIA,EAAKvJ,eAAegJ,GAAQ,CAC9B,GAAIiI,GA
 AY/W,KAAK+S,MAAMjE,EAC3BnC,GAAEmC,GAASnO,EAAKkG,QAAQwI,EAAKP,GAAQiI,GAIzC,MAAO1W,IASTQ,EAAQsS,UAAUmB,gBAAkB,SAAUiD,GAE5C,IAAK,GADDlD,MACKK,EAAM,EAAGC,EAAO4C,EAAUC,qBAA4B7C,EAAND,EAAYA,IACnEL,EAAQK,GAAO6C,EAAUE,YAAY/C,IAAQ6C,EAAUG,eAAehD,EAExE,OAAOL,IAUTxT,EAAQsS,UAAUyC,WAAa,SAAU2B,EAAWlD,EAAShF,GAG3D,IAAK,GAFDkF,GAAMgD,EAAUI,SAEXjD,EAAM,EAAGC,EAAON,EAAQ1O,OAAcgP,EAAND,EAAYA,IAAO,CAC1D,GAAI5F,GAAQuF,EAAQK,EACpB6C,GAAUK,SAASrD,EAAKG,EAAKrF,EAAKP,MAItCjP,EAAOD,QAAUiB,GAKb,SAAShB,EAAQD,EAASM,GAe9B,QAASY,GAAU4R,EAAMjE,GACvBzO,KAAK4S,MAAQ,KACb5S,KAAK6X,QACL7X,KAAK2S,SAAWlE,MAChBzO,KAAK6S,SAAW,KAChB7S,KAAKgT,eAEL,IAAImB,GAAKnU,IACTA,MAAK8I,SAAW,WACdqL,EAAG2D,SAASC,MAAM5D,EAAIzO,YAGxB1F,KAAKgY,QAAQtF,GAzBf,GAAI/R,GAAOT,EAAoB,GAC3BW,EAAUX,EAAoB,EAkClCY,GAASqS,UAAU6E,QAAU,SAAUtF,GACrC,GAAIyC,GAAK3P,EAAGC,CAEZ,IAAIzF,KAAK4S,MAAO,CAEV5S,KAAK4S,MAAMgB,aACb5T,KAAK4S,MAAMgB,YAAY,IAAK5T,KAAK8I,UAInCqM,IACA,KAAK,GAAI9U,KAAML,MAAK6X,KACd7X,KAAK6X,KAAK/R,eAAezF,IAC3B8U,EAAInN,KAAK3H,EAGbL,MAAK6X,QACL7
 X,KAAK6T,SAAS,UAAW5R,MAAOkT,IAKlC,GAFAnV,KAAK4S,MAAQF,EAET1S,KAAK4S,MAAO,CAQd,IANA5S,KAAK6S,SAAW7S,KAAK2S,SAASG,SACzB9S,KAAK4S,OAAS5S,KAAK4S,MAAMnE,SAAWzO,KAAK4S,MAAMnE,QAAQqE,SACxD,KAGJqC,EAAMnV,KAAK4S,MAAMiD,QAAQlC,OAAQ3T,KAAK2S,UAAY3S,KAAK2S,SAASgB,SAC3DnO,EAAI,EAAGC,EAAM0P,EAAIxP,OAAYF,EAAJD,EAASA,IACrCnF,EAAK8U,EAAI3P,GACTxF,KAAK6X,KAAKxX,IAAM,CAElBL,MAAK6T,SAAS,OAAQ5R,MAAOkT,IAGzBnV,KAAK4S,MAAMW,IACbvT,KAAK4S,MAAMW,GAAG,IAAKvT,KAAK8I,YAuC9BhI,EAASqS,UAAU+B,IAAM,WACvB,GAGIC,GAAK1G,EAASiE,EAHdyB,EAAKnU,KAILoV,EAAYzU,EAAKwG,QAAQzB,UAAU,GACtB,WAAb0P,GAAsC,UAAbA,GAAsC,SAAbA,GAEpDD,EAAMzP,UAAU,GAChB+I,EAAU/I,UAAU,GACpBgN,EAAOhN,UAAU,KAIjB+I,EAAU/I,UAAU,GACpBgN,EAAOhN,UAAU,GAInB,IAAIuS,GAActX,EAAK2E,UAAWtF,KAAK2S,SAAUlE,EAG7CzO,MAAK2S,SAASgB,QAAUlF,GAAWA,EAAQkF,SAC7CsE,EAAYtE,OAAS,SAAUtE,GAC7B,MAAO8E,GAAGxB,SAASgB,OAAOtE,IAASZ,EAAQkF,OAAOtE,IAKtD,IAAI6I,KAOJ,OANW1R,SAAP2O,GACF+C,EAAalQ,KAAKmN,GAEpB+C,EAAalQ,KAAKiQ,GAClBC,EAAalQ,KAAK0K,GAEX1S,KAAK4S,OAAS5S,KAAK4S,MAAMsC,IAAI6C,MAAM/X,
 KAAK4S,MAAOsF,IAWxDpX,EAASqS,UAAU0C,OAAS,SAAUpH,GACpC,GAAI0G,EAEJ,IAAInV,KAAK4S,MAAO,CACd,GACIe,GADAwE,EAAgBnY,KAAK2S,SAASgB,MAK9BA,GAFAlF,GAAWA,EAAQkF,OACjBwE,EACO,SAAU9I,GACjB,MAAO8I,GAAc9I,IAASZ,EAAQkF,OAAOtE,IAItCZ,EAAQkF,OAIVwE,EAGXhD,EAAMnV,KAAK4S,MAAMiD,QACflC,OAAQA,EACR8B,MAAOhH,GAAWA,EAAQgH,YAI5BN,KAGF,OAAOA,IAQTrU,EAASqS,UAAU2C,WAAa,WAE9B,IADA,GAAIsC,GAAUpY,KACPoY,YAAmBtX,IACxBsX,EAAUA,EAAQxF,KAEpB,OAAOwF,IAAW,MAYpBtX,EAASqS,UAAU2E,SAAW,SAAUxO,EAAOwK,EAAQC,GACrD,GAAIvO,GAAGC,EAAKpF,EAAIgP,EACZ8F,EAAMrB,GAAUA,EAAO7R,MACvByQ,EAAO1S,KAAK4S,MACZyF,KACAC,KACAC,IAEJ,IAAIpD,GAAOzC,EAAM,CACf,OAAQpJ,GACN,IAAK,MAEH,IAAK9D,EAAI,EAAGC,EAAM0P,EAAIxP,OAAYF,EAAJD,EAASA,IACrCnF,EAAK8U,EAAI3P,GACT6J,EAAOrP,KAAKkV,IAAI7U,GACZgP,IACFrP,KAAK6X,KAAKxX,IAAM,EAChBgY,EAAMrQ,KAAK3H,GAIf,MAEF,KAAK,SAGH,IAAKmF,EAAI,EAAGC,EAAM0P,EAAIxP,OAAYF,EAAJD,EAASA,IACrCnF,EAAK8U,EAAI3P,GACT6J,EAAOrP,KAAKkV,IAAI7U,GAEZgP,EACErP,KAAK6X,KAAKxX,GACZiY,EAAQtQ,KAAK3H,IAGbL,KAAK6X,KAAKxX,IAAM,EAChBgY,EAAMrQ,KAAK3H,IAI
 TL,KAAK6X,KAAKxX,WACLL,MAAK6X,KAAKxX,GACjBkY,EAAQvQ,KAAK3H,GAQnB,MAEF,KAAK,SAEH,IAAKmF,EAAI,EAAGC,EAAM0P,EAAIxP,OAAYF,EAAJD,EAASA,IACrCnF,EAAK8U,EAAI3P,GACLxF,KAAK6X,KAAKxX,WACLL,MAAK6X,KAAKxX,GACjBkY,EAAQvQ,KAAK3H,IAOjBgY,EAAM1S,QACR3F,KAAK6T,SAAS,OAAQ5R,MAAOoW,GAAQtE,GAEnCuE,EAAQ3S,QACV3F,KAAK6T,SAAS,UAAW5R,MAAOqW,GAAUvE,GAExCwE,EAAQ5S,QACV3F,KAAK6T,SAAS,UAAW5R,MAAOsW,GAAUxE,KAMhDjT,EAASqS,UAAUI,GAAK1S,EAAQsS,UAAUI,GAC1CzS,EAASqS,UAAUO,IAAM7S,EAAQsS,UAAUO,IAC3C5S,EAASqS,UAAUU,SAAWhT,EAAQsS,UAAUU,SAGhD/S,EAASqS,UAAUM,UAAY3S,EAASqS,UAAUI,GAClDzS,EAASqS,UAAUS,YAAc9S,EAASqS,UAAUO,IAEpD7T,EAAOD,QAAUkB,GAIb,SAASjB,GAeb,QAASkB,GAAM0N,GAEbzO,KAAKwY,MAAQ,KACbxY,KAAK0M,IAAM+L,IAGXzY,KAAKqT,UACLrT,KAAK0Y,SAAW,KAChB1Y,KAAK2Y,UAAY,KAEjB3Y,KAAKkT,WAAWzE,GAgBlB1N,EAAMoS,UAAUD,WAAa,SAAUzE,GACjCA,GAAoC,mBAAlBA,GAAQ+J,QAC5BxY,KAAKwY,MAAQ/J,EAAQ+J,OAEnB/J,GAAkC,mBAAhBA,GAAQ/B,MAC5B1M,KAAK0M,IAAM+B,EAAQ/B,KAGrB1M,KAAK4Y,kBAsBP7X,EAAMuE,OAAS,SAAUrB,EAAQwK,GAC/B,GAAI2E,GAAQ,GAAIrS,GAAM0N,EAEtB,IAAqBjI,
 SAAjBvC,EAAO4U,MACT,KAAM,IAAIjV,OAAM,6CAElBK,GAAO4U,MAAQ,WACbzF,EAAMyF,QAGR,IAAIC,KACF7C,KAAM,QACN8C,SAAUvS,QAGZ,IAAIiI,GAAWA,EAAQjE,QACrB,IAAK,GAAIhF,GAAI,EAAGA,EAAIiJ,EAAQjE,QAAQ7E,OAAQH,IAAK,CAC/C,GAAIyQ,GAAOxH,EAAQjE,QAAQhF,EAC3BsT,GAAQ9Q,MACNiO,KAAMA,EACN8C,SAAU9U,EAAOgS,KAEnB7C,EAAM5I,QAAQvG,EAAQgS,GAS1B,MALA7C,GAAMuF,WACJ1U,OAAQA,EACR6U,QAASA,GAGJ1F,GAOTrS,EAAMoS,UAAUG,QAAU,WAGxB,GAFAtT,KAAK6Y,QAED7Y,KAAK2Y,UAAW,CAGlB,IAAK,GAFD1U,GAASjE,KAAK2Y,UAAU1U,OACxB6U,EAAU9Y,KAAK2Y,UAAUG,QACpBtT,EAAI,EAAGA,EAAIsT,EAAQnT,OAAQH,IAAK,CACvC,GAAIwT,GAASF,EAAQtT,EACjBwT,GAAOD,SACT9U,EAAO+U,EAAO/C,MAAQ+C,EAAOD,eAGtB9U,GAAO+U,EAAO/C,MAGzBjW,KAAK2Y,UAAY,OASrB5X,EAAMoS,UAAU3I,QAAU,SAASvG,EAAQ+U,GACzC,GAAI7E,GAAKnU,KACL+Y,EAAW9U,EAAO+U,EACtB,KAAKD,EACH,KAAM,IAAInV,OAAM,UAAYoV,EAAS,aAGvC/U,GAAO+U,GAAU,WAGf,IAAK,GADDC,MACKzT,EAAI,EAAGA,EAAIE,UAAUC,OAAQH,IACpCyT,EAAKzT,GAAKE,UAAUF,EAItB2O,GAAGf,OACD6F,KAAMA,EACNC,GAAIH,EACJI,QAASnZ,SASfe,EAAMoS,UAAUC,MAAQ,SAASgG,GAE7BpZ,KAAKqT,OAAOrL,KADO,kBAAVoR,IA
 CSF,GAAIE,GAGLA,GAGnBpZ,KAAK4Y,kBAOP7X,EAAMoS,UAAUyF,eAAiB,WAQ/B,GANI5Y,KAAKqT,OAAO1N,OAAS3F,KAAK0M,KAC5B1M,KAAK6Y,QAIPQ,aAAarZ,KAAK0Y,UACd1Y,KAAKoT,MAAMzN,OAAS,GAA2B,gBAAf3F,MAAKwY,MAAoB,CAC3D,GAAIrE,GAAKnU,IACTA,MAAK0Y,SAAWY,WAAW,WACzBnF,EAAG0E,SACF7Y,KAAKwY,SAOZzX,EAAMoS,UAAU0F,MAAQ,WACtB,KAAO7Y,KAAKqT,OAAO1N,OAAS,GAAG,CAC7B,GAAIyT,GAAQpZ,KAAKqT,OAAO/B,OACxB8H,GAAMF,GAAGnB,MAAMqB,EAAMD,SAAWC,EAAMF,GAAIE,EAAMH,YAIpDpZ,EAAOD,QAAUmB,GAKb,SAASlB,EAAQD,EAASM,GAwB9B,QAASc,GAAQuY,EAAW7G,EAAMjE,GAChC,KAAMzO,eAAgBgB,IACpB,KAAM,IAAIwY,aAAY,mDAIxBxZ,MAAKyZ,iBAAmBF,EACxBvZ,KAAKuS,MAAQ,QACbvS,KAAKwS,OAAS,QACdxS,KAAK0Z,OAAS,GACd1Z,KAAK2Z,eAAiB,MACtB3Z,KAAK4Z,eAAiB,MAEtB5Z,KAAK6Z,OAAS,IACd7Z,KAAK8Z,OAAS,IACd9Z,KAAK+Z,OAAS,GAEd,IAAIC,GAAc,SAASnO,GAAK,MAAOA,GACvC7L,MAAKia,YAAcD,EACnBha,KAAKka,YAAcF,EACnBha,KAAKma,YAAcH,EAEnBha,KAAKoa,YAAc,OACnBpa,KAAKqa,YAAc,QAEnBra,KAAKiN,MAAQjM,EAAQsZ,MAAMC,IAC3Bva,KAAKwa,iBAAkB,EACvBxa,KAAKya,UAAW,EAChBza,KAAK0a,iBAAkB,EACvB1a,KAAK2a,YAAa,EAClB3a,KAAK4a,gBAAiB
 ,EACtB5a,KAAK6a,aAAc,EACnB7a,KAAK8a,cAAgB,GAErB9a,KAAK+a,kBAAoB,IACzB/a,KAAKgb,kBAAmB,EAExBhb,KAAKib,OAAS,GAAI/Z,GAClBlB,KAAKkb,IAAM,GAAI7Z,GAAQ,EAAG,EAAG,IAE7BrB,KAAKuX,UAAY,KACjBvX,KAAKmb,WAAa,KAGlBnb,KAAKob,KAAO5U,OACZxG,KAAKqb,KAAO7U,OACZxG,KAAKsb,KAAO9U,OACZxG,KAAKub,SAAW/U,OAChBxG,KAAKwb,UAAYhV,OAEjBxG,KAAKyb,KAAO,EACZzb,KAAK0b,MAAQlV,OACbxG,KAAK2b,KAAO,EACZ3b,KAAK4b,KAAO,EACZ5b,KAAK6b,MAAQrV,OACbxG,KAAK8b,KAAO,EACZ9b,KAAK+b,KAAO,EACZ/b,KAAKgc,MAAQxV,OACbxG,KAAKic,KAAO,EACZjc,KAAKkc,SAAW,EAChBlc,KAAKmc,SAAW,EAChBnc,KAAKoc,UAAY,EACjBpc,KAAKqc,UAAY,EAIjBrc,KAAKsc,UAAY,UACjBtc,KAAKuc,UAAY,UACjBvc,KAAKwc,SAAW,UAChBxc,KAAKyc,eAAiB,UAGtBzc,KAAKqO,SAGLrO,KAAKkT,WAAWzE,GAGZiE,GACF1S,KAAKgY,QAAQtF,GAknEjB,QAASgK,GAAWpT,GAClB,MAAI,WAAaA,GAAcA,EAAMqT,QAC9BrT,EAAMsT,cAAc,IAAMtT,EAAMsT,cAAc,GAAGD,SAAW,EAQrE,QAASE,GAAWvT,GAClB,MAAI,WAAaA,GAAcA,EAAMwT,QAC9BxT,EAAMsT,cAAc,IAAMtT,EAAMsT,cAAc,GAAGE,SAAW,EAnuErE,GAAIC,GAAU7c,EAAoB,IAC9BW,EAAUX,EAAoB,GAC9BY,EAAWZ,EAAoB,GAC/BS,EAAOT,EAAoB,GAC3BmB,
 EAAUnB,EAAoB,IAC9BkB,EAAUlB,EAAoB,GAC9BgB,EAAShB,EAAoB,GAC7BiB,EAASjB,EAAoB,GAC7BoB,EAASpB,EAAoB,IAC7BqB,EAAarB,EAAoB,GAiGrC6c,GAAQ/b,EAAQmS,WAKhBnS,EAAQmS,UAAU6J,UAAY,WAC5Bhd,KAAKid,MAAQ,GAAI5b,GAAQ,GAAKrB,KAAK2b,KAAO3b,KAAKyb,MAC7C,GAAKzb,KAAK8b,KAAO9b,KAAK4b,MACtB,GAAK5b,KAAKic,KAAOjc,KAAK+b,OAGpB/b,KAAK0a,kBACH1a,KAAKid,MAAMlL,EAAI/R,KAAKid,MAAMjL,EAE5BhS,KAAKid,MAAMjL,EAAIhS,KAAKid,MAAMlL,EAI1B/R,KAAKid,MAAMlL,EAAI/R,KAAKid,MAAMjL,GAK9BhS,KAAKid,MAAMC,GAAKld,KAAK8a,cAIrB9a,KAAKid,MAAM5V,MAAQ,GAAKrH,KAAKmc,SAAWnc,KAAKkc,SAG7C,IAAIiB,IAAWnd,KAAK2b,KAAO3b,KAAKyb,MAAQ,EAAIzb,KAAKid,MAAMlL,EACnDqL,GAAWpd,KAAK8b,KAAO9b,KAAK4b,MAAQ,EAAI5b,KAAKid,MAAMjL,EACnDqL,GAAWrd,KAAKic,KAAOjc,KAAK+b,MAAQ,EAAI/b,KAAKid,MAAMC,CACvDld,MAAKib,OAAOqC,eAAeH,EAASC,EAASC,IAU/Crc,EAAQmS,UAAUoK,eAAiB,SAASC,GAC1C,GAAIC,GAAczd,KAAK0d,2BAA2BF,EAClD,OAAOxd,MAAK2d,4BAA4BF,IAW1Czc,EAAQmS,UAAUuK,2BAA6B,SAASF,GACtD,GAAII,GAAKJ,EAAQzL,EAAI/R,KAAKid,MAAMlL,EAC9B8L,EAAKL,EAAQxL,EAAIhS,KAAKid,MAAMjL,EAC5B8L,EAAKN,EAAQ
 N,EAAIld,KAAKid,MAAMC,EAE5Ba,EAAK/d,KAAKib,OAAO+C,oBAAoBjM,EACrCkM,EAAKje,KAAKib,OAAO+C,oBAAoBhM,EACrCkM,EAAKle,KAAKib,OAAO+C,oBAAoBd,EAGrCiB,EAAQjZ,KAAKkZ,IAAIpe,KAAKib,OAAOoD,oBAAoBtM,GACjDuM,EAAQpZ,KAAKqZ,IAAIve,KAAKib,OAAOoD,oBAAoBtM,GACjDyM,EAAQtZ,KAAKkZ,IAAIpe,KAAKib,OAAOoD,oBAAoBrM,GACjDyM,EAAQvZ,KAAKqZ,IAAIve,KAAKib,OAAOoD,oBAAoBrM,GACjD0M,EAAQxZ,KAAKkZ,IAAIpe,KAAKib,OAAOoD,oBAAoBnB,GACjDyB,EAAQzZ,KAAKqZ,IAAIve,KAAKib,OAAOoD,oBAAoBnB,GAGjD0B,EAAKH,GAASC,GAASb,EAAKI,GAAMU,GAASf,EAAKG,IAAOS,GAASV,EAAKI,GACrEW,EAAKV,GAASM,GAASX,EAAKI,GAAMM,GAASE,GAASb,EAAKI,GAAMU,GAASf,EAAKG,KAAQO,GAASK,GAASd,EAAKI,GAAMS,GAASd,EAAGG,IAC9He,EAAKR,GAASG,GAASX,EAAKI,GAAMM,GAASE,GAASb,EAAKI,GAAMU,GAASf,EAAKG,KAAQI,GAASQ,GAASd,EAAKI,GAAMS,GAASd,EAAGG,GAEhI,OAAO,IAAI1c,GAAQud,EAAIC,EAAIC,IAU7B9d,EAAQmS,UAAUwK,4BAA8B,SAASF,GACvD,GAQIsB,GACAC,EATAC,EAAKjf,KAAKkb,IAAInJ,EAChBmN,EAAKlf,KAAKkb,IAAIlJ,EACdmN,EAAKnf,KAAKkb,IAAIgC,EACd0B,EAAKnB,EAAY1L,EACjB8M,EAAKpB,EAAYzL,EACjB8M,EAAKrB,EAAYP,CAgBnB,OAXIld,
 MAAKwa,iBACPuE,GAAMH,EAAKK,IAAOE,EAAKL,GACvBE,GAAMH,EAAKK,IAAOC,EAAKL,KAGvBC,EAAKH,IAAOO,EAAKnf,KAAKib,OAAOmE,gBAC7BJ,EAAKH,IAAOM,EAAKnf,KAAKib,OAAOmE,iBAKxB,GAAIhe,GACTpB,KAAKqf,QAAUN,EAAK/e,KAAKsf,MAAMC,OAAOC,YACtCxf,KAAKyf,QAAUT,EAAKhf,KAAKsf,MAAMC,OAAOC,cAO1Cxe,EAAQmS,UAAUuM,oBAAsB,SAASC,GAC/C,GAAIC,GAAO,QACPC,EAAS,OACTC,EAAc,CAElB,IAAgC,gBAAtB,GACRF,EAAOD,EACPE,EAAS,OACTC,EAAc,MAEX,IAAgC,gBAAtB,GACgBtZ,SAAzBmZ,EAAgBC,OAAuBA,EAAOD,EAAgBC,MACnCpZ,SAA3BmZ,EAAgBE,SAAyBA,EAASF,EAAgBE,QAClCrZ,SAAhCmZ,EAAgBG,cAA2BA,EAAcH,EAAgBG,iBAE1E,IAAyBtZ,SAApBmZ,EAIR,KAAM,qCAGR3f,MAAKsf,MAAMrS,MAAM0S,gBAAkBC,EACnC5f,KAAKsf,MAAMrS,MAAM8S,YAAcF,EAC/B7f,KAAKsf,MAAMrS,MAAM+S,YAAcF,EAAc,KAC7C9f,KAAKsf,MAAMrS,MAAMgT,YAAc,SAKjCjf,EAAQsZ,OACN4F,IAAK,EACLC,SAAU,EACVC,QAAS,EACT7F,IAAM,EACN8F,QAAU,EACVC,SAAU,EACVC,QAAS,EACTC,KAAO,EACPC,KAAM,EACNC,QAAU,GASZ1f,EAAQmS,UAAUwN,gBAAkB,SAASC,GAC3C,OAAQA,GACN,IAAK,MAAW,MAAO5f,GAAQsZ,MAAMC,GACrC,KAAK,WAAa,MAAOvZ,GAAQsZ,MAAM+F,OACvC,KAAK,YAAe,MAAOrf,GAAQsZ,MAAMgG,QAC
 zC,KAAK,WAAa,MAAOtf,GAAQsZ,MAAMiG,OACvC,KAAK,OAAW,MAAOvf,GAAQsZ,MAAMmG,IACrC,KAAK,OAAW,MAAOzf,GAAQsZ,MAAMkG,IACrC,KAAK,UAAa,MAAOxf,GAAQsZ,MAAMoG,OACvC,KAAK,MAAW,MAAO1f,GAAQsZ,MAAM4F,GACrC,KAAK,YAAe,MAAOlf,GAAQsZ,MAAM6F,QACzC,KAAK,WAAa,MAAOnf,GAAQsZ,MAAM8F,QAGzC,MAAO,IAQTpf,EAAQmS,UAAU0N,wBAA0B,SAASnO,GACnD,GAAI1S,KAAKiN,QAAUjM,EAAQsZ,MAAMC,KAC/Bva,KAAKiN,QAAUjM,EAAQsZ,MAAM+F,SAC7BrgB,KAAKiN,QAAUjM,EAAQsZ,MAAMmG,MAC7BzgB,KAAKiN,QAAUjM,EAAQsZ,MAAMkG,MAC7BxgB,KAAKiN,QAAUjM,EAAQsZ,MAAMoG,SAC7B1gB,KAAKiN,QAAUjM,EAAQsZ,MAAM4F,IAE7BlgB,KAAKob,KAAO,EACZpb,KAAKqb,KAAO,EACZrb,KAAKsb,KAAO,EACZtb,KAAKub,SAAW/U,OAEZkM,EAAK8E,qBAAuB,IAC9BxX,KAAKwb,UAAY,OAGhB,CAAA,GAAIxb,KAAKiN,QAAUjM,EAAQsZ,MAAMgG,UACpCtgB,KAAKiN,QAAUjM,EAAQsZ,MAAMiG,SAC7BvgB,KAAKiN,QAAUjM,EAAQsZ,MAAM6F,UAC7BngB,KAAKiN,QAAUjM,EAAQsZ,MAAM8F,QAY7B,KAAM,kBAAoBpgB,KAAKiN,MAAQ,GAVvCjN,MAAKob,KAAO,EACZpb,KAAKqb,KAAO,EACZrb,KAAKsb,KAAO,EACZtb,KAAKub,SAAW,EAEZ7I,EAAK8E,qBAAuB,IAC9BxX,KAAKwb,UAAY,KAQvBxa,EAAQmS,UAAUsB,gBAAkB,SAAS/B,GAC3C,
 MAAOA,GAAK/M,QAId3E,EAAQmS,UAAUqE,mBAAqB,SAAS9E,GAC9C,GAAIoO,GAAU,CACd,KAAK,GAAIC,KAAUrO,GAAK,GAClBA,EAAK,GAAG5M,eAAeib,IACzBD,GAGJ,OAAOA,IAIT9f,EAAQmS,UAAU6N,kBAAoB,SAAStO,EAAMqO,GAEnD,IAAK,GADDE,MACKzb,EAAI,EAAGA,EAAIkN,EAAK/M,OAAQH,IACgB,IAA3Cyb,EAAeta,QAAQ+L,EAAKlN,GAAGub,KACjCE,EAAejZ,KAAK0K,EAAKlN,GAAGub,GAGhC,OAAOE,IAITjgB,EAAQmS,UAAU+N,eAAiB,SAASxO,EAAKqO,GAE/C,IAAK,GADDI,IAAUrV,IAAI4G,EAAK,GAAGqO,GAAQrU,IAAIgG,EAAK,GAAGqO,IACrCvb,EAAI,EAAGA,EAAIkN,EAAK/M,OAAQH,IAC3B2b,EAAOrV,IAAM4G,EAAKlN,GAAGub,KAAWI,EAAOrV,IAAM4G,EAAKlN,GAAGub,IACrDI,EAAOzU,IAAMgG,EAAKlN,GAAGub,KAAWI,EAAOzU,IAAMgG,EAAKlN,GAAGub,GAE3D,OAAOI,IASTngB,EAAQmS,UAAUiO,gBAAkB,SAAUC,GAC5C,GAAIlN,GAAKnU,IAOT,IAJIA,KAAKoY,SACPpY,KAAKoY,QAAQ1E,IAAI,IAAK1T,KAAKshB,WAGb9a,SAAZ6a,EAAJ,CAGIpb,MAAMC,QAAQmb,KAChBA,EAAU,GAAIxgB,GAAQwgB,GAGxB,IAAI3O,EACJ,MAAI2O,YAAmBxgB,IAAWwgB,YAAmBvgB,IAInD,KAAM,IAAI8C,OAAM,uCAGlB,IANE8O,EAAO2O,EAAQnM,MAME,GAAfxC,EAAK/M,OAAT,CAGA3F,KAAKoY,QAAUiJ,EACfrhB,KAAKuX,UAAY7E,EAGjB1S,KAAKshB,UAAY,W
 ACfnN,EAAG6D,QAAQ7D,EAAGiE,UAEhBpY,KAAKoY,QAAQ7E,GAAG,IAAKvT,KAAKshB,WAS1BthB,KAAKob,KAAO,IACZpb,KAAKqb,KAAO,IACZrb,KAAKsb,KAAO,IACZtb,KAAKub,SAAW,QAChBvb,KAAKwb,UAAY,SAKb9I,EAAK,GAAG5M,eAAe,WACDU,SAApBxG,KAAKuhB,aACPvhB,KAAKuhB,WAAa,GAAIpgB,GAAOkgB,EAASrhB,KAAKwb,UAAWxb,MACtDA,KAAKuhB,WAAWC,kBAAkB,WAAYrN,EAAGsN,WAKrD,IAAIC,GAAW1hB,KAAKiN,OAASjM,EAAQsZ,MAAM4F,KACzClgB,KAAKiN,OAASjM,EAAQsZ,MAAM6F,UAC5BngB,KAAKiN,OAASjM,EAAQsZ,MAAM8F,OAG9B,IAAIsB,EAAU,CACZ,GAA8Blb,SAA1BxG,KAAK2hB,iBACP3hB,KAAKoc,UAAYpc,KAAK2hB,qBAEnB,CACH,GAAIC,GAAQ5hB,KAAKghB,kBAAkBtO,EAAK1S,KAAKob,KAC7Cpb,MAAKoc,UAAawF,EAAM,GAAKA,EAAM,IAAO,EAG5C,GAA8Bpb,SAA1BxG,KAAK6hB,iBACP7hB,KAAKqc,UAAYrc,KAAK6hB,qBAEnB,CACH,GAAIC,GAAQ9hB,KAAKghB,kBAAkBtO,EAAK1S,KAAKqb,KAC7Crb,MAAKqc,UAAayF,EAAM,GAAKA,EAAM,IAAO,GAK9C,GAAIC,GAAS/hB,KAAKkhB,eAAexO,EAAK1S,KAAKob,KACvCsG,KACFK,EAAOjW,KAAO9L,KAAKoc,UAAY,EAC/B2F,EAAOrV,KAAO1M,KAAKoc,UAAY,GAEjCpc,KAAKyb,KAA6BjV,SAArBxG,KAAKgiB,YAA6BhiB,KAAKgiB,YAAcD,EAAOjW,IACzE9L,KAAK2b,KAA6BnV,SAArBxG
 ,KAAKiiB,YAA6BjiB,KAAKiiB,YAAcF,EAAOrV,IACrE1M,KAAK2b,MAAQ3b,KAAKyb,OAAMzb,KAAK2b,KAAO3b,KAAKyb,KAAO,GACpDzb,KAAK0b,MAA+BlV,SAAtBxG,KAAKkiB,aAA8BliB,KAAKkiB,cAAgBliB,KAAK2b,KAAK3b,KAAKyb,MAAM,CAE3F,IAAI0G,GAASniB,KAAKkhB,eAAexO,EAAK1S,KAAKqb,KACvCqG,KACFS,EAAOrW,KAAO9L,KAAKqc,UAAY,EAC/B8F,EAAOzV,KAAO1M,KAAKqc,UAAY,GAEjCrc,KAAK4b,KAA6BpV,SAArBxG,KAAKoiB,YAA6BpiB,KAAKoiB,YAAcD,EAAOrW,IACzE9L,KAAK8b,KAA6BtV,SAArBxG,KAAKqiB,YAA6BriB,KAAKqiB,YAAcF,EAAOzV,IACrE1M,KAAK8b,MAAQ9b,KAAK4b,OAAM5b,KAAK8b,KAAO9b,KAAK4b,KAAO,GACpD5b,KAAK6b,MAA+BrV,SAAtBxG,KAAKsiB,aAA8BtiB,KAAKsiB,cAAgBtiB,KAAK8b,KAAK9b,KAAK4b,MAAM,CAE3F;GAAI2G,GAASviB,KAAKkhB,eAAexO,EAAK1S,KAAKsb,KAM3C,IALAtb,KAAK+b,KAA6BvV,SAArBxG,KAAKwiB,YAA6BxiB,KAAKwiB,YAAcD,EAAOzW,IACzE9L,KAAKic,KAA6BzV,SAArBxG,KAAKyiB,YAA6BziB,KAAKyiB,YAAcF,EAAO7V,IACrE1M,KAAKic,MAAQjc,KAAK+b,OAAM/b,KAAKic,KAAOjc,KAAK+b,KAAO,GACpD/b,KAAKgc,MAA+BxV,SAAtBxG,KAAK0iB,aAA8B1iB,KAAK0iB,cAAgB1iB,KAAKic,KAAKjc,KAAK+b,MAAM,EAErEvV,SAAlBxG,KAAKub,SAAwB,CAC/B,GAAIoH,GA
 Aa3iB,KAAKkhB,eAAexO,EAAK1S,KAAKub,SAC/Cvb,MAAKkc,SAAqC1V,SAAzBxG,KAAK4iB,gBAAiC5iB,KAAK4iB,gBAAkBD,EAAW7W,IACzF9L,KAAKmc,SAAqC3V,SAAzBxG,KAAK6iB,gBAAiC7iB,KAAK6iB,gBAAkBF,EAAWjW,IACrF1M,KAAKmc,UAAYnc,KAAKkc,WAAUlc,KAAKmc,SAAWnc,KAAKkc,SAAW,GAItElc,KAAKgd,eAUPhc,EAAQmS,UAAU2P,eAAiB,SAAUpQ,GAE3C,GAAIX,GAAGC,EAAGxM,EAAG0X,EAAG6F,EAAK7Q,EAEjBiJ,IAEJ,IAAInb,KAAKiN,QAAUjM,EAAQsZ,MAAMkG,MAC/BxgB,KAAKiN,QAAUjM,EAAQsZ,MAAMoG,QAAS,CAKtC,GAAIkB,MACAE,IACJ,KAAKtc,EAAI,EAAGA,EAAIxF,KAAKyU,gBAAgB/B,GAAOlN,IAC1CuM,EAAIW,EAAKlN,GAAGxF,KAAKob,OAAS,EAC1BpJ,EAAIU,EAAKlN,GAAGxF,KAAKqb,OAAS,EAED,KAArBuG,EAAMjb,QAAQoL,IAChB6P,EAAM5Z,KAAK+J,GAEY,KAArB+P,EAAMnb,QAAQqL,IAChB8P,EAAM9Z,KAAKgK,EAIf,IAAIgR,GAAa,SAAUzd,EAAGa,GAC5B,MAAOb,GAAIa,EAEbwb,GAAM1L,KAAK8M,GACXlB,EAAM5L,KAAK8M,EAGX,IAAIC,KACJ,KAAKzd,EAAI,EAAGA,EAAIkN,EAAK/M,OAAQH,IAAK,CAChCuM,EAAIW,EAAKlN,GAAGxF,KAAKob,OAAS,EAC1BpJ,EAAIU,EAAKlN,GAAGxF,KAAKqb,OAAS,EAC1B6B,EAAIxK,EAAKlN,GAAGxF,KAAKsb,OAAS,CAE1B,IAAI4H,GAAStB,EAAMjb,QAAQoL,GACvBoR,EAASrB,EA
 AMnb,QAAQqL,EAEAxL,UAAvByc,EAAWC,KACbD,EAAWC,MAGb,IAAI1F,GAAU,GAAInc,EAClBmc,GAAQzL,EAAIA,EACZyL,EAAQxL,EAAIA,EACZwL,EAAQN,EAAIA,EAEZ6F,KACAA,EAAI7Q,MAAQsL,EACZuF,EAAIK,MAAQ5c,OACZuc,EAAIM,OAAS7c,OACbuc,EAAIO,OAAS,GAAIjiB,GAAQ0Q,EAAGC,EAAGhS,KAAK+b,MAEpCkH,EAAWC,GAAQC,GAAUJ,EAE7B5H,EAAWnT,KAAK+a,GAIlB,IAAKhR,EAAI,EAAGA,EAAIkR,EAAWtd,OAAQoM,IACjC,IAAKC,EAAI,EAAGA,EAAIiR,EAAWlR,GAAGpM,OAAQqM,IAChCiR,EAAWlR,GAAGC,KAChBiR,EAAWlR,GAAGC,GAAGuR,WAAcxR,EAAIkR,EAAWtd,OAAO,EAAKsd,EAAWlR,EAAE,GAAGC,GAAKxL,OAC/Eyc,EAAWlR,GAAGC,GAAGwR,SAAcxR,EAAIiR,EAAWlR,GAAGpM,OAAO,EAAKsd,EAAWlR,GAAGC,EAAE,GAAKxL,OAClFyc,EAAWlR,GAAGC,GAAGyR,WACd1R,EAAIkR,EAAWtd,OAAO,GAAKqM,EAAIiR,EAAWlR,GAAGpM,OAAO,EACnDsd,EAAWlR,EAAE,GAAGC,EAAE,GAClBxL,YAOV,KAAKhB,EAAI,EAAGA,EAAIkN,EAAK/M,OAAQH,IAC3B0M,EAAQ,GAAI7Q,GACZ6Q,EAAMH,EAAIW,EAAKlN,GAAGxF,KAAKob,OAAS,EAChClJ,EAAMF,EAAIU,EAAKlN,GAAGxF,KAAKqb,OAAS,EAChCnJ,EAAMgL,EAAIxK,EAAKlN,GAAGxF,KAAKsb,OAAS,EAEV9U,SAAlBxG,KAAKub,WACPrJ,EAAM7K,MAAQqL,EAAKlN,GAAGxF,KAAKub,WAAa,GAG1CwH
 ,KACAA,EAAI7Q,MAAQA,EACZ6Q,EAAIO,OAAS,GAAIjiB,GAAQ6Q,EAAMH,EAAGG,EAAMF,EAAGhS,KAAK+b,MAChDgH,EAAIK,MAAQ5c,OACZuc,EAAIM,OAAS7c,OAEb2U,EAAWnT,KAAK+a,EAIpB,OAAO5H,IASTna,EAAQmS,UAAU9E,OAAS,WAEzB,KAAOrO,KAAKyZ,iBAAiBiK,iBAC3B1jB,KAAKyZ,iBAAiBtI,YAAYnR,KAAKyZ,iBAAiBkK,WAG1D3jB,MAAKsf,MAAQ/N,SAASM,cAAc,OACpC7R,KAAKsf,MAAMrS,MAAM2W,SAAW,WAC5B5jB,KAAKsf,MAAMrS,MAAM4W,SAAW,SAG5B7jB,KAAKsf,MAAMC,OAAShO,SAASM,cAAe,UAC5C7R,KAAKsf,MAAMC,OAAOtS,MAAM2W,SAAW,WACnC5jB,KAAKsf,MAAM7N,YAAYzR,KAAKsf,MAAMC,OAGhC,IAAIuE,GAAWvS,SAASM,cAAe,MACvCiS,GAAS7W,MAAM9B,MAAQ,MACvB2Y,EAAS7W,MAAM8W,WAAc,OAC7BD,EAAS7W,MAAM+W,QAAW,OAC1BF,EAASG,UAAa,mDACtBjkB,KAAKsf,MAAMC,OAAO9N,YAAYqS,GAGhC9jB,KAAKsf,MAAM3L,OAASpC,SAASM,cAAe,OAC5C7R,KAAKsf,MAAM3L,OAAO1G,MAAM2W,SAAW,WACnC5jB,KAAKsf,MAAM3L,OAAO1G,MAAMqW,OAAS,MACjCtjB,KAAKsf,MAAM3L,OAAO1G,MAAMxF,KAAO,MAC/BzH,KAAKsf,MAAM3L,OAAO1G,MAAMsF,MAAQ,OAChCvS,KAAKsf,MAAM7N,YAAYzR,KAAKsf,MAAM3L,OAGlC,IAAIQ,GAAKnU,KACLkkB,EAAc,SAAU5a,GAAQ6K,EAAGgQ,aAAa7a,IAChD8a,EAAe,SAAU9a,GAAQ6K,EAAG
 kQ,cAAc/a,IAClDgb,EAAe,SAAUhb,GAAQ6K,EAAGoQ,SAASjb,IAC7Ckb,EAAY,SAAUlb,GAAQ6K,EAAGsQ,WAAWnb,GAGhD3I,GAAKgI,iBAAiB3I,KAAKsf,MAAMC,OAAQ,UAAWmF,WACpD/jB,EAAKgI,iBAAiB3I,KAAKsf,MAAMC,OAAQ,YAAa2E,GACtDvjB,EAAKgI,iBAAiB3I,KAAKsf,MAAMC,OAAQ,aAAc6E,GACvDzjB,EAAKgI,iBAAiB3I,KAAKsf,MAAMC,OAAQ,aAAc+E,GACv

<TRUNCATED>

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