From bdd4024e20fdfb0accb6121a68780ce3a0c218c0 Mon Sep 17 00:00:00 2001 From: mjsax Date: Sat, 26 Sep 2015 13:14:43 +0200 Subject: [PATCH] [FLINK-2111] Add "stop" signal to cleanly shutdown streaming jobs - added JobType to JobGraph and ExecutionGraph - added interface Stoppable, applied to SourceStreamTask - added STOP signal logic to JobManager, TaskManager, ExecutionGraph - extended Client to support stop - extended Cli frontend, JobManager frontend - updated documenation Fix JobManagerTest.testStopSignal and testStopSignalFail The StoppableInvokable could not be instantiated by Task because it was declared as a private class. Adds additional checks to verify that the stop signal behaves correctly. Auto-detect if job is stoppable A job is stoppable iff all sources are stoppable - Replace JobType by stoppable flag - Add StoppableFunction and StoppableInvokable to support the optional stop operation - added REST get/delete test (no extra YARN test -- think not required as get/delete is both tested) - bug fix: job got canceld instead of stopped in web interface - Add StoppingException - Allow to stop jobs when they are not in state RUNNING Second round of Till's comments --- docs/apis/cli.md | 14 + .../org/apache/flink/client/CliFrontend.java | 86 +- .../flink/client/cli/CliFrontendParser.java | 41 +- .../apache/flink/client/cli/StopOptions.java | 37 + .../apache/flink/client/program/Client.java | 59 +- .../client/program/ProgramStopException.java | 53 + .../flink/client/CliFrontendStopTest.java | 159 ++ .../apache/flink/storm/api/FlinkClient.java | 6 +- .../flink/storm/wrappers/SpoutWrapper.java | 13 +- .../common/functions/StoppableFunction.java | 33 + flink-runtime-web/pom.xml | 14 + .../runtime/webmonitor/WebRuntimeMonitor.java | 11 +- .../handlers/JobCancellationHandler.java | 3 + .../handlers/JobDetailsHandler.java | 1 + .../handlers/JobStoppingHandler.java | 49 + .../webmonitor/testutils/HttpTestClient.java | 19 + .../web-dashboard/app/partials/jobs/job.jade | 4 + .../app/scripts/modules/jobs/jobs.ctrl.coffee | 5 + .../app/scripts/modules/jobs/jobs.svc.coffee | 5 + .../web-dashboard/web/css/vendor.css | 1 + .../web-dashboard/web/js/vendor.js | 2053 ++++++++++------- .../web-dashboard/web/partials/jobs/job.html | 1 + .../flink/runtime/StoppingException.java | 35 + .../runtime/executiongraph/Execution.java | 93 +- .../executiongraph/ExecutionGraph.java | 93 +- .../executiongraph/ExecutionVertex.java | 5 +- .../flink/runtime/jobgraph/JobGraph.java | 1 + .../flink/runtime/jobgraph/JobVertex.java | 85 +- .../runtime/jobgraph/tasks/StoppableTask.java | 25 + .../flink/runtime/taskmanager/Task.java | 60 +- .../flink/runtime/jobmanager/JobManager.scala | 27 + .../runtime/messages/JobManagerMessages.scala | 25 + .../messages/TaskControlMessages.scala | 9 + .../runtime/taskmanager/TaskManager.scala | 22 + .../ExecutionGraphConstructionTest.java | 97 +- .../ExecutionGraphDeploymentTest.java | 27 +- .../ExecutionGraphSignalsTest.java | 224 ++ .../ExecutionGraphTestUtils.java | 12 +- .../ExecutionStateProgressTest.java | 173 +- .../ExecutionVertexStopTest.java | 132 ++ .../executiongraph/LocalInputSplitsTest.java | 25 +- .../executiongraph/PointwisePatternTest.java | 84 +- .../runtime/jobmanager/JobManagerTest.java | 107 +- .../runtime/taskmanager/TaskManagerTest.java | 131 +- .../runtime/taskmanager/TaskStopTest.java | 114 + .../runtime/testutils/StoppableInvokable.java | 32 + .../StreamExecutionEnvironment.java | 9 +- .../streaming/api/graph/StreamGraph.java | 7 +- .../api/operators/StoppableStreamSource.java | 51 + .../tasks/StoppableSourceStreamTask.java | 33 + .../runtime/tasks/SourceStreamTaskTest.java | 53 +- flink-tests/pom.xml | 16 + .../flink/test/web/WebFrontendITCase.java | 89 +- 53 files changed, 3389 insertions(+), 1174 deletions(-) create mode 100644 flink-clients/src/main/java/org/apache/flink/client/cli/StopOptions.java create mode 100644 flink-clients/src/main/java/org/apache/flink/client/program/ProgramStopException.java create mode 100644 flink-clients/src/test/java/org/apache/flink/client/CliFrontendStopTest.java create mode 100644 flink-core/src/main/java/org/apache/flink/api/common/functions/StoppableFunction.java create mode 100644 flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobStoppingHandler.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/StoppingException.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/StoppableTask.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSignalsTest.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexStopTest.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskStopTest.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/testutils/StoppableInvokable.java create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StoppableStreamSource.java create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StoppableSourceStreamTask.java diff --git a/docs/apis/cli.md b/docs/apis/cli.md index c9145bc36db..421ed9407fc 100644 --- a/docs/apis/cli.md +++ b/docs/apis/cli.md @@ -108,6 +108,10 @@ The command line can be used to ./bin/flink cancel +- Stop a job (streaming jobs only): + + ./bin/flink stop + ### Savepoints [Savepoints]({{site.baseurl}}/apis/streaming/savepoints.html) are controlled via the command line client: @@ -248,6 +252,16 @@ Action "cancel" cancels a running program. configuration. +Action "stop" stops a running program (streaming jobs only). + + Syntax: stop [OPTIONS] + "stop" action options: + -m,--jobmanager Address of the JobManager (master) to which + to connect. Use this flag to connect to a + different JobManager than the one specified + in the configuration. + + Action "savepoint" triggers savepoints for a running job or disposes existing ones. Syntax: savepoint [OPTIONS] diff --git a/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java b/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java index da91bcad8f4..98bf056c71e 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java +++ b/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java @@ -36,6 +36,7 @@ import org.apache.flink.client.cli.ListOptions; import org.apache.flink.client.cli.ProgramOptions; import org.apache.flink.client.cli.RunOptions; import org.apache.flink.client.cli.SavepointOptions; +import org.apache.flink.client.cli.StopOptions; import org.apache.flink.client.program.Client; import org.apache.flink.client.program.PackagedProgram; import org.apache.flink.client.program.ProgramInvocationException; @@ -56,7 +57,10 @@ import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; import org.apache.flink.runtime.messages.JobManagerMessages; import org.apache.flink.runtime.messages.JobManagerMessages.CancelJob; +import org.apache.flink.runtime.messages.JobManagerMessages.CancellationFailure; import org.apache.flink.runtime.messages.JobManagerMessages.RunningJobsStatus; +import org.apache.flink.runtime.messages.JobManagerMessages.StopJob; +import org.apache.flink.runtime.messages.JobManagerMessages.StoppingFailure; import org.apache.flink.runtime.messages.JobManagerMessages.TriggerSavepoint; import org.apache.flink.runtime.messages.JobManagerMessages.TriggerSavepointSuccess; import org.apache.flink.runtime.security.SecurityUtils; @@ -110,6 +114,7 @@ public class CliFrontend { public static final String ACTION_INFO = "info"; private static final String ACTION_LIST = "list"; private static final String ACTION_CANCEL = "cancel"; + private static final String ACTION_STOP = "stop"; private static final String ACTION_SAVEPOINT = "savepoint"; // config dir parameters @@ -290,9 +295,6 @@ public class CliFrontend { catch (FileNotFoundException e) { return handleArgException(e); } - catch (ProgramInvocationException e) { - return handleError(e); - } catch (Throwable t) { return handleError(t); } @@ -362,7 +364,7 @@ public class CliFrontend { /** * Executes the info action. * - * @param args Command line arguments for the info action. + * @param args Command line arguments for the info action. */ protected int info(String[] args) { LOG.info("Running 'info' command."); @@ -567,6 +569,65 @@ public class CliFrontend { } } + /** + * Executes the STOP action. + * + * @param args Command line arguments for the stop action. + */ + protected int stop(String[] args) { + LOG.info("Running 'stop' command."); + + StopOptions options; + try { + options = CliFrontendParser.parseStopCommand(args); + } + catch (CliArgsException e) { + return handleArgException(e); + } + catch (Throwable t) { + return handleError(t); + } + + // evaluate help flag + if (options.isPrintHelp()) { + CliFrontendParser.printHelpForStop(); + return 0; + } + + String[] stopArgs = options.getArgs(); + JobID jobId; + + if (stopArgs.length > 0) { + String jobIdString = stopArgs[0]; + try { + jobId = new JobID(StringUtils.hexStringToByte(jobIdString)); + } + catch (Exception e) { + return handleError(e); + } + } + else { + return handleArgException(new CliArgsException("Missing JobID")); + } + + try { + ActorGateway jobManager = getJobManagerGateway(options); + Future response = jobManager.ask(new StopJob(jobId), clientTimeout); + + final Object rc = Await.result(response, clientTimeout); + + if (rc instanceof StoppingFailure) { + throw new Exception("Stopping the job with ID " + jobId + " failed.", + ((StoppingFailure) rc).cause()); + } + + return 0; + } + catch (Throwable t) { + return handleError(t); + } + } + /** * Executes the CANCEL action. * @@ -616,13 +677,14 @@ public class CliFrontend { ActorGateway jobManager = getJobManagerGateway(options); Future response = jobManager.ask(new CancelJob(jobId), clientTimeout); - try { - Await.result(response, clientTimeout); - return 0; - } - catch (Exception e) { - throw new Exception("Canceling the job with ID " + jobId + " failed.", e); + final Object rc = Await.result(response, clientTimeout); + + if (rc instanceof CancellationFailure) { + throw new Exception("Canceling the job with ID " + jobId + " failed.", + ((CancellationFailure) rc).cause()); } + + return 0; } catch (Throwable t) { return handleError(t); @@ -1123,6 +1185,8 @@ public class CliFrontend { return info(params); case ACTION_CANCEL: return cancel(params); + case ACTION_STOP: + return stop(params); case ACTION_SAVEPOINT: return savepoint(params); case "-h": @@ -1139,7 +1203,7 @@ public class CliFrontend { default: System.out.printf("\"%s\" is not a valid action.\n", action); System.out.println(); - System.out.println("Valid actions are \"run\", \"list\", \"info\", or \"cancel\"."); + System.out.println("Valid actions are \"run\", \"list\", \"info\", \"stop\", or \"cancel\"."); System.out.println(); System.out.println("Specify the version option (-v or --version) to print Flink version."); System.out.println(); diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java index 07d409e3edf..2ac53d24246 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java +++ b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java @@ -39,13 +39,13 @@ public class CliFrontendParser { static final Option HELP_OPTION = new Option("h", "help", false, - "Show the help message for the CLI Frontend or the action."); + "Show the help message for the CLI Frontend or the action."); static final Option JAR_OPTION = new Option("j", "jarfile", true, "Flink program JAR file."); public static final Option CLASS_OPTION = new Option("c", "class", true, "Class with the program entry point (\"main\" method or \"getPlan()\" method. Only needed if the " + - "JAR file does not specify the class in its manifest."); + "JAR file does not specify the class in its manifest."); static final Option CLASSPATH_OPTION = new Option("C", "classpath", true, "Adds a URL to each user code " + "classloader on all nodes in the cluster. The paths must specify a protocol (e.g. file://) and be " + @@ -55,7 +55,7 @@ public class CliFrontendParser { static final Option PARALLELISM_OPTION = new Option("p", "parallelism", true, "The parallelism with which to run the program. Optional flag to override the default value " + - "specified in the configuration."); + "specified in the configuration."); static final Option LOGGING_OPTION = new Option("q", "sysoutLogging", false, "If present, " + "supress logging output to standard out."); @@ -67,9 +67,9 @@ public class CliFrontendParser { "Program arguments. Arguments can also be added without -a, simply as trailing parameters."); static final Option ADDRESS_OPTION = new Option("m", "jobmanager", true, - "Address of the JobManager (master) to which to connect. Specify '" + CliFrontend.YARN_DEPLOY_JOBMANAGER - + "' as the JobManager to deploy a YARN cluster for the job. Use this flag to connect to a " + - "different JobManager than the one specified in the configuration."); + "Address of the JobManager (master) to which to connect. Specify '" + CliFrontend.YARN_DEPLOY_JOBMANAGER + + "' as the JobManager to deploy a YARN cluster for the job. Use this flag to connect to a " + + "different JobManager than the one specified in the configuration."); static final Option SAVEPOINT_PATH_OPTION = new Option("s", "fromSavepoint", true, "Path to a savepoint to reset the job back to (for example file:///flink/savepoint-1537)."); @@ -123,6 +123,7 @@ public class CliFrontendParser { private static final Options INFO_OPTIONS = getInfoOptions(buildGeneralOptions(new Options())); private static final Options LIST_OPTIONS = getListOptions(buildGeneralOptions(new Options())); private static final Options CANCEL_OPTIONS = getCancelOptions(buildGeneralOptions(new Options())); + private static final Options STOP_OPTIONS = getStopOptions(buildGeneralOptions(new Options())); private static final Options SAVEPOINT_OPTIONS = getSavepointOptions(buildGeneralOptions(new Options())); private static Options buildGeneralOptions(Options options) { @@ -197,6 +198,11 @@ public class CliFrontendParser { return options; } + private static Options getStopOptions(Options options) { + options = getJobManagerAddressOption(options); + return options; + } + private static Options getSavepointOptions(Options options) { options = getJobManagerAddressOption(options); options.addOption(SAVEPOINT_DISPOSE_OPTION); @@ -218,6 +224,7 @@ public class CliFrontendParser { printHelpForRun(); printHelpForInfo(); printHelpForList(); + printHelpForStop(); printHelpForCancel(); printHelpForSavepoint(); @@ -264,6 +271,18 @@ public class CliFrontendParser { System.out.println(); } + public static void printHelpForStop() { + HelpFormatter formatter = new HelpFormatter(); + formatter.setLeftPadding(5); + formatter.setWidth(80); + + System.out.println("\nAction \"stop\" stops a running program (streaming jobs only)."); + System.out.println("\n Syntax: stop [OPTIONS] "); + formatter.setSyntaxPrefix(" \"stop\" action options:"); + formatter.printHelp(" ", getStopOptions(new Options())); + System.out.println(); + } + public static void printHelpForCancel() { HelpFormatter formatter = new HelpFormatter(); formatter.setLeftPadding(5); @@ -325,6 +344,16 @@ public class CliFrontendParser { } } + public static StopOptions parseStopCommand(String[] args) throws CliArgsException { + try { + PosixParser parser = new PosixParser(); + CommandLine line = parser.parse(STOP_OPTIONS, args, false); + return new StopOptions(line); + } catch (ParseException e) { + throw new CliArgsException(e.getMessage()); + } + } + public static SavepointOptions parseSavepointCommand(String[] args) throws CliArgsException { try { PosixParser parser = new PosixParser(); diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/StopOptions.java b/flink-clients/src/main/java/org/apache/flink/client/cli/StopOptions.java new file mode 100644 index 00000000000..7f246c83d29 --- /dev/null +++ b/flink-clients/src/main/java/org/apache/flink/client/cli/StopOptions.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.client.cli; + +import org.apache.commons.cli.CommandLine; + +/** + * Command line options for the STOP command + */ +public class StopOptions extends CommandLineOptions { + + private final String[] args; + + public StopOptions(CommandLine line) { + super(line); + this.args = line.getArgs(); + } + + public String[] getArgs() { + return args == null ? new String[0] : args; + } +} diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/Client.java b/flink-clients/src/main/java/org/apache/flink/client/program/Client.java index 452710c8325..999b461f4f5 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/Client.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/Client.java @@ -29,7 +29,6 @@ import com.google.common.base.Preconditions; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobSubmissionResult; import org.apache.flink.api.common.accumulators.AccumulatorHelper; -import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.Plan; import org.apache.flink.optimizer.CompilerException; @@ -43,16 +42,17 @@ import org.apache.flink.optimizer.plandump.PlanJSONDumpGenerator; import org.apache.flink.optimizer.plantranslate.JobGraphGenerator; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.client.JobClient; import org.apache.flink.runtime.client.JobExecutionException; import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; -import org.apache.flink.runtime.messages.JobManagerMessages; -import org.apache.flink.runtime.util.LeaderRetrievalUtils; import org.apache.flink.runtime.messages.accumulators.AccumulatorResultsErroneous; import org.apache.flink.runtime.messages.accumulators.AccumulatorResultsFound; import org.apache.flink.runtime.messages.accumulators.RequestAccumulatorResults; +import org.apache.flink.runtime.messages.JobManagerMessages; +import org.apache.flink.runtime.util.LeaderRetrievalUtils; import org.apache.flink.util.SerializedValue; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -413,25 +413,60 @@ public class Client { * @throws Exception In case an error occurred. */ public void cancel(JobID jobId) throws Exception { - ActorGateway jobManagerGateway = getJobManagerGateway(); + final ActorGateway jobManagerGateway = getJobManagerGateway(); - Future response; + final Future response; try { response = jobManagerGateway.ask(new JobManagerMessages.CancelJob(jobId), timeout); - } catch (Exception e) { + } catch (final Exception e) { throw new ProgramInvocationException("Failed to query the job manager gateway.", e); } - Object result = Await.result(response, timeout); + final Object result = Await.result(response, timeout); if (result instanceof JobManagerMessages.CancellationSuccess) { - LOG.debug("Job cancellation with ID " + jobId + " succeeded."); + LOG.info("Job cancellation with ID " + jobId + " succeeded."); } else if (result instanceof JobManagerMessages.CancellationFailure) { - Throwable t = ((JobManagerMessages.CancellationFailure) result).cause(); - LOG.debug("Job cancellation with ID " + jobId + " failed.", t); + final Throwable t = ((JobManagerMessages.CancellationFailure) result).cause(); + LOG.info("Job cancellation with ID " + jobId + " failed.", t); throw new Exception("Failed to cancel the job because of \n" + t.getMessage()); } else { - throw new Exception("Unknown message received while cancelling."); + throw new Exception("Unknown message received while cancelling: " + result.getClass().getName()); + } + } + + /** + * Stops a program on Flink cluster whose job-manager is configured in this client's configuration. + * Stopping works only for streaming programs. Be aware, that the program might continue to run for + * a while after sending the stop command, because after sources stopped to emit data all operators + * need to finish processing. + * + * @param jobId + * the job ID of the streaming program to stop + * @throws ProgramStopException + * If the job ID is invalid (ie, is unknown or refers to a batch job) or if sending the stop signal + * failed. That might be due to an I/O problem, ie, the job-manager is unreachable. + */ + public void stop(final JobID jobId) throws Exception { + final ActorGateway jobManagerGateway = getJobManagerGateway(); + + final Future response; + try { + response = jobManagerGateway.ask(new JobManagerMessages.StopJob(jobId), timeout); + } catch (final Exception e) { + throw new ProgramInvocationException("Failed to query the job manager gateway.", e); + } + + final Object result = Await.result(response, timeout); + + if (result instanceof JobManagerMessages.StoppingSuccess) { + LOG.info("Job stopping with ID " + jobId + " succeeded."); + } else if (result instanceof JobManagerMessages.StoppingFailure) { + final Throwable t = ((JobManagerMessages.StoppingFailure) result).cause(); + LOG.info("Job stopping with ID " + jobId + " failed.", t); + throw new Exception("Failed to stop the job because of \n" + t.getMessage()); + } else { + throw new Exception("Unknown message received while stopping: " + result.getClass().getName()); } } @@ -482,7 +517,7 @@ public class Client { // ------------------------------------------------------------------------ // Sessions // ------------------------------------------------------------------------ - + /** * Tells the JobManager to finish the session (job) defined by the given ID. * diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/ProgramStopException.java b/flink-clients/src/main/java/org/apache/flink/client/program/ProgramStopException.java new file mode 100644 index 00000000000..a1d8a9b1ba1 --- /dev/null +++ b/flink-clients/src/main/java/org/apache/flink/client/program/ProgramStopException.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package org.apache.flink.client.program; + +/** + * Exception used to indicate that there is an error during stopping of a Flink program. + */ +public class ProgramStopException extends Exception { + private static final long serialVersionUID = -906791331829704450L; + + /** + * Creates a ProgramStopException with the given message. + * + * @param message + * The message for the exception. + */ + public ProgramStopException(String message) { + super(message); + } + + /** + * Creates a ProgramStopException for the given exception. + * + * @param cause + * The exception that causes the program invocation to fail. + */ + public ProgramStopException(Throwable cause) { + super(cause); + } + + /** + * Creates a ProgramStopException for the given exception with an additional message. + * + * @param message + * The additional message. + * @param cause + * The exception that causes the program invocation to fail. + */ + public ProgramStopException(String message, Throwable cause) { + super(message, cause); + } +} diff --git a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendStopTest.java b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendStopTest.java new file mode 100644 index 00000000000..7c34c755530 --- /dev/null +++ b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendStopTest.java @@ -0,0 +1,159 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.client; + +import akka.actor.*; +import akka.testkit.JavaTestKit; + +import org.apache.flink.client.cli.CommandLineOptions; +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.akka.FlinkUntypedActor; +import org.apache.flink.runtime.instance.ActorGateway; +import org.apache.flink.runtime.instance.AkkaActorGateway; +import org.apache.flink.runtime.messages.JobManagerMessages; +import org.apache.flink.util.TestLogger; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.util.UUID; + +import static org.apache.flink.client.CliFrontendTestUtils.pipeSystemOutToNull; +import static org.apache.flink.client.CliFrontendTestUtils.clearGlobalConfiguration; +import static org.junit.Assert.*; + +public class CliFrontendStopTest extends TestLogger { + + private static ActorSystem actorSystem; + + @BeforeClass + public static void setup() { + pipeSystemOutToNull(); + clearGlobalConfiguration(); + actorSystem = ActorSystem.create("TestingActorSystem"); + } + + @AfterClass + public static void teardown() { + JavaTestKit.shutdownActorSystem(actorSystem); + actorSystem = null; + } + + @Test + public void testStop() throws Exception { + // test unrecognized option + { + String[] parameters = { "-v", "-l" }; + CliFrontend testFrontend = new CliFrontend(CliFrontendTestUtils.getConfigDir()); + int retCode = testFrontend.stop(parameters); + assertTrue(retCode != 0); + } + + // test missing job id + { + String[] parameters = {}; + CliFrontend testFrontend = new CliFrontend(CliFrontendTestUtils.getConfigDir()); + int retCode = testFrontend.stop(parameters); + assertTrue(retCode != 0); + } + + // test stop properly + { + JobID jid = new JobID(); + String jidString = jid.toString(); + + final UUID leaderSessionID = UUID.randomUUID(); + final ActorRef jm = actorSystem.actorOf(Props.create(CliJobManager.class, jid, leaderSessionID)); + + final ActorGateway gateway = new AkkaActorGateway(jm, leaderSessionID); + + String[] parameters = { jidString }; + StopTestCliFrontend testFrontend = new StopTestCliFrontend(gateway); + + int retCode = testFrontend.stop(parameters); + assertTrue(retCode == 0); + } + + // test unknown job Id + { + JobID jid1 = new JobID(); + JobID jid2 = new JobID(); + + final UUID leaderSessionID = UUID.randomUUID(); + final ActorRef jm = actorSystem.actorOf(Props.create(CliJobManager.class, jid1, leaderSessionID)); + + final ActorGateway gateway = new AkkaActorGateway(jm, leaderSessionID); + + String[] parameters = { jid2.toString() }; + StopTestCliFrontend testFrontend = new StopTestCliFrontend(gateway); + + assertTrue(testFrontend.stop(parameters) != 0); + } + } + + protected static final class StopTestCliFrontend extends CliFrontend { + + private ActorGateway jobManagerGateway; + + public StopTestCliFrontend(ActorGateway jobManagerGateway) throws Exception { + super(CliFrontendTestUtils.getConfigDir()); + this.jobManagerGateway = jobManagerGateway; + } + + @Override + public ActorGateway getJobManagerGateway(CommandLineOptions options) { + return jobManagerGateway; + } + } + + protected static final class CliJobManager extends FlinkUntypedActor { + private final JobID jobID; + private final UUID leaderSessionID; + + public CliJobManager(final JobID jobID, final UUID leaderSessionID) { + this.jobID = jobID; + this.leaderSessionID = leaderSessionID; + } + + @Override + public void handleMessage(Object message) { + if (message instanceof JobManagerMessages.RequestTotalNumberOfSlots$) { + getSender().tell(decorateMessage(1), getSelf()); + } else if (message instanceof JobManagerMessages.StopJob) { + JobManagerMessages.StopJob stopJob = (JobManagerMessages.StopJob) message; + + if (jobID != null && jobID.equals(stopJob.jobID())) { + getSender().tell(decorateMessage(new Status.Success(new Object())), getSelf()); + } else { + getSender() + .tell(decorateMessage(new Status.Failure(new Exception( + "Wrong or no JobID"))), getSelf()); + } + } else if (message instanceof JobManagerMessages.RequestRunningJobsStatus$) { + getSender().tell(decorateMessage(new JobManagerMessages.RunningJobsStatus()), + getSelf()); + } + } + + @Override + protected UUID getLeaderSessionID() { + return leaderSessionID; + } + } +} diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkClient.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkClient.java index 767f1b1fd9f..254134559c9 100644 --- a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkClient.java +++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkClient.java @@ -208,7 +208,7 @@ public class FlinkClient { final Client client; try { client = new Client(configuration); - } catch (IOException e) { + } catch (final IOException e) { throw new RuntimeException("Could not establish a connection to the job manager", e); } @@ -253,7 +253,7 @@ public class FlinkClient { } try { - client.cancel(jobId); + client.stop(jobId); } catch (final Exception e) { throw new RuntimeException("Cannot stop job.", e); } @@ -282,7 +282,7 @@ public class FlinkClient { final Future response = Patterns.ask(jobManager, JobManagerMessages.getRequestRunningJobsStatus(), new Timeout(askTimeout)); - Object result; + final Object result; try { result = Await.result(response, askTimeout); } catch (final Exception e) { diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/SpoutWrapper.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/SpoutWrapper.java index e077aeb7b27..66b05c6efb5 100644 --- a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/SpoutWrapper.java +++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/SpoutWrapper.java @@ -25,6 +25,7 @@ import backtype.storm.topology.IRichSpout; import com.google.common.collect.Sets; import org.apache.flink.api.common.ExecutionConfig.GlobalJobParameters; +import org.apache.flink.api.common.functions.StoppableFunction; import org.apache.flink.api.java.tuple.Tuple0; import org.apache.flink.api.java.tuple.Tuple1; import org.apache.flink.api.java.tuple.Tuple25; @@ -52,7 +53,7 @@ import java.util.HashMap; * is {@code null}, {@link SpoutWrapper} calls {@link IRichSpout#nextTuple() nextTuple()} method until * {@link FiniteSpout#reachedEnd()} returns true. */ -public final class SpoutWrapper extends RichParallelSourceFunction { +public final class SpoutWrapper extends RichParallelSourceFunction implements StoppableFunction { private static final long serialVersionUID = -218340336648247605L; /** Number of attributes of the spouts's output tuples per stream. */ @@ -299,6 +300,16 @@ public final class SpoutWrapper extends RichParallelSourceFunction { this.isRunning = false; } + /** + * {@inheritDoc} + *

+ * Sets the {@link #isRunning} flag to {@code false}. + */ + @Override + public void stop() { + this.isRunning = false; + } + @Override public void close() throws Exception { this.spout.close(); diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/StoppableFunction.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/StoppableFunction.java new file mode 100644 index 00000000000..a83b73fa3c4 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/StoppableFunction.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package org.apache.flink.api.common.functions; + +/** + * Must be implemented by stoppable functions, eg, source functions of streaming jobs. The method {@link #stop()} will + * be called when the job received the STOP signal. On this signal, the source function must stop emitting new data and + * terminate gracefully. + */ +public interface StoppableFunction { + /** + * Stops the source. In contrast to {@code cancel()} this is a request to the source function to shut down + * gracefully. Pending data can still be emitted and it is not required to stop immediately -- however, in the near + * future. The job will keep running until all emitted data is processed completely. + *

+ * Most streaming sources will have a while loop inside the {@code run()} method. You need to ensure that the source + * will break out of this loop. This can be achieved by having a volatile field "isRunning" that is checked in the + * loop and that is set to false in this method. + *

+ * The call to {@code stop()} should not block and not throw any exception. + */ + public void stop(); +} diff --git a/flink-runtime-web/pom.xml b/flink-runtime-web/pom.xml index 1a19fb185f8..27bbbd5b7bb 100644 --- a/flink-runtime-web/pom.xml +++ b/flink-runtime-web/pom.xml @@ -44,6 +44,20 @@ under the License. + + + org.apache.maven.plugins + maven-jar-plugin + true + + + + test-jar + + + + + diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java index 0b5de1ff3a4..67c0dab842b 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java @@ -53,6 +53,7 @@ import org.apache.flink.runtime.webmonitor.handlers.JobDetailsHandler; import org.apache.flink.runtime.webmonitor.handlers.JobExceptionsHandler; import org.apache.flink.runtime.webmonitor.handlers.JobManagerConfigHandler; import org.apache.flink.runtime.webmonitor.handlers.JobPlanHandler; +import org.apache.flink.runtime.webmonitor.handlers.JobStoppingHandler; import org.apache.flink.runtime.webmonitor.handlers.JobVertexAccumulatorsHandler; import org.apache.flink.runtime.webmonitor.handlers.JobVertexBackPressureHandler; import org.apache.flink.runtime.webmonitor.handlers.JobVertexCheckpointsHandler; @@ -245,8 +246,14 @@ public class WebRuntimeMonitor implements WebMonitor { // Cancel a job via GET (for proper integration with YARN this has to be performed via GET) .GET("/jobs/:jobid/yarn-cancel", handler(new JobCancellationHandler())) - // DELETE is the preferred way of cancelling a job (Rest-conform) - .DELETE("/jobs/:jobid", handler(new JobCancellationHandler())); + // DELETE is the preferred way of canceling a job (Rest-conform) + .DELETE("/jobs/:jobid/cancel", handler(new JobCancellationHandler())) + + // stop a job via GET (for proper integration with YARN this has to be performed via GET) + .GET("/jobs/:jobid/yarn-stop", handler(new JobStoppingHandler())) + + // DELETE is the preferred way of stopping a job (Rest-conform) + .DELETE("/jobs/:jobid/stop", handler(new JobStoppingHandler())); if (webSubmitAllow) { router diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationHandler.java index aae8b349048..b17acdc08ba 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationHandler.java @@ -25,6 +25,9 @@ import org.apache.flink.util.StringUtils; import java.util.Map; +/** + * Request handler for the CANCEL request. + */ public class JobCancellationHandler implements RequestHandler { @Override diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobDetailsHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobDetailsHandler.java index 4511c170e17..4f311285344 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobDetailsHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobDetailsHandler.java @@ -61,6 +61,7 @@ public class JobDetailsHandler extends AbstractExecutionGraphRequestHandler { // basic info gen.writeStringField("jid", graph.getJobID().toString()); gen.writeStringField("name", graph.getJobName()); + gen.writeBooleanField("isStoppable", graph.isStoppable()); gen.writeStringField("state", graph.getState().name()); // times and duration diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobStoppingHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobStoppingHandler.java new file mode 100644 index 00000000000..791790a241f --- /dev/null +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobStoppingHandler.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.webmonitor.handlers; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.instance.ActorGateway; +import org.apache.flink.runtime.messages.JobManagerMessages; +import org.apache.flink.util.StringUtils; + +import java.util.Map; + +/** + * Request handler for the STOP request. + */ +public class JobStoppingHandler implements RequestHandler { + + @Override + public String handleRequest(Map pathParams, Map queryParams, ActorGateway jobManager) throws Exception { + try { + JobID jobid = new JobID(StringUtils.hexStringToByte(pathParams.get("jobid"))); + if (jobManager != null) { + jobManager.tell(new JobManagerMessages.StopJob(jobid)); + return "{}"; + } + else { + throw new Exception("No connection to the leading JobManager."); + } + } + catch (Exception e) { + throw new Exception("Failed to stop the job with id: " + pathParams.get("jobid") + e.getMessage(), e); + } + } +} diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/testutils/HttpTestClient.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/testutils/HttpTestClient.java index d7d44573fc9..9a396d3d86f 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/testutils/HttpTestClient.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/testutils/HttpTestClient.java @@ -164,6 +164,25 @@ public class HttpTestClient implements AutoCloseable { sendRequest(getRequest, timeout); } + /** + * Sends a simple DELETE request to the given path. You only specify the $path part of + * http://$host:$host/$path. + * + * @param path The $path to DELETE (http://$host:$host/$path) + */ + public void sendDeleteRequest(String path, FiniteDuration timeout) throws TimeoutException, InterruptedException { + if (!path.startsWith("/")) { + path = "/" + path; + } + + HttpRequest getRequest = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, + HttpMethod.DELETE, path); + getRequest.headers().set(HttpHeaders.Names.HOST, host); + getRequest.headers().set(HttpHeaders.Names.CONNECTION, HttpHeaders.Values.CLOSE); + + sendRequest(getRequest, timeout); + } + /** * Returns the next available HTTP response. A call to this method blocks until a response * becomes available. diff --git a/flink-runtime-web/web-dashboard/app/partials/jobs/job.jade b/flink-runtime-web/web-dashboard/app/partials/jobs/job.jade index 5b541ae701f..fe3e0fce39d 100644 --- a/flink-runtime-web/web-dashboard/app/partials/jobs/job.jade +++ b/flink-runtime-web/web-dashboard/app/partials/jobs/job.jade @@ -43,6 +43,10 @@ nav.navbar.navbar-default.navbar-fixed-top.navbar-main(ng-if="job") span.navbar-info-button.btn.btn-default(ng-click="cancelJob($event)") | Cancel + .navbar-info.last.first(ng-if!="job.isStoppable && (job.state=='CREATED' || job.state=='RUNNING' || job.state=='RESTARTING')") + span.navbar-info-button.btn.btn-default(ng-click="stopJob($event)") + | Stop + nav.navbar.navbar-default.navbar-fixed-top.navbar-main-additional(ng-if="job") ul.nav.nav-tabs li(ui-sref-active='active') diff --git a/flink-runtime-web/web-dashboard/app/scripts/modules/jobs/jobs.ctrl.coffee b/flink-runtime-web/web-dashboard/app/scripts/modules/jobs/jobs.ctrl.coffee index f0ce892ccb1..931976da837 100644 --- a/flink-runtime-web/web-dashboard/app/scripts/modules/jobs/jobs.ctrl.coffee +++ b/flink-runtime-web/web-dashboard/app/scripts/modules/jobs/jobs.ctrl.coffee @@ -80,6 +80,11 @@ angular.module('flinkApp') JobsService.cancelJob($stateParams.jobid).then (data) -> {} + $scope.stopJob = (stopEvent) -> + angular.element(stopEvent.currentTarget).removeClass("btn").removeClass("btn-default").html('Stopping...') + JobsService.stopJob($stateParams.jobid).then (data) -> + {} + $scope.toggleHistory = -> $scope.showHistory = !$scope.showHistory diff --git a/flink-runtime-web/web-dashboard/app/scripts/modules/jobs/jobs.svc.coffee b/flink-runtime-web/web-dashboard/app/scripts/modules/jobs/jobs.svc.coffee index 65ae5cb3237..71f09218070 100644 --- a/flink-runtime-web/web-dashboard/app/scripts/modules/jobs/jobs.svc.coffee +++ b/flink-runtime-web/web-dashboard/app/scripts/modules/jobs/jobs.svc.coffee @@ -282,4 +282,9 @@ angular.module('flinkApp') # proper "DELETE jobs//" $http.get flinkConfig.jobServer + "jobs/" + jobid + "/yarn-cancel" + @stopJob = (jobid) -> + # uses the non REST-compliant GET yarn-cancel handler which is available in addition to the + # proper "DELETE jobs//" + $http.get "jobs/" + jobid + "/yarn-stop" + @ diff --git a/flink-runtime-web/web-dashboard/web/css/vendor.css b/flink-runtime-web/web-dashboard/web/css/vendor.css index 2a8d00ff1ff..e0c9259af9c 100644 --- a/flink-runtime-web/web-dashboard/web/css/vendor.css +++ b/flink-runtime-web/web-dashboard/web/css/vendor.css @@ -5902,6 +5902,7 @@ button.close { .modal-header { padding: 15px; border-bottom: 1px solid #e5e5e5; + min-height: 16.42857143px; } .modal-header .close { margin-top: -2px; diff --git a/flink-runtime-web/web-dashboard/web/js/vendor.js b/flink-runtime-web/web-dashboard/web/js/vendor.js index bebddae81a2..ab517636406 100644 --- a/flink-runtime-web/web-dashboard/web/js/vendor.js +++ b/flink-runtime-web/web-dashboard/web/js/vendor.js @@ -40551,7 +40551,7 @@ $provide.value("$locale", { if (key) { duration[key] = input; } else { - duration.milliseconds = input; + makeDateFromStringAndFormat(config); } } else if (!!(match = aspNetRegex.exec(input))) { sign = (match[1] === '-') ? -1 : 1; @@ -40689,6 +40689,9 @@ $provide.value("$locale", { function clone () { return new Moment(this); } + relativeTimeThresholds[threshold] = limit; + return true; + }; function isAfter (input, units) { var inputMs; @@ -40701,6 +40704,7 @@ $provide.value("$locale", { return inputMs < +this.clone().startOf(units); } } + ); function isBefore (input, units) { var inputMs; @@ -46414,7 +46418,7 @@ angular.module('ui.router.state') .filter('includedByState', $IncludedByStateFilter); })(window, window.angular); /*! - * Bootstrap v3.3.6 (http://getbootstrap.com) + * Bootstrap v3.3.5 (http://getbootstrap.com) * Copyright 2011-2015 Twitter, Inc. * Licensed under the MIT license */ @@ -46426,13 +46430,13 @@ if (typeof jQuery === 'undefined') { +function ($) { 'use strict'; var version = $.fn.jquery.split(' ')[0].split('.') - if ((version[0] < 2 && version[1] < 9) || (version[0] == 1 && version[1] == 9 && version[2] < 1) || (version[0] > 2)) { - throw new Error('Bootstrap\'s JavaScript requires jQuery version 1.9.1 or higher, but lower than version 3') + if ((version[0] < 2 && version[1] < 9) || (version[0] == 1 && version[1] == 9 && version[2] < 1)) { + throw new Error('Bootstrap\'s JavaScript requires jQuery version 1.9.1 or higher') } }(jQuery); /* ======================================================================== - * Bootstrap: transition.js v3.3.6 + * Bootstrap: transition.js v3.3.5 * http://getbootstrap.com/javascript/#transitions * ======================================================================== * Copyright 2011-2015 Twitter, Inc. @@ -46492,7 +46496,7 @@ if (typeof jQuery === 'undefined') { }(jQuery); /* ======================================================================== - * Bootstrap: alert.js v3.3.6 + * Bootstrap: alert.js v3.3.5 * http://getbootstrap.com/javascript/#alerts * ======================================================================== * Copyright 2011-2015 Twitter, Inc. @@ -46511,7 +46515,7 @@ if (typeof jQuery === 'undefined') { $(el).on('click', dismiss, this.close) } - Alert.VERSION = '3.3.6' + Alert.VERSION = '3.3.5' Alert.TRANSITION_DURATION = 150 @@ -46587,7 +46591,7 @@ if (typeof jQuery === 'undefined') { }(jQuery); /* ======================================================================== - * Bootstrap: button.js v3.3.6 + * Bootstrap: button.js v3.3.5 * http://getbootstrap.com/javascript/#buttons * ======================================================================== * Copyright 2011-2015 Twitter, Inc. @@ -46607,7 +46611,7 @@ if (typeof jQuery === 'undefined') { this.isLoading = false } - Button.VERSION = '3.3.6' + Button.VERSION = '3.3.5' Button.DEFAULTS = { loadingText: 'loading...' @@ -46708,7 +46712,7 @@ if (typeof jQuery === 'undefined') { }(jQuery); /* ======================================================================== - * Bootstrap: carousel.js v3.3.6 + * Bootstrap: carousel.js v3.3.5 * http://getbootstrap.com/javascript/#carousel * ======================================================================== * Copyright 2011-2015 Twitter, Inc. @@ -46739,7 +46743,7 @@ if (typeof jQuery === 'undefined') { .on('mouseleave.bs.carousel', $.proxy(this.cycle, this)) } - Carousel.VERSION = '3.3.6' + Carousel.VERSION = '3.3.5' Carousel.TRANSITION_DURATION = 600 @@ -46946,7 +46950,7 @@ if (typeof jQuery === 'undefined') { }(jQuery); /* ======================================================================== - * Bootstrap: collapse.js v3.3.6 + * Bootstrap: collapse.js v3.3.5 * http://getbootstrap.com/javascript/#collapse * ======================================================================== * Copyright 2011-2015 Twitter, Inc. @@ -46976,7 +46980,7 @@ if (typeof jQuery === 'undefined') { if (this.options.toggle) this.toggle() } - Collapse.VERSION = '3.3.6' + Collapse.VERSION = '3.3.5' Collapse.TRANSITION_DURATION = 350 @@ -47158,7 +47162,7 @@ if (typeof jQuery === 'undefined') { }(jQuery); /* ======================================================================== - * Bootstrap: dropdown.js v3.3.6 + * Bootstrap: dropdown.js v3.3.5 * http://getbootstrap.com/javascript/#dropdowns * ======================================================================== * Copyright 2011-2015 Twitter, Inc. @@ -47178,7 +47182,7 @@ if (typeof jQuery === 'undefined') { $(element).on('click.bs.dropdown', this.toggle) } - Dropdown.VERSION = '3.3.6' + Dropdown.VERSION = '3.3.5' function getParent($this) { var selector = $this.attr('data-target') @@ -47210,7 +47214,7 @@ if (typeof jQuery === 'undefined') { if (e.isDefaultPrevented()) return $this.attr('aria-expanded', 'false') - $parent.removeClass('open').trigger($.Event('hidden.bs.dropdown', relatedTarget)) + $parent.removeClass('open').trigger('hidden.bs.dropdown', relatedTarget) }) } @@ -47244,7 +47248,7 @@ if (typeof jQuery === 'undefined') { $parent .toggleClass('open') - .trigger($.Event('shown.bs.dropdown', relatedTarget)) + .trigger('shown.bs.dropdown', relatedTarget) } return false @@ -47324,7 +47328,7 @@ if (typeof jQuery === 'undefined') { }(jQuery); /* ======================================================================== - * Bootstrap: modal.js v3.3.6 + * Bootstrap: modal.js v3.3.5 * http://getbootstrap.com/javascript/#modals * ======================================================================== * Copyright 2011-2015 Twitter, Inc. @@ -47358,7 +47362,7 @@ if (typeof jQuery === 'undefined') { } } - Modal.VERSION = '3.3.6' + Modal.VERSION = '3.3.5' Modal.TRANSITION_DURATION = 300 Modal.BACKDROP_TRANSITION_DURATION = 150 @@ -47662,7 +47666,7 @@ if (typeof jQuery === 'undefined') { }(jQuery); /* ======================================================================== - * Bootstrap: tooltip.js v3.3.6 + * Bootstrap: tooltip.js v3.3.5 * http://getbootstrap.com/javascript/#tooltip * Inspired by the original jQuery.tipsy by Jason Frame * ======================================================================== @@ -47689,7 +47693,7 @@ if (typeof jQuery === 'undefined') { this.init('tooltip', element, options) } - Tooltip.VERSION = '3.3.6' + Tooltip.VERSION = '3.3.5' Tooltip.TRANSITION_DURATION = 150 @@ -48177,7 +48181,7 @@ if (typeof jQuery === 'undefined') { }(jQuery); /* ======================================================================== - * Bootstrap: popover.js v3.3.6 + * Bootstrap: popover.js v3.3.5 * http://getbootstrap.com/javascript/#popovers * ======================================================================== * Copyright 2011-2015 Twitter, Inc. @@ -48197,7 +48201,7 @@ if (typeof jQuery === 'undefined') { if (!$.fn.tooltip) throw new Error('Popover requires tooltip.js') - Popover.VERSION = '3.3.6' + Popover.VERSION = '3.3.5' Popover.DEFAULTS = $.extend({}, $.fn.tooltip.Constructor.DEFAULTS, { placement: 'right', @@ -48286,7 +48290,7 @@ if (typeof jQuery === 'undefined') { }(jQuery); /* ======================================================================== - * Bootstrap: scrollspy.js v3.3.6 + * Bootstrap: scrollspy.js v3.3.5 * http://getbootstrap.com/javascript/#scrollspy * ======================================================================== * Copyright 2011-2015 Twitter, Inc. @@ -48315,7 +48319,7 @@ if (typeof jQuery === 'undefined') { this.process() } - ScrollSpy.VERSION = '3.3.6' + ScrollSpy.VERSION = '3.3.5' ScrollSpy.DEFAULTS = { offset: 10 @@ -48459,7 +48463,7 @@ if (typeof jQuery === 'undefined') { }(jQuery); /* ======================================================================== - * Bootstrap: tab.js v3.3.6 + * Bootstrap: tab.js v3.3.5 * http://getbootstrap.com/javascript/#tabs * ======================================================================== * Copyright 2011-2015 Twitter, Inc. @@ -48479,7 +48483,7 @@ if (typeof jQuery === 'undefined') { // jscs:enable requireDollarBeforejQueryAssignment } - Tab.VERSION = '3.3.6' + Tab.VERSION = '3.3.5' Tab.TRANSITION_DURATION = 150 @@ -48615,7 +48619,7 @@ if (typeof jQuery === 'undefined') { }(jQuery); /* ======================================================================== - * Bootstrap: affix.js v3.3.6 + * Bootstrap: affix.js v3.3.5 * http://getbootstrap.com/javascript/#affix * ======================================================================== * Copyright 2011-2015 Twitter, Inc. @@ -48644,7 +48648,7 @@ if (typeof jQuery === 'undefined') { this.checkPosition() } - Affix.VERSION = '3.3.6' + Affix.VERSION = '3.3.5' Affix.RESET = 'affix affix-top affix-bottom' @@ -48779,7 +48783,7 @@ if (typeof jQuery === 'undefined') { !function() { var d3 = { - version: "3.5.12" + version: "3.5.9" }; var d3_arraySlice = [].slice, d3_array = function(list) { return d3_arraySlice.call(list); @@ -54936,7 +54940,7 @@ if (typeof jQuery === 'undefined') { index: di, startAngle: x0, endAngle: x, - value: groupSums[di] + value: (x - x0) / k }; x += padding; } @@ -55146,7 +55150,7 @@ if (typeof jQuery === 'undefined') { alpha = x; } else { timer.c = null, timer.t = NaN, timer = null; - event.end({ + event.start({ type: "end", alpha: alpha = 0 }); @@ -56344,9 +56348,7 @@ if (typeof jQuery === 'undefined') { return d3.rebind(scale, linear, "range", "rangeRound", "interpolate", "clamp"); } function d3_scale_linearNice(domain, m) { - d3_scale_nice(domain, d3_scale_niceStep(d3_scale_linearTickRange(domain, m)[2])); - d3_scale_nice(domain, d3_scale_niceStep(d3_scale_linearTickRange(domain, m)[2])); - return domain; + return d3_scale_nice(domain, d3_scale_niceStep(d3_scale_linearTickRange(domain, m)[2])); } function d3_scale_linearTickRange(domain, m) { if (m == null) m = 10; @@ -56448,11 +56450,10 @@ if (typeof jQuery === 'undefined') { scale.tickFormat = function(n, format) { if (!arguments.length) return d3_scale_logFormat; if (arguments.length < 2) format = d3_scale_logFormat; else if (typeof format !== "function") format = d3.format(format); - var k = Math.max(1, base * n / scale.ticks().length); + var k = Math.max(.1, n / scale.ticks().length), f = positive ? (e = 1e-12, Math.ceil) : (e = -1e-12, + Math.floor), e; return function(d) { - var i = d / pow(Math.round(log(d))); - if (i * base < base - .5) i *= base; - return i <= k ? format(d) : ""; + return d / pow(f(log(d) + e)) <= k ? format(d) : ""; }; }; scale.copy = function() { @@ -71166,115 +71167,561 @@ if (typeof jQuery === 'undefined') { })(this); -/** - * EvEmitter v1.0.1 - * Lil' event emitter - * MIT License +/*! + * EventEmitter v4.2.11 - git.io/ee + * Unlicense - http://unlicense.org/ + * Oliver Caldwell - http://oli.me.uk/ + * @preserve */ -/* jshint unused: true, undef: true, strict: true */ +;(function () { + 'use strict'; -( function( global, factory ) { - // universal module definition - /* jshint strict: false */ /* globals define, module */ - if ( typeof define == 'function' && define.amd ) { - // AMD - RequireJS - define( factory ); - } else if ( typeof module == 'object' && module.exports ) { - // CommonJS - Browserify, Webpack - module.exports = factory(); - } else { - // Browser globals - global.EvEmitter = factory(); - } + /** + * Class for managing events. + * Can be extended to provide event functionality in other classes. + * + * @class EventEmitter Manages event registering and emitting. + */ + function EventEmitter() {} -}( this, function() { + // Shortcuts to improve speed and size + var proto = EventEmitter.prototype; + var exports = this; + var originalGlobalValue = exports.EventEmitter; -"use strict"; + /** + * Finds the index of the listener for the event in its storage array. + * + * @param {Function[]} listeners Array of listeners to search through. + * @param {Function} listener Method to look for. + * @return {Number} Index of the specified listener, -1 if not found + * @api private + */ + function indexOfListener(listeners, listener) { + var i = listeners.length; + while (i--) { + if (listeners[i].listener === listener) { + return i; + } + } -function EvEmitter() {} + return -1; + } -var proto = EvEmitter.prototype; + /** + * Alias a method while keeping the context correct, to allow for overwriting of target method. + * + * @param {String} name The name of the target method. + * @return {Function} The aliased method + * @api private + */ + function alias(name) { + return function aliasClosure() { + return this[name].apply(this, arguments); + }; + } -proto.on = function( eventName, listener ) { - if ( !eventName || !listener ) { - return; - } - // set events hash - var events = this._events = this._events || {}; - // set listeners array - var listeners = events[ eventName ] = events[ eventName ] || []; - // only add once - if ( listeners.indexOf( listener ) == -1 ) { - listeners.push( listener ); - } + /** + * Returns the listener array for the specified event. + * Will initialise the event object and listener arrays if required. + * Will return an object if you use a regex search. The object contains keys for each matched event. So /ba[rz]/ might return an object containing bar and baz. But only if you have either defined them with defineEvent or added some listeners to them. + * Each property in the object response is an array of listener functions. + * + * @param {String|RegExp} evt Name of the event to return the listeners from. + * @return {Function[]|Object} All listener functions for the event. + */ + proto.getListeners = function getListeners(evt) { + var events = this._getEvents(); + var response; + var key; + + // Return a concatenated array of all matching events if + // the selector is a regular expression. + if (evt instanceof RegExp) { + response = {}; + for (key in events) { + if (events.hasOwnProperty(key) && evt.test(key)) { + response[key] = events[key]; + } + } + } + else { + response = events[evt] || (events[evt] = []); + } - return this; -}; + return response; + }; -proto.once = function( eventName, listener ) { - if ( !eventName || !listener ) { - return; - } - // add event - this.on( eventName, listener ); - // set once flag - // set onceEvents hash - var onceEvents = this._onceEvents = this._onceEvents || {}; - // set onceListeners array - var onceListeners = onceEvents[ eventName ] = onceEvents[ eventName ] || []; - // set flag - onceListeners[ listener ] = true; + /** + * Takes a list of listener objects and flattens it into a list of listener functions. + * + * @param {Object[]} listeners Raw listener objects. + * @return {Function[]} Just the listener functions. + */ + proto.flattenListeners = function flattenListeners(listeners) { + var flatListeners = []; + var i; - return this; -}; + for (i = 0; i < listeners.length; i += 1) { + flatListeners.push(listeners[i].listener); + } -proto.off = function( eventName, listener ) { - var listeners = this._events && this._events[ eventName ]; - if ( !listeners || !listeners.length ) { - return; - } - var index = listeners.indexOf( listener ); - if ( index != -1 ) { - listeners.splice( index, 1 ); - } + return flatListeners; + }; - return this; -}; + /** + * Fetches the requested listeners via getListeners but will always return the results inside an object. This is mainly for internal use but others may find it useful. + * + * @param {String|RegExp} evt Name of the event to return the listeners from. + * @return {Object} All listener functions for an event in an object. + */ + proto.getListenersAsObject = function getListenersAsObject(evt) { + var listeners = this.getListeners(evt); + var response; -proto.emitEvent = function( eventName, args ) { - var listeners = this._events && this._events[ eventName ]; - if ( !listeners || !listeners.length ) { - return; - } - var i = 0; - var listener = listeners[i]; - args = args || []; - // once stuff - var onceListeners = this._onceEvents && this._onceEvents[ eventName ]; + if (listeners instanceof Array) { + response = {}; + response[evt] = listeners; + } + + return response || listeners; + }; + + /** + * Adds a listener function to the specified event. + * The listener will not be added if it is a duplicate. + * If the listener returns true then it will be removed after it is called. + * If you pass a regular expression as the event name then the listener will be added to all events that match it. + * + * @param {String|RegExp} evt Name of the event to attach the listener to. + * @param {Function} listener Method to be called when the event is emitted. If the function returns true then it will be removed after calling. + * @return {Object} Current instance of EventEmitter for chaining. + */ + proto.addListener = function addListener(evt, listener) { + var listeners = this.getListenersAsObject(evt); + var listenerIsWrapped = typeof listener === 'object'; + var key; + + for (key in listeners) { + if (listeners.hasOwnProperty(key) && indexOfListener(listeners[key], listener) === -1) { + listeners[key].push(listenerIsWrapped ? listener : { + listener: listener, + once: false + }); + } + } + + return this; + }; + + /** + * Alias of addListener + */ + proto.on = alias('addListener'); + + /** + * Semi-alias of addListener. It will add a listener that will be + * automatically removed after its first execution. + * + * @param {String|RegExp} evt Name of the event to attach the listener to. + * @param {Function} listener Method to be called when the event is emitted. If the function returns true then it will be removed after calling. + * @return {Object} Current instance of EventEmitter for chaining. + */ + proto.addOnceListener = function addOnceListener(evt, listener) { + return this.addListener(evt, { + listener: listener, + once: true + }); + }; + + /** + * Alias of addOnceListener. + */ + proto.once = alias('addOnceListener'); + + /** + * Defines an event name. This is required if you want to use a regex to add a listener to multiple events at once. If you don't do this then how do you expect it to know what event to add to? Should it just add to every possible match for a regex? No. That is scary and bad. + * You need to tell it what event names should be matched by a regex. + * + * @param {String} evt Name of the event to create. + * @return {Object} Current instance of EventEmitter for chaining. + */ + proto.defineEvent = function defineEvent(evt) { + this.getListeners(evt); + return this; + }; + + /** + * Uses defineEvent to define multiple events. + * + * @param {String[]} evts An array of event names to define. + * @return {Object} Current instance of EventEmitter for chaining. + */ + proto.defineEvents = function defineEvents(evts) { + for (var i = 0; i < evts.length; i += 1) { + this.defineEvent(evts[i]); + } + return this; + }; + + /** + * Removes a listener function from the specified event. + * When passed a regular expression as the event name, it will remove the listener from all events that match it. + * + * @param {String|RegExp} evt Name of the event to remove the listener from. + * @param {Function} listener Method to remove from the event. + * @return {Object} Current instance of EventEmitter for chaining. + */ + proto.removeListener = function removeListener(evt, listener) { + var listeners = this.getListenersAsObject(evt); + var index; + var key; + + for (key in listeners) { + if (listeners.hasOwnProperty(key)) { + index = indexOfListener(listeners[key], listener); + + if (index !== -1) { + listeners[key].splice(index, 1); + } + } + } + + return this; + }; + + /** + * Alias of removeListener + */ + proto.off = alias('removeListener'); + + /** + * Adds listeners in bulk using the manipulateListeners method. + * If you pass an object as the second argument you can add to multiple events at once. The object should contain key value pairs of events and listeners or listener arrays. You can also pass it an event name and an array of listeners to be added. + * You can also pass it a regular expression to add the array of listeners to all events that match it. + * Yeah, this function does quite a bit. That's probably a bad thing. + * + * @param {String|Object|RegExp} evt An event name if you will pass an array of listeners next. An object if you wish to add to multiple events at once. + * @param {Function[]} [listeners] An optional array of listener functions to add. + * @return {Object} Current instance of EventEmitter for chaining. + */ + proto.addListeners = function addListeners(evt, listeners) { + // Pass through to manipulateListeners + return this.manipulateListeners(false, evt, listeners); + }; + + /** + * Removes listeners in bulk using the manipulateListeners method. + * If you pass an object as the second argument you can remove from multiple events at once. The object should contain key value pairs of events and listeners or listener arrays. + * You can also pass it an event name and an array of listeners to be removed. + * You can also pass it a regular expression to remove the listeners from all events that match it. + * + * @param {String|Object|RegExp} evt An event name if you will pass an array of listeners next. An object if you wish to remove from multiple events at once. + * @param {Function[]} [listeners] An optional array of listener functions to remove. + * @return {Object} Current instance of EventEmitter for chaining. + */ + proto.removeListeners = function removeListeners(evt, listeners) { + // Pass through to manipulateListeners + return this.manipulateListeners(true, evt, listeners); + }; + + /** + * Edits listeners in bulk. The addListeners and removeListeners methods both use this to do their job. You should really use those instead, this is a little lower level. + * The first argument will determine if the listeners are removed (true) or added (false). + * If you pass an object as the second argument you can add/remove from multiple events at once. The object should contain key value pairs of events and listeners or listener arrays. + * You can also pass it an event name and an array of listeners to be added/removed. + * You can also pass it a regular expression to manipulate the listeners of all events that match it. + * + * @param {Boolean} remove True if you want to remove listeners, false if you want to add. + * @param {String|Object|RegExp} evt An event name if you will pass an array of listeners next. An object if you wish to add/remove from multiple events at once. + * @param {Function[]} [listeners] An optional array of listener functions to add/remove. + * @return {Object} Current instance of EventEmitter for chaining. + */ + proto.manipulateListeners = function manipulateListeners(remove, evt, listeners) { + var i; + var value; + var single = remove ? this.removeListener : this.addListener; + var multiple = remove ? this.removeListeners : this.addListeners; + + // If evt is an object then pass each of its properties to this method + if (typeof evt === 'object' && !(evt instanceof RegExp)) { + for (i in evt) { + if (evt.hasOwnProperty(i) && (value = evt[i])) { + // Pass the single listener straight through to the singular method + if (typeof value === 'function') { + single.call(this, i, value); + } + else { + // Otherwise pass back to the multiple function + multiple.call(this, i, value); + } + } + } + } + else { + // So evt must be a string + // And listeners must be an array of listeners + // Loop over it and pass each one to the multiple method + i = listeners.length; + while (i--) { + single.call(this, evt, listeners[i]); + } + } + + return this; + }; + + /** + * Removes all listeners from a specified event. + * If you do not specify an event then all listeners will be removed. + * That means every event will be emptied. + * You can also pass a regex to remove all events that match it. + * + * @param {String|RegExp} [evt] Optional name of the event to remove all listeners for. Will remove from every event if not passed. + * @return {Object} Current instance of EventEmitter for chaining. + */ + proto.removeEvent = function removeEvent(evt) { + var type = typeof evt; + var events = this._getEvents(); + var key; + + // Remove different things depending on the state of evt + if (type === 'string') { + // Remove all listeners for the specified event + delete events[evt]; + } + else if (evt instanceof RegExp) { + // Remove all events matching the regex. + for (key in events) { + if (events.hasOwnProperty(key) && evt.test(key)) { + delete events[key]; + } + } + } + else { + // Remove all listeners in all events + delete this._events; + } + + return this; + }; + + /** + * Alias of removeEvent. + * + * Added to mirror the node API. + */ + proto.removeAllListeners = alias('removeEvent'); + + /** + * Emits an event of your choice. + * When emitted, every listener attached to that event will be executed. + * If you pass the optional argument array then those arguments will be passed to every listener upon execution. + * Because it uses `apply`, your array of arguments will be passed as if you wrote them out separately. + * So they will not arrive within the array on the other side, they will be separate. + * You can also pass a regular expression to emit to all events that match it. + * + * @param {String|RegExp} evt Name of the event to emit and execute listeners for. + * @param {Array} [args] Optional array of arguments to be passed to each listener. + * @return {Object} Current instance of EventEmitter for chaining. + */ + proto.emitEvent = function emitEvent(evt, args) { + var listenersMap = this.getListenersAsObject(evt); + var listeners; + var listener; + var i; + var key; + var response; + + for (key in listenersMap) { + if (listenersMap.hasOwnProperty(key)) { + listeners = listenersMap[key].slice(0); + i = listeners.length; + + while (i--) { + // If the listener returns true then it shall be removed from the event + // The function is executed either with a basic call or an apply if there is an args array + listener = listeners[i]; + + if (listener.once === true) { + this.removeListener(evt, listener.listener); + } + + response = listener.listener.apply(this, args || []); + + if (response === this._getOnceReturnValue()) { + this.removeListener(evt, listener.listener); + } + } + } + } + + return this; + }; + + /** + * Alias of emitEvent + */ + proto.trigger = alias('emitEvent'); + + /** + * Subtly different from emitEvent in that it will pass its arguments on to the listeners, as opposed to taking a single array of arguments to pass on. + * As with emitEvent, you can pass a regex in place of the event name to emit to all events that match it. + * + * @param {String|RegExp} evt Name of the event to emit and execute listeners for. + * @param {...*} Optional additional arguments to be passed to each listener. + * @return {Object} Current instance of EventEmitter for chaining. + */ + proto.emit = function emit(evt) { + var args = Array.prototype.slice.call(arguments, 1); + return this.emitEvent(evt, args); + }; + + /** + * Sets the current value to check against when executing listeners. If a + * listeners return value matches the one set here then it will be removed + * after execution. This value defaults to true. + * + * @param {*} value The new value to check for when executing listeners. + * @return {Object} Current instance of EventEmitter for chaining. + */ + proto.setOnceReturnValue = function setOnceReturnValue(value) { + this._onceReturnValue = value; + return this; + }; - while ( listener ) { - var isOnce = onceListeners && onceListeners[ listener ]; - if ( isOnce ) { - // remove listener - // remove before trigger to prevent recursion - this.off( eventName, listener ); - // unset once flag - delete onceListeners[ listener ]; + /** + * Fetches the current value to check against when executing listeners. If + * the listeners return value matches this one then it should be removed + * automatically. It will return true by default. + * + * @return {*|Boolean} The current value to check for or the default, true. + * @api private + */ + proto._getOnceReturnValue = function _getOnceReturnValue() { + if (this.hasOwnProperty('_onceReturnValue')) { + return this._onceReturnValue; + } + else { + return true; + } + }; + + /** + * Fetches the events object and creates one if required. + * + * @return {Object} The events storage object. + * @api private + */ + proto._getEvents = function _getEvents() { + return this._events || (this._events = {}); + }; + + /** + * Reverts the global {@link EventEmitter} to its previous value and returns a reference to this version. + * + * @return {Function} Non conflicting EventEmitter class. + */ + EventEmitter.noConflict = function noConflict() { + exports.EventEmitter = originalGlobalValue; + return EventEmitter; + }; + + // Expose the class either via AMD, CommonJS or the global object + if (typeof define === 'function' && define.amd) { + define(function () { + return EventEmitter; + }); } - // trigger listener - listener.apply( this, args ); - // get next listener - i += isOnce ? 0 : 1; - listener = listeners[i]; - } + else if (typeof module === 'object' && module.exports){ + module.exports = EventEmitter; + } + else { + exports.EventEmitter = EventEmitter; + } +}.call(this)); - return this; +/*! + * eventie v1.0.6 + * event binding helper + * eventie.bind( elem, 'click', myFn ) + * eventie.unbind( elem, 'click', myFn ) + * MIT license + */ + +/*jshint browser: true, undef: true, unused: true */ +/*global define: false, module: false */ + +( function( window ) { + +'use strict'; + +var docElem = document.documentElement; + +var bind = function() {}; + +function getIEEvent( obj ) { + var event = window.event; + // add event.target + event.target = event.target || event.srcElement || obj; + return event; +} + +if ( docElem.addEventListener ) { + bind = function( obj, type, fn ) { + obj.addEventListener( type, fn, false ); + }; +} else if ( docElem.attachEvent ) { + bind = function( obj, type, fn ) { + obj[ type + fn ] = fn.handleEvent ? + function() { + var event = getIEEvent( obj ); + fn.handleEvent.call( fn, event ); + } : + function() { + var event = getIEEvent( obj ); + fn.call( obj, event ); + }; + obj.attachEvent( "on" + type, obj[ type + fn ] ); + }; +} + +var unbind = function() {}; + +if ( docElem.removeEventListener ) { + unbind = function( obj, type, fn ) { + obj.removeEventListener( type, fn, false ); + }; +} else if ( docElem.detachEvent ) { + unbind = function( obj, type, fn ) { + obj.detachEvent( "on" + type, obj[ type + fn ] ); + try { + delete obj[ type + fn ]; + } catch ( err ) { + // can't delete window object properties + obj[ type + fn ] = undefined; + } + }; +} + +var eventie = { + bind: bind, + unbind: unbind }; -return EvEmitter; +// ----- module definition ----- // -})); +if ( typeof define === 'function' && define.amd ) { + // AMD + define( eventie ); +} else if ( typeof exports === 'object' ) { + // CommonJS + module.exports = eventie; +} else { + // browser global + window.eventie = eventie; +} /* angular-moment.js / v0.10.3 / (c) 2013, 2014, 2015 Uri Shaked / MIT Licence */ @@ -73148,7 +73595,7 @@ module.exports = '1.0.7'; },{}]},{},[1]); /*! - * imagesLoaded v4.1.0 + * imagesLoaded v3.2.0 * JavaScript is all like "You images are done yet or what?" * MIT License */ @@ -73161,21 +73608,24 @@ module.exports = '1.0.7'; if ( typeof define == 'function' && define.amd ) { // AMD define( [ - 'ev-emitter/ev-emitter' - ], function( EvEmitter ) { - return factory( window, EvEmitter ); + 'eventEmitter/EventEmitter', + 'eventie/eventie' + ], function( EventEmitter, eventie ) { + return factory( window, EventEmitter, eventie ); }); } else if ( typeof module == 'object' && module.exports ) { // CommonJS module.exports = factory( window, - require('ev-emitter') + require('wolfy87-eventemitter'), + require('eventie') ); } else { // browser global window.imagesLoaded = factory( window, - window.EvEmitter + window.EventEmitter, + window.eventie ); } @@ -73183,7 +73633,7 @@ module.exports = '1.0.7'; // -------------------------- factory -------------------------- // -function factory( window, EvEmitter ) { +function factory( window, EventEmitter, eventie ) { 'use strict'; @@ -73200,10 +73650,15 @@ function extend( a, b ) { return a; } +var objToString = Object.prototype.toString; +function isArray( obj ) { + return objToString.call( obj ) == '[object Array]'; +} + // turn element or nodeList into an array function makeArray( obj ) { var ary = []; - if ( Array.isArray( obj ) ) { + if ( isArray( obj ) ) { // use object if already an array ary = obj; } else if ( typeof obj.length == 'number' ) { @@ -73218,303 +73673,309 @@ function makeArray( obj ) { return ary; } -// -------------------------- imagesLoaded -------------------------- // + // -------------------------- imagesLoaded -------------------------- // -/** - * @param {Array, Element, NodeList, String} elem - * @param {Object or Function} options - if function, use as callback - * @param {Function} onAlways - callback function - */ -function ImagesLoaded( elem, options, onAlways ) { - // coerce ImagesLoaded() without new, to be new ImagesLoaded() - if ( !( this instanceof ImagesLoaded ) ) { - return new ImagesLoaded( elem, options, onAlways ); - } - // use elem as selector string - if ( typeof elem == 'string' ) { - elem = document.querySelectorAll( elem ); - } + /** + * @param {Array, Element, NodeList, String} elem + * @param {Object or Function} options - if function, use as callback + * @param {Function} onAlways - callback function + */ + function ImagesLoaded( elem, options, onAlways ) { + // coerce ImagesLoaded() without new, to be new ImagesLoaded() + if ( !( this instanceof ImagesLoaded ) ) { + return new ImagesLoaded( elem, options, onAlways ); + } + // use elem as selector string + if ( typeof elem == 'string' ) { + elem = document.querySelectorAll( elem ); + } - this.elements = makeArray( elem ); - this.options = extend( {}, this.options ); + this.elements = makeArray( elem ); + this.options = extend( {}, this.options ); - if ( typeof options == 'function' ) { - onAlways = options; - } else { - extend( this.options, options ); - } + if ( typeof options == 'function' ) { + onAlways = options; + } else { + extend( this.options, options ); + } - if ( onAlways ) { - this.on( 'always', onAlways ); - } + if ( onAlways ) { + this.on( 'always', onAlways ); + } - this.getImages(); + this.getImages(); - if ( $ ) { - // add jQuery Deferred object - this.jqDeferred = new $.Deferred(); - } + if ( $ ) { + // add jQuery Deferred object + this.jqDeferred = new $.Deferred(); + } - // HACK check async to allow time to bind listeners - setTimeout( function() { - this.check(); - }.bind( this )); -} + // HACK check async to allow time to bind listeners + var _this = this; + setTimeout( function() { + _this.check(); + }); + } -ImagesLoaded.prototype = Object.create( EvEmitter.prototype ); + ImagesLoaded.prototype = new EventEmitter(); -ImagesLoaded.prototype.options = {}; + ImagesLoaded.prototype.options = {}; -ImagesLoaded.prototype.getImages = function() { - this.images = []; + ImagesLoaded.prototype.getImages = function() { + this.images = []; - // filter & find items if we have an item selector - this.elements.forEach( this.addElementImages, this ); -}; + // filter & find items if we have an item selector + for ( var i=0; i < this.elements.length; i++ ) { + var elem = this.elements[i]; + this.addElementImages( elem ); + } + }; -/** - * @param {Node} element - */ -ImagesLoaded.prototype.addElementImages = function( elem ) { - // filter siblings - if ( elem.nodeName == 'IMG' ) { - this.addImage( elem ); - } - // get background image on element - if ( this.options.background === true ) { - this.addElementBackgroundImages( elem ); - } + /** + * @param {Node} element + */ + ImagesLoaded.prototype.addElementImages = function( elem ) { + // filter siblings + if ( elem.nodeName == 'IMG' ) { + this.addImage( elem ); + } + // get background image on element + if ( this.options.background === true ) { + this.addElementBackgroundImages( elem ); + } - // find children - // no non-element nodes, #143 - var nodeType = elem.nodeType; - if ( !nodeType || !elementNodeTypes[ nodeType ] ) { - return; - } - var childImgs = elem.querySelectorAll('img'); - // concat childElems to filterFound array - for ( var i=0; i < childImgs.length; i++ ) { - var img = childImgs[i]; - this.addImage( img ); - } + // find children + // no non-element nodes, #143 + var nodeType = elem.nodeType; + if ( !nodeType || !elementNodeTypes[ nodeType ] ) { + return; + } + var childImgs = elem.querySelectorAll('img'); + // concat childElems to filterFound array + for ( var i=0; i < childImgs.length; i++ ) { + var img = childImgs[i]; + this.addImage( img ); + } - // get child background images - if ( typeof this.options.background == 'string' ) { - var children = elem.querySelectorAll( this.options.background ); - for ( i=0; i < children.length; i++ ) { - var child = children[i]; - this.addElementBackgroundImages( child ); + // get child background images + if ( typeof this.options.background == 'string' ) { + var children = elem.querySelectorAll( this.options.background ); + for ( i=0; i < children.length; i++ ) { + var child = children[i]; + this.addElementBackgroundImages( child ); + } } - } -}; + }; -var elementNodeTypes = { - 1: true, - 9: true, - 11: true -}; + var elementNodeTypes = { + 1: true, + 9: true, + 11: true + }; -ImagesLoaded.prototype.addElementBackgroundImages = function( elem ) { - var style = getComputedStyle( elem ); - if ( !style ) { - // Firefox returns null if in a hidden iframe https://bugzil.la/548397 - return; - } - // get url inside url("...") - var reURL = /url\((['"])?(.*?)\1\)/gi; - var matches = reURL.exec( style.backgroundImage ); - while ( matches !== null ) { - var url = matches && matches[2]; - if ( url ) { - this.addBackground( url, elem ); + ImagesLoaded.prototype.addElementBackgroundImages = function( elem ) { + var style = getStyle( elem ); + // get url inside url("...") + var reURL = /url\(['"]*([^'"\)]+)['"]*\)/gi; + var matches = reURL.exec( style.backgroundImage ); + while ( matches !== null ) { + var url = matches && matches[1]; + if ( url ) { + this.addBackground( url, elem ); + } + matches = reURL.exec( style.backgroundImage ); } - matches = reURL.exec( style.backgroundImage ); - } -}; - -/** - * @param {Image} img - */ -ImagesLoaded.prototype.addImage = function( img ) { - var loadingImage = new LoadingImage( img ); - this.images.push( loadingImage ); -}; + }; -ImagesLoaded.prototype.addBackground = function( url, elem ) { - var background = new Background( url, elem ); - this.images.push( background ); -}; + // IE8 + var getStyle = window.getComputedStyle || function( elem ) { + return elem.currentStyle; + }; -ImagesLoaded.prototype.check = function() { - var _this = this; - this.progressedCount = 0; - this.hasAnyBroken = false; - // complete if no images - if ( !this.images.length ) { - this.complete(); - return; - } + /** + * @param {Image} img + */ + ImagesLoaded.prototype.addImage = function( img ) { + var loadingImage = new LoadingImage( img ); + this.images.push( loadingImage ); + }; - function onProgress( image, elem, message ) { - // HACK - Chrome triggers event before object properties have changed. #83 - setTimeout( function() { - _this.progress( image, elem, message ); - }); - } + ImagesLoaded.prototype.addBackground = function( url, elem ) { + var background = new Background( url, elem ); + this.images.push( background ); + }; - this.images.forEach( function( loadingImage ) { - loadingImage.once( 'progress', onProgress ); - loadingImage.check(); - }); -}; + ImagesLoaded.prototype.check = function() { + var _this = this; + this.progressedCount = 0; + this.hasAnyBroken = false; + // complete if no images + if ( !this.images.length ) { + this.complete(); + return; + } -ImagesLoaded.prototype.progress = function( image, elem, message ) { - this.progressedCount++; - this.hasAnyBroken = this.hasAnyBroken || !image.isLoaded; - // progress event - this.emitEvent( 'progress', [ this, image, elem ] ); - if ( this.jqDeferred && this.jqDeferred.notify ) { - this.jqDeferred.notify( this, image ); - } - // check if completed - if ( this.progressedCount == this.images.length ) { - this.complete(); - } + function onProgress( image, elem, message ) { + // HACK - Chrome triggers event before object properties have changed. #83 + setTimeout( function() { + _this.progress( image, elem, message ); + }); + } - if ( this.options.debug && console ) { - console.log( 'progress: ' + message, image, elem ); - } -}; + for ( var i=0; i < this.images.length; i++ ) { + var loadingImage = this.images[i]; + loadingImage.once( 'progress', onProgress ); + loadingImage.check(); + } + }; -ImagesLoaded.prototype.complete = function() { - var eventName = this.hasAnyBroken ? 'fail' : 'done'; - this.isComplete = true; - this.emitEvent( eventName, [ this ] ); - this.emitEvent( 'always', [ this ] ); - if ( this.jqDeferred ) { - var jqMethod = this.hasAnyBroken ? 'reject' : 'resolve'; - this.jqDeferred[ jqMethod ]( this ); - } -}; + ImagesLoaded.prototype.progress = function( image, elem, message ) { + this.progressedCount++; + this.hasAnyBroken = this.hasAnyBroken || !image.isLoaded; + // progress event + this.emit( 'progress', this, image, elem ); + if ( this.jqDeferred && this.jqDeferred.notify ) { + this.jqDeferred.notify( this, image ); + } + // check if completed + if ( this.progressedCount == this.images.length ) { + this.complete(); + } -// -------------------------- -------------------------- // + if ( this.options.debug && console ) { + console.log( 'progress: ' + message, image, elem ); + } + }; -function LoadingImage( img ) { - this.img = img; -} + ImagesLoaded.prototype.complete = function() { + var eventName = this.hasAnyBroken ? 'fail' : 'done'; + this.isComplete = true; + this.emit( eventName, this ); + this.emit( 'always', this ); + if ( this.jqDeferred ) { + var jqMethod = this.hasAnyBroken ? 'reject' : 'resolve'; + this.jqDeferred[ jqMethod ]( this ); + } + }; -LoadingImage.prototype = Object.create( EvEmitter.prototype ); + // -------------------------- -------------------------- // -LoadingImage.prototype.check = function() { - // If complete is true and browser supports natural sizes, - // try to check for image status manually. - var isComplete = this.getIsImageComplete(); - if ( isComplete ) { - // report based on naturalWidth - this.confirm( this.img.naturalWidth !== 0, 'naturalWidth' ); - return; + function LoadingImage( img ) { + this.img = img; } - // If none of the checks above matched, simulate loading on detached element. - this.proxyImage = new Image(); - this.proxyImage.addEventListener( 'load', this ); - this.proxyImage.addEventListener( 'error', this ); - // bind to image as well for Firefox. #191 - this.img.addEventListener( 'load', this ); - this.img.addEventListener( 'error', this ); - this.proxyImage.src = this.img.src; -}; + LoadingImage.prototype = new EventEmitter(); -LoadingImage.prototype.getIsImageComplete = function() { - return this.img.complete && this.img.naturalWidth !== undefined; -}; + LoadingImage.prototype.check = function() { + // If complete is true and browser supports natural sizes, + // try to check for image status manually. + var isComplete = this.getIsImageComplete(); + if ( isComplete ) { + // report based on naturalWidth + this.confirm( this.img.naturalWidth !== 0, 'naturalWidth' ); + return; + } -LoadingImage.prototype.confirm = function( isLoaded, message ) { - this.isLoaded = isLoaded; - this.emitEvent( 'progress', [ this, this.img, message ] ); -}; + // If none of the checks above matched, simulate loading on detached element. + this.proxyImage = new Image(); + eventie.bind( this.proxyImage, 'load', this ); + eventie.bind( this.proxyImage, 'error', this ); + // bind to image as well for Firefox. #191 + eventie.bind( this.img, 'load', this ); + eventie.bind( this.img, 'error', this ); + this.proxyImage.src = this.img.src; + }; -// ----- events ----- // + LoadingImage.prototype.getIsImageComplete = function() { + return this.img.complete && this.img.naturalWidth !== undefined; + }; -// trigger specified handler for event type -LoadingImage.prototype.handleEvent = function( event ) { - var method = 'on' + event.type; - if ( this[ method ] ) { - this[ method ]( event ); - } -}; + LoadingImage.prototype.confirm = function( isLoaded, message ) { + this.isLoaded = isLoaded; + this.emit( 'progress', this, this.img, message ); + }; -LoadingImage.prototype.onload = function() { - this.confirm( true, 'onload' ); - this.unbindEvents(); -}; + // ----- events ----- // -LoadingImage.prototype.onerror = function() { - this.confirm( false, 'onerror' ); - this.unbindEvents(); -}; + // trigger specified handler for event type + LoadingImage.prototype.handleEvent = function( event ) { + var method = 'on' + event.type; + if ( this[ method ] ) { + this[ method ]( event ); + } + }; -LoadingImage.prototype.unbindEvents = function() { - this.proxyImage.removeEventListener( 'load', this ); - this.proxyImage.removeEventListener( 'error', this ); - this.img.removeEventListener( 'load', this ); - this.img.removeEventListener( 'error', this ); -}; + LoadingImage.prototype.onload = function() { + this.confirm( true, 'onload' ); + this.unbindEvents(); + }; -// -------------------------- Background -------------------------- // + LoadingImage.prototype.onerror = function() { + this.confirm( false, 'onerror' ); + this.unbindEvents(); + }; -function Background( url, element ) { - this.url = url; - this.element = element; - this.img = new Image(); -} + LoadingImage.prototype.unbindEvents = function() { + eventie.unbind( this.proxyImage, 'load', this ); + eventie.unbind( this.proxyImage, 'error', this ); + eventie.unbind( this.img, 'load', this ); + eventie.unbind( this.img, 'error', this ); + }; -// inherit LoadingImage prototype -Background.prototype = Object.create( LoadingImage.prototype ); + // -------------------------- Background -------------------------- // -Background.prototype.check = function() { - this.img.addEventListener( 'load', this ); - this.img.addEventListener( 'error', this ); - this.img.src = this.url; - // check if image is already complete - var isComplete = this.getIsImageComplete(); - if ( isComplete ) { - this.confirm( this.img.naturalWidth !== 0, 'naturalWidth' ); - this.unbindEvents(); + function Background( url, element ) { + this.url = url; + this.element = element; + this.img = new Image(); } -}; -Background.prototype.unbindEvents = function() { - this.img.removeEventListener( 'load', this ); - this.img.removeEventListener( 'error', this ); -}; + // inherit LoadingImage prototype + Background.prototype = new LoadingImage(); -Background.prototype.confirm = function( isLoaded, message ) { - this.isLoaded = isLoaded; - this.emitEvent( 'progress', [ this, this.element, message ] ); -}; + Background.prototype.check = function() { + eventie.bind( this.img, 'load', this ); + eventie.bind( this.img, 'error', this ); + this.img.src = this.url; + // check if image is already complete + var isComplete = this.getIsImageComplete(); + if ( isComplete ) { + this.confirm( this.img.naturalWidth !== 0, 'naturalWidth' ); + this.unbindEvents(); + } + }; -// -------------------------- jQuery -------------------------- // + Background.prototype.unbindEvents = function() { + eventie.unbind( this.img, 'load', this ); + eventie.unbind( this.img, 'error', this ); + }; -ImagesLoaded.makeJQueryPlugin = function( jQuery ) { - jQuery = jQuery || window.jQuery; - if ( !jQuery ) { - return; - } - // set local variable - $ = jQuery; - // $().imagesLoaded() - $.fn.imagesLoaded = function( options, callback ) { - var instance = new ImagesLoaded( this, options, callback ); - return instance.jqDeferred.promise( $(this) ); + Background.prototype.confirm = function( isLoaded, message ) { + this.isLoaded = isLoaded; + this.emit( 'progress', this, this.element, message ); }; -}; -// try making plugin -ImagesLoaded.makeJQueryPlugin(); -// -------------------------- -------------------------- // + // -------------------------- jQuery -------------------------- // + + ImagesLoaded.makeJQueryPlugin = function( jQuery ) { + jQuery = jQuery || window.jQuery; + if ( !jQuery ) { + return; + } + // set local variable + $ = jQuery; + // $().imagesLoaded() + $.fn.imagesLoaded = function( options, callback ) { + var instance = new ImagesLoaded( this, options, callback ); + return instance.jqDeferred.promise( $(this) ); + }; + }; + // try making plugin + ImagesLoaded.makeJQueryPlugin(); + + // -------------------------- -------------------------- // -return ImagesLoaded; + return ImagesLoaded; }); @@ -76518,211 +76979,211 @@ BROWSER = { .replace('undefined', '3_2').replace('_', '.').replace('_', '') ) || FALSE }; -;function QTip(target, options, id, attr) { - // Elements and ID - this.id = id; - this.target = target; - this.tooltip = NULL; - this.elements = { target: target }; - - // Internal constructs - this._id = NAMESPACE + '-' + id; - this.timers = { img: {} }; - this.options = options; - this.plugins = {}; - - // Cache object - this.cache = { - event: {}, - target: $(), - disabled: FALSE, - attr: attr, - onTooltip: FALSE, - lastClass: '' - }; - - // Set the initial flags - this.rendered = this.destroyed = this.disabled = this.waiting = - this.hiddenDuringWait = this.positioning = this.triggering = FALSE; -} -PROTOTYPE = QTip.prototype; - -PROTOTYPE._when = function(deferreds) { - return $.when.apply($, deferreds); -}; - -PROTOTYPE.render = function(show) { - if(this.rendered || this.destroyed) { return this; } // If tooltip has already been rendered, exit - - var self = this, - options = this.options, - cache = this.cache, - elements = this.elements, - text = options.content.text, - title = options.content.title, - button = options.content.button, - posOptions = options.position, - namespace = '.'+this._id+' ', - deferreds = [], - tooltip; - - // Add ARIA attributes to target - $.attr(this.target[0], 'aria-describedby', this._id); - - // Create public position object that tracks current position corners - cache.posClass = this._createPosClass( - (this.position = { my: posOptions.my, at: posOptions.at }).my - ); - - // Create tooltip element - this.tooltip = elements.tooltip = tooltip = $('

', { - 'id': this._id, - 'class': [ NAMESPACE, CLASS_DEFAULT, options.style.classes, cache.posClass ].join(' '), - 'width': options.style.width || '', - 'height': options.style.height || '', - 'tracking': posOptions.target === 'mouse' && posOptions.adjust.mouse, - - /* ARIA specific attributes */ - 'role': 'alert', - 'aria-live': 'polite', - 'aria-atomic': FALSE, - 'aria-describedby': this._id + '-content', - 'aria-hidden': TRUE - }) - .toggleClass(CLASS_DISABLED, this.disabled) - .attr(ATTR_ID, this.id) - .data(NAMESPACE, this) - .appendTo(posOptions.container) - .append( - // Create content element - elements.content = $('
', { - 'class': NAMESPACE + '-content', - 'id': this._id + '-content', - 'aria-atomic': TRUE - }) - ); - - // Set rendered flag and prevent redundant reposition calls for now - this.rendered = -1; - this.positioning = TRUE; - - // Create title... - if(title) { - this._createTitle(); - - // Update title only if its not a callback (called in toggle if so) - if(!$.isFunction(title)) { - deferreds.push( this._updateTitle(title, FALSE) ); - } - } - - // Create button - if(button) { this._createButton(); } - - // Set proper rendered flag and update content if not a callback function (called in toggle) - if(!$.isFunction(text)) { - deferreds.push( this._updateContent(text, FALSE) ); - } - this.rendered = TRUE; - - // Setup widget classes - this._setWidget(); - - // Initialize 'render' plugins - $.each(PLUGINS, function(name) { - var instance; - if(this.initialize === 'render' && (instance = this(self))) { - self.plugins[name] = instance; - } - }); - - // Unassign initial events and assign proper events - this._unassignEvents(); - this._assignEvents(); - - // When deferreds have completed - this._when(deferreds).then(function() { - // tooltiprender event - self._trigger('render'); - - // Reset flags - self.positioning = FALSE; - - // Show tooltip if not hidden during wait period - if(!self.hiddenDuringWait && (options.show.ready || show)) { - self.toggle(TRUE, cache.event, FALSE); - } - self.hiddenDuringWait = FALSE; - }); - - // Expose API - QTIP.api[this.id] = this; - - return this; -}; - -PROTOTYPE.destroy = function(immediate) { - // Set flag the signify destroy is taking place to plugins - // and ensure it only gets destroyed once! - if(this.destroyed) { return this.target; } - - function process() { - if(this.destroyed) { return; } - this.destroyed = TRUE; - - var target = this.target, - title = target.attr(oldtitle), - timer; - - // Destroy tooltip if rendered - if(this.rendered) { - this.tooltip.stop(1,0).find('*').remove().end().remove(); - } - - // Destroy all plugins - $.each(this.plugins, function(name) { - this.destroy && this.destroy(); - }); - - // Clear timers - for(timer in this.timers) { - clearTimeout(this.timers[timer]); - } - - // Remove api object and ARIA attributes - target.removeData(NAMESPACE) - .removeAttr(ATTR_ID) - .removeAttr(ATTR_HAS) - .removeAttr('aria-describedby'); - - // Reset old title attribute if removed - if(this.options.suppress && title) { - target.attr('title', title).removeAttr(oldtitle); - } - - // Remove qTip events associated with this API - this._unassignEvents(); - - // Remove ID from used id objects, and delete object references - // for better garbage collection and leak protection - this.options = this.elements = this.cache = this.timers = - this.plugins = this.mouse = NULL; - - // Delete epoxsed API object - delete QTIP.api[this.id]; - } - - // If an immediate destory is needed - if((immediate !== TRUE || this.triggering === 'hide') && this.rendered) { - this.tooltip.one('tooltiphidden', $.proxy(process, this)); - !this.triggering && this.hide(); - } - - // If we're not in the process of hiding... process - else { process.call(this); } - - return this.target; -}; +;function QTip(target, options, id, attr) { + // Elements and ID + this.id = id; + this.target = target; + this.tooltip = NULL; + this.elements = { target: target }; + + // Internal constructs + this._id = NAMESPACE + '-' + id; + this.timers = { img: {} }; + this.options = options; + this.plugins = {}; + + // Cache object + this.cache = { + event: {}, + target: $(), + disabled: FALSE, + attr: attr, + onTooltip: FALSE, + lastClass: '' + }; + + // Set the initial flags + this.rendered = this.destroyed = this.disabled = this.waiting = + this.hiddenDuringWait = this.positioning = this.triggering = FALSE; +} +PROTOTYPE = QTip.prototype; + +PROTOTYPE._when = function(deferreds) { + return $.when.apply($, deferreds); +}; + +PROTOTYPE.render = function(show) { + if(this.rendered || this.destroyed) { return this; } // If tooltip has already been rendered, exit + + var self = this, + options = this.options, + cache = this.cache, + elements = this.elements, + text = options.content.text, + title = options.content.title, + button = options.content.button, + posOptions = options.position, + namespace = '.'+this._id+' ', + deferreds = [], + tooltip; + + // Add ARIA attributes to target + $.attr(this.target[0], 'aria-describedby', this._id); + + // Create public position object that tracks current position corners + cache.posClass = this._createPosClass( + (this.position = { my: posOptions.my, at: posOptions.at }).my + ); + + // Create tooltip element + this.tooltip = elements.tooltip = tooltip = $('
', { + 'id': this._id, + 'class': [ NAMESPACE, CLASS_DEFAULT, options.style.classes, cache.posClass ].join(' '), + 'width': options.style.width || '', + 'height': options.style.height || '', + 'tracking': posOptions.target === 'mouse' && posOptions.adjust.mouse, + + /* ARIA specific attributes */ + 'role': 'alert', + 'aria-live': 'polite', + 'aria-atomic': FALSE, + 'aria-describedby': this._id + '-content', + 'aria-hidden': TRUE + }) + .toggleClass(CLASS_DISABLED, this.disabled) + .attr(ATTR_ID, this.id) + .data(NAMESPACE, this) + .appendTo(posOptions.container) + .append( + // Create content element + elements.content = $('
', { + 'class': NAMESPACE + '-content', + 'id': this._id + '-content', + 'aria-atomic': TRUE + }) + ); + + // Set rendered flag and prevent redundant reposition calls for now + this.rendered = -1; + this.positioning = TRUE; + + // Create title... + if(title) { + this._createTitle(); + + // Update title only if its not a callback (called in toggle if so) + if(!$.isFunction(title)) { + deferreds.push( this._updateTitle(title, FALSE) ); + } + } + + // Create button + if(button) { this._createButton(); } + + // Set proper rendered flag and update content if not a callback function (called in toggle) + if(!$.isFunction(text)) { + deferreds.push( this._updateContent(text, FALSE) ); + } + this.rendered = TRUE; + + // Setup widget classes + this._setWidget(); + + // Initialize 'render' plugins + $.each(PLUGINS, function(name) { + var instance; + if(this.initialize === 'render' && (instance = this(self))) { + self.plugins[name] = instance; + } + }); + + // Unassign initial events and assign proper events + this._unassignEvents(); + this._assignEvents(); + + // When deferreds have completed + this._when(deferreds).then(function() { + // tooltiprender event + self._trigger('render'); + + // Reset flags + self.positioning = FALSE; + + // Show tooltip if not hidden during wait period + if(!self.hiddenDuringWait && (options.show.ready || show)) { + self.toggle(TRUE, cache.event, FALSE); + } + self.hiddenDuringWait = FALSE; + }); + + // Expose API + QTIP.api[this.id] = this; + + return this; +}; + +PROTOTYPE.destroy = function(immediate) { + // Set flag the signify destroy is taking place to plugins + // and ensure it only gets destroyed once! + if(this.destroyed) { return this.target; } + + function process() { + if(this.destroyed) { return; } + this.destroyed = TRUE; + + var target = this.target, + title = target.attr(oldtitle), + timer; + + // Destroy tooltip if rendered + if(this.rendered) { + this.tooltip.stop(1,0).find('*').remove().end().remove(); + } + + // Destroy all plugins + $.each(this.plugins, function(name) { + this.destroy && this.destroy(); + }); + + // Clear timers + for(timer in this.timers) { + clearTimeout(this.timers[timer]); + } + + // Remove api object and ARIA attributes + target.removeData(NAMESPACE) + .removeAttr(ATTR_ID) + .removeAttr(ATTR_HAS) + .removeAttr('aria-describedby'); + + // Reset old title attribute if removed + if(this.options.suppress && title) { + target.attr('title', title).removeAttr(oldtitle); + } + + // Remove qTip events associated with this API + this._unassignEvents(); + + // Remove ID from used id objects, and delete object references + // for better garbage collection and leak protection + this.options = this.elements = this.cache = this.timers = + this.plugins = this.mouse = NULL; + + // Delete epoxsed API object + delete QTIP.api[this.id]; + } + + // If an immediate destory is needed + if((immediate !== TRUE || this.triggering === 'hide') && this.rendered) { + this.tooltip.one('tooltiphidden', $.proxy(process, this)); + !this.triggering && this.hide(); + } + + // If we're not in the process of hiding... process + else { process.call(this); } + + return this.target; +}; ;function invalidOpt(a) { return a === NULL || $.type(a) !== 'object'; } @@ -79970,211 +80431,211 @@ BROWSER = { .replace('undefined', '3_2').replace('_', '.').replace('_', '') ) || FALSE }; -;function QTip(target, options, id, attr) { - // Elements and ID - this.id = id; - this.target = target; - this.tooltip = NULL; - this.elements = { target: target }; - - // Internal constructs - this._id = NAMESPACE + '-' + id; - this.timers = { img: {} }; - this.options = options; - this.plugins = {}; - - // Cache object - this.cache = { - event: {}, - target: $(), - disabled: FALSE, - attr: attr, - onTooltip: FALSE, - lastClass: '' - }; - - // Set the initial flags - this.rendered = this.destroyed = this.disabled = this.waiting = - this.hiddenDuringWait = this.positioning = this.triggering = FALSE; -} -PROTOTYPE = QTip.prototype; - -PROTOTYPE._when = function(deferreds) { - return $.when.apply($, deferreds); -}; - -PROTOTYPE.render = function(show) { - if(this.rendered || this.destroyed) { return this; } // If tooltip has already been rendered, exit - - var self = this, - options = this.options, - cache = this.cache, - elements = this.elements, - text = options.content.text, - title = options.content.title, - button = options.content.button, - posOptions = options.position, - namespace = '.'+this._id+' ', - deferreds = [], - tooltip; - - // Add ARIA attributes to target - $.attr(this.target[0], 'aria-describedby', this._id); - - // Create public position object that tracks current position corners - cache.posClass = this._createPosClass( - (this.position = { my: posOptions.my, at: posOptions.at }).my - ); - - // Create tooltip element - this.tooltip = elements.tooltip = tooltip = $('
', { - 'id': this._id, - 'class': [ NAMESPACE, CLASS_DEFAULT, options.style.classes, cache.posClass ].join(' '), - 'width': options.style.width || '', - 'height': options.style.height || '', - 'tracking': posOptions.target === 'mouse' && posOptions.adjust.mouse, - - /* ARIA specific attributes */ - 'role': 'alert', - 'aria-live': 'polite', - 'aria-atomic': FALSE, - 'aria-describedby': this._id + '-content', - 'aria-hidden': TRUE - }) - .toggleClass(CLASS_DISABLED, this.disabled) - .attr(ATTR_ID, this.id) - .data(NAMESPACE, this) - .appendTo(posOptions.container) - .append( - // Create content element - elements.content = $('
', { - 'class': NAMESPACE + '-content', - 'id': this._id + '-content', - 'aria-atomic': TRUE - }) - ); - - // Set rendered flag and prevent redundant reposition calls for now - this.rendered = -1; - this.positioning = TRUE; - - // Create title... - if(title) { - this._createTitle(); - - // Update title only if its not a callback (called in toggle if so) - if(!$.isFunction(title)) { - deferreds.push( this._updateTitle(title, FALSE) ); - } - } - - // Create button - if(button) { this._createButton(); } - - // Set proper rendered flag and update content if not a callback function (called in toggle) - if(!$.isFunction(text)) { - deferreds.push( this._updateContent(text, FALSE) ); - } - this.rendered = TRUE; - - // Setup widget classes - this._setWidget(); - - // Initialize 'render' plugins - $.each(PLUGINS, function(name) { - var instance; - if(this.initialize === 'render' && (instance = this(self))) { - self.plugins[name] = instance; - } - }); - - // Unassign initial events and assign proper events - this._unassignEvents(); - this._assignEvents(); - - // When deferreds have completed - this._when(deferreds).then(function() { - // tooltiprender event - self._trigger('render'); - - // Reset flags - self.positioning = FALSE; - - // Show tooltip if not hidden during wait period - if(!self.hiddenDuringWait && (options.show.ready || show)) { - self.toggle(TRUE, cache.event, FALSE); - } - self.hiddenDuringWait = FALSE; - }); - - // Expose API - QTIP.api[this.id] = this; - - return this; -}; - -PROTOTYPE.destroy = function(immediate) { - // Set flag the signify destroy is taking place to plugins - // and ensure it only gets destroyed once! - if(this.destroyed) { return this.target; } - - function process() { - if(this.destroyed) { return; } - this.destroyed = TRUE; - - var target = this.target, - title = target.attr(oldtitle), - timer; - - // Destroy tooltip if rendered - if(this.rendered) { - this.tooltip.stop(1,0).find('*').remove().end().remove(); - } - - // Destroy all plugins - $.each(this.plugins, function(name) { - this.destroy && this.destroy(); - }); - - // Clear timers - for(timer in this.timers) { - clearTimeout(this.timers[timer]); - } - - // Remove api object and ARIA attributes - target.removeData(NAMESPACE) - .removeAttr(ATTR_ID) - .removeAttr(ATTR_HAS) - .removeAttr('aria-describedby'); - - // Reset old title attribute if removed - if(this.options.suppress && title) { - target.attr('title', title).removeAttr(oldtitle); - } - - // Remove qTip events associated with this API - this._unassignEvents(); - - // Remove ID from used id objects, and delete object references - // for better garbage collection and leak protection - this.options = this.elements = this.cache = this.timers = - this.plugins = this.mouse = NULL; - - // Delete epoxsed API object - delete QTIP.api[this.id]; - } - - // If an immediate destory is needed - if((immediate !== TRUE || this.triggering === 'hide') && this.rendered) { - this.tooltip.one('tooltiphidden', $.proxy(process, this)); - !this.triggering && this.hide(); - } - - // If we're not in the process of hiding... process - else { process.call(this); } - - return this.target; -}; +;function QTip(target, options, id, attr) { + // Elements and ID + this.id = id; + this.target = target; + this.tooltip = NULL; + this.elements = { target: target }; + + // Internal constructs + this._id = NAMESPACE + '-' + id; + this.timers = { img: {} }; + this.options = options; + this.plugins = {}; + + // Cache object + this.cache = { + event: {}, + target: $(), + disabled: FALSE, + attr: attr, + onTooltip: FALSE, + lastClass: '' + }; + + // Set the initial flags + this.rendered = this.destroyed = this.disabled = this.waiting = + this.hiddenDuringWait = this.positioning = this.triggering = FALSE; +} +PROTOTYPE = QTip.prototype; + +PROTOTYPE._when = function(deferreds) { + return $.when.apply($, deferreds); +}; + +PROTOTYPE.render = function(show) { + if(this.rendered || this.destroyed) { return this; } // If tooltip has already been rendered, exit + + var self = this, + options = this.options, + cache = this.cache, + elements = this.elements, + text = options.content.text, + title = options.content.title, + button = options.content.button, + posOptions = options.position, + namespace = '.'+this._id+' ', + deferreds = [], + tooltip; + + // Add ARIA attributes to target + $.attr(this.target[0], 'aria-describedby', this._id); + + // Create public position object that tracks current position corners + cache.posClass = this._createPosClass( + (this.position = { my: posOptions.my, at: posOptions.at }).my + ); + + // Create tooltip element + this.tooltip = elements.tooltip = tooltip = $('
', { + 'id': this._id, + 'class': [ NAMESPACE, CLASS_DEFAULT, options.style.classes, cache.posClass ].join(' '), + 'width': options.style.width || '', + 'height': options.style.height || '', + 'tracking': posOptions.target === 'mouse' && posOptions.adjust.mouse, + + /* ARIA specific attributes */ + 'role': 'alert', + 'aria-live': 'polite', + 'aria-atomic': FALSE, + 'aria-describedby': this._id + '-content', + 'aria-hidden': TRUE + }) + .toggleClass(CLASS_DISABLED, this.disabled) + .attr(ATTR_ID, this.id) + .data(NAMESPACE, this) + .appendTo(posOptions.container) + .append( + // Create content element + elements.content = $('
', { + 'class': NAMESPACE + '-content', + 'id': this._id + '-content', + 'aria-atomic': TRUE + }) + ); + + // Set rendered flag and prevent redundant reposition calls for now + this.rendered = -1; + this.positioning = TRUE; + + // Create title... + if(title) { + this._createTitle(); + + // Update title only if its not a callback (called in toggle if so) + if(!$.isFunction(title)) { + deferreds.push( this._updateTitle(title, FALSE) ); + } + } + + // Create button + if(button) { this._createButton(); } + + // Set proper rendered flag and update content if not a callback function (called in toggle) + if(!$.isFunction(text)) { + deferreds.push( this._updateContent(text, FALSE) ); + } + this.rendered = TRUE; + + // Setup widget classes + this._setWidget(); + + // Initialize 'render' plugins + $.each(PLUGINS, function(name) { + var instance; + if(this.initialize === 'render' && (instance = this(self))) { + self.plugins[name] = instance; + } + }); + + // Unassign initial events and assign proper events + this._unassignEvents(); + this._assignEvents(); + + // When deferreds have completed + this._when(deferreds).then(function() { + // tooltiprender event + self._trigger('render'); + + // Reset flags + self.positioning = FALSE; + + // Show tooltip if not hidden during wait period + if(!self.hiddenDuringWait && (options.show.ready || show)) { + self.toggle(TRUE, cache.event, FALSE); + } + self.hiddenDuringWait = FALSE; + }); + + // Expose API + QTIP.api[this.id] = this; + + return this; +}; + +PROTOTYPE.destroy = function(immediate) { + // Set flag the signify destroy is taking place to plugins + // and ensure it only gets destroyed once! + if(this.destroyed) { return this.target; } + + function process() { + if(this.destroyed) { return; } + this.destroyed = TRUE; + + var target = this.target, + title = target.attr(oldtitle), + timer; + + // Destroy tooltip if rendered + if(this.rendered) { + this.tooltip.stop(1,0).find('*').remove().end().remove(); + } + + // Destroy all plugins + $.each(this.plugins, function(name) { + this.destroy && this.destroy(); + }); + + // Clear timers + for(timer in this.timers) { + clearTimeout(this.timers[timer]); + } + + // Remove api object and ARIA attributes + target.removeData(NAMESPACE) + .removeAttr(ATTR_ID) + .removeAttr(ATTR_HAS) + .removeAttr('aria-describedby'); + + // Reset old title attribute if removed + if(this.options.suppress && title) { + target.attr('title', title).removeAttr(oldtitle); + } + + // Remove qTip events associated with this API + this._unassignEvents(); + + // Remove ID from used id objects, and delete object references + // for better garbage collection and leak protection + this.options = this.elements = this.cache = this.timers = + this.plugins = this.mouse = NULL; + + // Delete epoxsed API object + delete QTIP.api[this.id]; + } + + // If an immediate destory is needed + if((immediate !== TRUE || this.triggering === 'hide') && this.rendered) { + this.tooltip.one('tooltiphidden', $.proxy(process, this)); + !this.triggering && this.hide(); + } + + // If we're not in the process of hiding... process + else { process.call(this); } + + return this.target; +}; ;function invalidOpt(a) { return a === NULL || $.type(a) !== 'object'; } @@ -82367,7 +82828,7 @@ function intersectLine(p1, p2, q1, q2) { c2 = (q2.x * q1.y) - (q1.x * q2.y); // Compute r1 and r2 - r1 = (a2 * p1.x) + (b2 * p1.y) + c2; + r1 = (a2 * p1.x) + (b2 * p1.yy) + c2; r2 = (a2 * p2.x) + (b2 * p2.y) + c2; // Check signs of r1 and r2. If both point 1 and point 2 lie @@ -83054,11 +83515,11 @@ function applyTransition(selection, g) { } },{"./lodash":21}],28:[function(require,module,exports){ -module.exports = "0.4.11"; +module.exports = "0.4.10"; },{}]},{},[1])(1) }); -//# sourceMappingURL=data:application/json;charset:utf-8;base64,eyJ2ZXJzaW9uIjozLCJzb3VyY2VzIjpbIm5vZGVfbW9kdWxlcy9icm93c2VyaWZ5L25vZGVfbW9kdWxlcy9icm93c2VyLXBhY2svX3ByZWx1ZGUuanMiLCJpbmRleC5qcyIsImxpYi9hcnJvd3MuanMiLCJsaWIvY3JlYXRlLWNsdXN0ZXJzLmpzIiwibGliL2NyZWF0ZS1lZGdlLWxhYmVscy5qcyIsImxpYi9jcmVhdGUtZWRnZS1wYXRocy5qcyIsImxpYi9jcmVhdGUtbm9kZXMuanMiLCJsaWIvZDMuanMiLCJsaWIvZGFncmUuanMiLCJsaWIvZ3JhcGhsaWIuanMiLCJsaWIvaW50ZXJzZWN0L2luZGV4LmpzIiwibGliL2ludGVyc2VjdC9pbnRlcnNlY3QtY2lyY2xlLmpzIiwibGliL2ludGVyc2VjdC9pbnRlcnNlY3QtZWxsaXBzZS5qcyIsImxpYi9pbnRlcnNlY3QvaW50ZXJzZWN0LWxpbmUuanMiLCJsaWIvaW50ZXJzZWN0L2ludGVyc2VjdC1ub2RlLmpzIiwibGliL2ludGVyc2VjdC9pbnRlcnNlY3QtcG9seWdvbi5qcyIsImxpYi9pbnRlcnNlY3QvaW50ZXJzZWN0LXJlY3QuanMiLCJsaWIvbGFiZWwvYWRkLWh0bWwtbGFiZWwuanMiLCJsaWIvbGFiZWwvYWRkLWxhYmVsLmpzIiwibGliL2xhYmVsL2FkZC1zdmctbGFiZWwuanMiLCJsaWIvbGFiZWwvYWRkLXRleHQtbGFiZWwuanMiLCJsaWIvbG9kYXNoLmpzIiwibGliL3Bvc2l0aW9uLWNsdXN0ZXJzLmpzIiwibGliL3Bvc2l0aW9uLWVkZ2UtbGFiZWxzLmpzIiwibGliL3Bvc2l0aW9uLW5vZGVzLmpzIiwibGliL3JlbmRlci5qcyIsImxpYi9zaGFwZXMuanMiLCJsaWIvdXRpbC5qcyIsImxpYi92ZXJzaW9uLmpzIl0sIm5hbWVzIjpbXSwibWFwcGluZ3MiOiJBQUFBO0FDQUE7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7O0FDOUJBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTs7QUM5REE7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTs7QUMzQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBOztBQ25DQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBOztBQ2xJQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBOztBQzFEQTtBQUNBO0FBQ0E7O0FDRkE7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7O0FDZkE7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7O0FDZkE7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTs7QUNQQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBOztBQ1BBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7O0FDekJBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7O0FDdEVBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTs7QUNMQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBOztBQ3ZEQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7O0FDaENBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7O0FDM0NBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7O0FDckNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7O0FDYkE7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7O0FDN0NBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBOztBQ2ZBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7O0FDbENBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7O0FDdEJBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBOztBQ3JCQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7O0FDdktBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBOztBQ2pGQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTs7QUN0REE7QUFDQSIsImZpbGUiOiJnZW5lcmF0ZWQuanMiLCJzb3VyY2VSb290IjoiIiwic291cmNlc0NvbnRlbnQiOlsiKGZ1bmN0aW9uIGUodCxuLHIpe2Z1bmN0aW9uIHMobyx1KXtpZighbltvXSl7aWYoIXRbb10pe3ZhciBhPXR5cGVvZiByZXF1aXJlPT1cImZ1bmN0aW9uXCImJnJlcXVpcmU7aWYoIXUmJmEpcmV0dXJuIGEobywhMCk7aWYoaSlyZXR1cm4gaShvLCEwKTt2YXIgZj1uZXcgRXJyb3IoXCJDYW5ub3QgZmluZCBtb2R1bGUgJ1wiK28rXCInXCIpO3Rocm93IGYuY29kZT1cIk1PRFVMRV9OT1RfRk9VTkRcIixmfXZhciBsPW5bb109e2V4cG9ydHM6e319O3Rbb11bMF0uY2FsbChsLmV4cG9ydHMsZnVuY3Rpb24oZSl7dmFyIG49dFtvXVsxXVtlXTtyZXR1cm4gcyhuP246ZSl9LGwsbC5leHBvcnRzLGUsdCxuLHIpfXJldHVybiBuW29dLmV4cG9ydHN9dmFyIGk9dHlwZW9mIHJlcXVpcmU9PVwiZnVuY3Rpb25cIiYmcmVxdWlyZTtmb3IodmFyIG89MDtvPHIubGVuZ3RoO28rKylzKHJbb10pO3JldHVybiBzfSkiLCIvKipcbiAqIEBsaWNlbnNlXG4gKiBDb3B5cmlnaHQgKGMpIDIwMTItMjAxMyBDaHJpcyBQZXR0aXR0XG4gKlxuICogUGVybWlzc2lvbiBpcyBoZXJlYnkgZ3JhbnRlZCwgZnJlZSBvZiBjaGFyZ2UsIHRvIGFueSBwZXJzb24gb2J0YWluaW5nIGEgY29weVxuICogb2YgdGhpcyBzb2Z0d2FyZSBhbmQgYXNzb2NpYXRlZCBkb2N1bWVudGF0aW9uIGZpbGVzICh0aGUgXCJTb2Z0d2FyZVwiKSwgdG8gZGVhbFxuICogaW4gdGhlIFNvZnR3YXJlIHdpdGhvdXQgcmVzdHJpY3Rpb24sIGluY2x1ZGluZyB3aXRob3V0IGxpbWl0YXRpb24gdGhlIHJpZ2h0c1xuICogdG8gdXNlLCBjb3B5LCBtb2RpZnksIG1lcmdlLCBwdWJsaXNoLCBkaXN0cmlidXRlLCBzdWJsaWNlbnNlLCBhbmQvb3Igc2VsbFxuICogY29waWVzIG9mIHRoZSBTb2Z0d2FyZSwgYW5kIHRvIHBlcm1pdCBwZXJzb25zIHRvIHdob20gdGhlIFNvZnR3YXJlIGlzXG4gKiBmdXJuaXNoZWQgdG8gZG8gc28sIHN1YmplY3QgdG8gdGhlIGZvbGxvd2luZyBjb25kaXRpb25zOlxuICpcbiAqIFRoZSBhYm92ZSBjb3B5cmlnaHQgbm90aWNlIGFuZCB0aGlzIHBlcm1pc3Npb24gbm90aWNlIHNoYWxsIGJlIGluY2x1ZGVkIGluXG4gKiBhbGwgY29waWVzIG9yIHN1YnN0YW50aWFsIHBvcnRpb25zIG9mIHRoZSBTb2Z0d2FyZS5cbiAqXG4gKiBUSEUgU09GVFdBUkUgSVMgUFJPVklERUQgXCJBUyBJU1wiLCBXSVRIT1VUIFdBUlJBTlRZIE9GIEFOWSBLSU5ELCBFWFBSRVNTIE9SXG4gKiBJTVBMSUVELCBJTkNMVURJTkcgQlVUIE5PVCBMSU1JVEVEIFRPIFRIRSBXQVJSQU5USUVTIE9GIE1FUkNIQU5UQUJJTElUWSxcbiAqIEZJVE5FU1MgRk9SIEEgUEFSVElDVUxBUiBQVVJQT1NFIEFORCBOT05JTkZSSU5HRU1FTlQuIElOIE5PIEVWRU5UIFNIQUxMIFRIRVxuICogQVVUSE9SUyBPUiBDT1BZUklHSFQgSE9MREVSUyBCRSBMSUFCTEUgRk9SIEFOWSBDTEFJTSwgREFNQUdFUyBPUiBPVEhFUlxuICogTElBQklMSVRZLCBXSEVUSEVSIElOIEFOIEFDVElPTiBPRiBDT05UUkFDVCwgVE9SVCBPUiBPVEhFUldJU0UsIEFSSVNJTkcgRlJPTSxcbiAqIE9VVCBPRiBPUiBJTiBDT05ORUNUSU9OIFdJVEggVEhFIFNPRlRXQVJFIE9SIFRIRSBVU0UgT1IgT1RIRVIgREVBTElOR1MgSU5cbiAqIFRIRSBTT0ZUV0FSRS5cbiAqL1xubW9kdWxlLmV4cG9ydHMgPSAge1xuICBncmFwaGxpYjogcmVxdWlyZShcIi4vbGliL2dyYXBobGliXCIpLFxuICBkYWdyZTogcmVxdWlyZShcIi4vbGliL2RhZ3JlXCIpLFxuICBpbnRlcnNlY3Q6IHJlcXVpcmUoXCIuL2xpYi9pbnRlcnNlY3RcIiksXG4gIHJlbmRlcjogcmVxdWlyZShcIi4vbGliL3JlbmRlclwiKSxcbiAgdXRpbDogcmVxdWlyZShcIi4vbGliL3V0aWxcIiksXG4gIHZlcnNpb246IHJlcXVpcmUoXCIuL2xpYi92ZXJzaW9uXCIpXG59O1xuIiwidmFyIHV0aWwgPSByZXF1aXJlKFwiLi91dGlsXCIpO1xuXG5tb2R1bGUuZXhwb3J0cyA9IHtcbiAgXCJkZWZhdWx0XCI6IG5vcm1hbCxcbiAgXCJub3JtYWxcIjogbm9ybWFsLFxuICBcInZlZVwiOiB2ZWUsXG4gIFwidW5kaXJlY3RlZFwiOiB1bmRpcmVjdGVkXG59O1xuXG5mdW5jdGlvbiBub3JtYWwocGFyZW50LCBpZCwgZWRnZSwgdHlwZSkge1xuICB2YXIgbWFya2VyID0gcGFyZW50LmFwcGVuZChcIm1hcmtlclwiKVxuICAgIC5hdHRyKFwiaWRcIiwgaWQpXG4gICAgLmF0dHIoXCJ2aWV3Qm94XCIsIFwiMCAwIDEwIDEwXCIpXG4gICAgLmF0dHIoXCJyZWZYXCIsIDkpXG4gICAgLmF0dHIoXCJyZWZZXCIsIDUpXG4gICAgLmF0dHIoXCJtYXJrZXJVbml0c1wiLCBcInN0cm9rZVdpZHRoXCIpXG4gICAgLmF0dHIoXCJtYXJrZXJXaWR0aFwiLCA4KVxuICAgIC5hdHRyKFwibWFya2VySGVpZ2h0XCIsIDYpXG4gICAgLmF0dHIoXCJvcmllbnRcIiwgXCJhdXRvXCIpO1xuXG4gIHZhciBwYXRoID0gbWFya2VyLmFwcGVuZChcInBhdGhcIilcbiAgICAuYXR0cihcImRcIiwgXCJNIDAgMCBMIDEwIDUgTCAwIDEwIHpcIilcbiAgICAuc3R5bGUoXCJzdHJva2Utd2lkdGhcIiwgMSlcbiAgICAuc3R5bGUoXCJzdHJva2UtZGFzaGFycmF5XCIsIFwiMSwwXCIpO1xuICB1dGlsLmFwcGx5U3R5bGUocGF0aCwgZWRnZVt0eXBlICsgXCJTdHlsZVwiXSk7XG59XG5cbmZ1bmN0aW9uIHZlZShwYXJlbnQsIGlkLCBlZGdlLCB0eXBlKSB7XG4gIHZhciBtYXJrZXIgPSBwYXJlbnQuYXBwZW5kKFwibWFya2VyXCIpXG4gICAgLmF0dHIoXCJpZFwiLCBpZClcbiAgICAuYXR0cihcInZpZXdCb3hcIiwgXCIwIDAgMTAgMTBcIilcbiAgICAuYXR0cihcInJlZlhcIiwgOSlcbiAgICAuYXR0cihcInJlZllcIiwgNSlcbiAgICAuYXR0cihcIm1hcmtlclVuaXRzXCIsIFwic3Ryb2tlV2lkdGhcIilcbiAgICAuYXR0cihcIm1hcmtlcldpZHRoXCIsIDgpXG4gICAgLmF0dHIoXCJtYXJrZXJIZWlnaHRcIiwgNilcbiAgICAuYXR0cihcIm9yaWVudFwiLCBcImF1dG9cIik7XG5cbiAgdmFyIHBhdGggPSBtYXJrZXIuYXBwZW5kKFwicGF0aFwiKVxuICAgIC5hdHRyKFwiZFwiLCBcIk0gMCAwIEwgMTAgNSBMIDAgMTAgTCA0IDUgelwiKVxuICAgIC5zdHlsZShcInN0cm9rZS13aWR0aFwiLCAxKVxuICAgIC5zdHlsZShcInN0cm9rZS1kYXNoYXJyYXlcIiwgXCIxLDBcIik7XG4gIHV0aWwuYXBwbHlTdHlsZShwYXRoLCBlZGdlW3R5cGUgKyBcIlN0eWxlXCJdKTtcbn1cblxuZnVuY3Rpb24gdW5kaXJlY3RlZChwYXJlbnQsIGlkLCBlZGdlLCB0eXBlKSB7XG4gIHZhciBtYXJrZXIgPSBwYXJlbnQuYXBwZW5kKFwibWFya2VyXCIpXG4gICAgLmF0dHIoXCJpZFwiLCBpZClcbiAgICAuYXR0cihcInZpZXdCb3hcIiwgXCIwIDAgMTAgMTBcIilcbiAgICAuYXR0cihcInJlZlhcIiwgOSlcbiAgICAuYXR0cihcInJlZllcIiwgNSlcbiAgICAuYXR0cihcIm1hcmtlclVuaXRzXCIsIFwic3Ryb2tlV2lkdGhcIilcbiAgICAuYXR0cihcIm1hcmtlcldpZHRoXCIsIDgpXG4gICAgLmF0dHIoXCJtYXJrZXJIZWlnaHRcIiwgNilcbiAgICAuYXR0cihcIm9yaWVudFwiLCBcImF1dG9cIik7XG5cbiAgdmFyIHBhdGggPSBtYXJrZXIuYXBwZW5kKFwicGF0aFwiKVxuICAgIC5hdHRyKFwiZFwiLCBcIk0gMCA1IEwgMTAgNVwiKVxuICAgIC5zdHlsZShcInN0cm9rZS13aWR0aFwiLCAxKVxuICAgIC5zdHlsZShcInN0cm9rZS1kYXNoYXJyYXlcIiwgXCIxLDBcIik7XG4gIHV0aWwuYXBwbHlTdHlsZShwYXRoLCBlZGdlW3R5cGUgKyBcIlN0eWxlXCJdKTtcbn1cbiIsInZhciB1dGlsID0gcmVxdWlyZShcIi4vdXRpbFwiKSxcbiAgICBhZGRMYWJlbCA9IHJlcXVpcmUoXCIuL2xhYmVsL2FkZC1sYWJlbFwiKTtcblxubW9kdWxlLmV4cG9ydHMgPSBjcmVhdGVDbHVzdGVycztcblxuZnVuY3Rpb24gY3JlYXRlQ2x1c3RlcnMoc2VsZWN0aW9uLCBnKSB7XG4gIHZhciBjbHVzdGVycyA9IGcubm9kZXMoKS5maWx0ZXIoZnVuY3Rpb24odikgeyByZXR1cm4gdXRpbC5pc1N1YmdyYXBoKGcsIHYpOyB9KSxcbiAgICAgIHN2Z0NsdXN0ZXJzID0gc2VsZWN0aW9uLnNlbGVjdEFsbChcImcuY2x1c3RlclwiKVxuICAgICAgICAuZGF0YShjbHVzdGVycywgZnVuY3Rpb24odikgeyByZXR1cm4gdjsgfSk7XG5cbiAgc3ZnQ2x1c3RlcnMuc2VsZWN0QWxsKFwiKlwiKS5yZW1vdmUoKTtcbiAgc3ZnQ2x1c3RlcnMuZW50ZXIoKVxuICAgIC5hcHBlbmQoXCJnXCIpXG4gICAgICAuYXR0cihcImNsYXNzXCIsIFwiY2x1c3RlclwiKVxuICAgICAgLmF0dHIoXCJpZFwiLGZ1bmN0aW9uKHYpe1xuICAgICAgICAgIHZhciBub2RlID0gZy5ub2RlKHYpO1xuICAgICAgICAgIHJldHVybiBub2RlLmlkO1xuICAgICAgfSlcbiAgICAgIC5zdHlsZShcIm9wYWNpdHlcIiwgMCk7XG5cbiAgdXRpbC5hcHBseVRyYW5zaXRpb24oc3ZnQ2x1c3RlcnMsIGcpXG4gICAgLnN0eWxlKFwib3BhY2l0eVwiLCAxKTtcblxuICBzdmdDbHVzdGVycy5lYWNoKGZ1bmN0aW9uKHYpIHtcbiAgICB2YXIgbm9kZSA9IGcubm9kZSh2KSxcbiAgICAgICAgdGhpc0dyb3VwID0gZDMuc2VsZWN0KHRoaXMpO1xuICAgIGQzLnNlbGVjdCh0aGlzKS5hcHBlbmQoXCJyZWN0XCIpO1xuICAgIHZhciBsYWJlbEdyb3VwID0gdGhpc0dyb3VwLmFwcGVuZChcImdcIikuYXR0cihcImNsYXNzXCIsIFwibGFiZWxcIik7XG4gICAgYWRkTGFiZWwobGFiZWxHcm91cCwgbm9kZSwgbm9kZS5jbHVzdGVyTGFiZWxQb3MpO1xuICB9KTtcblxuICBzdmdDbHVzdGVycy5zZWxlY3RBbGwoXCJyZWN0XCIpLmVhY2goZnVuY3Rpb24oYykge1xuICAgIHZhciBub2RlID0gZy5ub2RlKGMpO1xuICAgIHZhciBkb21DbHVzdGVyID0gZDMuc2VsZWN0KHRoaXMpO1xuICAgIHV0aWwuYXBwbHlTdHlsZShkb21DbHVzdGVyLCBub2RlLnN0eWxlKTtcbiAgfSk7XG5cbiAgdXRpbC5hcHBseVRyYW5zaXRpb24oc3ZnQ2x1c3RlcnMuZXhpdCgpLCBnKVxuICAgIC5zdHlsZShcIm9wYWNpdHlcIiwgMClcbiAgICAucmVtb3ZlKCk7XG5cbiAgcmV0dXJuIHN2Z0NsdXN0ZXJzO1xufVxuIiwiXCJ1c2Ugc3RyaWN0XCI7XG5cbnZhciBfID0gcmVxdWlyZShcIi4vbG9kYXNoXCIpLFxuICAgIGFkZExhYmVsID0gcmVxdWlyZShcIi4vbGFiZWwvYWRkLWxhYmVsXCIpLFxuICAgIHV0aWwgPSByZXF1aXJlKFwiLi91dGlsXCIpLFxuICAgIGQzID0gcmVxdWlyZShcIi4vZDNcIik7XG5cbm1vZHVsZS5leHBvcnRzID0gY3JlYXRlRWRnZUxhYmVscztcblxuZnVuY3Rpb24gY3JlYXRlRWRnZUxhYmVscyhzZWxlY3Rpb24sIGcpIHtcbiAgdmFyIHN2Z0VkZ2VMYWJlbHMgPSBzZWxlY3Rpb24uc2VsZWN0QWxsKFwiZy5lZGdlTGFiZWxcIilcbiAgICAuZGF0YShnLmVkZ2VzKCksIGZ1bmN0aW9uKGUpIHsgcmV0dXJuIHV0aWwuZWRnZVRvSWQoZSk7IH0pXG4gICAgLmNsYXNzZWQoXCJ1cGRhdGVcIiwgdHJ1ZSk7XG5cbiAgc3ZnRWRnZUxhYmVscy5zZWxlY3RBbGwoXCIqXCIpLnJlbW92ZSgpO1xuICBzdmdFZGdlTGFiZWxzLmVudGVyKClcbiAgICAuYXBwZW5kKFwiZ1wiKVxuICAgICAgLmNsYXNzZWQoXCJlZGdlTGFiZWxcIiwgdHJ1ZSlcbiAgICAgIC5zdHlsZShcIm9wYWNpdHlcIiwgMCk7XG4gIHN2Z0VkZ2VMYWJlbHMuZWFjaChmdW5jdGlvbihlKSB7XG4gICAgdmFyIGVkZ2UgPSBnLmVkZ2UoZSksXG4gICAgICAgIGxhYmVsID0gYWRkTGFiZWwoZDMuc2VsZWN0KHRoaXMpLCBnLmVkZ2UoZSksIDAsIDApLmNsYXNzZWQoXCJsYWJlbFwiLCB0cnVlKSxcbiAgICAgICAgYmJveCA9IGxhYmVsLm5vZGUoKS5nZXRCQm94KCk7XG5cbiAgICBpZiAoZWRnZS5sYWJlbElkKSB7IGxhYmVsLmF0dHIoXCJpZFwiLCBlZGdlLmxhYmVsSWQpOyB9XG4gICAgaWYgKCFfLmhhcyhlZGdlLCBcIndpZHRoXCIpKSB7IGVkZ2Uud2lkdGggPSBiYm94LndpZHRoOyB9XG4gICAgaWYgKCFfLmhhcyhlZGdlLCBcImhlaWdodFwiKSkgeyBlZGdlLmhlaWdodCA9IGJib3guaGVpZ2h0OyB9XG4gIH0pO1xuXG4gIHV0aWwuYXBwbHlUcmFuc2l0aW9uKHN2Z0VkZ2VMYWJlbHMuZXhpdCgpLCBnKVxuICAgIC5zdHlsZShcIm9wYWNpdHlcIiwgMClcbiAgICAucmVtb3ZlKCk7XG5cbiAgcmV0dXJuIHN2Z0VkZ2VMYWJlbHM7XG59XG4iLCJcInVzZSBzdHJpY3RcIjtcblxudmFyIF8gPSByZXF1aXJlKFwiLi9sb2Rhc2hcIiksXG4gICAgaW50ZXJzZWN0Tm9kZSA9IHJlcXVpcmUoXCIuL2ludGVyc2VjdC9pbnRlcnNlY3Qtbm9kZVwiKSxcbiAgICB1dGlsID0gcmVxdWlyZShcIi4vdXRpbFwiKSxcbiAgICBkMyA9IHJlcXVpcmUoXCIuL2QzXCIpO1xuXG5tb2R1bGUuZXhwb3J0cyA9IGNyZWF0ZUVkZ2VQYXRocztcblxuZnVuY3Rpb24gY3JlYXRlRWRnZVBhdGhzKHNlbGVjdGlvbiwgZywgYXJyb3dzKSB7XG4gIHZhciBzdmdQYXRocyA9IHNlbGVjdGlvbi5zZWxlY3RBbGwoXCJnLmVkZ2VQYXRoXCIpXG4gICAgLmRhdGEoZy5lZGdlcygpLCBmdW5jdGlvbihlKSB7IHJldHVybiB1dGlsLmVkZ2VUb0lkKGUpOyB9KVxuICAgIC5jbGFzc2VkKFwidXBkYXRlXCIsIHRydWUpO1xuXG4gIGVudGVyKHN2Z1BhdGhzLCBnKTtcbiAgZXhpdChzdmdQYXRocywgZyk7XG5cbiAgdXRpbC5hcHBseVRyYW5zaXRpb24oc3ZnUGF0aHMsIGcpXG4gICAgLnN0eWxlKFwib3BhY2l0eVwiLCAxKTtcblxuICAvLyBTYXZlIERPTSBlbGVtZW50IGluIHRoZSBwYXRoIGdyb3VwLCBhbmQgc2V0IElEIGFuZCBjbGFzc1xuICBzdmdQYXRocy5lYWNoKGZ1bmN0aW9uKGUpIHtcbiAgICB2YXIgZG9tRWRnZSA9IGQzLnNlbGVjdCh0aGlzKTtcbiAgICB2YXIgZWRnZSA9IGcuZWRnZShlKTtcbiAgICBlZGdlLmVsZW0gPSB0aGlzO1xuXG4gICAgaWYgKGVkZ2UuaWQpIHtcbiAgICAgIGRvbUVkZ2UuYXR0cihcImlkXCIsIGVkZ2UuaWQpO1xuICAgIH1cblxuICAgIHV0aWwuYXBwbHlDbGFzcyhkb21FZGdlLCBlZGdlW1wiY2xhc3NcIl0sXG4gICAgICAoZG9tRWRnZS5jbGFzc2VkKFwidXBkYXRlXCIpID8gXCJ1cGRhdGUgXCIgOiBcIlwiKSArIFwiZWRnZVBhdGhcIik7XG4gIH0pO1xuXG4gIHN2Z1BhdGhzLnNlbGVjdEFsbChcInBhdGgucGF0aFwiKVxuICAgIC5lYWNoKGZ1bmN0aW9uKGUpIHtcbiAgICAgIHZhciBlZGdlID0gZy5lZGdlKGUpO1xuICAgICAgZWRnZS5hcnJvd2hlYWRJZCA9IF8udW5pcXVlSWQoXCJhcnJvd2hlYWRcIik7XG5cbiAgICAgIHZhciBkb21FZGdlID0gZDMuc2VsZWN0KHRoaXMpXG4gICAgICAgIC5hdHRyKFwibWFya2VyLWVuZFwiLCBmdW5jdGlvbigpIHtcbiAgICAgICAgICByZXR1cm4gXCJ1cmwoI1wiICsgZWRnZS5hcnJvd2hlYWRJZCArIFwiKVwiO1xuICAgICAgICB9KVxuICAgICAgICAuc3R5bGUoXCJmaWxsXCIsIFwibm9uZVwiKTtcblxuICAgICAgdXRpbC5hcHBseVRyYW5zaXRpb24oZG9tRWRnZSwgZylcbiAgICAgICAgLmF0dHIoXCJkXCIsIGZ1bmN0aW9uKGUpIHsgcmV0dXJuIGNhbGNQb2ludHMoZywgZSk7IH0pO1xuXG4gICAgICB1dGlsLmFwcGx5U3R5bGUoZG9tRWRnZSwgZWRnZS5zdHlsZSk7XG4gICAgfSk7XG5cbiAgc3ZnUGF0aHMuc2VsZWN0QWxsKFwiZGVmcyAqXCIpLnJlbW92ZSgpO1xuICBzdmdQYXRocy5zZWxlY3RBbGwoXCJkZWZzXCIpXG4gICAgLmVhY2goZnVuY3Rpb24oZSkge1xuICAgICAgdmFyIGVkZ2UgPSBnLmVkZ2UoZSksXG4gICAgICAgICAgYXJyb3doZWFkID0gYXJyb3dzW2VkZ2UuYXJyb3doZWFkXTtcbiAgICAgIGFycm93aGVhZChkMy5zZWxlY3QodGhpcyksIGVkZ2UuYXJyb3doZWFkSWQsIGVkZ2UsIFwiYXJyb3doZWFkXCIpO1xuICAgIH0pO1xuXG4gIHJldHVybiBzdmdQYXRocztcbn1cblxuZnVuY3Rpb24gY2FsY1BvaW50cyhnLCBlKSB7XG4gIHZhciBlZGdlID0gZy5lZGdlKGUpLFxuICAgICAgdGFpbCA9IGcubm9kZShlLnYpLFxuICAgICAgaGVhZCA9IGcubm9kZShlLncpLFxuICAgICAgcG9pbnRzID0gZWRnZS5wb2ludHMuc2xpY2UoMSwgZWRnZS5wb2ludHMubGVuZ3RoIC0gMSk7XG4gIHBvaW50cy51bnNoaWZ0KGludGVyc2VjdE5vZGUodGFpbCwgcG9pbnRzWzBdKSk7XG4gIHBvaW50cy5wdXNoKGludGVyc2VjdE5vZGUoaGVhZCwgcG9pbnRzW3BvaW50cy5sZW5ndGggLSAxXSkpO1xuXG4gIHJldHVybiBjcmVhdGVMaW5lKGVkZ2UsIHBvaW50cyk7XG59XG5cbmZ1bmN0aW9uIGNyZWF0ZUxpbmUoZWRnZSwgcG9pbnRzKSB7XG4gIHZhciBsaW5lID0gZDMuc3ZnLmxpbmUoKVxuICAgIC54KGZ1bmN0aW9uKGQpIHsgcmV0dXJuIGQueDsgfSlcbiAgICAueShmdW5jdGlvbihkKSB7IHJldHVybiBkLnk7IH0pO1xuXG4gIGlmIChfLmhhcyhlZGdlLCBcImxpbmVJbnRlcnBvbGF0ZVwiKSkge1xuICAgIGxpbmUuaW50ZXJwb2xhdGUoZWRnZS5saW5lSW50ZXJwb2xhdGUpO1xuICB9XG5cbiAgaWYgKF8uaGFzKGVkZ2UsIFwibGluZVRlbnNpb25cIikpIHtcbiAgICBsaW5lLnRlbnNpb24oTnVtYmVyKGVkZ2UubGluZVRlbnNpb24pKTtcbiAgfVxuXG4gIHJldHVybiBsaW5lKHBvaW50cyk7XG59XG5cbmZ1bmN0aW9uIGdldENvb3JkcyhlbGVtKSB7XG4gIHZhciBiYm94ID0gZWxlbS5nZXRCQm94KCksXG4gICAgICBtYXRyaXggPSBlbGVtLmdldFRyYW5zZm9ybVRvRWxlbWVudChlbGVtLm93bmVyU1ZHRWxlbWVudClcbiAgICAgICAgLnRyYW5zbGF0ZShiYm94LndpZHRoIC8gMiwgYmJveC5oZWlnaHQgLyAyKTtcbiAgcmV0dXJuIHsgeDogbWF0cml4LmUsIHk6IG1hdHJpeC5mIH07XG59XG5cbmZ1bmN0aW9uIGVudGVyKHN2Z1BhdGhzLCBnKSB7XG4gIHZhciBzdmdQYXRoc0VudGVyID0gc3ZnUGF0aHMuZW50ZXIoKVxuICAgIC5hcHBlbmQoXCJnXCIpXG4gICAgICAuYXR0cihcImNsYXNzXCIsIFwiZWRnZVBhdGhcIilcbiAgICAgIC5zdHlsZShcIm9wYWNpdHlcIiwgMCk7XG4gIHN2Z1BhdGhzRW50ZXIuYXBwZW5kKFwicGF0aFwiKVxuICAgIC5hdHRyKFwiY2xhc3NcIiwgXCJwYXRoXCIpXG4gICAgLmF0dHIoXCJkXCIsIGZ1bmN0aW9uKGUpIHtcbiAgICAgIHZhciBlZGdlID0gZy5lZGdlKGUpLFxuICAgICAgICAgIHNvdXJjZUVsZW0gPSBnLm5vZGUoZS52KS5lbGVtLFxuICAgICAgICAgIHBvaW50cyA9IF8ucmFuZ2UoZWRnZS5wb2ludHMubGVuZ3RoKS5tYXAoZnVuY3Rpb24oKSB7IHJldHVybiBnZXRDb29yZHMoc291cmNlRWxlbSk7IH0pO1xuICAgICAgcmV0dXJuIGNyZWF0ZUxpbmUoZWRnZSwgcG9pbnRzKTtcbiAgICB9KTtcbiAgc3ZnUGF0aHNFbnRlci5hcHBlbmQoXCJkZWZzXCIpO1xufVxuXG5mdW5jdGlvbiBleGl0KHN2Z1BhdGhzLCBnKSB7XG4gIHZhciBzdmdQYXRoRXhpdCA9IHN2Z1BhdGhzLmV4aXQoKTtcbiAgdXRpbC5hcHBseVRyYW5zaXRpb24oc3ZnUGF0aEV4aXQsIGcpXG4gICAgLnN0eWxlKFwib3BhY2l0eVwiLCAwKVxuICAgIC5yZW1vdmUoKTtcblxuICB1dGlsLmFwcGx5VHJhbnNpdGlvbihzdmdQYXRoRXhpdC5zZWxlY3QoXCJwYXRoLnBhdGhcIiksIGcpXG4gICAgLmF0dHIoXCJkXCIsIGZ1bmN0aW9uKGUpIHtcbiAgICAgIHZhciBzb3VyY2UgPSBnLm5vZGUoZS52KTtcblxuICAgICAgaWYgKHNvdXJjZSkge1xuICAgICAgICB2YXIgcG9pbnRzID0gXy5yYW5nZSh0aGlzLnBhdGhTZWdMaXN0Lmxlbmd0aCkubWFwKGZ1bmN0aW9uKCkgeyByZXR1cm4gc291cmNlOyB9KTtcbiAgICAgICAgcmV0dXJuIGNyZWF0ZUxpbmUoe30sIHBvaW50cyk7XG4gICAgICB9IGVsc2Uge1xuICAgICAgICByZXR1cm4gZDMuc2VsZWN0KHRoaXMpLmF0dHIoXCJkXCIpO1xuICAgICAgfVxuICAgIH0pO1xufVxuIiwiXCJ1c2Ugc3RyaWN0XCI7XG5cbnZhciBfID0gcmVxdWlyZShcIi4vbG9kYXNoXCIpLFxuICAgIGFkZExhYmVsID0gcmVxdWlyZShcIi4vbGFiZWwvYWRkLWxhYmVsXCIpLFxuICAgIHV0aWwgPSByZXF1aXJlKFwiLi91dGlsXCIpLFxuICAgIGQzID0gcmVxdWlyZShcIi4vZDNcIik7XG5cbm1vZHVsZS5leHBvcnRzID0gY3JlYXRlTm9kZXM7XG5cbmZ1bmN0aW9uIGNyZWF0ZU5vZGVzKHNlbGVjdGlvbiwgZywgc2hhcGVzKSB7XG4gIHZhciBzaW1wbGVOb2RlcyA9IGcubm9kZXMoKS5maWx0ZXIoZnVuY3Rpb24odikgeyByZXR1cm4gIXV0aWwuaXNTdWJncmFwaChnLCB2KTsgfSk7XG4gIHZhciBzdmdOb2RlcyA9IHNlbGVjdGlvbi5zZWxlY3RBbGwoXCJnLm5vZGVcIilcbiAgICAuZGF0YShzaW1wbGVOb2RlcywgZnVuY3Rpb24odikgeyByZXR1cm4gdjsgfSlcbiAgICAuY2xhc3NlZChcInVwZGF0ZVwiLCB0cnVlKTtcblxuICBzdmdOb2Rlcy5zZWxlY3RBbGwoXCIqXCIpLnJlbW92ZSgpO1xuICBzdmdOb2Rlcy5lbnRlcigpXG4gICAgLmFwcGVuZChcImdcIilcbiAgICAgIC5hdHRyKFwiY2xhc3NcIiwgXCJub2RlXCIpXG4gICAgICAuc3R5bGUoXCJvcGFjaXR5XCIsIDApO1xuICBzdmdOb2Rlcy5lYWNoKGZ1bmN0aW9uKHYpIHtcbiAgICB2YXIgbm9kZSA9IGcubm9kZSh2KSxcbiAgICAgICAgdGhpc0dyb3VwID0gZDMuc2VsZWN0KHRoaXMpLFxuICAgICAgICBsYWJlbEdyb3VwID0gdGhpc0dyb3VwLmFwcGVuZChcImdcIikuYXR0cihcImNsYXNzXCIsIFwibGFiZWxcIiksXG4gICAgICAgIGxhYmVsRG9tID0gYWRkTGFiZWwobGFiZWxHcm91cCwgbm9kZSksXG4gICAgICAgIHNoYXBlID0gc2hhcGVzW25vZGUuc2hhcGVdLFxuICAgICAgICBiYm94ID0gXy5waWNrKGxhYmVsRG9tLm5vZGUoKS5nZXRCQm94KCksIFwid2lkdGhcIiwgXCJoZWlnaHRcIik7XG5cbiAgICBub2RlLmVsZW0gPSB0aGlzO1xuXG4gICAgaWYgKG5vZGUuaWQpIHsgdGhpc0dyb3VwLmF0dHIoXCJpZFwiLCBub2RlLmlkKTsgfVxuICAgIGlmIChub2RlLmxhYmVsSWQpIHsgbGFiZWxHcm91cC5hdHRyKFwiaWRcIiwgbm9kZS5sYWJlbElkKTsgfVxuICAgIHV0aWwuYXBwbHlDbGFzcyh0aGlzR3JvdXAsIG5vZGVbXCJjbGFzc1wiXSxcbiAgICAgICh0aGlzR3JvdXAuY2xhc3NlZChcInVwZGF0ZVwiKSA/IFwidXBkYXRlIFwiIDogXCJcIikgKyBcIm5vZGVcIik7XG5cbiAgICBpZiAoXy5oYXMobm9kZSwgXCJ3aWR0aFwiKSkgeyBiYm94LndpZHRoID0gbm9kZS53aWR0aDsgfVxuICAgIGlmIChfLmhhcyhub2RlLCBcImhlaWdodFwiKSkgeyBiYm94LmhlaWdodCA9IG5vZGUuaGVpZ2h0OyB9XG5cbiAgICBiYm94LndpZHRoICs9IG5vZGUucGFkZGluZ0xlZnQgKyBub2RlLnBhZGRpbmdSaWdodDtcbiAgICBiYm94LmhlaWdodCArPSBub2RlLnBhZGRpbmdUb3AgKyBub2RlLnBhZGRpbmdCb3R0b207XG4gICAgbGFiZWxHcm91cC5hdHRyKFwidHJhbnNmb3JtXCIsIFwidHJhbnNsYXRlKFwiICtcbiAgICAgICgobm9kZS5wYWRkaW5nTGVmdCAtIG5vZGUucGFkZGluZ1JpZ2h0KSAvIDIpICsgXCIsXCIgK1xuICAgICAgKChub2RlLnBhZGRpbmdUb3AgLSBub2RlLnBhZGRpbmdCb3R0b20pIC8gMikgKyBcIilcIik7XG5cbiAgICB2YXIgc2hhcGVTdmcgPSBzaGFwZShkMy5zZWxlY3QodGhpcyksIGJib3gsIG5vZGUpO1xuICAgIHV0aWwuYXBwbHlTdHlsZShzaGFwZVN2Zywgbm9kZS5zdHlsZSk7XG5cbiAgICB2YXIgc2hhcGVCQm94ID0gc2hhcGVTdmcubm9kZSgpLmdldEJCb3goKTtcbiAgICBub2RlLndpZHRoID0gc2hhcGVCQm94LndpZHRoO1xuICAgIG5vZGUuaGVpZ2h0ID0gc2hhcGVCQm94LmhlaWdodDtcbiAgfSk7XG5cbiAgdXRpbC5hcHBseVRyYW5zaXRpb24oc3ZnTm9kZXMuZXhpdCgpLCBnKVxuICAgIC5zdHlsZShcIm9wYWNpdHlcIiwgMClcbiAgICAucmVtb3ZlKCk7XG5cbiAgcmV0dXJuIHN2Z05vZGVzO1xufVxuIiwiLy8gU3R1YiB0byBnZXQgRDMgZWl0aGVyIHZpYSBOUE0gb3IgZnJvbSB0aGUgZ2xvYmFsIG9iamVjdFxubW9kdWxlLmV4cG9ydHMgPSB3aW5kb3cuZDM7XG4iLCIvKiBnbG9iYWwgd2luZG93ICovXG5cbnZhciBkYWdyZTtcblxuaWYgKHJlcXVpcmUpIHtcbiAgdHJ5IHtcbiAgICBkYWdyZSA9IHJlcXVpcmUoXCJkYWdyZVwiKTtcbiAgfSBjYXRjaCAoZSkge31cbn1cblxuaWYgKCFkYWdyZSkge1xuICBkYWdyZSA9IHdpbmRvdy5kYWdyZTtcbn1cblxubW9kdWxlLmV4cG9ydHMgPSBkYWdyZTtcbiIsIi8qIGdsb2JhbCB3aW5kb3cgKi9cblxudmFyIGdyYXBobGliO1xuXG5pZiAocmVxdWlyZSkge1xuICB0cnkge1xuICAgIGdyYXBobGliID0gcmVxdWlyZShcImdyYXBobGliXCIpO1xuICB9IGNhdGNoIChlKSB7fVxufVxuXG5pZiAoIWdyYXBobGliKSB7XG4gIGdyYXBobGliID0gd2luZG93LmdyYXBobGliO1xufVxuXG5tb2R1bGUuZXhwb3J0cyA9IGdyYXBobGliO1xuIiwibW9kdWxlLmV4cG9ydHMgPSB7XG4gIG5vZGU6IHJlcXVpcmUoXCIuL2ludGVyc2VjdC1ub2RlXCIpLFxuICBjaXJjbGU6IHJlcXVpcmUoXCIuL2ludGVyc2VjdC1jaXJjbGVcIiksXG4gIGVsbGlwc2U6IHJlcXVpcmUoXCIuL2ludGVyc2VjdC1lbGxpcHNlXCIpLFxuICBwb2x5Z29uOiByZXF1aXJlKFwiLi9pbnRlcnNlY3QtcG9seWdvblwiKSxcbiAgcmVjdDogcmVxdWlyZShcIi4vaW50ZXJzZWN0LXJlY3RcIilcbn07XG4iLCJ2YXIgaW50ZXJzZWN0RWxsaXBzZSA9IHJlcXVpcmUoXCIuL2ludGVyc2VjdC1lbGxpcHNlXCIpO1xuXG5tb2R1bGUuZXhwb3J0cyA9IGludGVyc2VjdENpcmNsZTtcblxuZnVuY3Rpb24gaW50ZXJzZWN0Q2lyY2xlKG5vZGUsIHJ4LCBwb2ludCkge1xuICByZXR1cm4gaW50ZXJzZWN0RWxsaXBzZShub2RlLCByeCwgcngsIHBvaW50KTtcbn1cbiIsIm1vZHVsZS5leHBvcnRzID0gaW50ZXJzZWN0RWxsaXBzZTtcblxuZnVuY3Rpb24gaW50ZXJzZWN0RWxsaXBzZShub2RlLCByeCwgcnksIHBvaW50KSB7XG4gIC8vIEZvcm11bGFlIGZyb206IGh0dHA6Ly9tYXRod29ybGQud29sZnJhbS5jb20vRWxsaXBzZS1MaW5lSW50ZXJzZWN0aW9uLmh0bWxcblxuICB2YXIgY3ggPSBub2RlLng7XG4gIHZhciBjeSA9IG5vZGUueTtcblxuICB2YXIgcHggPSBjeCAtIHBvaW50Lng7XG4gIHZhciBweSA9IGN5IC0gcG9pbnQueTtcblxuICB2YXIgZGV0ID0gTWF0aC5zcXJ0KHJ4ICogcnggKiBweSAqIHB5ICsgcnkgKiByeSAqIHB4ICogcHgpO1xuXG4gIHZhciBkeCA9IE1hdGguYWJzKHJ4ICogcnkgKiBweCAvIGRldCk7XG4gIGlmIChwb2ludC54IDwgY3gpIHtcbiAgICBkeCA9IC1keDtcbiAgfVxuICB2YXIgZHkgPSBNYXRoLmFicyhyeCAqIHJ5ICogcHkgLyBkZXQpO1xuICBpZiAocG9pbnQueSA8IGN5KSB7XG4gICAgZHkgPSAtZHk7XG4gIH1cblxuICByZXR1cm4ge3g6IGN4ICsgZHgsIHk6IGN5ICsgZHl9O1xufVxuXG4iLCJtb2R1bGUuZXhwb3J0cyA9IGludGVyc2VjdExpbmU7XG5cbi8qXG4gKiBSZXR1cm5zIHRoZSBwb2ludCBhdCB3aGljaCB0d28gbGluZXMsIHAgYW5kIHEsIGludGVyc2VjdCBvciByZXR1cm5zXG4gKiB1bmRlZmluZWQgaWYgdGhleSBkbyBub3QgaW50ZXJzZWN0LlxuICovXG5mdW5jdGlvbiBpbnRlcnNlY3RMaW5lKHAxLCBwMiwgcTEsIHEyKSB7XG4gIC8vIEFsZ29yaXRobSBmcm9tIEouIEF2cm8sIChlZC4pIEdyYXBoaWNzIEdlbXMsIE5vIDIsIE1vcmdhbiBLYXVmbWFubiwgMTk5NCxcbiAgLy8gcDcgYW5kIHA0NzMuXG5cbiAgdmFyIGExLCBhMiwgYjEsIGIyLCBjMSwgYzI7XG4gIHZhciByMSwgcjIgLCByMywgcjQ7XG4gIHZhciBkZW5vbSwgb2Zmc2V0LCBudW07XG4gIHZhciB4LCB5O1xuXG4gIC8vIENvbXB1dGUgYTEsIGIxLCBjMSwgd2hlcmUgbGluZSBqb2luaW5nIHBvaW50cyAxIGFuZCAyIGlzIEYoeCx5KSA9IGExIHggK1xuICAvLyBiMSB5ICsgYzEgPSAwLlxuICBhMSA9IHAyLnkgLSBwMS55O1xuICBiMSA9IHAxLnggLSBwMi54O1xuICBjMSA9IChwMi54ICogcDEueSkgLSAocDEueCAqIHAyLnkpO1xuXG4gIC8vIENvbXB1dGUgcjMgYW5kIHI0LlxuICByMyA9ICgoYTEgKiBxMS54KSArIChiMSAqIHExLnkpICsgYzEpO1xuICByNCA9ICgoYTEgKiBxMi54KSArIChiMSAqIHEyLnkpICsgYzEpO1xuXG4gIC8vIENoZWNrIHNpZ25zIG9mIHIzIGFuZCByNC4gSWYgYm90aCBwb2ludCAzIGFuZCBwb2ludCA0IGxpZSBvblxuICAvLyBzYW1lIHNpZGUgb2YgbGluZSAxLCB0aGUgbGluZSBzZWdtZW50cyBkbyBub3QgaW50ZXJzZWN0LlxuICBpZiAoKHIzICE9PSAwKSAmJiAocjQgIT09IDApICYmIHNhbWVTaWduKHIzLCByNCkpIHtcbiAgICByZXR1cm4gLypET05UX0lOVEVSU0VDVCovO1xuICB9XG5cbiAgLy8gQ29tcHV0ZSBhMiwgYjIsIGMyIHdoZXJlIGxpbmUgam9pbmluZyBwb2ludHMgMyBhbmQgNCBpcyBHKHgseSkgPSBhMiB4ICsgYjIgeSArIGMyID0gMFxuICBhMiA9IHEyLnkgLSBxMS55O1xuICBiMiA9IHExLnggLSBxMi54O1xuICBjMiA9IChxMi54ICogcTEueSkgLSAocTEueCAqIHEyLnkpO1xuXG4gIC8vIENvbXB1dGUgcjEgYW5kIHIyXG4gIHIxID0gKGEyICogcDEueCkgKyAoYjIgKiBwMS55KSArIGMyO1xuICByMiA9IChhMiAqIHAyLngpICsgKGIyICogcDIueSkgKyBjMjtcblxuICAvLyBDaGVjayBzaWducyBvZiByMSBhbmQgcjIuIElmIGJvdGggcG9pbnQgMSBhbmQgcG9pbnQgMiBsaWVcbiAgLy8gb24gc2FtZSBzaWRlIG9mIHNlY29uZCBsaW5lIHNlZ21lbnQsIHRoZSBsaW5lIHNlZ21lbnRzIGRvXG4gIC8vIG5vdCBpbnRlcnNlY3QuXG4gIGlmICgocjEgIT09IDApICYmIChyMiAhPT0gMCkgJiYgKHNhbWVTaWduKHIxLCByMikpKSB7XG4gICAgcmV0dXJuIC8qRE9OVF9JTlRFUlNFQ1QqLztcbiAgfVxuXG4gIC8vIExpbmUgc2VnbWVudHMgaW50ZXJzZWN0OiBjb21wdXRlIGludGVyc2VjdGlvbiBwb2ludC5cbiAgZGVub20gPSAoYTEgKiBiMikgLSAoYTIgKiBiMSk7XG4gIGlmIChkZW5vbSA9PT0gMCkge1xuICAgIHJldHVybiAvKkNPTExJTkVBUiovO1xuICB9XG5cbiAgb2Zmc2V0ID0gTWF0aC5hYnMoZGVub20gLyAyKTtcblxuICAvLyBUaGUgZGVub20vMiBpcyB0byBnZXQgcm91bmRpbmcgaW5zdGVhZCBvZiB0cnVuY2F0aW5nLiBJdFxuICAvLyBpcyBhZGRlZCBvciBzdWJ0cmFjdGVkIHRvIHRoZSBudW1lcmF0b3IsIGRlcGVuZGluZyB1cG9uIHRoZVxuICAvLyBzaWduIG9mIHRoZSBudW1lcmF0b3IuXG4gIG51bSA9IChiMSAqIGMyKSAtIChiMiAqIGMxKTtcbiAgeCA9IChudW0gPCAwKSA/ICgobnVtIC0gb2Zmc2V0KSAvIGRlbm9tKSA6ICgobnVtICsgb2Zmc2V0KSAvIGRlbm9tKTtcblxuICBudW0gPSAoYTIgKiBjMSkgLSAoYTEgKiBjMik7XG4gIHkgPSAobnVtIDwgMCkgPyAoKG51bSAtIG9mZnNldCkgLyBkZW5vbSkgOiAoKG51bSArIG9mZnNldCkgLyBkZW5vbSk7XG5cbiAgcmV0dXJuIHsgeDogeCwgeTogeSB9O1xufVxuXG5mdW5jdGlvbiBzYW1lU2lnbihyMSwgcjIpIHtcbiAgcmV0dXJuIHIxICogcjIgPiAwO1xufVxuIiwibW9kdWxlLmV4cG9ydHMgPSBpbnRlcnNlY3ROb2RlO1xuXG5mdW5jdGlvbiBpbnRlcnNlY3ROb2RlKG5vZGUsIHBvaW50KSB7XG4gIHJldHVybiBub2RlLmludGVyc2VjdChwb2ludCk7XG59XG4iLCJ2YXIgaW50ZXJzZWN0TGluZSA9IHJlcXVpcmUoXCIuL2ludGVyc2VjdC1saW5lXCIpO1xuXG5tb2R1bGUuZXhwb3J0cyA9IGludGVyc2VjdFBvbHlnb247XG5cbi8qXG4gKiBSZXR1cm5zIHRoZSBwb2ludCAoe3gsIHl9KSBhdCB3aGljaCB0aGUgcG9pbnQgYXJndW1lbnQgaW50ZXJzZWN0cyB3aXRoIHRoZVxuICogbm9kZSBhcmd1bWVudCBhc3N1bWluZyB0aGF0IGl0IGhhcyB0aGUgc2hhcGUgc3BlY2lmaWVkIGJ5IHBvbHlnb24uXG4gKi9cbmZ1bmN0aW9uIGludGVyc2VjdFBvbHlnb24obm9kZSwgcG9seVBvaW50cywgcG9pbnQpIHtcbiAgdmFyIHgxID0gbm9kZS54O1xuICB2YXIgeTEgPSBub2RlLnk7XG5cbiAgdmFyIGludGVyc2VjdGlvbnMgPSBbXTtcblxuICB2YXIgbWluWCA9IE51bWJlci5QT1NJVElWRV9JTkZJTklUWSxcbiAgICAgIG1pblkgPSBOdW1iZXIuUE9TSVRJVkVfSU5GSU5JVFk7XG4gIHBvbHlQb2ludHMuZm9yRWFjaChmdW5jdGlvbihlbnRyeSkge1xuICAgIG1pblggPSBNYXRoLm1pbihtaW5YLCBlbnRyeS54KTtcbiAgICBtaW5ZID0gTWF0aC5taW4obWluWSwgZW50cnkueSk7XG4gIH0pO1xuXG4gIHZhciBsZWZ0ID0geDEgLSBub2RlLndpZHRoIC8gMiAtIG1pblg7XG4gIHZhciB0b3AgPSAgeTEgLSBub2RlLmhlaWdodCAvIDIgLSBtaW5ZO1xuXG4gIGZvciAodmFyIGkgPSAwOyBpIDwgcG9seVBvaW50cy5sZW5ndGg7IGkrKykge1xuICAgIHZhciBwMSA9IHBvbHlQb2ludHNbaV07XG4gICAgdmFyIHAyID0gcG9seVBvaW50c1tpIDwgcG9seVBvaW50cy5sZW5ndGggLSAxID8gaSArIDEgOiAwXTtcbiAgICB2YXIgaW50ZXJzZWN0ID0gaW50ZXJzZWN0TGluZShub2RlLCBwb2ludCxcbiAgICAgIHt4OiBsZWZ0ICsgcDEueCwgeTogdG9wICsgcDEueX0sIHt4OiBsZWZ0ICsgcDIueCwgeTogdG9wICsgcDIueX0pO1xuICAgIGlmIChpbnRlcnNlY3QpIHtcbiAgICAgIGludGVyc2VjdGlvbnMucHVzaChpbnRlcnNlY3QpO1xuICAgIH1cbiAgfVxuXG4gIGlmICghaW50ZXJzZWN0aW9ucy5sZW5ndGgpIHtcbiAgICBjb25zb2xlLmxvZyhcIk5PIElOVEVSU0VDVElPTiBGT1VORCwgUkVUVVJOIE5PREUgQ0VOVEVSXCIsIG5vZGUpO1xuICAgIHJldHVybiBub2RlO1xuICB9XG5cbiAgaWYgKGludGVyc2VjdGlvbnMubGVuZ3RoID4gMSkge1xuICAgIC8vIE1vcmUgaW50ZXJzZWN0aW9ucywgZmluZCB0aGUgb25lIG5lYXJlc3QgdG8gZWRnZSBlbmQgcG9pbnRcbiAgICBpbnRlcnNlY3Rpb25zLnNvcnQoZnVuY3Rpb24ocCwgcSkge1xuICAgICAgdmFyIHBkeCA9IHAueCAtIHBvaW50LngsXG4gICAgICAgICAgcGR5ID0gcC55IC0gcG9pbnQueSxcbiAgICAgICAgICBkaXN0cCA9IE1hdGguc3FydChwZHggKiBwZHggKyBwZHkgKiBwZHkpLFxuXG4gICAgICAgICAgcWR4ID0gcS54IC0gcG9pbnQueCxcbiAgICAgICAgICBxZHkgPSBxLnkgLSBwb2ludC55LFxuICAgICAgICAgIGRpc3RxID0gTWF0aC5zcXJ0KHFkeCAqIHFkeCArIHFkeSAqIHFkeSk7XG5cbiAgICAgIHJldHVybiAoZGlzdHAgPCBkaXN0cSkgPyAtMSA6IChkaXN0cCA9PT0gZGlzdHEgPyAwIDogMSk7XG4gICAgfSk7XG4gIH1cbiAgcmV0dXJuIGludGVyc2VjdGlvbnNbMF07XG59XG4iLCJtb2R1bGUuZXhwb3J0cyA9IGludGVyc2VjdFJlY3Q7XG5cbmZ1bmN0aW9uIGludGVyc2VjdFJlY3Qobm9kZSwgcG9pbnQpIHtcbiAgdmFyIHggPSBub2RlLng7XG4gIHZhciB5ID0gbm9kZS55O1xuXG4gIC8vIFJlY3RhbmdsZSBpbnRlcnNlY3Rpb24gYWxnb3JpdGhtIGZyb206XG4gIC8vIGh0dHA6Ly9tYXRoLnN0YWNrZXhjaGFuZ2UuY29tL3F1ZXN0aW9ucy8xMDgxMTMvZmluZC1lZGdlLWJldHdlZW4tdHdvLWJveGVzXG4gIHZhciBkeCA9IHBvaW50LnggLSB4O1xuICB2YXIgZHkgPSBwb2ludC55IC0geTtcbiAgdmFyIHcgPSBub2RlLndpZHRoIC8gMjtcbiAgdmFyIGggPSBub2RlLmhlaWdodCAvIDI7XG5cbiAgdmFyIHN4LCBzeTtcbiAgaWYgKE1hdGguYWJzKGR5KSAqIHcgPiBNYXRoLmFicyhkeCkgKiBoKSB7XG4gICAgLy8gSW50ZXJzZWN0aW9uIGlzIHRvcCBvciBib3R0b20gb2YgcmVjdC5cbiAgICBpZiAoZHkgPCAwKSB7XG4gICAgICBoID0gLWg7XG4gICAgfVxuICAgIHN4ID0gZHkgPT09IDAgPyAwIDogaCAqIGR4IC8gZHk7XG4gICAgc3kgPSBoO1xuICB9IGVsc2Uge1xuICAgIC8vIEludGVyc2VjdGlvbiBpcyBsZWZ0IG9yIHJpZ2h0IG9mIHJlY3QuXG4gICAgaWYgKGR4IDwgMCkge1xuICAgICAgdyA9IC13O1xuICAgIH1cbiAgICBzeCA9IHc7XG4gICAgc3kgPSBkeCA9PT0gMCA/IDAgOiB3ICogZHkgLyBkeDtcbiAgfVxuXG4gIHJldHVybiB7eDogeCArIHN4LCB5OiB5ICsgc3l9O1xufVxuIiwidmFyIHV0aWwgPSByZXF1aXJlKFwiLi4vdXRpbFwiKTtcblxubW9kdWxlLmV4cG9ydHMgPSBhZGRIdG1sTGFiZWw7XG5cbmZ1bmN0aW9uIGFkZEh0bWxMYWJlbChyb290LCBub2RlKSB7XG4gIHZhciBmbyA9IHJvb3RcbiAgICAuYXBwZW5kKFwiZm9yZWlnbk9iamVjdFwiKVxuICAgICAgLmF0dHIoXCJ3aWR0aFwiLCBcIjEwMDAwMFwiKTtcblxuICB2YXIgZGl2ID0gZm9cbiAgICAuYXBwZW5kKFwieGh0bWw6ZGl2XCIpO1xuXG4gIHZhciBsYWJlbCA9IG5vZGUubGFiZWw7XG4gIHN3aXRjaCh0eXBlb2YgbGFiZWwpIHtcbiAgICBjYXNlIFwiZnVuY3Rpb25cIjpcbiAgICAgIGRpdi5pbnNlcnQobGFiZWwpO1xuICAgICAgYnJlYWs7XG4gICAgY2FzZSBcIm9iamVjdFwiOlxuICAgICAgLy8gQ3VycmVudGx5IHdlIGFzc3VtZSB0aGlzIGlzIGEgRE9NIG9iamVjdC5cbiAgICAgIGRpdi5pbnNlcnQoZnVuY3Rpb24oKSB7IHJldHVybiBsYWJlbDsgfSk7XG4gICAgICBicmVhaztcbiAgICBkZWZhdWx0OiBkaXYuaHRtbChsYWJlbCk7XG4gIH1cblxuICB1dGlsLmFwcGx5U3R5bGUoZGl2LCBub2RlLmxhYmVsU3R5bGUpO1xuICBkaXYuc3R5bGUoXCJkaXNwbGF5XCIsIFwiaW5saW5lLWJsb2NrXCIpO1xuICAvLyBGaXggZm9yIGZpcmVmb3hcbiAgZGl2LnN0eWxlKFwid2hpdGUtc3BhY2VcIiwgXCJub3dyYXBcIik7XG5cbiAgLy8gVE9ETyBmaW5kIGEgYmV0dGVyIHdheSB0byBnZXQgZGltZW5zaW9ucyBmb3IgZm9yZWlnbk9iamVjdHMuLi5cbiAgdmFyIHcsIGg7XG4gIGRpdlxuICAgIC5lYWNoKGZ1bmN0aW9uKCkge1xuICAgICAgdyA9IHRoaXMuY2xpZW50V2lkdGg7XG4gICAgICBoID0gdGhpcy5jbGllbnRIZWlnaHQ7XG4gICAgfSk7XG5cbiAgZm9cbiAgICAuYXR0cihcIndpZHRoXCIsIHcpXG4gICAgLmF0dHIoXCJoZWlnaHRcIiwgaCk7XG5cbiAgcmV0dXJuIGZvO1xufVxuIiwidmFyIGFkZFRleHRMYWJlbCA9IHJlcXVpcmUoXCIuL2FkZC10ZXh0LWxhYmVsXCIpLFxuICAgIGFkZEh0bWxMYWJlbCA9IHJlcXVpcmUoXCIuL2FkZC1odG1sLWxhYmVsXCIpLFxuICAgIGFkZFNWR0xhYmVsICA9IHJlcXVpcmUoXCIuL2FkZC1zdmctbGFiZWxcIik7XG5cbm1vZHVsZS5leHBvcnRzID0gYWRkTGFiZWw7XG5cbmZ1bmN0aW9uIGFkZExhYmVsKHJvb3QsIG5vZGUsIGxvY2F0aW9uKSB7XG4gIHZhciBsYWJlbCA9IG5vZGUubGFiZWw7XG4gIHZhciBsYWJlbFN2ZyA9IHJvb3QuYXBwZW5kKFwiZ1wiKTtcblxuICAvLyBBbGxvdyB0aGUgbGFiZWwgdG8gYmUgYSBzdHJpbmcsIGEgZnVuY3Rpb24gdGhhdCByZXR1cm5zIGEgRE9NIGVsZW1lbnQsIG9yXG4gIC8vIGEgRE9NIGVsZW1lbnQgaXRzZWxmLlxuICBpZiAobm9kZS5sYWJlbFR5cGUgPT09IFwic3ZnXCIpIHtcbiAgICBhZGRTVkdMYWJlbChsYWJlbFN2Zywgbm9kZSk7XG4gIH0gZWxzZSBpZiAodHlwZW9mIGxhYmVsICE9PSBcInN0cmluZ1wiIHx8IG5vZGUubGFiZWxUeXBlID09PSBcImh0bWxcIikge1xuICAgIGFkZEh0bWxMYWJlbChsYWJlbFN2Zywgbm9kZSk7XG4gIH0gZWxzZSB7XG4gICAgYWRkVGV4dExhYmVsKGxhYmVsU3ZnLCBub2RlKTtcbiAgfVxuXG4gIHZhciBsYWJlbEJCb3ggPSBsYWJlbFN2Zy5ub2RlKCkuZ2V0QkJveCgpO1xuICB2YXIgeTtcbiAgc3dpdGNoKGxvY2F0aW9uKSB7XG4gICAgY2FzZSBcInRvcFwiOlxuICAgICAgeSA9ICgtbm9kZS5oZWlnaHQgLyAyKTtcbiAgICAgIGJyZWFrO1xuICAgIGNhc2UgXCJib3R0b21cIjpcbiAgICAgIHkgPSAobm9kZS5oZWlnaHQgLyAyKSAtIGxhYmVsQkJveC5oZWlnaHQ7XG4gICAgICBicmVhaztcbiAgICBkZWZhdWx0OlxuICAgICAgeSA9ICgtbGFiZWxCQm94LmhlaWdodCAvIDIpO1xuICB9XG4gIGxhYmVsU3ZnLmF0dHIoXCJ0cmFuc2Zvcm1cIixcbiAgICAgICAgICAgICAgICBcInRyYW5zbGF0ZShcIiArICgtbGFiZWxCQm94LndpZHRoIC8gMikgKyBcIixcIiArIHkgKyBcIilcIik7XG5cbiAgcmV0dXJuIGxhYmVsU3ZnO1xufVxuIiwidmFyIHV0aWwgPSByZXF1aXJlKFwiLi4vdXRpbFwiKTtcblxubW9kdWxlLmV4cG9ydHMgPSBhZGRTVkdMYWJlbDtcblxuZnVuY3Rpb24gYWRkU1ZHTGFiZWwocm9vdCwgbm9kZSkge1xuICB2YXIgZG9tTm9kZSA9IHJvb3Q7XG5cbiAgZG9tTm9kZS5ub2RlKCkuYXBwZW5kQ2hpbGQobm9kZS5sYWJlbCk7XG5cbiAgdXRpbC5hcHBseVN0eWxlKGRvbU5vZGUsIG5vZGUubGFiZWxTdHlsZSk7XG5cbiAgcmV0dXJuIGRvbU5vZGU7XG59XG4iLCJ2YXIgdXRpbCA9IHJlcXVpcmUoXCIuLi91dGlsXCIpO1xuXG5tb2R1bGUuZXhwb3J0cyA9IGFkZFRleHRMYWJlbDtcblxuLypcbiAqIEF0dGFjaGVzIGEgdGV4dCBsYWJlbCB0byB0aGUgc3BlY2lmaWVkIHJvb3QuIEhhbmRsZXMgZXNjYXBlIHNlcXVlbmNlcy5cbiAqL1xuZnVuY3Rpb24gYWRkVGV4dExhYmVsKHJvb3QsIG5vZGUpIHtcbiAgdmFyIGRvbU5vZGUgPSByb290LmFwcGVuZChcInRleHRcIik7XG5cbiAgdmFyIGxpbmVzID0gcHJvY2Vzc0VzY2FwZVNlcXVlbmNlcyhub2RlLmxhYmVsKS5zcGxpdChcIlxcblwiKTtcbiAgZm9yICh2YXIgaSA9IDA7IGkgPCBsaW5lcy5sZW5ndGg7IGkrKykge1xuICAgIGRvbU5vZGVcbiAgICAgIC5hcHBlbmQoXCJ0c3BhblwiKVxuICAgICAgICAuYXR0cihcInhtbDpzcGFjZVwiLCBcInByZXNlcnZlXCIpXG4gICAgICAgIC5hdHRyKFwiZHlcIiwgXCIxZW1cIilcbiAgICAgICAgLmF0dHIoXCJ4XCIsIFwiMVwiKVxuICAgICAgICAudGV4dChsaW5lc1tpXSk7XG4gIH1cblxuICB1dGlsLmFwcGx5U3R5bGUoZG9tTm9kZSwgbm9kZS5sYWJlbFN0eWxlKTtcblxuICByZXR1cm4gZG9tTm9kZTtcbn1cblxuZnVuY3Rpb24gcHJvY2Vzc0VzY2FwZVNlcXVlbmNlcyh0ZXh0KSB7XG4gIHZhciBuZXdUZXh0ID0gXCJcIixcbiAgICAgIGVzY2FwZWQgPSBmYWxzZSxcbiAgICAgIGNoO1xuICBmb3IgKHZhciBpID0gMDsgaSA8IHRleHQubGVuZ3RoOyArK2kpIHtcbiAgICBjaCA9IHRleHRbaV07XG4gICAgaWYgKGVzY2FwZWQpIHtcbiAgICAgIHN3aXRjaChjaCkge1xuICAgICAgICBjYXNlIFwiblwiOiBuZXdUZXh0ICs9IFwiXFxuXCI7IGJyZWFrO1xuICAgICAgICBkZWZhdWx0OiBuZXdUZXh0ICs9IGNoO1xuICAgICAgfVxuICAgICAgZXNjYXBlZCA9IGZhbHNlO1xuICAgIH0gZWxzZSBpZiAoY2ggPT09IFwiXFxcXFwiKSB7XG4gICAgICBlc2NhcGVkID0gdHJ1ZTtcbiAgICB9IGVsc2Uge1xuICAgICAgbmV3VGV4dCArPSBjaDtcbiAgICB9XG4gIH1cbiAgcmV0dXJuIG5ld1RleHQ7XG59XG4iLCIvKiBnbG9iYWwgd2luZG93ICovXG5cbnZhciBsb2Rhc2g7XG5cbmlmIChyZXF1aXJlKSB7XG4gIHRyeSB7XG4gICAgbG9kYXNoID0gcmVxdWlyZShcImxvZGFzaFwiKTtcbiAgfSBjYXRjaCAoZSkge31cbn1cblxuaWYgKCFsb2Rhc2gpIHtcbiAgbG9kYXNoID0gd2luZG93Ll87XG59XG5cbm1vZHVsZS5leHBvcnRzID0gbG9kYXNoO1xuIiwiXCJ1c2Ugc3RyaWN0XCI7XG5cbnZhciB1dGlsID0gcmVxdWlyZShcIi4vdXRpbFwiKSxcbiAgICBkMyA9IHJlcXVpcmUoXCIuL2QzXCIpO1xuXG5tb2R1bGUuZXhwb3J0cyA9IHBvc2l0aW9uQ2x1c3RlcnM7XG5cbmZ1bmN0aW9uIHBvc2l0aW9uQ2x1c3RlcnMoc2VsZWN0aW9uLCBnKSB7XG4gIHZhciBjcmVhdGVkID0gc2VsZWN0aW9uLmZpbHRlcihmdW5jdGlvbigpIHsgcmV0dXJuICFkMy5zZWxlY3QodGhpcykuY2xhc3NlZChcInVwZGF0ZVwiKTsgfSk7XG5cbiAgZnVuY3Rpb24gdHJhbnNsYXRlKHYpIHtcbiAgICB2YXIgbm9kZSA9IGcubm9kZSh2KTtcbiAgICByZXR1cm4gXCJ0cmFuc2xhdGUoXCIgKyBub2RlLnggKyBcIixcIiArIG5vZGUueSArIFwiKVwiO1xuICB9XG5cbiAgY3JlYXRlZC5hdHRyKFwidHJhbnNmb3JtXCIsIHRyYW5zbGF0ZSk7XG5cbiAgdXRpbC5hcHBseVRyYW5zaXRpb24oc2VsZWN0aW9uLCBnKVxuICAgICAgLnN0eWxlKFwib3BhY2l0eVwiLCAxKVxuICAgICAgLmF0dHIoXCJ0cmFuc2Zvcm1cIiwgdHJhbnNsYXRlKTtcblxuICB1dGlsLmFwcGx5VHJhbnNpdGlvbihjcmVhdGVkLnNlbGVjdEFsbChcInJlY3RcIiksIGcpXG4gICAgICAuYXR0cihcIndpZHRoXCIsIGZ1bmN0aW9uKHYpIHsgcmV0dXJuIGcubm9kZSh2KS53aWR0aDsgfSlcbiAgICAgIC5hdHRyKFwiaGVpZ2h0XCIsIGZ1bmN0aW9uKHYpIHsgcmV0dXJuIGcubm9kZSh2KS5oZWlnaHQ7IH0pXG4gICAgICAuYXR0cihcInhcIiwgZnVuY3Rpb24odikge1xuICAgICAgICB2YXIgbm9kZSA9IGcubm9kZSh2KTtcbiAgICAgICAgcmV0dXJuIC1ub2RlLndpZHRoIC8gMjtcbiAgICAgIH0pXG4gICAgICAuYXR0cihcInlcIiwgZnVuY3Rpb24odikge1xuICAgICAgICB2YXIgbm9kZSA9IGcubm9kZSh2KTtcbiAgICAgICAgcmV0dXJuIC1ub2RlLmhlaWdodCAvIDI7XG4gICAgICB9KTtcblxufVxuIiwiXCJ1c2Ugc3RyaWN0XCI7XG5cbnZhciB1dGlsID0gcmVxdWlyZShcIi4vdXRpbFwiKSxcbiAgICBkMyA9IHJlcXVpcmUoXCIuL2QzXCIpLFxuICAgIF8gPSByZXF1aXJlKFwiLi9sb2Rhc2hcIik7XG5cbm1vZHVsZS5leHBvcnRzID0gcG9zaXRpb25FZGdlTGFiZWxzO1xuXG5mdW5jdGlvbiBwb3NpdGlvbkVkZ2VMYWJlbHMoc2VsZWN0aW9uLCBnKSB7XG4gIHZhciBjcmVhdGVkID0gc2VsZWN0aW9uLmZpbHRlcihmdW5jdGlvbigpIHsgcmV0dXJuICFkMy5zZWxlY3QodGhpcykuY2xhc3NlZChcInVwZGF0ZVwiKTsgfSk7XG5cbiAgZnVuY3Rpb24gdHJhbnNsYXRlKGUpIHtcbiAgICB2YXIgZWRnZSA9IGcuZWRnZShlKTtcbiAgICByZXR1cm4gXy5oYXMoZWRnZSwgXCJ4XCIpID8gXCJ0cmFuc2xhdGUoXCIgKyBlZGdlLnggKyBcIixcIiArIGVkZ2UueSArIFwiKVwiIDogXCJcIjtcbiAgfVxuXG4gIGNyZWF0ZWQuYXR0cihcInRyYW5zZm9ybVwiLCB0cmFuc2xhdGUpO1xuXG4gIHV0aWwuYXBwbHlUcmFuc2l0aW9uKHNlbGVjdGlvbiwgZylcbiAgICAuc3R5bGUoXCJvcGFjaXR5XCIsIDEpXG4gICAgLmF0dHIoXCJ0cmFuc2Zvcm1cIiwgdHJhbnNsYXRlKTtcbn1cbiIsIlwidXNlIHN0cmljdFwiO1xuXG52YXIgdXRpbCA9IHJlcXVpcmUoXCIuL3V0aWxcIiksXG4gICAgZDMgPSByZXF1aXJlKFwiLi9kM1wiKTtcblxubW9kdWxlLmV4cG9ydHMgPSBwb3NpdGlvbk5vZGVzO1xuXG5mdW5jdGlvbiBwb3NpdGlvbk5vZGVzKHNlbGVjdGlvbiwgZykge1xuICB2YXIgY3JlYXRlZCA9IHNlbGVjdGlvbi5maWx0ZXIoZnVuY3Rpb24oKSB7IHJldHVybiAhZDMuc2VsZWN0KHRoaXMpLmNsYXNzZWQoXCJ1cGRhdGVcIik7IH0pO1xuXG4gIGZ1bmN0aW9uIHRyYW5zbGF0ZSh2KSB7XG4gICAgdmFyIG5vZGUgPSBnLm5vZGUodik7XG4gICAgcmV0dXJuIFwidHJhbnNsYXRlKFwiICsgbm9kZS54ICsgXCIsXCIgKyBub2RlLnkgKyBcIilcIjtcbiAgfVxuXG4gIGNyZWF0ZWQuYXR0cihcInRyYW5zZm9ybVwiLCB0cmFuc2xhdGUpO1xuXG4gIHV0aWwuYXBwbHlUcmFuc2l0aW9uKHNlbGVjdGlvbiwgZylcbiAgICAuc3R5bGUoXCJvcGFjaXR5XCIsIDEpXG4gICAgLmF0dHIoXCJ0cmFuc2Zvcm1cIiwgdHJhbnNsYXRlKTtcbn1cbiIsInZhciBfID0gcmVxdWlyZShcIi4vbG9kYXNoXCIpLFxuICAgIGxheW91dCA9IHJlcXVpcmUoXCIuL2RhZ3JlXCIpLmxheW91dDtcblxubW9kdWxlLmV4cG9ydHMgPSByZW5kZXI7XG5cbi8vIFRoaXMgZGVzaWduIGlzIGJhc2VkIG9uIGh0dHA6Ly9ib3N0Lm9ja3Mub3JnL21pa2UvY2hhcnQvLlxuZnVuY3Rpb24gcmVuZGVyKCkge1xuICB2YXIgY3JlYXRlTm9kZXMgPSByZXF1aXJlKFwiLi9jcmVhdGUtbm9kZXNcIiksXG4gICAgICBjcmVhdGVDbHVzdGVycyA9IHJlcXVpcmUoXCIuL2NyZWF0ZS1jbHVzdGVyc1wiKSxcbiAgICAgIGNyZWF0ZUVkZ2VMYWJlbHMgPSByZXF1aXJlKFwiLi9jcmVhdGUtZWRnZS1sYWJlbHNcIiksXG4gICAgICBjcmVhdGVFZGdlUGF0aHMgPSByZXF1aXJlKFwiLi9jcmVhdGUtZWRnZS1wYXRoc1wiKSxcbiAgICAgIHBvc2l0aW9uTm9kZXMgPSByZXF1aXJlKFwiLi9wb3NpdGlvbi1ub2Rlc1wiKSxcbiAgICAgIHBvc2l0aW9uRWRnZUxhYmVscyA9IHJlcXVpcmUoXCIuL3Bvc2l0aW9uLWVkZ2UtbGFiZWxzXCIpLFxuICAgICAgcG9zaXRpb25DbHVzdGVycyA9IHJlcXVpcmUoXCIuL3Bvc2l0aW9uLWNsdXN0ZXJzXCIpLFxuICAgICAgc2hhcGVzID0gcmVxdWlyZShcIi4vc2hhcGVzXCIpLFxuICAgICAgYXJyb3dzID0gcmVxdWlyZShcIi4vYXJyb3dzXCIpO1xuXG4gIHZhciBmbiA9IGZ1bmN0aW9uKHN2ZywgZykge1xuICAgIHByZVByb2Nlc3NHcmFwaChnKTtcblxuICAgIHZhciBvdXRwdXRHcm91cCA9IGNyZWF0ZU9yU2VsZWN0R3JvdXAoc3ZnLCBcIm91dHB1dFwiKSxcbiAgICAgICAgY2x1c3RlcnNHcm91cCA9IGNyZWF0ZU9yU2VsZWN0R3JvdXAob3V0cHV0R3JvdXAsIFwiY2x1c3RlcnNcIiksXG4gICAgICAgIGVkZ2VQYXRoc0dyb3VwID0gY3JlYXRlT3JTZWxlY3RHcm91cChvdXRwdXRHcm91cCwgXCJlZGdlUGF0aHNcIiksXG4gICAgICAgIGVkZ2VMYWJlbHMgPSBjcmVhdGVFZGdlTGFiZWxzKGNyZWF0ZU9yU2VsZWN0R3JvdXAob3V0cHV0R3JvdXAsIFwiZWRnZUxhYmVsc1wiKSwgZyksXG4gICAgICAgIG5vZGVzID0gY3JlYXRlTm9kZXMoY3JlYXRlT3JTZWxlY3RHcm91cChvdXRwdXRHcm91cCwgXCJub2Rlc1wiKSwgZywgc2hhcGVzKTtcblxuICAgIGxheW91dChnKTtcblxuICAgIHBvc2l0aW9uTm9kZXMobm9kZXMsIGcpO1xuICAgIHBvc2l0aW9uRWRnZUxhYmVscyhlZGdlTGFiZWxzLCBnKTtcbiAgICBjcmVhdGVFZGdlUGF0aHMoZWRnZVBhdGhzR3JvdXAsIGcsIGFycm93cyk7XG5cbiAgICB2YXIgY2x1c3RlcnMgPSBjcmVhdGVDbHVzdGVycyhjbHVzdGVyc0dyb3VwLCBnKTtcbiAgICBwb3NpdGlvbkNsdXN0ZXJzKGNsdXN0ZXJzLCBnKTtcblxuICAgIHBvc3RQcm9jZXNzR3JhcGgoZyk7XG4gIH07XG5cbiAgZm4uY3JlYXRlTm9kZXMgPSBmdW5jdGlvbih2YWx1ZSkge1xuICAgIGlmICghYXJndW1lbnRzLmxlbmd0aCkgcmV0dXJuIGNyZWF0ZU5vZGVzO1xuICAgIGNyZWF0ZU5vZGVzID0gdmFsdWU7XG4gICAgcmV0dXJuIGZuO1xuICB9O1xuXG4gIGZuLmNyZWF0ZUNsdXN0ZXJzID0gZnVuY3Rpb24odmFsdWUpIHtcbiAgICBpZiAoIWFyZ3VtZW50cy5sZW5ndGgpIHJldHVybiBjcmVhdGVDbHVzdGVycztcbiAgICBjcmVhdGVDbHVzdGVycyA9IHZhbHVlO1xuICAgIHJldHVybiBmbjtcbiAgfTtcblxuICBmbi5jcmVhdGVFZGdlTGFiZWxzID0gZnVuY3Rpb24odmFsdWUpIHtcbiAgICBpZiAoIWFyZ3VtZW50cy5sZW5ndGgpIHJldHVybiBjcmVhdGVFZGdlTGFiZWxzO1xuICAgIGNyZWF0ZUVkZ2VMYWJlbHMgPSB2YWx1ZTtcbiAgICByZXR1cm4gZm47XG4gIH07XG5cbiAgZm4uY3JlYXRlRWRnZVBhdGhzID0gZnVuY3Rpb24odmFsdWUpIHtcbiAgICBpZiAoIWFyZ3VtZW50cy5sZW5ndGgpIHJldHVybiBjcmVhdGVFZGdlUGF0aHM7XG4gICAgY3JlYXRlRWRnZVBhdGhzID0gdmFsdWU7XG4gICAgcmV0dXJuIGZuO1xuICB9O1xuXG4gIGZuLnNoYXBlcyA9IGZ1bmN0aW9uKHZhbHVlKSB7XG4gICAgaWYgKCFhcmd1bWVudHMubGVuZ3RoKSByZXR1cm4gc2hhcGVzO1xuICAgIHNoYXBlcyA9IHZhbHVlO1xuICAgIHJldHVybiBmbjtcbiAgfTtcblxuICBmbi5hcnJvd3MgPSBmdW5jdGlvbih2YWx1ZSkge1xuICAgIGlmICghYXJndW1lbnRzLmxlbmd0aCkgcmV0dXJuIGFycm93cztcbiAgICBhcnJvd3MgPSB2YWx1ZTtcbiAgICByZXR1cm4gZm47XG4gIH07XG5cbiAgcmV0dXJuIGZuO1xufVxuXG52YXIgTk9ERV9ERUZBVUxUX0FUVFJTID0ge1xuICBwYWRkaW5nTGVmdDogMTAsXG4gIHBhZGRpbmdSaWdodDogMTAsXG4gIHBhZGRpbmdUb3A6IDEwLFxuICBwYWRkaW5nQm90dG9tOiAxMCxcbiAgcng6IDAsXG4gIHJ5OiAwLFxuICBzaGFwZTogXCJyZWN0XCJcbn07XG5cbnZhciBFREdFX0RFRkFVTFRfQVRUUlMgPSB7XG4gIGFycm93aGVhZDogXCJub3JtYWxcIixcbiAgbGluZUludGVycG9sYXRlOiBcImxpbmVhclwiXG59O1xuXG5mdW5jdGlvbiBwcmVQcm9jZXNzR3JhcGgoZykge1xuICBnLm5vZGVzKCkuZm9yRWFjaChmdW5jdGlvbih2KSB7XG4gICAgdmFyIG5vZGUgPSBnLm5vZGUodik7XG4gICAgaWYgKCFfLmhhcyhub2RlLCBcImxhYmVsXCIpICYmICFnLmNoaWxkcmVuKHYpLmxlbmd0aCkgeyBub2RlLmxhYmVsID0gdjsgfVxuXG4gICAgaWYgKF8uaGFzKG5vZGUsIFwicGFkZGluZ1hcIikpIHtcbiAgICAgIF8uZGVmYXVsdHMobm9kZSwge1xuICAgICAgICBwYWRkaW5nTGVmdDogbm9kZS5wYWRkaW5nWCxcbiAgICAgICAgcGFkZGluZ1JpZ2h0OiBub2RlLnBhZGRpbmdYXG4gICAgICB9KTtcbiAgICB9XG5cbiAgICBpZiAoXy5oYXMobm9kZSwgXCJwYWRkaW5nWVwiKSkge1xuICAgICAgXy5kZWZhdWx0cyhub2RlLCB7XG4gICAgICAgIHBhZGRpbmdUb3A6IG5vZGUucGFkZGluZ1ksXG4gICAgICAgIHBhZGRpbmdCb3R0b206IG5vZGUucGFkZGluZ1lcbiAgICAgIH0pO1xuICAgIH1cblxuICAgIGlmIChfLmhhcyhub2RlLCBcInBhZGRpbmdcIikpIHtcbiAgICAgIF8uZGVmYXVsdHMobm9kZSwge1xuICAgICAgICBwYWRkaW5nTGVmdDogbm9kZS5wYWRkaW5nLFxuICAgICAgICBwYWRkaW5nUmlnaHQ6IG5vZGUucGFkZGluZyxcbiAgICAgICAgcGFkZGluZ1RvcDogbm9kZS5wYWRkaW5nLFxuICAgICAgICBwYWRkaW5nQm90dG9tOiBub2RlLnBhZGRpbmdcbiAgICAgIH0pO1xuICAgIH1cblxuICAgIF8uZGVmYXVsdHMobm9kZSwgTk9ERV9ERUZBVUxUX0FUVFJTKTtcblxuICAgIF8uZWFjaChbXCJwYWRkaW5nTGVmdFwiLCBcInBhZGRpbmdSaWdodFwiLCBcInBhZGRpbmdUb3BcIiwgXCJwYWRkaW5nQm90dG9tXCJdLCBmdW5jdGlvbihrKSB7XG4gICAgICBub2RlW2tdID0gTnVtYmVyKG5vZGVba10pO1xuICAgIH0pO1xuXG4gICAgLy8gU2F2ZSBkaW1lbnNpb25zIGZvciByZXN0b3JlIGR1cmluZyBwb3N0LXByb2Nlc3NpbmdcbiAgICBpZiAoXy5oYXMobm9kZSwgXCJ3aWR0aFwiKSkgeyBub2RlLl9wcmV2V2lkdGggPSBub2RlLndpZHRoOyB9XG4gICAgaWYgKF8uaGFzKG5vZGUsIFwiaGVpZ2h0XCIpKSB7IG5vZGUuX3ByZXZIZWlnaHQgPSBub2RlLmhlaWdodDsgfVxuICB9KTtcblxuICBnLmVkZ2VzKCkuZm9yRWFjaChmdW5jdGlvbihlKSB7XG4gICAgdmFyIGVkZ2UgPSBnLmVkZ2UoZSk7XG4gICAgaWYgKCFfLmhhcyhlZGdlLCBcImxhYmVsXCIpKSB7IGVkZ2UubGFiZWwgPSBcIlwiOyB9XG4gICAgXy5kZWZhdWx0cyhlZGdlLCBFREdFX0RFRkFVTFRfQVRUUlMpO1xuICB9KTtcbn1cblxuZnVuY3Rpb24gcG9zdFByb2Nlc3NHcmFwaChnKSB7XG4gIF8uZWFjaChnLm5vZGVzKCksIGZ1bmN0aW9uKHYpIHtcbiAgICB2YXIgbm9kZSA9IGcubm9kZSh2KTtcblxuICAgIC8vIFJlc3RvcmUgb3JpZ2luYWwgZGltZW5zaW9uc1xuICAgIGlmIChfLmhhcyhub2RlLCBcIl9wcmV2V2lkdGhcIikpIHtcbiAgICAgIG5vZGUud2lkdGggPSBub2RlLl9wcmV2V2lkdGg7XG4gICAgfSBlbHNlIHtcbiAgICAgIGRlbGV0ZSBub2RlLndpZHRoO1xuICAgIH1cblxuICAgIGlmIChfLmhhcyhub2RlLCBcIl9wcmV2SGVpZ2h0XCIpKSB7XG4gICAgICBub2RlLmhlaWdodCA9IG5vZGUuX3ByZXZIZWlnaHQ7XG4gICAgfSBlbHNlIHtcbiAgICAgIGRlbGV0ZSBub2RlLmhlaWdodDtcbiAgICB9XG5cbiAgICBkZWxldGUgbm9kZS5fcHJldldpZHRoO1xuICAgIGRlbGV0ZSBub2RlLl9wcmV2SGVpZ2h0O1xuICB9KTtcbn1cblxuZnVuY3Rpb24gY3JlYXRlT3JTZWxlY3RHcm91cChyb290LCBuYW1lKSB7XG4gIHZhciBzZWxlY3Rpb24gPSByb290LnNlbGVjdChcImcuXCIgKyBuYW1lKTtcbiAgaWYgKHNlbGVjdGlvbi5lbXB0eSgpKSB7XG4gICAgc2VsZWN0aW9uID0gcm9vdC5hcHBlbmQoXCJnXCIpLmF0dHIoXCJjbGFzc1wiLCBuYW1lKTtcbiAgfVxuICByZXR1cm4gc2VsZWN0aW9uO1xufVxuIiwiXCJ1c2Ugc3RyaWN0XCI7XG5cbnZhciBpbnRlcnNlY3RSZWN0ID0gcmVxdWlyZShcIi4vaW50ZXJzZWN0L2ludGVyc2VjdC1yZWN0XCIpLFxuICAgIGludGVyc2VjdEVsbGlwc2UgPSByZXF1aXJlKFwiLi9pbnRlcnNlY3QvaW50ZXJzZWN0LWVsbGlwc2VcIiksXG4gICAgaW50ZXJzZWN0Q2lyY2xlID0gcmVxdWlyZShcIi4vaW50ZXJzZWN0L2ludGVyc2VjdC1jaXJjbGVcIiksXG4gICAgaW50ZXJzZWN0UG9seWdvbiA9IHJlcXVpcmUoXCIuL2ludGVyc2VjdC9pbnRlcnNlY3QtcG9seWdvblwiKTtcblxubW9kdWxlLmV4cG9ydHMgPSB7XG4gIHJlY3Q6IHJlY3QsXG4gIGVsbGlwc2U6IGVsbGlwc2UsXG4gIGNpcmNsZTogY2lyY2xlLFxuICBkaWFtb25kOiBkaWFtb25kXG59O1xuXG5mdW5jdGlvbiByZWN0KHBhcmVudCwgYmJveCwgbm9kZSkge1xuICB2YXIgc2hhcGVTdmcgPSBwYXJlbnQuaW5zZXJ0KFwicmVjdFwiLCBcIjpmaXJzdC1jaGlsZFwiKVxuICAgICAgICAuYXR0cihcInJ4XCIsIG5vZGUucngpXG4gICAgICAgIC5hdHRyKFwicnlcIiwgbm9kZS5yeSlcbiAgICAgICAgLmF0dHIoXCJ4XCIsIC1iYm94LndpZHRoIC8gMilcbiAgICAgICAgLmF0dHIoXCJ5XCIsIC1iYm94LmhlaWdodCAvIDIpXG4gICAgICAgIC5hdHRyKFwid2lkdGhcIiwgYmJveC53aWR0aClcbiAgICAgICAgLmF0dHIoXCJoZWlnaHRcIiwgYmJveC5oZWlnaHQpO1xuXG4gIG5vZGUuaW50ZXJzZWN0ID0gZnVuY3Rpb24ocG9pbnQpIHtcbiAgICByZXR1cm4gaW50ZXJzZWN0UmVjdChub2RlLCBwb2ludCk7XG4gIH07XG5cbiAgcmV0dXJuIHNoYXBlU3ZnO1xufVxuXG5mdW5jdGlvbiBlbGxpcHNlKHBhcmVudCwgYmJveCwgbm9kZSkge1xuICB2YXIgcnggPSBiYm94LndpZHRoIC8gMixcbiAgICAgIHJ5ID0gYmJveC5oZWlnaHQgLyAyLFxuICAgICAgc2hhcGVTdmcgPSBwYXJlbnQuaW5zZXJ0KFwiZWxsaXBzZVwiLCBcIjpmaXJzdC1jaGlsZFwiKVxuICAgICAgICAuYXR0cihcInhcIiwgLWJib3gud2lkdGggLyAyKVxuICAgICAgICAuYXR0cihcInlcIiwgLWJib3guaGVpZ2h0IC8gMilcbiAgICAgICAgLmF0dHIoXCJyeFwiLCByeClcbiAgICAgICAgLmF0dHIoXCJyeVwiLCByeSk7XG5cbiAgbm9kZS5pbnRlcnNlY3QgPSBmdW5jdGlvbihwb2ludCkge1xuICAgIHJldHVybiBpbnRlcnNlY3RFbGxpcHNlKG5vZGUsIHJ4LCByeSwgcG9pbnQpO1xuICB9O1xuXG4gIHJldHVybiBzaGFwZVN2Zztcbn1cblxuZnVuY3Rpb24gY2lyY2xlKHBhcmVudCwgYmJveCwgbm9kZSkge1xuICB2YXIgciA9IE1hdGgubWF4KGJib3gud2lkdGgsIGJib3guaGVpZ2h0KSAvIDIsXG4gICAgICBzaGFwZVN2ZyA9IHBhcmVudC5pbnNlcnQoXCJjaXJjbGVcIiwgXCI6Zmlyc3QtY2hpbGRcIilcbiAgICAgICAgLmF0dHIoXCJ4XCIsIC1iYm94LndpZHRoIC8gMilcbiAgICAgICAgLmF0dHIoXCJ5XCIsIC1iYm94LmhlaWdodCAvIDIpXG4gICAgICAgIC5hdHRyKFwiclwiLCByKTtcblxuICBub2RlLmludGVyc2VjdCA9IGZ1bmN0aW9uKHBvaW50KSB7XG4gICAgcmV0dXJuIGludGVyc2VjdENpcmNsZShub2RlLCByLCBwb2ludCk7XG4gIH07XG5cbiAgcmV0dXJuIHNoYXBlU3ZnO1xufVxuXG4vLyBDaXJjdW1zY3JpYmUgYW4gZWxsaXBzZSBmb3IgdGhlIGJvdW5kaW5nIGJveCB3aXRoIGEgZGlhbW9uZCBzaGFwZS4gSSBkZXJpdmVkXG4vLyB0aGUgZnVuY3Rpb24gdG8gY2FsY3VsYXRlIHRoZSBkaWFtb25kIHNoYXBlIGZyb206XG4vLyBodHRwOi8vbWF0aGZvcnVtLm9yZy9rYi9tZXNzYWdlLmpzcGE/bWVzc2FnZUlEPTM3NTAyMzZcbmZ1bmN0aW9uIGRpYW1vbmQocGFyZW50LCBiYm94LCBub2RlKSB7XG4gIHZhciB3ID0gKGJib3gud2lkdGggKiBNYXRoLlNRUlQyKSAvIDIsXG4gICAgICBoID0gKGJib3guaGVpZ2h0ICogTWF0aC5TUVJUMikgLyAyLFxuICAgICAgcG9pbnRzID0gW1xuICAgICAgICB7IHg6ICAwLCB5OiAtaCB9LFxuICAgICAgICB7IHg6IC13LCB5OiAgMCB9LFxuICAgICAgICB7IHg6ICAwLCB5OiAgaCB9LFxuICAgICAgICB7IHg6ICB3LCB5OiAgMCB9XG4gICAgICBdLFxuICAgICAgc2hhcGVTdmcgPSBwYXJlbnQuaW5zZXJ0KFwicG9seWdvblwiLCBcIjpmaXJzdC1jaGlsZFwiKVxuICAgICAgICAuYXR0cihcInBvaW50c1wiLCBwb2ludHMubWFwKGZ1bmN0aW9uKHApIHsgcmV0dXJuIHAueCArIFwiLFwiICsgcC55OyB9KS5qb2luKFwiIFwiKSk7XG5cbiAgbm9kZS5pbnRlcnNlY3QgPSBmdW5jdGlvbihwKSB7XG4gICAgcmV0dXJuIGludGVyc2VjdFBvbHlnb24obm9kZSwgcG9pbnRzLCBwKTtcbiAgfTtcblxuICByZXR1cm4gc2hhcGVTdmc7XG59XG4iLCJ2YXIgXyA9IHJlcXVpcmUoXCIuL2xvZGFzaFwiKTtcblxuLy8gUHVibGljIHV0aWxpdHkgZnVuY3Rpb25zXG5tb2R1bGUuZXhwb3J0cyA9IHtcbiAgaXNTdWJncmFwaDogaXNTdWJncmFwaCxcbiAgZWRnZVRvSWQ6IGVkZ2VUb0lkLFxuICBhcHBseVN0eWxlOiBhcHBseVN0eWxlLFxuICBhcHBseUNsYXNzOiBhcHBseUNsYXNzLFxuICBhcHBseVRyYW5zaXRpb246IGFwcGx5VHJhbnNpdGlvblxufTtcblxuLypcbiAqIFJldHVybnMgdHJ1ZSBpZiB0aGUgc3BlY2lmaWVkIG5vZGUgaW4gdGhlIGdyYXBoIGlzIGEgc3ViZ3JhcGggbm9kZS4gQVxuICogc3ViZ3JhcGggbm9kZSBpcyBvbmUgdGhhdCBjb250YWlucyBvdGhlciBub2Rlcy5cbiAqL1xuZnVuY3Rpb24gaXNTdWJncmFwaChnLCB2KSB7XG4gIHJldHVybiAhIWcuY2hpbGRyZW4odikubGVuZ3RoO1xufVxuXG5mdW5jdGlvbiBlZGdlVG9JZChlKSB7XG4gIHJldHVybiBlc2NhcGVJZChlLnYpICsgXCI6XCIgKyBlc2NhcGVJZChlLncpICsgXCI6XCIgKyBlc2NhcGVJZChlLm5hbWUpO1xufVxuXG52YXIgSURfREVMSU0gPSAvOi9nO1xuZnVuY3Rpb24gZXNjYXBlSWQoc3RyKSB7XG4gIHJldHVybiBzdHIgPyBTdHJpbmcoc3RyKS5yZXBsYWNlKElEX0RFTElNLCBcIlxcXFw6XCIpIDogXCJcIjtcbn1cblxuZnVuY3Rpb24gYXBwbHlTdHlsZShkb20sIHN0eWxlRm4pIHtcbiAgaWYgKHN0eWxlRm4pIHtcbiAgICBkb20uYXR0cihcInN0eWxlXCIsIHN0eWxlRm4pO1xuICB9XG59XG5cbmZ1bmN0aW9uIGFwcGx5Q2xhc3MoZG9tLCBjbGFzc0ZuLCBvdGhlckNsYXNzZXMpIHtcbiAgaWYgKGNsYXNzRm4pIHtcbiAgICBkb21cbiAgICAgIC5hdHRyKFwiY2xhc3NcIiwgY2xhc3NGbilcbiAgICAgIC5hdHRyKFwiY2xhc3NcIiwgb3RoZXJDbGFzc2VzICsgXCIgXCIgKyBkb20uYXR0cihcImNsYXNzXCIpKTtcbiAgfVxufVxuXG5mdW5jdGlvbiBhcHBseVRyYW5zaXRpb24oc2VsZWN0aW9uLCBnKSB7XG4gIHZhciBncmFwaCA9IGcuZ3JhcGgoKTtcblxuICBpZiAoXy5pc1BsYWluT2JqZWN0KGdyYXBoKSkge1xuICAgIHZhciB0cmFuc2l0aW9uID0gZ3JhcGgudHJhbnNpdGlvbjtcbiAgICBpZiAoXy5pc0Z1bmN0aW9uKHRyYW5zaXRpb24pKSB7XG4gICAgICByZXR1cm4gdHJhbnNpdGlvbihzZWxlY3Rpb24pO1xuICAgIH1cbiAgfVxuXG4gIHJldHVybiBzZWxlY3Rpb247XG59XG4iLCJtb2R1bGUuZXhwb3J0cyA9IFwiMC40LjExXCI7XG4iXX0= +//# sourceMappingURL=data:application/json;charset:utf-8;base64,eyJ2ZXJzaW9uIjozLCJzb3VyY2VzIjpbIm5vZGVfbW9kdWxlcy9icm93c2VyaWZ5L25vZGVfbW9kdWxlcy9icm93c2VyLXBhY2svX3ByZWx1ZGUuanMiLCJpbmRleC5qcyIsImxpYi9hcnJvd3MuanMiLCJsaWIvY3JlYXRlLWNsdXN0ZXJzLmpzIiwibGliL2NyZWF0ZS1lZGdlLWxhYmVscy5qcyIsImxpYi9jcmVhdGUtZWRnZS1wYXRocy5qcyIsImxpYi9jcmVhdGUtbm9kZXMuanMiLCJsaWIvZDMuanMiLCJsaWIvZGFncmUuanMiLCJsaWIvZ3JhcGhsaWIuanMiLCJsaWIvaW50ZXJzZWN0L2luZGV4LmpzIiwibGliL2ludGVyc2VjdC9pbnRlcnNlY3QtY2lyY2xlLmpzIiwibGliL2ludGVyc2VjdC9pbnRlcnNlY3QtZWxsaXBzZS5qcyIsImxpYi9pbnRlcnNlY3QvaW50ZXJzZWN0LWxpbmUuanMiLCJsaWIvaW50ZXJzZWN0L2ludGVyc2VjdC1ub2RlLmpzIiwibGliL2ludGVyc2VjdC9pbnRlcnNlY3QtcG9seWdvbi5qcyIsImxpYi9pbnRlcnNlY3QvaW50ZXJzZWN0LXJlY3QuanMiLCJsaWIvbGFiZWwvYWRkLWh0bWwtbGFiZWwuanMiLCJsaWIvbGFiZWwvYWRkLWxhYmVsLmpzIiwibGliL2xhYmVsL2FkZC1zdmctbGFiZWwuanMiLCJsaWIvbGFiZWwvYWRkLXRleHQtbGFiZWwuanMiLCJsaWIvbG9kYXNoLmpzIiwibGliL3Bvc2l0aW9uLWNsdXN0ZXJzLmpzIiwibGliL3Bvc2l0aW9uLWVkZ2UtbGFiZWxzLmpzIiwibGliL3Bvc2l0aW9uLW5vZGVzLmpzIiwibGliL3JlbmRlci5qcyIsImxpYi9zaGFwZXMuanMiLCJsaWIvdXRpbC5qcyIsImxpYi92ZXJzaW9uLmpzIl0sIm5hbWVzIjpbXSwibWFwcGluZ3MiOiJBQUFBO0FDQUE7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7O0FDOUJBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTs7QUM5REE7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTs7QUMzQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBOztBQ25DQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBOztBQ2xJQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBOztBQzFEQTtBQUNBO0FBQ0E7O0FDRkE7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7O0FDZkE7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7O0FDZkE7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTs7QUNQQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBOztBQ1BBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7O0FDekJBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7O0FDdEVBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTs7QUNMQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBOztBQ3ZEQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7O0FDaENBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7O0FDM0NBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7O0FDckNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7O0FDYkE7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7O0FDN0NBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBOztBQ2ZBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7O0FDbENBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7O0FDdEJBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBOztBQ3JCQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7O0FDdktBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBOztBQ2pGQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTtBQUNBO0FBQ0E7QUFDQTs7QUN0REE7QUFDQSIsImZpbGUiOiJnZW5lcmF0ZWQuanMiLCJzb3VyY2VSb290IjoiIiwic291cmNlc0NvbnRlbnQiOlsiKGZ1bmN0aW9uIGUodCxuLHIpe2Z1bmN0aW9uIHMobyx1KXtpZighbltvXSl7aWYoIXRbb10pe3ZhciBhPXR5cGVvZiByZXF1aXJlPT1cImZ1bmN0aW9uXCImJnJlcXVpcmU7aWYoIXUmJmEpcmV0dXJuIGEobywhMCk7aWYoaSlyZXR1cm4gaShvLCEwKTt2YXIgZj1uZXcgRXJyb3IoXCJDYW5ub3QgZmluZCBtb2R1bGUgJ1wiK28rXCInXCIpO3Rocm93IGYuY29kZT1cIk1PRFVMRV9OT1RfRk9VTkRcIixmfXZhciBsPW5bb109e2V4cG9ydHM6e319O3Rbb11bMF0uY2FsbChsLmV4cG9ydHMsZnVuY3Rpb24oZSl7dmFyIG49dFtvXVsxXVtlXTtyZXR1cm4gcyhuP246ZSl9LGwsbC5leHBvcnRzLGUsdCxuLHIpfXJldHVybiBuW29dLmV4cG9ydHN9dmFyIGk9dHlwZW9mIHJlcXVpcmU9PVwiZnVuY3Rpb25cIiYmcmVxdWlyZTtmb3IodmFyIG89MDtvPHIubGVuZ3RoO28rKylzKHJbb10pO3JldHVybiBzfSkiLCIvKipcbiAqIEBsaWNlbnNlXG4gKiBDb3B5cmlnaHQgKGMpIDIwMTItMjAxMyBDaHJpcyBQZXR0aXR0XG4gKlxuICogUGVybWlzc2lvbiBpcyBoZXJlYnkgZ3JhbnRlZCwgZnJlZSBvZiBjaGFyZ2UsIHRvIGFueSBwZXJzb24gb2J0YWluaW5nIGEgY29weVxuICogb2YgdGhpcyBzb2Z0d2FyZSBhbmQgYXNzb2NpYXRlZCBkb2N1bWVudGF0aW9uIGZpbGVzICh0aGUgXCJTb2Z0d2FyZVwiKSwgdG8gZGVhbFxuICogaW4gdGhlIFNvZnR3YXJlIHdpdGhvdXQgcmVzdHJpY3Rpb24sIGluY2x1ZGluZyB3aXRob3V0IGxpbWl0YXRpb24gdGhlIHJpZ2h0c1xuICogdG8gdXNlLCBjb3B5LCBtb2RpZnksIG1lcmdlLCBwdWJsaXNoLCBkaXN0cmlidXRlLCBzdWJsaWNlbnNlLCBhbmQvb3Igc2VsbFxuICogY29waWVzIG9mIHRoZSBTb2Z0d2FyZSwgYW5kIHRvIHBlcm1pdCBwZXJzb25zIHRvIHdob20gdGhlIFNvZnR3YXJlIGlzXG4gKiBmdXJuaXNoZWQgdG8gZG8gc28sIHN1YmplY3QgdG8gdGhlIGZvbGxvd2luZyBjb25kaXRpb25zOlxuICpcbiAqIFRoZSBhYm92ZSBjb3B5cmlnaHQgbm90aWNlIGFuZCB0aGlzIHBlcm1pc3Npb24gbm90aWNlIHNoYWxsIGJlIGluY2x1ZGVkIGluXG4gKiBhbGwgY29waWVzIG9yIHN1YnN0YW50aWFsIHBvcnRpb25zIG9mIHRoZSBTb2Z0d2FyZS5cbiAqXG4gKiBUSEUgU09GVFdBUkUgSVMgUFJPVklERUQgXCJBUyBJU1wiLCBXSVRIT1VUIFdBUlJBTlRZIE9GIEFOWSBLSU5ELCBFWFBSRVNTIE9SXG4gKiBJTVBMSUVELCBJTkNMVURJTkcgQlVUIE5PVCBMSU1JVEVEIFRPIFRIRSBXQVJSQU5USUVTIE9GIE1FUkNIQU5UQUJJTElUWSxcbiAqIEZJVE5FU1MgRk9SIEEgUEFSVElDVUxBUiBQVVJQT1NFIEFORCBOT05JTkZSSU5HRU1FTlQuIElOIE5PIEVWRU5UIFNIQUxMIFRIRVxuICogQVVUSE9SUyBPUiBDT1BZUklHSFQgSE9MREVSUyBCRSBMSUFCTEUgRk9SIEFOWSBDTEFJTSwgREFNQUdFUyBPUiBPVEhFUlxuICogTElBQklMSVRZLCBXSEVUSEVSIElOIEFOIEFDVElPTiBPRiBDT05UUkFDVCwgVE9SVCBPUiBPVEhFUldJU0UsIEFSSVNJTkcgRlJPTSxcbiAqIE9VVCBPRiBPUiBJTiBDT05ORUNUSU9OIFdJVEggVEhFIFNPRlRXQVJFIE9SIFRIRSBVU0UgT1IgT1RIRVIgREVBTElOR1MgSU5cbiAqIFRIRSBTT0ZUV0FSRS5cbiAqL1xubW9kdWxlLmV4cG9ydHMgPSAge1xuICBncmFwaGxpYjogcmVxdWlyZShcIi4vbGliL2dyYXBobGliXCIpLFxuICBkYWdyZTogcmVxdWlyZShcIi4vbGliL2RhZ3JlXCIpLFxuICBpbnRlcnNlY3Q6IHJlcXVpcmUoXCIuL2xpYi9pbnRlcnNlY3RcIiksXG4gIHJlbmRlcjogcmVxdWlyZShcIi4vbGliL3JlbmRlclwiKSxcbiAgdXRpbDogcmVxdWlyZShcIi4vbGliL3V0aWxcIiksXG4gIHZlcnNpb246IHJlcXVpcmUoXCIuL2xpYi92ZXJzaW9uXCIpXG59O1xuIiwidmFyIHV0aWwgPSByZXF1aXJlKFwiLi91dGlsXCIpO1xuXG5tb2R1bGUuZXhwb3J0cyA9IHtcbiAgXCJkZWZhdWx0XCI6IG5vcm1hbCxcbiAgXCJub3JtYWxcIjogbm9ybWFsLFxuICBcInZlZVwiOiB2ZWUsXG4gIFwidW5kaXJlY3RlZFwiOiB1bmRpcmVjdGVkXG59O1xuXG5mdW5jdGlvbiBub3JtYWwocGFyZW50LCBpZCwgZWRnZSwgdHlwZSkge1xuICB2YXIgbWFya2VyID0gcGFyZW50LmFwcGVuZChcIm1hcmtlclwiKVxuICAgIC5hdHRyKFwiaWRcIiwgaWQpXG4gICAgLmF0dHIoXCJ2aWV3Qm94XCIsIFwiMCAwIDEwIDEwXCIpXG4gICAgLmF0dHIoXCJyZWZYXCIsIDkpXG4gICAgLmF0dHIoXCJyZWZZXCIsIDUpXG4gICAgLmF0dHIoXCJtYXJrZXJVbml0c1wiLCBcInN0cm9rZVdpZHRoXCIpXG4gICAgLmF0dHIoXCJtYXJrZXJXaWR0aFwiLCA4KVxuICAgIC5hdHRyKFwibWFya2VySGVpZ2h0XCIsIDYpXG4gICAgLmF0dHIoXCJvcmllbnRcIiwgXCJhdXRvXCIpO1xuXG4gIHZhciBwYXRoID0gbWFya2VyLmFwcGVuZChcInBhdGhcIilcbiAgICAuYXR0cihcImRcIiwgXCJNIDAgMCBMIDEwIDUgTCAwIDEwIHpcIilcbiAgICAuc3R5bGUoXCJzdHJva2Utd2lkdGhcIiwgMSlcbiAgICAuc3R5bGUoXCJzdHJva2UtZGFzaGFycmF5XCIsIFwiMSwwXCIpO1xuICB1dGlsLmFwcGx5U3R5bGUocGF0aCwgZWRnZVt0eXBlICsgXCJTdHlsZVwiXSk7XG59XG5cbmZ1bmN0aW9uIHZlZShwYXJlbnQsIGlkLCBlZGdlLCB0eXBlKSB7XG4gIHZhciBtYXJrZXIgPSBwYXJlbnQuYXBwZW5kKFwibWFya2VyXCIpXG4gICAgLmF0dHIoXCJpZFwiLCBpZClcbiAgICAuYXR0cihcInZpZXdCb3hcIiwgXCIwIDAgMTAgMTBcIilcbiAgICAuYXR0cihcInJlZlhcIiwgOSlcbiAgICAuYXR0cihcInJlZllcIiwgNSlcbiAgICAuYXR0cihcIm1hcmtlclVuaXRzXCIsIFwic3Ryb2tlV2lkdGhcIilcbiAgICAuYXR0cihcIm1hcmtlcldpZHRoXCIsIDgpXG4gICAgLmF0dHIoXCJtYXJrZXJIZWlnaHRcIiwgNilcbiAgICAuYXR0cihcIm9yaWVudFwiLCBcImF1dG9cIik7XG5cbiAgdmFyIHBhdGggPSBtYXJrZXIuYXBwZW5kKFwicGF0aFwiKVxuICAgIC5hdHRyKFwiZFwiLCBcIk0gMCAwIEwgMTAgNSBMIDAgMTAgTCA0IDUgelwiKVxuICAgIC5zdHlsZShcInN0cm9rZS13aWR0aFwiLCAxKVxuICAgIC5zdHlsZShcInN0cm9rZS1kYXNoYXJyYXlcIiwgXCIxLDBcIik7XG4gIHV0aWwuYXBwbHlTdHlsZShwYXRoLCBlZGdlW3R5cGUgKyBcIlN0eWxlXCJdKTtcbn1cblxuZnVuY3Rpb24gdW5kaXJlY3RlZChwYXJlbnQsIGlkLCBlZGdlLCB0eXBlKSB7XG4gIHZhciBtYXJrZXIgPSBwYXJlbnQuYXBwZW5kKFwibWFya2VyXCIpXG4gICAgLmF0dHIoXCJpZFwiLCBpZClcbiAgICAuYXR0cihcInZpZXdCb3hcIiwgXCIwIDAgMTAgMTBcIilcbiAgICAuYXR0cihcInJlZlhcIiwgOSlcbiAgICAuYXR0cihcInJlZllcIiwgNSlcbiAgICAuYXR0cihcIm1hcmtlclVuaXRzXCIsIFwic3Ryb2tlV2lkdGhcIilcbiAgICAuYXR0cihcIm1hcmtlcldpZHRoXCIsIDgpXG4gICAgLmF0dHIoXCJtYXJrZXJIZWlnaHRcIiwgNilcbiAgICAuYXR0cihcIm9yaWVudFwiLCBcImF1dG9cIik7XG5cbiAgdmFyIHBhdGggPSBtYXJrZXIuYXBwZW5kKFwicGF0aFwiKVxuICAgIC5hdHRyKFwiZFwiLCBcIk0gMCA1IEwgMTAgNVwiKVxuICAgIC5zdHlsZShcInN0cm9rZS13aWR0aFwiLCAxKVxuICAgIC5zdHlsZShcInN0cm9rZS1kYXNoYXJyYXlcIiwgXCIxLDBcIik7XG4gIHV0aWwuYXBwbHlTdHlsZShwYXRoLCBlZGdlW3R5cGUgKyBcIlN0eWxlXCJdKTtcbn1cbiIsInZhciB1dGlsID0gcmVxdWlyZShcIi4vdXRpbFwiKSxcbiAgICBhZGRMYWJlbCA9IHJlcXVpcmUoXCIuL2xhYmVsL2FkZC1sYWJlbFwiKTtcblxubW9kdWxlLmV4cG9ydHMgPSBjcmVhdGVDbHVzdGVycztcblxuZnVuY3Rpb24gY3JlYXRlQ2x1c3RlcnMoc2VsZWN0aW9uLCBnKSB7XG4gIHZhciBjbHVzdGVycyA9IGcubm9kZXMoKS5maWx0ZXIoZnVuY3Rpb24odikgeyByZXR1cm4gdXRpbC5pc1N1YmdyYXBoKGcsIHYpOyB9KSxcbiAgICAgIHN2Z0NsdXN0ZXJzID0gc2VsZWN0aW9uLnNlbGVjdEFsbChcImcuY2x1c3RlclwiKVxuICAgICAgICAuZGF0YShjbHVzdGVycywgZnVuY3Rpb24odikgeyByZXR1cm4gdjsgfSk7XG5cbiAgc3ZnQ2x1c3RlcnMuc2VsZWN0QWxsKFwiKlwiKS5yZW1vdmUoKTtcbiAgc3ZnQ2x1c3RlcnMuZW50ZXIoKVxuICAgIC5hcHBlbmQoXCJnXCIpXG4gICAgICAuYXR0cihcImNsYXNzXCIsIFwiY2x1c3RlclwiKVxuICAgICAgLmF0dHIoXCJpZFwiLGZ1bmN0aW9uKHYpe1xuICAgICAgICAgIHZhciBub2RlID0gZy5ub2RlKHYpO1xuICAgICAgICAgIHJldHVybiBub2RlLmlkO1xuICAgICAgfSlcbiAgICAgIC5zdHlsZShcIm9wYWNpdHlcIiwgMCk7XG5cbiAgdXRpbC5hcHBseVRyYW5zaXRpb24oc3ZnQ2x1c3RlcnMsIGcpXG4gICAgLnN0eWxlKFwib3BhY2l0eVwiLCAxKTtcblxuICBzdmdDbHVzdGVycy5lYWNoKGZ1bmN0aW9uKHYpIHtcbiAgICB2YXIgbm9kZSA9IGcubm9kZSh2KSxcbiAgICAgICAgdGhpc0dyb3VwID0gZDMuc2VsZWN0KHRoaXMpO1xuICAgIGQzLnNlbGVjdCh0aGlzKS5hcHBlbmQoXCJyZWN0XCIpO1xuICAgIHZhciBsYWJlbEdyb3VwID0gdGhpc0dyb3VwLmFwcGVuZChcImdcIikuYXR0cihcImNsYXNzXCIsIFwibGFiZWxcIik7XG4gICAgYWRkTGFiZWwobGFiZWxHcm91cCwgbm9kZSwgbm9kZS5jbHVzdGVyTGFiZWxQb3MpO1xuICB9KTtcblxuICBzdmdDbHVzdGVycy5zZWxlY3RBbGwoXCJyZWN0XCIpLmVhY2goZnVuY3Rpb24oYykge1xuICAgIHZhciBub2RlID0gZy5ub2RlKGMpO1xuICAgIHZhciBkb21DbHVzdGVyID0gZDMuc2VsZWN0KHRoaXMpO1xuICAgIHV0aWwuYXBwbHlTdHlsZShkb21DbHVzdGVyLCBub2RlLnN0eWxlKTtcbiAgfSk7XG5cbiAgdXRpbC5hcHBseVRyYW5zaXRpb24oc3ZnQ2x1c3RlcnMuZXhpdCgpLCBnKVxuICAgIC5zdHlsZShcIm9wYWNpdHlcIiwgMClcbiAgICAucmVtb3ZlKCk7XG5cbiAgcmV0dXJuIHN2Z0NsdXN0ZXJzO1xufVxuIiwiXCJ1c2Ugc3RyaWN0XCI7XG5cbnZhciBfID0gcmVxdWlyZShcIi4vbG9kYXNoXCIpLFxuICAgIGFkZExhYmVsID0gcmVxdWlyZShcIi4vbGFiZWwvYWRkLWxhYmVsXCIpLFxuICAgIHV0aWwgPSByZXF1aXJlKFwiLi91dGlsXCIpLFxuICAgIGQzID0gcmVxdWlyZShcIi4vZDNcIik7XG5cbm1vZHVsZS5leHBvcnRzID0gY3JlYXRlRWRnZUxhYmVscztcblxuZnVuY3Rpb24gY3JlYXRlRWRnZUxhYmVscyhzZWxlY3Rpb24sIGcpIHtcbiAgdmFyIHN2Z0VkZ2VMYWJlbHMgPSBzZWxlY3Rpb24uc2VsZWN0QWxsKFwiZy5lZGdlTGFiZWxcIilcbiAgICAuZGF0YShnLmVkZ2VzKCksIGZ1bmN0aW9uKGUpIHsgcmV0dXJuIHV0aWwuZWRnZVRvSWQoZSk7IH0pXG4gICAgLmNsYXNzZWQoXCJ1cGRhdGVcIiwgdHJ1ZSk7XG5cbiAgc3ZnRWRnZUxhYmVscy5zZWxlY3RBbGwoXCIqXCIpLnJlbW92ZSgpO1xuICBzdmdFZGdlTGFiZWxzLmVudGVyKClcbiAgICAuYXBwZW5kKFwiZ1wiKVxuICAgICAgLmNsYXNzZWQoXCJlZGdlTGFiZWxcIiwgdHJ1ZSlcbiAgICAgIC5zdHlsZShcIm9wYWNpdHlcIiwgMCk7XG4gIHN2Z0VkZ2VMYWJlbHMuZWFjaChmdW5jdGlvbihlKSB7XG4gICAgdmFyIGVkZ2UgPSBnLmVkZ2UoZSksXG4gICAgICAgIGxhYmVsID0gYWRkTGFiZWwoZDMuc2VsZWN0KHRoaXMpLCBnLmVkZ2UoZSksIDAsIDApLmNsYXNzZWQoXCJsYWJlbFwiLCB0cnVlKSxcbiAgICAgICAgYmJveCA9IGxhYmVsLm5vZGUoKS5nZXRCQm94KCk7XG5cbiAgICBpZiAoZWRnZS5sYWJlbElkKSB7IGxhYmVsLmF0dHIoXCJpZFwiLCBlZGdlLmxhYmVsSWQpOyB9XG4gICAgaWYgKCFfLmhhcyhlZGdlLCBcIndpZHRoXCIpKSB7IGVkZ2Uud2lkdGggPSBiYm94LndpZHRoOyB9XG4gICAgaWYgKCFfLmhhcyhlZGdlLCBcImhlaWdodFwiKSkgeyBlZGdlLmhlaWdodCA9IGJib3guaGVpZ2h0OyB9XG4gIH0pO1xuXG4gIHV0aWwuYXBwbHlUcmFuc2l0aW9uKHN2Z0VkZ2VMYWJlbHMuZXhpdCgpLCBnKVxuICAgIC5zdHlsZShcIm9wYWNpdHlcIiwgMClcbiAgICAucmVtb3ZlKCk7XG5cbiAgcmV0dXJuIHN2Z0VkZ2VMYWJlbHM7XG59XG4iLCJcInVzZSBzdHJpY3RcIjtcblxudmFyIF8gPSByZXF1aXJlKFwiLi9sb2Rhc2hcIiksXG4gICAgaW50ZXJzZWN0Tm9kZSA9IHJlcXVpcmUoXCIuL2ludGVyc2VjdC9pbnRlcnNlY3Qtbm9kZVwiKSxcbiAgICB1dGlsID0gcmVxdWlyZShcIi4vdXRpbFwiKSxcbiAgICBkMyA9IHJlcXVpcmUoXCIuL2QzXCIpO1xuXG5tb2R1bGUuZXhwb3J0cyA9IGNyZWF0ZUVkZ2VQYXRocztcblxuZnVuY3Rpb24gY3JlYXRlRWRnZVBhdGhzKHNlbGVjdGlvbiwgZywgYXJyb3dzKSB7XG4gIHZhciBzdmdQYXRocyA9IHNlbGVjdGlvbi5zZWxlY3RBbGwoXCJnLmVkZ2VQYXRoXCIpXG4gICAgLmRhdGEoZy5lZGdlcygpLCBmdW5jdGlvbihlKSB7IHJldHVybiB1dGlsLmVkZ2VUb0lkKGUpOyB9KVxuICAgIC5jbGFzc2VkKFwidXBkYXRlXCIsIHRydWUpO1xuXG4gIGVudGVyKHN2Z1BhdGhzLCBnKTtcbiAgZXhpdChzdmdQYXRocywgZyk7XG5cbiAgdXRpbC5hcHBseVRyYW5zaXRpb24oc3ZnUGF0aHMsIGcpXG4gICAgLnN0eWxlKFwib3BhY2l0eVwiLCAxKTtcblxuICAvLyBTYXZlIERPTSBlbGVtZW50IGluIHRoZSBwYXRoIGdyb3VwLCBhbmQgc2V0IElEIGFuZCBjbGFzc1xuICBzdmdQYXRocy5lYWNoKGZ1bmN0aW9uKGUpIHtcbiAgICB2YXIgZG9tRWRnZSA9IGQzLnNlbGVjdCh0aGlzKTtcbiAgICB2YXIgZWRnZSA9IGcuZWRnZShlKTtcbiAgICBlZGdlLmVsZW0gPSB0aGlzO1xuXG4gICAgaWYgKGVkZ2UuaWQpIHtcbiAgICAgIGRvbUVkZ2UuYXR0cihcImlkXCIsIGVkZ2UuaWQpO1xuICAgIH1cblxuICAgIHV0aWwuYXBwbHlDbGFzcyhkb21FZGdlLCBlZGdlW1wiY2xhc3NcIl0sXG4gICAgICAoZG9tRWRnZS5jbGFzc2VkKFwidXBkYXRlXCIpID8gXCJ1cGRhdGUgXCIgOiBcIlwiKSArIFwiZWRnZVBhdGhcIik7XG4gIH0pO1xuXG4gIHN2Z1BhdGhzLnNlbGVjdEFsbChcInBhdGgucGF0aFwiKVxuICAgIC5lYWNoKGZ1bmN0aW9uKGUpIHtcbiAgICAgIHZhciBlZGdlID0gZy5lZGdlKGUpO1xuICAgICAgZWRnZS5hcnJvd2hlYWRJZCA9IF8udW5pcXVlSWQoXCJhcnJvd2hlYWRcIik7XG5cbiAgICAgIHZhciBkb21FZGdlID0gZDMuc2VsZWN0KHRoaXMpXG4gICAgICAgIC5hdHRyKFwibWFya2VyLWVuZFwiLCBmdW5jdGlvbigpIHtcbiAgICAgICAgICByZXR1cm4gXCJ1cmwoI1wiICsgZWRnZS5hcnJvd2hlYWRJZCArIFwiKVwiO1xuICAgICAgICB9KVxuICAgICAgICAuc3R5bGUoXCJmaWxsXCIsIFwibm9uZVwiKTtcblxuICAgICAgdXRpbC5hcHBseVRyYW5zaXRpb24oZG9tRWRnZSwgZylcbiAgICAgICAgLmF0dHIoXCJkXCIsIGZ1bmN0aW9uKGUpIHsgcmV0dXJuIGNhbGNQb2ludHMoZywgZSk7IH0pO1xuXG4gICAgICB1dGlsLmFwcGx5U3R5bGUoZG9tRWRnZSwgZWRnZS5zdHlsZSk7XG4gICAgfSk7XG5cbiAgc3ZnUGF0aHMuc2VsZWN0QWxsKFwiZGVmcyAqXCIpLnJlbW92ZSgpO1xuICBzdmdQYXRocy5zZWxlY3RBbGwoXCJkZWZzXCIpXG4gICAgLmVhY2goZnVuY3Rpb24oZSkge1xuICAgICAgdmFyIGVkZ2UgPSBnLmVkZ2UoZSksXG4gICAgICAgICAgYXJyb3doZWFkID0gYXJyb3dzW2VkZ2UuYXJyb3doZWFkXTtcbiAgICAgIGFycm93aGVhZChkMy5zZWxlY3QodGhpcyksIGVkZ2UuYXJyb3doZWFkSWQsIGVkZ2UsIFwiYXJyb3doZWFkXCIpO1xuICAgIH0pO1xuXG4gIHJldHVybiBzdmdQYXRocztcbn1cblxuZnVuY3Rpb24gY2FsY1BvaW50cyhnLCBlKSB7XG4gIHZhciBlZGdlID0gZy5lZGdlKGUpLFxuICAgICAgdGFpbCA9IGcubm9kZShlLnYpLFxuICAgICAgaGVhZCA9IGcubm9kZShlLncpLFxuICAgICAgcG9pbnRzID0gZWRnZS5wb2ludHMuc2xpY2UoMSwgZWRnZS5wb2ludHMubGVuZ3RoIC0gMSk7XG4gIHBvaW50cy51bnNoaWZ0KGludGVyc2VjdE5vZGUodGFpbCwgcG9pbnRzWzBdKSk7XG4gIHBvaW50cy5wdXNoKGludGVyc2VjdE5vZGUoaGVhZCwgcG9pbnRzW3BvaW50cy5sZW5ndGggLSAxXSkpO1xuXG4gIHJldHVybiBjcmVhdGVMaW5lKGVkZ2UsIHBvaW50cyk7XG59XG5cbmZ1bmN0aW9uIGNyZWF0ZUxpbmUoZWRnZSwgcG9pbnRzKSB7XG4gIHZhciBsaW5lID0gZDMuc3ZnLmxpbmUoKVxuICAgIC54KGZ1bmN0aW9uKGQpIHsgcmV0dXJuIGQueDsgfSlcbiAgICAueShmdW5jdGlvbihkKSB7IHJldHVybiBkLnk7IH0pO1xuXG4gIGlmIChfLmhhcyhlZGdlLCBcImxpbmVJbnRlcnBvbGF0ZVwiKSkge1xuICAgIGxpbmUuaW50ZXJwb2xhdGUoZWRnZS5saW5lSW50ZXJwb2xhdGUpO1xuICB9XG5cbiAgaWYgKF8uaGFzKGVkZ2UsIFwibGluZVRlbnNpb25cIikpIHtcbiAgICBsaW5lLnRlbnNpb24oTnVtYmVyKGVkZ2UubGluZVRlbnNpb24pKTtcbiAgfVxuXG4gIHJldHVybiBsaW5lKHBvaW50cyk7XG59XG5cbmZ1bmN0aW9uIGdldENvb3JkcyhlbGVtKSB7XG4gIHZhciBiYm94ID0gZWxlbS5nZXRCQm94KCksXG4gICAgICBtYXRyaXggPSBlbGVtLmdldFRyYW5zZm9ybVRvRWxlbWVudChlbGVtLm93bmVyU1ZHRWxlbWVudClcbiAgICAgICAgLnRyYW5zbGF0ZShiYm94LndpZHRoIC8gMiwgYmJveC5oZWlnaHQgLyAyKTtcbiAgcmV0dXJuIHsgeDogbWF0cml4LmUsIHk6IG1hdHJpeC5mIH07XG59XG5cbmZ1bmN0aW9uIGVudGVyKHN2Z1BhdGhzLCBnKSB7XG4gIHZhciBzdmdQYXRoc0VudGVyID0gc3ZnUGF0aHMuZW50ZXIoKVxuICAgIC5hcHBlbmQoXCJnXCIpXG4gICAgICAuYXR0cihcImNsYXNzXCIsIFwiZWRnZVBhdGhcIilcbiAgICAgIC5zdHlsZShcIm9wYWNpdHlcIiwgMCk7XG4gIHN2Z1BhdGhzRW50ZXIuYXBwZW5kKFwicGF0aFwiKVxuICAgIC5hdHRyKFwiY2xhc3NcIiwgXCJwYXRoXCIpXG4gICAgLmF0dHIoXCJkXCIsIGZ1bmN0aW9uKGUpIHtcbiAgICAgIHZhciBlZGdlID0gZy5lZGdlKGUpLFxuICAgICAgICAgIHNvdXJjZUVsZW0gPSBnLm5vZGUoZS52KS5lbGVtLFxuICAgICAgICAgIHBvaW50cyA9IF8ucmFuZ2UoZWRnZS5wb2ludHMubGVuZ3RoKS5tYXAoZnVuY3Rpb24oKSB7IHJldHVybiBnZXRDb29yZHMoc291cmNlRWxlbSk7IH0pO1xuICAgICAgcmV0dXJuIGNyZWF0ZUxpbmUoZWRnZSwgcG9pbnRzKTtcbiAgICB9KTtcbiAgc3ZnUGF0aHNFbnRlci5hcHBlbmQoXCJkZWZzXCIpO1xufVxuXG5mdW5jdGlvbiBleGl0KHN2Z1BhdGhzLCBnKSB7XG4gIHZhciBzdmdQYXRoRXhpdCA9IHN2Z1BhdGhzLmV4aXQoKTtcbiAgdXRpbC5hcHBseVRyYW5zaXRpb24oc3ZnUGF0aEV4aXQsIGcpXG4gICAgLnN0eWxlKFwib3BhY2l0eVwiLCAwKVxuICAgIC5yZW1vdmUoKTtcblxuICB1dGlsLmFwcGx5VHJhbnNpdGlvbihzdmdQYXRoRXhpdC5zZWxlY3QoXCJwYXRoLnBhdGhcIiksIGcpXG4gICAgLmF0dHIoXCJkXCIsIGZ1bmN0aW9uKGUpIHtcbiAgICAgIHZhciBzb3VyY2UgPSBnLm5vZGUoZS52KTtcblxuICAgICAgaWYgKHNvdXJjZSkge1xuICAgICAgICB2YXIgcG9pbnRzID0gXy5yYW5nZSh0aGlzLnBhdGhTZWdMaXN0Lmxlbmd0aCkubWFwKGZ1bmN0aW9uKCkgeyByZXR1cm4gc291cmNlOyB9KTtcbiAgICAgICAgcmV0dXJuIGNyZWF0ZUxpbmUoe30sIHBvaW50cyk7XG4gICAgICB9IGVsc2Uge1xuICAgICAgICByZXR1cm4gZDMuc2VsZWN0KHRoaXMpLmF0dHIoXCJkXCIpO1xuICAgICAgfVxuICAgIH0pO1xufVxuIiwiXCJ1c2Ugc3RyaWN0XCI7XG5cbnZhciBfID0gcmVxdWlyZShcIi4vbG9kYXNoXCIpLFxuICAgIGFkZExhYmVsID0gcmVxdWlyZShcIi4vbGFiZWwvYWRkLWxhYmVsXCIpLFxuICAgIHV0aWwgPSByZXF1aXJlKFwiLi91dGlsXCIpLFxuICAgIGQzID0gcmVxdWlyZShcIi4vZDNcIik7XG5cbm1vZHVsZS5leHBvcnRzID0gY3JlYXRlTm9kZXM7XG5cbmZ1bmN0aW9uIGNyZWF0ZU5vZGVzKHNlbGVjdGlvbiwgZywgc2hhcGVzKSB7XG4gIHZhciBzaW1wbGVOb2RlcyA9IGcubm9kZXMoKS5maWx0ZXIoZnVuY3Rpb24odikgeyByZXR1cm4gIXV0aWwuaXNTdWJncmFwaChnLCB2KTsgfSk7XG4gIHZhciBzdmdOb2RlcyA9IHNlbGVjdGlvbi5zZWxlY3RBbGwoXCJnLm5vZGVcIilcbiAgICAuZGF0YShzaW1wbGVOb2RlcywgZnVuY3Rpb24odikgeyByZXR1cm4gdjsgfSlcbiAgICAuY2xhc3NlZChcInVwZGF0ZVwiLCB0cnVlKTtcblxuICBzdmdOb2Rlcy5zZWxlY3RBbGwoXCIqXCIpLnJlbW92ZSgpO1xuICBzdmdOb2Rlcy5lbnRlcigpXG4gICAgLmFwcGVuZChcImdcIilcbiAgICAgIC5hdHRyKFwiY2xhc3NcIiwgXCJub2RlXCIpXG4gICAgICAuc3R5bGUoXCJvcGFjaXR5XCIsIDApO1xuICBzdmdOb2Rlcy5lYWNoKGZ1bmN0aW9uKHYpIHtcbiAgICB2YXIgbm9kZSA9IGcubm9kZSh2KSxcbiAgICAgICAgdGhpc0dyb3VwID0gZDMuc2VsZWN0KHRoaXMpLFxuICAgICAgICBsYWJlbEdyb3VwID0gdGhpc0dyb3VwLmFwcGVuZChcImdcIikuYXR0cihcImNsYXNzXCIsIFwibGFiZWxcIiksXG4gICAgICAgIGxhYmVsRG9tID0gYWRkTGFiZWwobGFiZWxHcm91cCwgbm9kZSksXG4gICAgICAgIHNoYXBlID0gc2hhcGVzW25vZGUuc2hhcGVdLFxuICAgICAgICBiYm94ID0gXy5waWNrKGxhYmVsRG9tLm5vZGUoKS5nZXRCQm94KCksIFwid2lkdGhcIiwgXCJoZWlnaHRcIik7XG5cbiAgICBub2RlLmVsZW0gPSB0aGlzO1xuXG4gICAgaWYgKG5vZGUuaWQpIHsgdGhpc0dyb3VwLmF0dHIoXCJpZFwiLCBub2RlLmlkKTsgfVxuICAgIGlmIChub2RlLmxhYmVsSWQpIHsgbGFiZWxHcm91cC5hdHRyKFwiaWRcIiwgbm9kZS5sYWJlbElkKTsgfVxuICAgIHV0aWwuYXBwbHlDbGFzcyh0aGlzR3JvdXAsIG5vZGVbXCJjbGFzc1wiXSxcbiAgICAgICh0aGlzR3JvdXAuY2xhc3NlZChcInVwZGF0ZVwiKSA/IFwidXBkYXRlIFwiIDogXCJcIikgKyBcIm5vZGVcIik7XG5cbiAgICBpZiAoXy5oYXMobm9kZSwgXCJ3aWR0aFwiKSkgeyBiYm94LndpZHRoID0gbm9kZS53aWR0aDsgfVxuICAgIGlmIChfLmhhcyhub2RlLCBcImhlaWdodFwiKSkgeyBiYm94LmhlaWdodCA9IG5vZGUuaGVpZ2h0OyB9XG5cbiAgICBiYm94LndpZHRoICs9IG5vZGUucGFkZGluZ0xlZnQgKyBub2RlLnBhZGRpbmdSaWdodDtcbiAgICBiYm94LmhlaWdodCArPSBub2RlLnBhZGRpbmdUb3AgKyBub2RlLnBhZGRpbmdCb3R0b207XG4gICAgbGFiZWxHcm91cC5hdHRyKFwidHJhbnNmb3JtXCIsIFwidHJhbnNsYXRlKFwiICtcbiAgICAgICgobm9kZS5wYWRkaW5nTGVmdCAtIG5vZGUucGFkZGluZ1JpZ2h0KSAvIDIpICsgXCIsXCIgK1xuICAgICAgKChub2RlLnBhZGRpbmdUb3AgLSBub2RlLnBhZGRpbmdCb3R0b20pIC8gMikgKyBcIilcIik7XG5cbiAgICB2YXIgc2hhcGVTdmcgPSBzaGFwZShkMy5zZWxlY3QodGhpcyksIGJib3gsIG5vZGUpO1xuICAgIHV0aWwuYXBwbHlTdHlsZShzaGFwZVN2Zywgbm9kZS5zdHlsZSk7XG5cbiAgICB2YXIgc2hhcGVCQm94ID0gc2hhcGVTdmcubm9kZSgpLmdldEJCb3goKTtcbiAgICBub2RlLndpZHRoID0gc2hhcGVCQm94LndpZHRoO1xuICAgIG5vZGUuaGVpZ2h0ID0gc2hhcGVCQm94LmhlaWdodDtcbiAgfSk7XG5cbiAgdXRpbC5hcHBseVRyYW5zaXRpb24oc3ZnTm9kZXMuZXhpdCgpLCBnKVxuICAgIC5zdHlsZShcIm9wYWNpdHlcIiwgMClcbiAgICAucmVtb3ZlKCk7XG5cbiAgcmV0dXJuIHN2Z05vZGVzO1xufVxuIiwiLy8gU3R1YiB0byBnZXQgRDMgZWl0aGVyIHZpYSBOUE0gb3IgZnJvbSB0aGUgZ2xvYmFsIG9iamVjdFxubW9kdWxlLmV4cG9ydHMgPSB3aW5kb3cuZDM7XG4iLCIvKiBnbG9iYWwgd2luZG93ICovXG5cbnZhciBkYWdyZTtcblxuaWYgKHJlcXVpcmUpIHtcbiAgdHJ5IHtcbiAgICBkYWdyZSA9IHJlcXVpcmUoXCJkYWdyZVwiKTtcbiAgfSBjYXRjaCAoZSkge31cbn1cblxuaWYgKCFkYWdyZSkge1xuICBkYWdyZSA9IHdpbmRvdy5kYWdyZTtcbn1cblxubW9kdWxlLmV4cG9ydHMgPSBkYWdyZTtcbiIsIi8qIGdsb2JhbCB3aW5kb3cgKi9cblxudmFyIGdyYXBobGliO1xuXG5pZiAocmVxdWlyZSkge1xuICB0cnkge1xuICAgIGdyYXBobGliID0gcmVxdWlyZShcImdyYXBobGliXCIpO1xuICB9IGNhdGNoIChlKSB7fVxufVxuXG5pZiAoIWdyYXBobGliKSB7XG4gIGdyYXBobGliID0gd2luZG93LmdyYXBobGliO1xufVxuXG5tb2R1bGUuZXhwb3J0cyA9IGdyYXBobGliO1xuIiwibW9kdWxlLmV4cG9ydHMgPSB7XG4gIG5vZGU6IHJlcXVpcmUoXCIuL2ludGVyc2VjdC1ub2RlXCIpLFxuICBjaXJjbGU6IHJlcXVpcmUoXCIuL2ludGVyc2VjdC1jaXJjbGVcIiksXG4gIGVsbGlwc2U6IHJlcXVpcmUoXCIuL2ludGVyc2VjdC1lbGxpcHNlXCIpLFxuICBwb2x5Z29uOiByZXF1aXJlKFwiLi9pbnRlcnNlY3QtcG9seWdvblwiKSxcbiAgcmVjdDogcmVxdWlyZShcIi4vaW50ZXJzZWN0LXJlY3RcIilcbn07XG4iLCJ2YXIgaW50ZXJzZWN0RWxsaXBzZSA9IHJlcXVpcmUoXCIuL2ludGVyc2VjdC1lbGxpcHNlXCIpO1xuXG5tb2R1bGUuZXhwb3J0cyA9IGludGVyc2VjdENpcmNsZTtcblxuZnVuY3Rpb24gaW50ZXJzZWN0Q2lyY2xlKG5vZGUsIHJ4LCBwb2ludCkge1xuICByZXR1cm4gaW50ZXJzZWN0RWxsaXBzZShub2RlLCByeCwgcngsIHBvaW50KTtcbn1cbiIsIm1vZHVsZS5leHBvcnRzID0gaW50ZXJzZWN0RWxsaXBzZTtcblxuZnVuY3Rpb24gaW50ZXJzZWN0RWxsaXBzZShub2RlLCByeCwgcnksIHBvaW50KSB7XG4gIC8vIEZvcm11bGFlIGZyb206IGh0dHA6Ly9tYXRod29ybGQud29sZnJhbS5jb20vRWxsaXBzZS1MaW5lSW50ZXJzZWN0aW9uLmh0bWxcblxuICB2YXIgY3ggPSBub2RlLng7XG4gIHZhciBjeSA9IG5vZGUueTtcblxuICB2YXIgcHggPSBjeCAtIHBvaW50Lng7XG4gIHZhciBweSA9IGN5IC0gcG9pbnQueTtcblxuICB2YXIgZGV0ID0gTWF0aC5zcXJ0KHJ4ICogcnggKiBweSAqIHB5ICsgcnkgKiByeSAqIHB4ICogcHgpO1xuXG4gIHZhciBkeCA9IE1hdGguYWJzKHJ4ICogcnkgKiBweCAvIGRldCk7XG4gIGlmIChwb2ludC54IDwgY3gpIHtcbiAgICBkeCA9IC1keDtcbiAgfVxuICB2YXIgZHkgPSBNYXRoLmFicyhyeCAqIHJ5ICogcHkgLyBkZXQpO1xuICBpZiAocG9pbnQueSA8IGN5KSB7XG4gICAgZHkgPSAtZHk7XG4gIH1cblxuICByZXR1cm4ge3g6IGN4ICsgZHgsIHk6IGN5ICsgZHl9O1xufVxuXG4iLCJtb2R1bGUuZXhwb3J0cyA9IGludGVyc2VjdExpbmU7XG5cbi8qXG4gKiBSZXR1cm5zIHRoZSBwb2ludCBhdCB3aGljaCB0d28gbGluZXMsIHAgYW5kIHEsIGludGVyc2VjdCBvciByZXR1cm5zXG4gKiB1bmRlZmluZWQgaWYgdGhleSBkbyBub3QgaW50ZXJzZWN0LlxuICovXG5mdW5jdGlvbiBpbnRlcnNlY3RMaW5lKHAxLCBwMiwgcTEsIHEyKSB7XG4gIC8vIEFsZ29yaXRobSBmcm9tIEouIEF2cm8sIChlZC4pIEdyYXBoaWNzIEdlbXMsIE5vIDIsIE1vcmdhbiBLYXVmbWFubiwgMTk5NCxcbiAgLy8gcDcgYW5kIHA0NzMuXG5cbiAgdmFyIGExLCBhMiwgYjEsIGIyLCBjMSwgYzI7XG4gIHZhciByMSwgcjIgLCByMywgcjQ7XG4gIHZhciBkZW5vbSwgb2Zmc2V0LCBudW07XG4gIHZhciB4LCB5O1xuXG4gIC8vIENvbXB1dGUgYTEsIGIxLCBjMSwgd2hlcmUgbGluZSBqb2luaW5nIHBvaW50cyAxIGFuZCAyIGlzIEYoeCx5KSA9IGExIHggK1xuICAvLyBiMSB5ICsgYzEgPSAwLlxuICBhMSA9IHAyLnkgLSBwMS55O1xuICBiMSA9IHAxLnggLSBwMi54O1xuICBjMSA9IChwMi54ICogcDEueSkgLSAocDEueCAqIHAyLnkpO1xuXG4gIC8vIENvbXB1dGUgcjMgYW5kIHI0LlxuICByMyA9ICgoYTEgKiBxMS54KSArIChiMSAqIHExLnkpICsgYzEpO1xuICByNCA9ICgoYTEgKiBxMi54KSArIChiMSAqIHEyLnkpICsgYzEpO1xuXG4gIC8vIENoZWNrIHNpZ25zIG9mIHIzIGFuZCByNC4gSWYgYm90aCBwb2ludCAzIGFuZCBwb2ludCA0IGxpZSBvblxuICAvLyBzYW1lIHNpZGUgb2YgbGluZSAxLCB0aGUgbGluZSBzZWdtZW50cyBkbyBub3QgaW50ZXJzZWN0LlxuICBpZiAoKHIzICE9PSAwKSAmJiAocjQgIT09IDApICYmIHNhbWVTaWduKHIzLCByNCkpIHtcbiAgICByZXR1cm4gLypET05UX0lOVEVSU0VDVCovO1xuICB9XG5cbiAgLy8gQ29tcHV0ZSBhMiwgYjIsIGMyIHdoZXJlIGxpbmUgam9pbmluZyBwb2ludHMgMyBhbmQgNCBpcyBHKHgseSkgPSBhMiB4ICsgYjIgeSArIGMyID0gMFxuICBhMiA9IHEyLnkgLSBxMS55O1xuICBiMiA9IHExLnggLSBxMi54O1xuICBjMiA9IChxMi54ICogcTEueSkgLSAocTEueCAqIHEyLnkpO1xuXG4gIC8vIENvbXB1dGUgcjEgYW5kIHIyXG4gIHIxID0gKGEyICogcDEueCkgKyAoYjIgKiBwMS55eSkgKyBjMjtcbiAgcjIgPSAoYTIgKiBwMi54KSArIChiMiAqIHAyLnkpICsgYzI7XG5cbiAgLy8gQ2hlY2sgc2lnbnMgb2YgcjEgYW5kIHIyLiBJZiBib3RoIHBvaW50IDEgYW5kIHBvaW50IDIgbGllXG4gIC8vIG9uIHNhbWUgc2lkZSBvZiBzZWNvbmQgbGluZSBzZWdtZW50LCB0aGUgbGluZSBzZWdtZW50cyBkb1xuICAvLyBub3QgaW50ZXJzZWN0LlxuICBpZiAoKHIxICE9PSAwKSAmJiAocjIgIT09IDApICYmIChzYW1lU2lnbihyMSwgcjIpKSkge1xuICAgIHJldHVybiAvKkRPTlRfSU5URVJTRUNUKi87XG4gIH1cblxuICAvLyBMaW5lIHNlZ21lbnRzIGludGVyc2VjdDogY29tcHV0ZSBpbnRlcnNlY3Rpb24gcG9pbnQuXG4gIGRlbm9tID0gKGExICogYjIpIC0gKGEyICogYjEpO1xuICBpZiAoZGVub20gPT09IDApIHtcbiAgICByZXR1cm4gLypDT0xMSU5FQVIqLztcbiAgfVxuXG4gIG9mZnNldCA9IE1hdGguYWJzKGRlbm9tIC8gMik7XG5cbiAgLy8gVGhlIGRlbm9tLzIgaXMgdG8gZ2V0IHJvdW5kaW5nIGluc3RlYWQgb2YgdHJ1bmNhdGluZy4gSXRcbiAgLy8gaXMgYWRkZWQgb3Igc3VidHJhY3RlZCB0byB0aGUgbnVtZXJhdG9yLCBkZXBlbmRpbmcgdXBvbiB0aGVcbiAgLy8gc2lnbiBvZiB0aGUgbnVtZXJhdG9yLlxuICBudW0gPSAoYjEgKiBjMikgLSAoYjIgKiBjMSk7XG4gIHggPSAobnVtIDwgMCkgPyAoKG51bSAtIG9mZnNldCkgLyBkZW5vbSkgOiAoKG51bSArIG9mZnNldCkgLyBkZW5vbSk7XG5cbiAgbnVtID0gKGEyICogYzEpIC0gKGExICogYzIpO1xuICB5ID0gKG51bSA8IDApID8gKChudW0gLSBvZmZzZXQpIC8gZGVub20pIDogKChudW0gKyBvZmZzZXQpIC8gZGVub20pO1xuXG4gIHJldHVybiB7IHg6IHgsIHk6IHkgfTtcbn1cblxuZnVuY3Rpb24gc2FtZVNpZ24ocjEsIHIyKSB7XG4gIHJldHVybiByMSAqIHIyID4gMDtcbn1cbiIsIm1vZHVsZS5leHBvcnRzID0gaW50ZXJzZWN0Tm9kZTtcblxuZnVuY3Rpb24gaW50ZXJzZWN0Tm9kZShub2RlLCBwb2ludCkge1xuICByZXR1cm4gbm9kZS5pbnRlcnNlY3QocG9pbnQpO1xufVxuIiwidmFyIGludGVyc2VjdExpbmUgPSByZXF1aXJlKFwiLi9pbnRlcnNlY3QtbGluZVwiKTtcblxubW9kdWxlLmV4cG9ydHMgPSBpbnRlcnNlY3RQb2x5Z29uO1xuXG4vKlxuICogUmV0dXJucyB0aGUgcG9pbnQgKHt4LCB5fSkgYXQgd2hpY2ggdGhlIHBvaW50IGFyZ3VtZW50IGludGVyc2VjdHMgd2l0aCB0aGVcbiAqIG5vZGUgYXJndW1lbnQgYXNzdW1pbmcgdGhhdCBpdCBoYXMgdGhlIHNoYXBlIHNwZWNpZmllZCBieSBwb2x5Z29uLlxuICovXG5mdW5jdGlvbiBpbnRlcnNlY3RQb2x5Z29uKG5vZGUsIHBvbHlQb2ludHMsIHBvaW50KSB7XG4gIHZhciB4MSA9IG5vZGUueDtcbiAgdmFyIHkxID0gbm9kZS55O1xuXG4gIHZhciBpbnRlcnNlY3Rpb25zID0gW107XG5cbiAgdmFyIG1pblggPSBOdW1iZXIuUE9TSVRJVkVfSU5GSU5JVFksXG4gICAgICBtaW5ZID0gTnVtYmVyLlBPU0lUSVZFX0lORklOSVRZO1xuICBwb2x5UG9pbnRzLmZvckVhY2goZnVuY3Rpb24oZW50cnkpIHtcbiAgICBtaW5YID0gTWF0aC5taW4obWluWCwgZW50cnkueCk7XG4gICAgbWluWSA9IE1hdGgubWluKG1pblksIGVudHJ5LnkpO1xuICB9KTtcblxuICB2YXIgbGVmdCA9IHgxIC0gbm9kZS53aWR0aCAvIDIgLSBtaW5YO1xuICB2YXIgdG9wID0gIHkxIC0gbm9kZS5oZWlnaHQgLyAyIC0gbWluWTtcblxuICBmb3IgKHZhciBpID0gMDsgaSA8IHBvbHlQb2ludHMubGVuZ3RoOyBpKyspIHtcbiAgICB2YXIgcDEgPSBwb2x5UG9pbnRzW2ldO1xuICAgIHZhciBwMiA9IHBvbHlQb2ludHNbaSA8IHBvbHlQb2ludHMubGVuZ3RoIC0gMSA/IGkgKyAxIDogMF07XG4gICAgdmFyIGludGVyc2VjdCA9IGludGVyc2VjdExpbmUobm9kZSwgcG9pbnQsXG4gICAgICB7eDogbGVmdCArIHAxLngsIHk6IHRvcCArIHAxLnl9LCB7eDogbGVmdCArIHAyLngsIHk6IHRvcCArIHAyLnl9KTtcbiAgICBpZiAoaW50ZXJzZWN0KSB7XG4gICAgICBpbnRlcnNlY3Rpb25zLnB1c2goaW50ZXJzZWN0KTtcbiAgICB9XG4gIH1cblxuICBpZiAoIWludGVyc2VjdGlvbnMubGVuZ3RoKSB7XG4gICAgY29uc29sZS5sb2coXCJOTyBJTlRFUlNFQ1RJT04gRk9VTkQsIFJFVFVSTiBOT0RFIENFTlRFUlwiLCBub2RlKTtcbiAgICByZXR1cm4gbm9kZTtcbiAgfVxuXG4gIGlmIChpbnRlcnNlY3Rpb25zLmxlbmd0aCA+IDEpIHtcbiAgICAvLyBNb3JlIGludGVyc2VjdGlvbnMsIGZpbmQgdGhlIG9uZSBuZWFyZXN0IHRvIGVkZ2UgZW5kIHBvaW50XG4gICAgaW50ZXJzZWN0aW9ucy5zb3J0KGZ1bmN0aW9uKHAsIHEpIHtcbiAgICAgIHZhciBwZHggPSBwLnggLSBwb2ludC54LFxuICAgICAgICAgIHBkeSA9IHAueSAtIHBvaW50LnksXG4gICAgICAgICAgZGlzdHAgPSBNYXRoLnNxcnQocGR4ICogcGR4ICsgcGR5ICogcGR5KSxcblxuICAgICAgICAgIHFkeCA9IHEueCAtIHBvaW50LngsXG4gICAgICAgICAgcWR5ID0gcS55IC0gcG9pbnQueSxcbiAgICAgICAgICBkaXN0cSA9IE1hdGguc3FydChxZHggKiBxZHggKyBxZHkgKiBxZHkpO1xuXG4gICAgICByZXR1cm4gKGRpc3RwIDwgZGlzdHEpID8gLTEgOiAoZGlzdHAgPT09IGRpc3RxID8gMCA6IDEpO1xuICAgIH0pO1xuICB9XG4gIHJldHVybiBpbnRlcnNlY3Rpb25zWzBdO1xufVxuIiwibW9kdWxlLmV4cG9ydHMgPSBpbnRlcnNlY3RSZWN0O1xuXG5mdW5jdGlvbiBpbnRlcnNlY3RSZWN0KG5vZGUsIHBvaW50KSB7XG4gIHZhciB4ID0gbm9kZS54O1xuICB2YXIgeSA9IG5vZGUueTtcblxuICAvLyBSZWN0YW5nbGUgaW50ZXJzZWN0aW9uIGFsZ29yaXRobSBmcm9tOlxuICAvLyBodHRwOi8vbWF0aC5zdGFja2V4Y2hhbmdlLmNvbS9xdWVzdGlvbnMvMTA4MTEzL2ZpbmQtZWRnZS1iZXR3ZWVuLXR3by1ib3hlc1xuICB2YXIgZHggPSBwb2ludC54IC0geDtcbiAgdmFyIGR5ID0gcG9pbnQueSAtIHk7XG4gIHZhciB3ID0gbm9kZS53aWR0aCAvIDI7XG4gIHZhciBoID0gbm9kZS5oZWlnaHQgLyAyO1xuXG4gIHZhciBzeCwgc3k7XG4gIGlmIChNYXRoLmFicyhkeSkgKiB3ID4gTWF0aC5hYnMoZHgpICogaCkge1xuICAgIC8vIEludGVyc2VjdGlvbiBpcyB0b3Agb3IgYm90dG9tIG9mIHJlY3QuXG4gICAgaWYgKGR5IDwgMCkge1xuICAgICAgaCA9IC1oO1xuICAgIH1cbiAgICBzeCA9IGR5ID09PSAwID8gMCA6IGggKiBkeCAvIGR5O1xuICAgIHN5ID0gaDtcbiAgfSBlbHNlIHtcbiAgICAvLyBJbnRlcnNlY3Rpb24gaXMgbGVmdCBvciByaWdodCBvZiByZWN0LlxuICAgIGlmIChkeCA8IDApIHtcbiAgICAgIHcgPSAtdztcbiAgICB9XG4gICAgc3ggPSB3O1xuICAgIHN5ID0gZHggPT09IDAgPyAwIDogdyAqIGR5IC8gZHg7XG4gIH1cblxuICByZXR1cm4ge3g6IHggKyBzeCwgeTogeSArIHN5fTtcbn1cbiIsInZhciB1dGlsID0gcmVxdWlyZShcIi4uL3V0aWxcIik7XG5cbm1vZHVsZS5leHBvcnRzID0gYWRkSHRtbExhYmVsO1xuXG5mdW5jdGlvbiBhZGRIdG1sTGFiZWwocm9vdCwgbm9kZSkge1xuICB2YXIgZm8gPSByb290XG4gICAgLmFwcGVuZChcImZvcmVpZ25PYmplY3RcIilcbiAgICAgIC5hdHRyKFwid2lkdGhcIiwgXCIxMDAwMDBcIik7XG5cbiAgdmFyIGRpdiA9IGZvXG4gICAgLmFwcGVuZChcInhodG1sOmRpdlwiKTtcblxuICB2YXIgbGFiZWwgPSBub2RlLmxhYmVsO1xuICBzd2l0Y2godHlwZW9mIGxhYmVsKSB7XG4gICAgY2FzZSBcImZ1bmN0aW9uXCI6XG4gICAgICBkaXYuaW5zZXJ0KGxhYmVsKTtcbiAgICAgIGJyZWFrO1xuICAgIGNhc2UgXCJvYmplY3RcIjpcbiAgICAgIC8vIEN1cnJlbnRseSB3ZSBhc3N1bWUgdGhpcyBpcyBhIERPTSBvYmplY3QuXG4gICAgICBkaXYuaW5zZXJ0KGZ1bmN0aW9uKCkgeyByZXR1cm4gbGFiZWw7IH0pO1xuICAgICAgYnJlYWs7XG4gICAgZGVmYXVsdDogZGl2Lmh0bWwobGFiZWwpO1xuICB9XG5cbiAgdXRpbC5hcHBseVN0eWxlKGRpdiwgbm9kZS5sYWJlbFN0eWxlKTtcbiAgZGl2LnN0eWxlKFwiZGlzcGxheVwiLCBcImlubGluZS1ibG9ja1wiKTtcbiAgLy8gRml4IGZvciBmaXJlZm94XG4gIGRpdi5zdHlsZShcIndoaXRlLXNwYWNlXCIsIFwibm93cmFwXCIpO1xuXG4gIC8vIFRPRE8gZmluZCBhIGJldHRlciB3YXkgdG8gZ2V0IGRpbWVuc2lvbnMgZm9yIGZvcmVpZ25PYmplY3RzLi4uXG4gIHZhciB3LCBoO1xuICBkaXZcbiAgICAuZWFjaChmdW5jdGlvbigpIHtcbiAgICAgIHcgPSB0aGlzLmNsaWVudFdpZHRoO1xuICAgICAgaCA9IHRoaXMuY2xpZW50SGVpZ2h0O1xuICAgIH0pO1xuXG4gIGZvXG4gICAgLmF0dHIoXCJ3aWR0aFwiLCB3KVxuICAgIC5hdHRyKFwiaGVpZ2h0XCIsIGgpO1xuXG4gIHJldHVybiBmbztcbn1cbiIsInZhciBhZGRUZXh0TGFiZWwgPSByZXF1aXJlKFwiLi9hZGQtdGV4dC1sYWJlbFwiKSxcbiAgICBhZGRIdG1sTGFiZWwgPSByZXF1aXJlKFwiLi9hZGQtaHRtbC1sYWJlbFwiKSxcbiAgICBhZGRTVkdMYWJlbCAgPSByZXF1aXJlKFwiLi9hZGQtc3ZnLWxhYmVsXCIpO1xuXG5tb2R1bGUuZXhwb3J0cyA9IGFkZExhYmVsO1xuXG5mdW5jdGlvbiBhZGRMYWJlbChyb290LCBub2RlLCBsb2NhdGlvbikge1xuICB2YXIgbGFiZWwgPSBub2RlLmxhYmVsO1xuICB2YXIgbGFiZWxTdmcgPSByb290LmFwcGVuZChcImdcIik7XG5cbiAgLy8gQWxsb3cgdGhlIGxhYmVsIHRvIGJlIGEgc3RyaW5nLCBhIGZ1bmN0aW9uIHRoYXQgcmV0dXJucyBhIERPTSBlbGVtZW50LCBvclxuICAvLyBhIERPTSBlbGVtZW50IGl0c2VsZi5cbiAgaWYgKG5vZGUubGFiZWxUeXBlID09PSBcInN2Z1wiKSB7XG4gICAgYWRkU1ZHTGFiZWwobGFiZWxTdmcsIG5vZGUpO1xuICB9IGVsc2UgaWYgKHR5cGVvZiBsYWJlbCAhPT0gXCJzdHJpbmdcIiB8fCBub2RlLmxhYmVsVHlwZSA9PT0gXCJodG1sXCIpIHtcbiAgICBhZGRIdG1sTGFiZWwobGFiZWxTdmcsIG5vZGUpO1xuICB9IGVsc2Uge1xuICAgIGFkZFRleHRMYWJlbChsYWJlbFN2Zywgbm9kZSk7XG4gIH1cblxuICB2YXIgbGFiZWxCQm94ID0gbGFiZWxTdmcubm9kZSgpLmdldEJCb3goKTtcbiAgdmFyIHk7XG4gIHN3aXRjaChsb2NhdGlvbikge1xuICAgIGNhc2UgXCJ0b3BcIjpcbiAgICAgIHkgPSAoLW5vZGUuaGVpZ2h0IC8gMik7XG4gICAgICBicmVhaztcbiAgICBjYXNlIFwiYm90dG9tXCI6XG4gICAgICB5ID0gKG5vZGUuaGVpZ2h0IC8gMikgLSBsYWJlbEJCb3guaGVpZ2h0O1xuICAgICAgYnJlYWs7XG4gICAgZGVmYXVsdDpcbiAgICAgIHkgPSAoLWxhYmVsQkJveC5oZWlnaHQgLyAyKTtcbiAgfVxuICBsYWJlbFN2Zy5hdHRyKFwidHJhbnNmb3JtXCIsXG4gICAgICAgICAgICAgICAgXCJ0cmFuc2xhdGUoXCIgKyAoLWxhYmVsQkJveC53aWR0aCAvIDIpICsgXCIsXCIgKyB5ICsgXCIpXCIpO1xuXG4gIHJldHVybiBsYWJlbFN2Zztcbn1cbiIsInZhciB1dGlsID0gcmVxdWlyZShcIi4uL3V0aWxcIik7XG5cbm1vZHVsZS5leHBvcnRzID0gYWRkU1ZHTGFiZWw7XG5cbmZ1bmN0aW9uIGFkZFNWR0xhYmVsKHJvb3QsIG5vZGUpIHtcbiAgdmFyIGRvbU5vZGUgPSByb290O1xuXG4gIGRvbU5vZGUubm9kZSgpLmFwcGVuZENoaWxkKG5vZGUubGFiZWwpO1xuXG4gIHV0aWwuYXBwbHlTdHlsZShkb21Ob2RlLCBub2RlLmxhYmVsU3R5bGUpO1xuXG4gIHJldHVybiBkb21Ob2RlO1xufVxuIiwidmFyIHV0aWwgPSByZXF1aXJlKFwiLi4vdXRpbFwiKTtcblxubW9kdWxlLmV4cG9ydHMgPSBhZGRUZXh0TGFiZWw7XG5cbi8qXG4gKiBBdHRhY2hlcyBhIHRleHQgbGFiZWwgdG8gdGhlIHNwZWNpZmllZCByb290LiBIYW5kbGVzIGVzY2FwZSBzZXF1ZW5jZXMuXG4gKi9cbmZ1bmN0aW9uIGFkZFRleHRMYWJlbChyb290LCBub2RlKSB7XG4gIHZhciBkb21Ob2RlID0gcm9vdC5hcHBlbmQoXCJ0ZXh0XCIpO1xuXG4gIHZhciBsaW5lcyA9IHByb2Nlc3NFc2NhcGVTZXF1ZW5jZXMobm9kZS5sYWJlbCkuc3BsaXQoXCJcXG5cIik7XG4gIGZvciAodmFyIGkgPSAwOyBpIDwgbGluZXMubGVuZ3RoOyBpKyspIHtcbiAgICBkb21Ob2RlXG4gICAgICAuYXBwZW5kKFwidHNwYW5cIilcbiAgICAgICAgLmF0dHIoXCJ4bWw6c3BhY2VcIiwgXCJwcmVzZXJ2ZVwiKVxuICAgICAgICAuYXR0cihcImR5XCIsIFwiMWVtXCIpXG4gICAgICAgIC5hdHRyKFwieFwiLCBcIjFcIilcbiAgICAgICAgLnRleHQobGluZXNbaV0pO1xuICB9XG5cbiAgdXRpbC5hcHBseVN0eWxlKGRvbU5vZGUsIG5vZGUubGFiZWxTdHlsZSk7XG5cbiAgcmV0dXJuIGRvbU5vZGU7XG59XG5cbmZ1bmN0aW9uIHByb2Nlc3NFc2NhcGVTZXF1ZW5jZXModGV4dCkge1xuICB2YXIgbmV3VGV4dCA9IFwiXCIsXG4gICAgICBlc2NhcGVkID0gZmFsc2UsXG4gICAgICBjaDtcbiAgZm9yICh2YXIgaSA9IDA7IGkgPCB0ZXh0Lmxlbmd0aDsgKytpKSB7XG4gICAgY2ggPSB0ZXh0W2ldO1xuICAgIGlmIChlc2NhcGVkKSB7XG4gICAgICBzd2l0Y2goY2gpIHtcbiAgICAgICAgY2FzZSBcIm5cIjogbmV3VGV4dCArPSBcIlxcblwiOyBicmVhaztcbiAgICAgICAgZGVmYXVsdDogbmV3VGV4dCArPSBjaDtcbiAgICAgIH1cbiAgICAgIGVzY2FwZWQgPSBmYWxzZTtcbiAgICB9IGVsc2UgaWYgKGNoID09PSBcIlxcXFxcIikge1xuICAgICAgZXNjYXBlZCA9IHRydWU7XG4gICAgfSBlbHNlIHtcbiAgICAgIG5ld1RleHQgKz0gY2g7XG4gICAgfVxuICB9XG4gIHJldHVybiBuZXdUZXh0O1xufVxuIiwiLyogZ2xvYmFsIHdpbmRvdyAqL1xuXG52YXIgbG9kYXNoO1xuXG5pZiAocmVxdWlyZSkge1xuICB0cnkge1xuICAgIGxvZGFzaCA9IHJlcXVpcmUoXCJsb2Rhc2hcIik7XG4gIH0gY2F0Y2ggKGUpIHt9XG59XG5cbmlmICghbG9kYXNoKSB7XG4gIGxvZGFzaCA9IHdpbmRvdy5fO1xufVxuXG5tb2R1bGUuZXhwb3J0cyA9IGxvZGFzaDtcbiIsIlwidXNlIHN0cmljdFwiO1xuXG52YXIgdXRpbCA9IHJlcXVpcmUoXCIuL3V0aWxcIiksXG4gICAgZDMgPSByZXF1aXJlKFwiLi9kM1wiKTtcblxubW9kdWxlLmV4cG9ydHMgPSBwb3NpdGlvbkNsdXN0ZXJzO1xuXG5mdW5jdGlvbiBwb3NpdGlvbkNsdXN0ZXJzKHNlbGVjdGlvbiwgZykge1xuICB2YXIgY3JlYXRlZCA9IHNlbGVjdGlvbi5maWx0ZXIoZnVuY3Rpb24oKSB7IHJldHVybiAhZDMuc2VsZWN0KHRoaXMpLmNsYXNzZWQoXCJ1cGRhdGVcIik7IH0pO1xuXG4gIGZ1bmN0aW9uIHRyYW5zbGF0ZSh2KSB7XG4gICAgdmFyIG5vZGUgPSBnLm5vZGUodik7XG4gICAgcmV0dXJuIFwidHJhbnNsYXRlKFwiICsgbm9kZS54ICsgXCIsXCIgKyBub2RlLnkgKyBcIilcIjtcbiAgfVxuXG4gIGNyZWF0ZWQuYXR0cihcInRyYW5zZm9ybVwiLCB0cmFuc2xhdGUpO1xuXG4gIHV0aWwuYXBwbHlUcmFuc2l0aW9uKHNlbGVjdGlvbiwgZylcbiAgICAgIC5zdHlsZShcIm9wYWNpdHlcIiwgMSlcbiAgICAgIC5hdHRyKFwidHJhbnNmb3JtXCIsIHRyYW5zbGF0ZSk7XG5cbiAgdXRpbC5hcHBseVRyYW5zaXRpb24oY3JlYXRlZC5zZWxlY3RBbGwoXCJyZWN0XCIpLCBnKVxuICAgICAgLmF0dHIoXCJ3aWR0aFwiLCBmdW5jdGlvbih2KSB7IHJldHVybiBnLm5vZGUodikud2lkdGg7IH0pXG4gICAgICAuYXR0cihcImhlaWdodFwiLCBmdW5jdGlvbih2KSB7IHJldHVybiBnLm5vZGUodikuaGVpZ2h0OyB9KVxuICAgICAgLmF0dHIoXCJ4XCIsIGZ1bmN0aW9uKHYpIHtcbiAgICAgICAgdmFyIG5vZGUgPSBnLm5vZGUodik7XG4gICAgICAgIHJldHVybiAtbm9kZS53aWR0aCAvIDI7XG4gICAgICB9KVxuICAgICAgLmF0dHIoXCJ5XCIsIGZ1bmN0aW9uKHYpIHtcbiAgICAgICAgdmFyIG5vZGUgPSBnLm5vZGUodik7XG4gICAgICAgIHJldHVybiAtbm9kZS5oZWlnaHQgLyAyO1xuICAgICAgfSk7XG5cbn1cbiIsIlwidXNlIHN0cmljdFwiO1xuXG52YXIgdXRpbCA9IHJlcXVpcmUoXCIuL3V0aWxcIiksXG4gICAgZDMgPSByZXF1aXJlKFwiLi9kM1wiKSxcbiAgICBfID0gcmVxdWlyZShcIi4vbG9kYXNoXCIpO1xuXG5tb2R1bGUuZXhwb3J0cyA9IHBvc2l0aW9uRWRnZUxhYmVscztcblxuZnVuY3Rpb24gcG9zaXRpb25FZGdlTGFiZWxzKHNlbGVjdGlvbiwgZykge1xuICB2YXIgY3JlYXRlZCA9IHNlbGVjdGlvbi5maWx0ZXIoZnVuY3Rpb24oKSB7IHJldHVybiAhZDMuc2VsZWN0KHRoaXMpLmNsYXNzZWQoXCJ1cGRhdGVcIik7IH0pO1xuXG4gIGZ1bmN0aW9uIHRyYW5zbGF0ZShlKSB7XG4gICAgdmFyIGVkZ2UgPSBnLmVkZ2UoZSk7XG4gICAgcmV0dXJuIF8uaGFzKGVkZ2UsIFwieFwiKSA/IFwidHJhbnNsYXRlKFwiICsgZWRnZS54ICsgXCIsXCIgKyBlZGdlLnkgKyBcIilcIiA6IFwiXCI7XG4gIH1cblxuICBjcmVhdGVkLmF0dHIoXCJ0cmFuc2Zvcm1cIiwgdHJhbnNsYXRlKTtcblxuICB1dGlsLmFwcGx5VHJhbnNpdGlvbihzZWxlY3Rpb24sIGcpXG4gICAgLnN0eWxlKFwib3BhY2l0eVwiLCAxKVxuICAgIC5hdHRyKFwidHJhbnNmb3JtXCIsIHRyYW5zbGF0ZSk7XG59XG4iLCJcInVzZSBzdHJpY3RcIjtcblxudmFyIHV0aWwgPSByZXF1aXJlKFwiLi91dGlsXCIpLFxuICAgIGQzID0gcmVxdWlyZShcIi4vZDNcIik7XG5cbm1vZHVsZS5leHBvcnRzID0gcG9zaXRpb25Ob2RlcztcblxuZnVuY3Rpb24gcG9zaXRpb25Ob2RlcyhzZWxlY3Rpb24sIGcpIHtcbiAgdmFyIGNyZWF0ZWQgPSBzZWxlY3Rpb24uZmlsdGVyKGZ1bmN0aW9uKCkgeyByZXR1cm4gIWQzLnNlbGVjdCh0aGlzKS5jbGFzc2VkKFwidXBkYXRlXCIpOyB9KTtcblxuICBmdW5jdGlvbiB0cmFuc2xhdGUodikge1xuICAgIHZhciBub2RlID0gZy5ub2RlKHYpO1xuICAgIHJldHVybiBcInRyYW5zbGF0ZShcIiArIG5vZGUueCArIFwiLFwiICsgbm9kZS55ICsgXCIpXCI7XG4gIH1cblxuICBjcmVhdGVkLmF0dHIoXCJ0cmFuc2Zvcm1cIiwgdHJhbnNsYXRlKTtcblxuICB1dGlsLmFwcGx5VHJhbnNpdGlvbihzZWxlY3Rpb24sIGcpXG4gICAgLnN0eWxlKFwib3BhY2l0eVwiLCAxKVxuICAgIC5hdHRyKFwidHJhbnNmb3JtXCIsIHRyYW5zbGF0ZSk7XG59XG4iLCJ2YXIgXyA9IHJlcXVpcmUoXCIuL2xvZGFzaFwiKSxcbiAgICBsYXlvdXQgPSByZXF1aXJlKFwiLi9kYWdyZVwiKS5sYXlvdXQ7XG5cbm1vZHVsZS5leHBvcnRzID0gcmVuZGVyO1xuXG4vLyBUaGlzIGRlc2lnbiBpcyBiYXNlZCBvbiBodHRwOi8vYm9zdC5vY2tzLm9yZy9taWtlL2NoYXJ0Ly5cbmZ1bmN0aW9uIHJlbmRlcigpIHtcbiAgdmFyIGNyZWF0ZU5vZGVzID0gcmVxdWlyZShcIi4vY3JlYXRlLW5vZGVzXCIpLFxuICAgICAgY3JlYXRlQ2x1c3RlcnMgPSByZXF1aXJlKFwiLi9jcmVhdGUtY2x1c3RlcnNcIiksXG4gICAgICBjcmVhdGVFZGdlTGFiZWxzID0gcmVxdWlyZShcIi4vY3JlYXRlLWVkZ2UtbGFiZWxzXCIpLFxuICAgICAgY3JlYXRlRWRnZVBhdGhzID0gcmVxdWlyZShcIi4vY3JlYXRlLWVkZ2UtcGF0aHNcIiksXG4gICAgICBwb3NpdGlvbk5vZGVzID0gcmVxdWlyZShcIi4vcG9zaXRpb24tbm9kZXNcIiksXG4gICAgICBwb3NpdGlvbkVkZ2VMYWJlbHMgPSByZXF1aXJlKFwiLi9wb3NpdGlvbi1lZGdlLWxhYmVsc1wiKSxcbiAgICAgIHBvc2l0aW9uQ2x1c3RlcnMgPSByZXF1aXJlKFwiLi9wb3NpdGlvbi1jbHVzdGVyc1wiKSxcbiAgICAgIHNoYXBlcyA9IHJlcXVpcmUoXCIuL3NoYXBlc1wiKSxcbiAgICAgIGFycm93cyA9IHJlcXVpcmUoXCIuL2Fycm93c1wiKTtcblxuICB2YXIgZm4gPSBmdW5jdGlvbihzdmcsIGcpIHtcbiAgICBwcmVQcm9jZXNzR3JhcGgoZyk7XG5cbiAgICB2YXIgb3V0cHV0R3JvdXAgPSBjcmVhdGVPclNlbGVjdEdyb3VwKHN2ZywgXCJvdXRwdXRcIiksXG4gICAgICAgIGNsdXN0ZXJzR3JvdXAgPSBjcmVhdGVPclNlbGVjdEdyb3VwKG91dHB1dEdyb3VwLCBcImNsdXN0ZXJzXCIpLFxuICAgICAgICBlZGdlUGF0aHNHcm91cCA9IGNyZWF0ZU9yU2VsZWN0R3JvdXAob3V0cHV0R3JvdXAsIFwiZWRnZVBhdGhzXCIpLFxuICAgICAgICBlZGdlTGFiZWxzID0gY3JlYXRlRWRnZUxhYmVscyhjcmVhdGVPclNlbGVjdEdyb3VwKG91dHB1dEdyb3VwLCBcImVkZ2VMYWJlbHNcIiksIGcpLFxuICAgICAgICBub2RlcyA9IGNyZWF0ZU5vZGVzKGNyZWF0ZU9yU2VsZWN0R3JvdXAob3V0cHV0R3JvdXAsIFwibm9kZXNcIiksIGcsIHNoYXBlcyk7XG5cbiAgICBsYXlvdXQoZyk7XG5cbiAgICBwb3NpdGlvbk5vZGVzKG5vZGVzLCBnKTtcbiAgICBwb3NpdGlvbkVkZ2VMYWJlbHMoZWRnZUxhYmVscywgZyk7XG4gICAgY3JlYXRlRWRnZVBhdGhzKGVkZ2VQYXRoc0dyb3VwLCBnLCBhcnJvd3MpO1xuXG4gICAgdmFyIGNsdXN0ZXJzID0gY3JlYXRlQ2x1c3RlcnMoY2x1c3RlcnNHcm91cCwgZyk7XG4gICAgcG9zaXRpb25DbHVzdGVycyhjbHVzdGVycywgZyk7XG5cbiAgICBwb3N0UHJvY2Vzc0dyYXBoKGcpO1xuICB9O1xuXG4gIGZuLmNyZWF0ZU5vZGVzID0gZnVuY3Rpb24odmFsdWUpIHtcbiAgICBpZiAoIWFyZ3VtZW50cy5sZW5ndGgpIHJldHVybiBjcmVhdGVOb2RlcztcbiAgICBjcmVhdGVOb2RlcyA9IHZhbHVlO1xuICAgIHJldHVybiBmbjtcbiAgfTtcblxuICBmbi5jcmVhdGVDbHVzdGVycyA9IGZ1bmN0aW9uKHZhbHVlKSB7XG4gICAgaWYgKCFhcmd1bWVudHMubGVuZ3RoKSByZXR1cm4gY3JlYXRlQ2x1c3RlcnM7XG4gICAgY3JlYXRlQ2x1c3RlcnMgPSB2YWx1ZTtcbiAgICByZXR1cm4gZm47XG4gIH07XG5cbiAgZm4uY3JlYXRlRWRnZUxhYmVscyA9IGZ1bmN0aW9uKHZhbHVlKSB7XG4gICAgaWYgKCFhcmd1bWVudHMubGVuZ3RoKSByZXR1cm4gY3JlYXRlRWRnZUxhYmVscztcbiAgICBjcmVhdGVFZGdlTGFiZWxzID0gdmFsdWU7XG4gICAgcmV0dXJuIGZuO1xuICB9O1xuXG4gIGZuLmNyZWF0ZUVkZ2VQYXRocyA9IGZ1bmN0aW9uKHZhbHVlKSB7XG4gICAgaWYgKCFhcmd1bWVudHMubGVuZ3RoKSByZXR1cm4gY3JlYXRlRWRnZVBhdGhzO1xuICAgIGNyZWF0ZUVkZ2VQYXRocyA9IHZhbHVlO1xuICAgIHJldHVybiBmbjtcbiAgfTtcblxuICBmbi5zaGFwZXMgPSBmdW5jdGlvbih2YWx1ZSkge1xuICAgIGlmICghYXJndW1lbnRzLmxlbmd0aCkgcmV0dXJuIHNoYXBlcztcbiAgICBzaGFwZXMgPSB2YWx1ZTtcbiAgICByZXR1cm4gZm47XG4gIH07XG5cbiAgZm4uYXJyb3dzID0gZnVuY3Rpb24odmFsdWUpIHtcbiAgICBpZiAoIWFyZ3VtZW50cy5sZW5ndGgpIHJldHVybiBhcnJvd3M7XG4gICAgYXJyb3dzID0gdmFsdWU7XG4gICAgcmV0dXJuIGZuO1xuICB9O1xuXG4gIHJldHVybiBmbjtcbn1cblxudmFyIE5PREVfREVGQVVMVF9BVFRSUyA9IHtcbiAgcGFkZGluZ0xlZnQ6IDEwLFxuICBwYWRkaW5nUmlnaHQ6IDEwLFxuICBwYWRkaW5nVG9wOiAxMCxcbiAgcGFkZGluZ0JvdHRvbTogMTAsXG4gIHJ4OiAwLFxuICByeTogMCxcbiAgc2hhcGU6IFwicmVjdFwiXG59O1xuXG52YXIgRURHRV9ERUZBVUxUX0FUVFJTID0ge1xuICBhcnJvd2hlYWQ6IFwibm9ybWFsXCIsXG4gIGxpbmVJbnRlcnBvbGF0ZTogXCJsaW5lYXJcIlxufTtcblxuZnVuY3Rpb24gcHJlUHJvY2Vzc0dyYXBoKGcpIHtcbiAgZy5ub2RlcygpLmZvckVhY2goZnVuY3Rpb24odikge1xuICAgIHZhciBub2RlID0gZy5ub2RlKHYpO1xuICAgIGlmICghXy5oYXMobm9kZSwgXCJsYWJlbFwiKSAmJiAhZy5jaGlsZHJlbih2KS5sZW5ndGgpIHsgbm9kZS5sYWJlbCA9IHY7IH1cblxuICAgIGlmIChfLmhhcyhub2RlLCBcInBhZGRpbmdYXCIpKSB7XG4gICAgICBfLmRlZmF1bHRzKG5vZGUsIHtcbiAgICAgICAgcGFkZGluZ0xlZnQ6IG5vZGUucGFkZGluZ1gsXG4gICAgICAgIHBhZGRpbmdSaWdodDogbm9kZS5wYWRkaW5nWFxuICAgICAgfSk7XG4gICAgfVxuXG4gICAgaWYgKF8uaGFzKG5vZGUsIFwicGFkZGluZ1lcIikpIHtcbiAgICAgIF8uZGVmYXVsdHMobm9kZSwge1xuICAgICAgICBwYWRkaW5nVG9wOiBub2RlLnBhZGRpbmdZLFxuICAgICAgICBwYWRkaW5nQm90dG9tOiBub2RlLnBhZGRpbmdZXG4gICAgICB9KTtcbiAgICB9XG5cbiAgICBpZiAoXy5oYXMobm9kZSwgXCJwYWRkaW5nXCIpKSB7XG4gICAgICBfLmRlZmF1bHRzKG5vZGUsIHtcbiAgICAgICAgcGFkZGluZ0xlZnQ6IG5vZGUucGFkZGluZyxcbiAgICAgICAgcGFkZGluZ1JpZ2h0OiBub2RlLnBhZGRpbmcsXG4gICAgICAgIHBhZGRpbmdUb3A6IG5vZGUucGFkZGluZyxcbiAgICAgICAgcGFkZGluZ0JvdHRvbTogbm9kZS5wYWRkaW5nXG4gICAgICB9KTtcbiAgICB9XG5cbiAgICBfLmRlZmF1bHRzKG5vZGUsIE5PREVfREVGQVVMVF9BVFRSUyk7XG5cbiAgICBfLmVhY2goW1wicGFkZGluZ0xlZnRcIiwgXCJwYWRkaW5nUmlnaHRcIiwgXCJwYWRkaW5nVG9wXCIsIFwicGFkZGluZ0JvdHRvbVwiXSwgZnVuY3Rpb24oaykge1xuICAgICAgbm9kZVtrXSA9IE51bWJlcihub2RlW2tdKTtcbiAgICB9KTtcblxuICAgIC8vIFNhdmUgZGltZW5zaW9ucyBmb3IgcmVzdG9yZSBkdXJpbmcgcG9zdC1wcm9jZXNzaW5nXG4gICAgaWYgKF8uaGFzKG5vZGUsIFwid2lkdGhcIikpIHsgbm9kZS5fcHJldldpZHRoID0gbm9kZS53aWR0aDsgfVxuICAgIGlmIChfLmhhcyhub2RlLCBcImhlaWdodFwiKSkgeyBub2RlLl9wcmV2SGVpZ2h0ID0gbm9kZS5oZWlnaHQ7IH1cbiAgfSk7XG5cbiAgZy5lZGdlcygpLmZvckVhY2goZnVuY3Rpb24oZSkge1xuICAgIHZhciBlZGdlID0gZy5lZGdlKGUpO1xuICAgIGlmICghXy5oYXMoZWRnZSwgXCJsYWJlbFwiKSkgeyBlZGdlLmxhYmVsID0gXCJcIjsgfVxuICAgIF8uZGVmYXVsdHMoZWRnZSwgRURHRV9ERUZBVUxUX0FUVFJTKTtcbiAgfSk7XG59XG5cbmZ1bmN0aW9uIHBvc3RQcm9jZXNzR3JhcGgoZykge1xuICBfLmVhY2goZy5ub2RlcygpLCBmdW5jdGlvbih2KSB7XG4gICAgdmFyIG5vZGUgPSBnLm5vZGUodik7XG5cbiAgICAvLyBSZXN0b3JlIG9yaWdpbmFsIGRpbWVuc2lvbnNcbiAgICBpZiAoXy5oYXMobm9kZSwgXCJfcHJldldpZHRoXCIpKSB7XG4gICAgICBub2RlLndpZHRoID0gbm9kZS5fcHJldldpZHRoO1xuICAgIH0gZWxzZSB7XG4gICAgICBkZWxldGUgbm9kZS53aWR0aDtcbiAgICB9XG5cbiAgICBpZiAoXy5oYXMobm9kZSwgXCJfcHJldkhlaWdodFwiKSkge1xuICAgICAgbm9kZS5oZWlnaHQgPSBub2RlLl9wcmV2SGVpZ2h0O1xuICAgIH0gZWxzZSB7XG4gICAgICBkZWxldGUgbm9kZS5oZWlnaHQ7XG4gICAgfVxuXG4gICAgZGVsZXRlIG5vZGUuX3ByZXZXaWR0aDtcbiAgICBkZWxldGUgbm9kZS5fcHJldkhlaWdodDtcbiAgfSk7XG59XG5cbmZ1bmN0aW9uIGNyZWF0ZU9yU2VsZWN0R3JvdXAocm9vdCwgbmFtZSkge1xuICB2YXIgc2VsZWN0aW9uID0gcm9vdC5zZWxlY3QoXCJnLlwiICsgbmFtZSk7XG4gIGlmIChzZWxlY3Rpb24uZW1wdHkoKSkge1xuICAgIHNlbGVjdGlvbiA9IHJvb3QuYXBwZW5kKFwiZ1wiKS5hdHRyKFwiY2xhc3NcIiwgbmFtZSk7XG4gIH1cbiAgcmV0dXJuIHNlbGVjdGlvbjtcbn1cbiIsIlwidXNlIHN0cmljdFwiO1xuXG52YXIgaW50ZXJzZWN0UmVjdCA9IHJlcXVpcmUoXCIuL2ludGVyc2VjdC9pbnRlcnNlY3QtcmVjdFwiKSxcbiAgICBpbnRlcnNlY3RFbGxpcHNlID0gcmVxdWlyZShcIi4vaW50ZXJzZWN0L2ludGVyc2VjdC1lbGxpcHNlXCIpLFxuICAgIGludGVyc2VjdENpcmNsZSA9IHJlcXVpcmUoXCIuL2ludGVyc2VjdC9pbnRlcnNlY3QtY2lyY2xlXCIpLFxuICAgIGludGVyc2VjdFBvbHlnb24gPSByZXF1aXJlKFwiLi9pbnRlcnNlY3QvaW50ZXJzZWN0LXBvbHlnb25cIik7XG5cbm1vZHVsZS5leHBvcnRzID0ge1xuICByZWN0OiByZWN0LFxuICBlbGxpcHNlOiBlbGxpcHNlLFxuICBjaXJjbGU6IGNpcmNsZSxcbiAgZGlhbW9uZDogZGlhbW9uZFxufTtcblxuZnVuY3Rpb24gcmVjdChwYXJlbnQsIGJib3gsIG5vZGUpIHtcbiAgdmFyIHNoYXBlU3ZnID0gcGFyZW50Lmluc2VydChcInJlY3RcIiwgXCI6Zmlyc3QtY2hpbGRcIilcbiAgICAgICAgLmF0dHIoXCJyeFwiLCBub2RlLnJ4KVxuICAgICAgICAuYXR0cihcInJ5XCIsIG5vZGUucnkpXG4gICAgICAgIC5hdHRyKFwieFwiLCAtYmJveC53aWR0aCAvIDIpXG4gICAgICAgIC5hdHRyKFwieVwiLCAtYmJveC5oZWlnaHQgLyAyKVxuICAgICAgICAuYXR0cihcIndpZHRoXCIsIGJib3gud2lkdGgpXG4gICAgICAgIC5hdHRyKFwiaGVpZ2h0XCIsIGJib3guaGVpZ2h0KTtcblxuICBub2RlLmludGVyc2VjdCA9IGZ1bmN0aW9uKHBvaW50KSB7XG4gICAgcmV0dXJuIGludGVyc2VjdFJlY3Qobm9kZSwgcG9pbnQpO1xuICB9O1xuXG4gIHJldHVybiBzaGFwZVN2Zztcbn1cblxuZnVuY3Rpb24gZWxsaXBzZShwYXJlbnQsIGJib3gsIG5vZGUpIHtcbiAgdmFyIHJ4ID0gYmJveC53aWR0aCAvIDIsXG4gICAgICByeSA9IGJib3guaGVpZ2h0IC8gMixcbiAgICAgIHNoYXBlU3ZnID0gcGFyZW50Lmluc2VydChcImVsbGlwc2VcIiwgXCI6Zmlyc3QtY2hpbGRcIilcbiAgICAgICAgLmF0dHIoXCJ4XCIsIC1iYm94LndpZHRoIC8gMilcbiAgICAgICAgLmF0dHIoXCJ5XCIsIC1iYm94LmhlaWdodCAvIDIpXG4gICAgICAgIC5hdHRyKFwicnhcIiwgcngpXG4gICAgICAgIC5hdHRyKFwicnlcIiwgcnkpO1xuXG4gIG5vZGUuaW50ZXJzZWN0ID0gZnVuY3Rpb24ocG9pbnQpIHtcbiAgICByZXR1cm4gaW50ZXJzZWN0RWxsaXBzZShub2RlLCByeCwgcnksIHBvaW50KTtcbiAgfTtcblxuICByZXR1cm4gc2hhcGVTdmc7XG59XG5cbmZ1bmN0aW9uIGNpcmNsZShwYXJlbnQsIGJib3gsIG5vZGUpIHtcbiAgdmFyIHIgPSBNYXRoLm1heChiYm94LndpZHRoLCBiYm94LmhlaWdodCkgLyAyLFxuICAgICAgc2hhcGVTdmcgPSBwYXJlbnQuaW5zZXJ0KFwiY2lyY2xlXCIsIFwiOmZpcnN0LWNoaWxkXCIpXG4gICAgICAgIC5hdHRyKFwieFwiLCAtYmJveC53aWR0aCAvIDIpXG4gICAgICAgIC5hdHRyKFwieVwiLCAtYmJveC5oZWlnaHQgLyAyKVxuICAgICAgICAuYXR0cihcInJcIiwgcik7XG5cbiAgbm9kZS5pbnRlcnNlY3QgPSBmdW5jdGlvbihwb2ludCkge1xuICAgIHJldHVybiBpbnRlcnNlY3RDaXJjbGUobm9kZSwgciwgcG9pbnQpO1xuICB9O1xuXG4gIHJldHVybiBzaGFwZVN2Zztcbn1cblxuLy8gQ2lyY3Vtc2NyaWJlIGFuIGVsbGlwc2UgZm9yIHRoZSBib3VuZGluZyBib3ggd2l0aCBhIGRpYW1vbmQgc2hhcGUuIEkgZGVyaXZlZFxuLy8gdGhlIGZ1bmN0aW9uIHRvIGNhbGN1bGF0ZSB0aGUgZGlhbW9uZCBzaGFwZSBmcm9tOlxuLy8gaHR0cDovL21hdGhmb3J1bS5vcmcva2IvbWVzc2FnZS5qc3BhP21lc3NhZ2VJRD0zNzUwMjM2XG5mdW5jdGlvbiBkaWFtb25kKHBhcmVudCwgYmJveCwgbm9kZSkge1xuICB2YXIgdyA9IChiYm94LndpZHRoICogTWF0aC5TUVJUMikgLyAyLFxuICAgICAgaCA9IChiYm94LmhlaWdodCAqIE1hdGguU1FSVDIpIC8gMixcbiAgICAgIHBvaW50cyA9IFtcbiAgICAgICAgeyB4OiAgMCwgeTogLWggfSxcbiAgICAgICAgeyB4OiAtdywgeTogIDAgfSxcbiAgICAgICAgeyB4OiAgMCwgeTogIGggfSxcbiAgICAgICAgeyB4OiAgdywgeTogIDAgfVxuICAgICAgXSxcbiAgICAgIHNoYXBlU3ZnID0gcGFyZW50Lmluc2VydChcInBvbHlnb25cIiwgXCI6Zmlyc3QtY2hpbGRcIilcbiAgICAgICAgLmF0dHIoXCJwb2ludHNcIiwgcG9pbnRzLm1hcChmdW5jdGlvbihwKSB7IHJldHVybiBwLnggKyBcIixcIiArIHAueTsgfSkuam9pbihcIiBcIikpO1xuXG4gIG5vZGUuaW50ZXJzZWN0ID0gZnVuY3Rpb24ocCkge1xuICAgIHJldHVybiBpbnRlcnNlY3RQb2x5Z29uKG5vZGUsIHBvaW50cywgcCk7XG4gIH07XG5cbiAgcmV0dXJuIHNoYXBlU3ZnO1xufVxuIiwidmFyIF8gPSByZXF1aXJlKFwiLi9sb2Rhc2hcIik7XG5cbi8vIFB1YmxpYyB1dGlsaXR5IGZ1bmN0aW9uc1xubW9kdWxlLmV4cG9ydHMgPSB7XG4gIGlzU3ViZ3JhcGg6IGlzU3ViZ3JhcGgsXG4gIGVkZ2VUb0lkOiBlZGdlVG9JZCxcbiAgYXBwbHlTdHlsZTogYXBwbHlTdHlsZSxcbiAgYXBwbHlDbGFzczogYXBwbHlDbGFzcyxcbiAgYXBwbHlUcmFuc2l0aW9uOiBhcHBseVRyYW5zaXRpb25cbn07XG5cbi8qXG4gKiBSZXR1cm5zIHRydWUgaWYgdGhlIHNwZWNpZmllZCBub2RlIGluIHRoZSBncmFwaCBpcyBhIHN1YmdyYXBoIG5vZGUuIEFcbiAqIHN1YmdyYXBoIG5vZGUgaXMgb25lIHRoYXQgY29udGFpbnMgb3RoZXIgbm9kZXMuXG4gKi9cbmZ1bmN0aW9uIGlzU3ViZ3JhcGgoZywgdikge1xuICByZXR1cm4gISFnLmNoaWxkcmVuKHYpLmxlbmd0aDtcbn1cblxuZnVuY3Rpb24gZWRnZVRvSWQoZSkge1xuICByZXR1cm4gZXNjYXBlSWQoZS52KSArIFwiOlwiICsgZXNjYXBlSWQoZS53KSArIFwiOlwiICsgZXNjYXBlSWQoZS5uYW1lKTtcbn1cblxudmFyIElEX0RFTElNID0gLzovZztcbmZ1bmN0aW9uIGVzY2FwZUlkKHN0cikge1xuICByZXR1cm4gc3RyID8gU3RyaW5nKHN0cikucmVwbGFjZShJRF9ERUxJTSwgXCJcXFxcOlwiKSA6IFwiXCI7XG59XG5cbmZ1bmN0aW9uIGFwcGx5U3R5bGUoZG9tLCBzdHlsZUZuKSB7XG4gIGlmIChzdHlsZUZuKSB7XG4gICAgZG9tLmF0dHIoXCJzdHlsZVwiLCBzdHlsZUZuKTtcbiAgfVxufVxuXG5mdW5jdGlvbiBhcHBseUNsYXNzKGRvbSwgY2xhc3NGbiwgb3RoZXJDbGFzc2VzKSB7XG4gIGlmIChjbGFzc0ZuKSB7XG4gICAgZG9tXG4gICAgICAuYXR0cihcImNsYXNzXCIsIGNsYXNzRm4pXG4gICAgICAuYXR0cihcImNsYXNzXCIsIG90aGVyQ2xhc3NlcyArIFwiIFwiICsgZG9tLmF0dHIoXCJjbGFzc1wiKSk7XG4gIH1cbn1cblxuZnVuY3Rpb24gYXBwbHlUcmFuc2l0aW9uKHNlbGVjdGlvbiwgZykge1xuICB2YXIgZ3JhcGggPSBnLmdyYXBoKCk7XG5cbiAgaWYgKF8uaXNQbGFpbk9iamVjdChncmFwaCkpIHtcbiAgICB2YXIgdHJhbnNpdGlvbiA9IGdyYXBoLnRyYW5zaXRpb247XG4gICAgaWYgKF8uaXNGdW5jdGlvbih0cmFuc2l0aW9uKSkge1xuICAgICAgcmV0dXJuIHRyYW5zaXRpb24oc2VsZWN0aW9uKTtcbiAgICB9XG4gIH1cblxuICByZXR1cm4gc2VsZWN0aW9uO1xufVxuIiwibW9kdWxlLmV4cG9ydHMgPSBcIjAuNC4xMFwiO1xuIl19 // ---------------------------------------------------------------------------- // D3 timeline diff --git a/flink-runtime-web/web-dashboard/web/partials/jobs/job.html b/flink-runtime-web/web-dashboard/web/partials/jobs/job.html index ab61ee7ec0d..c7dc0bc18e5 100644 --- a/flink-runtime-web/web-dashboard/web/partials/jobs/job.html +++ b/flink-runtime-web/web-dashboard/web/partials/jobs/job.html @@ -33,6 +33,7 @@ limitations under the License. {{ job['end-time'] | amDateFormat:'YYYY-MM-DD, H:mm:ss' }}
+